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 2016/12/16 15:46:30 UTC

[01/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Repository: flink
Updated Branches:
  refs/heads/master 48ef46a4d -> 67c4be648


http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/DataStreamCorrelateITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/DataStreamCorrelateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/DataStreamCorrelateITCase.scala
new file mode 100644
index 0000000..eb20517
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/datastream/DataStreamCorrelateITCase.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.datastream
+
+import org.apache.flink.api.scala._
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.scala.stream.utils.StreamITCase
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableFunc0
+import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.TableEnvironment
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.mutable
+
+class DataStreamCorrelateITCase extends StreamingMultipleProgramsTestBase {
+
+  @Test
+  def testCrossJoin(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.clear
+
+    val t = testData(env).toTable(tEnv).as('a, 'b, 'c)
+    val func0 = new TableFunc0
+
+    val result = t
+      .join(func0('c) as('d, 'e))
+      .select('c, 'd, 'e)
+      .toDataStream[Row]
+
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("Jack#22,Jack,22", "John#19,John,19", "Anna#44,Anna,44")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testLeftOuterJoin(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.clear
+
+    val t = testData(env).toTable(tEnv).as('a, 'b, 'c)
+    val func0 = new TableFunc0
+
+    val result = t
+      .leftOuterJoin(func0('c) as('d, 'e))
+      .select('c, 'd, 'e)
+      .toDataStream[Row]
+
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "nosharp,null,null", "Jack#22,Jack,22",
+      "John#19,John,19", "Anna#44,Anna,44")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  private def testData(
+    env: StreamExecutionEnvironment)
+  : DataStream[(Int, Long, String)] = {
+
+    val data = new mutable.MutableList[(Int, Long, String)]
+    data.+=((1, 1L, "Jack#22"))
+    data.+=((2, 2L, "John#19"))
+    data.+=((3, 2L, "Anna#44"))
+    data.+=((4, 3L, "nosharp"))
+    env.fromCollection(data)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..ff2c5d3
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/TableTestBase.scala
@@ -0,0 +1,229 @@
+/*
+ * 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.utils
+
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.flink.api.common.typeinfo.TypeInformation
+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.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.scala.{DataStream, StreamExecutionEnvironment}
+import org.junit.Assert.assertEquals
+import org.mockito.Mockito.{mock, when}
+
+/**
+  * Test base for testing Table API / SQL plans.
+  */
+class TableTestBase {
+
+  def batchTestUtil(): BatchTableTestUtil = {
+    BatchTableTestUtil()
+  }
+
+  def streamTestUtil(): StreamTableTestUtil = {
+    StreamTableTestUtil()
+  }
+
+  def verifyTableEquals(expected: Table, actual: Table): Unit = {
+    assertEquals(
+      "Logical plans do not match",
+      RelOptUtil.toString(expected.getRelNode),
+      RelOptUtil.toString(actual.getRelNode))
+  }
+
+}
+
+abstract class TableTestUtil {
+
+  private var counter = 0
+
+  def addTable[T: TypeInformation](fields: Expression*): Table = {
+    addTable[T](s"Table${counter += 1}", fields: _*)
+  }
+
+  def addTable[T: TypeInformation](name: String, fields: Expression*): Table
+  def addFunction[T: TypeInformation](name: String, function: TableFunction[T]): TableFunction[T]
+  def addFunction(name: String, function: ScalarFunction): Unit
+
+  def verifySql(query: String, expected: String): Unit
+  def verifyTable(resultTable: Table, expected: String): Unit
+
+  // the print methods are for debugging purposes only
+  def printTable(resultTable: Table): Unit
+  def printSql(query: String): Unit
+}
+
+object TableTestUtil {
+
+  // this methods are currently just for simplifying string construction,
+  // we could replace it with logic later
+
+  def unaryNode(node: String, input: String, term: String*): String = {
+    s"""$node(${term.mkString(", ")})
+       |$input
+       |""".stripMargin.stripLineEnd
+  }
+
+  def binaryNode(node: String, left: String, right: String, term: String*): String = {
+    s"""$node(${term.mkString(", ")})
+       |$left
+       |$right
+       |""".stripMargin.stripLineEnd
+  }
+
+  def values(node: String, term: String*): String = {
+    s"$node(${term.mkString(", ")})"
+  }
+
+  def term(term: AnyRef, value: AnyRef*): String = {
+    s"$term=[${value.mkString(", ")}]"
+  }
+
+  def tuples(value:List[AnyRef]*): String={
+    val listValues = value.map( listValue => s"{ ${listValue.mkString(", ")} }")
+    term("tuples","[" + listValues.mkString(", ") + "]")
+  }
+
+  def batchTableNode(idx: Int): String = {
+    s"DataSetScan(table=[[_DataSetTable_$idx]])"
+  }
+
+  def streamTableNode(idx: Int): String = {
+    s"DataStreamScan(table=[[_DataStreamTable_$idx]])"
+  }
+
+}
+
+case class BatchTableTestUtil() extends TableTestUtil {
+
+  val env = mock(classOf[ExecutionEnvironment])
+  val tEnv = TableEnvironment.getTableEnvironment(env)
+
+  def addTable[T: TypeInformation](
+      name: String,
+      fields: Expression*)
+    : Table = {
+    val ds = mock(classOf[DataSet[T]])
+    val jDs = mock(classOf[JDataSet[T]])
+    when(ds.javaSet).thenReturn(jDs)
+    val typeInfo: TypeInformation[T] = implicitly[TypeInformation[T]]
+    when(jDs.getType).thenReturn(typeInfo)
+
+    val t = ds.toTable(tEnv, fields: _*)
+    tEnv.registerTable(name, t)
+    t
+  }
+
+  def addFunction[T: TypeInformation](
+      name: String,
+      function: TableFunction[T])
+    : TableFunction[T] = {
+    tEnv.registerFunction(name, function)
+    function
+  }
+
+  def addFunction(name: String, function: ScalarFunction): Unit = {
+    tEnv.registerFunction(name, function)
+  }
+
+  def verifySql(query: String, expected: String): Unit = {
+    verifyTable(tEnv.sql(query), expected)
+  }
+
+  def verifyTable(resultTable: Table, expected: String): Unit = {
+    val relNode = resultTable.getRelNode
+    val optimized = tEnv.optimize(relNode)
+    val actual = RelOptUtil.toString(optimized)
+    assertEquals(
+      expected.split("\n").map(_.trim).mkString("\n"),
+      actual.split("\n").map(_.trim).mkString("\n"))
+  }
+
+  def printTable(resultTable: Table): Unit = {
+    val relNode = resultTable.getRelNode
+    val optimized = tEnv.optimize(relNode)
+    println(RelOptUtil.toString(optimized))
+  }
+
+  def printSql(query: String): Unit = {
+    printTable(tEnv.sql(query))
+  }
+}
+
+case class StreamTableTestUtil() extends TableTestUtil {
+
+  val env = mock(classOf[StreamExecutionEnvironment])
+  val tEnv = TableEnvironment.getTableEnvironment(env)
+
+  def addTable[T: TypeInformation](
+      name: String,
+      fields: Expression*)
+    : Table = {
+
+    val ds = mock(classOf[DataStream[T]])
+    val jDs = mock(classOf[JDataStream[T]])
+    when(ds.javaStream).thenReturn(jDs)
+    val typeInfo: TypeInformation[T] = implicitly[TypeInformation[T]]
+    when(jDs.getType).thenReturn(typeInfo)
+
+    val t = ds.toTable(tEnv, fields: _*)
+    tEnv.registerTable(name, t)
+    t
+  }
+
+  def addFunction[T: TypeInformation](
+      name: String,
+      function: TableFunction[T])
+    : TableFunction[T] = {
+    tEnv.registerFunction(name, function)
+    function
+  }
+
+  def addFunction(name: String, function: ScalarFunction): Unit = {
+    tEnv.registerFunction(name, function)
+  }
+
+  def verifySql(query: String, expected: String): Unit = {
+    verifyTable(tEnv.sql(query), expected)
+  }
+
+  def verifyTable(resultTable: Table, expected: String): Unit = {
+    val relNode = resultTable.getRelNode
+    val optimized = tEnv.optimize(relNode)
+    val actual = RelOptUtil.toString(optimized)
+    assertEquals(
+      expected.split("\n").map(_.trim).mkString("\n"),
+      actual.split("\n").map(_.trim).mkString("\n"))
+  }
+
+  // the print methods are for debugging purposes only
+  def printTable(resultTable: Table): Unit = {
+    val relNode = resultTable.getRelNode
+    val optimized = tEnv.optimize(relNode)
+    println(RelOptUtil.toString(optimized))
+  }
+
+  def printSql(query: String): Unit = {
+    printTable(tEnv.sql(query))
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala
new file mode 100644
index 0000000..54861ea
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefinedTableFunctions.scala
@@ -0,0 +1,117 @@
+/*
+ * 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.utils
+
+import java.lang.Boolean
+
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.java.tuple.Tuple3
+import org.apache.flink.types.Row
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.functions.TableFunction
+
+
+case class SimpleUser(name: String, age: Int)
+
+class TableFunc0 extends TableFunction[SimpleUser] {
+  // make sure input element's format is "<string>#<int>"
+  def eval(user: String): Unit = {
+    if (user.contains("#")) {
+      val splits = user.split("#")
+      collect(SimpleUser(splits(0), splits(1).toInt))
+    }
+  }
+}
+
+class TableFunc1 extends TableFunction[String] {
+  def eval(str: String): Unit = {
+    if (str.contains("#")){
+      str.split("#").foreach(collect)
+    }
+  }
+
+  def eval(str: String, prefix: String): Unit = {
+    if (str.contains("#")) {
+      str.split("#").foreach(s => collect(prefix + s))
+    }
+  }
+}
+
+
+class TableFunc2 extends TableFunction[Row] {
+  def eval(str: String): Unit = {
+    if (str.contains("#")) {
+      str.split("#").foreach({ s =>
+        val row = new Row(2)
+        row.setField(0, s)
+        row.setField(1, s.length)
+        collect(row)
+      })
+    }
+  }
+
+  override def getResultType: TypeInformation[Row] = {
+    new RowTypeInfo(BasicTypeInfo.STRING_TYPE_INFO,
+                    BasicTypeInfo.INT_TYPE_INFO)
+  }
+}
+
+class HierarchyTableFunction extends SplittableTableFunction[Boolean, Integer] {
+  def eval(user: String) {
+    if (user.contains("#")) {
+      val splits = user.split("#")
+      val age = splits(1).toInt
+      collect(new Tuple3[String, Boolean, Integer](splits(0), age >= 20, age))
+    }
+  }
+}
+
+abstract class SplittableTableFunction[A, B] extends TableFunction[Tuple3[String, A, B]] {}
+
+class PojoTableFunc extends TableFunction[PojoUser] {
+  def eval(user: String) {
+    if (user.contains("#")) {
+      val splits = user.split("#")
+      collect(new PojoUser(splits(0), splits(1).toInt))
+    }
+  }
+}
+
+class PojoUser() {
+  var name: String = _
+  var age: Int = 0
+
+  def this(name: String, age: Int) {
+    this()
+    this.name = name
+    this.age = age
+  }
+}
+
+// ----------------------------------------------------------------------------------------------
+// Invalid Table Functions
+// ----------------------------------------------------------------------------------------------
+
+
+// this is used to check whether scala object is forbidden
+object ObjectTableFunction extends TableFunction[Integer] {
+  def eval(a: Int, b: Int): Unit = {
+    collect(a)
+    collect(b)
+  }
+}


[12/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala
deleted file mode 100644
index 074f70b..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/JoinITCase.scala
+++ /dev/null
@@ -1,366 +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.api.scala.batch.sql
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.{ValidationException, TableEnvironment, TableException}
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class JoinITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testJoin(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE b = e"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testJoinWithFilter(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE b = e AND b < 2"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hi,Hallo\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testJoinWithJoinFilter(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE b = e AND a < 6 AND h < b"
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hello world, how are you?,Hallo Welt wie\n" +
-      "I am fine.,Hallo Welt wie\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testJoinWithMultipleKeys(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE a = d AND b = h"
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" +
-      "Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testJoinNonExistingKey(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE foo = e"
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    tEnv.sql(sqlQuery)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testJoinNonMatchingKeyTypes(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE a = g"
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    tEnv.sql(sqlQuery).toDataSet[Row].collect()
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testJoinWithAmbiguousFields(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE a = d"
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'c)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    tEnv.sql(sqlQuery)
-  }
-
-  @Test
-  def testJoinWithAlias(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT Table5.c, Table3.c FROM Table3, Table5 WHERE a = d AND a < 4"
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'c)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    val result = tEnv.sql(sqlQuery)
-    val expected = "1,Hi\n" + "2,Hello\n" + "1,Hello\n" +
-      "2,Hello world\n" + "2,Hello world\n" + "3,Hello world\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testJoinNoEqualityPredicate(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE d = f"
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    tEnv.sql(sqlQuery).toDataSet[Row].collect()
-  }
-
-  @Test
-  def testDataSetJoinWithAggregation(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT COUNT(g), COUNT(b) FROM Table3, Table5 WHERE a = d"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env)
-    tEnv.registerDataSet("Table3", ds1, 'a, 'b, 'c)
-    tEnv.registerDataSet("Table5", ds2, 'd, 'e, 'f, 'g, 'h)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "6,6"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testTableJoinWithAggregation(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT COUNT(b), COUNT(g) FROM Table3, Table5 WHERE a = d"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "6,6"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testFullOuterJoin(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    tEnv.getConfig.setNullCheck(true)
-
-    val sqlQuery = "SELECT c, g FROM Table3 FULL OUTER JOIN Table5 ON b = e"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" +
-      "null,Hallo Welt wie\n" + "null,Hallo Welt wie gehts?\n" + "null,ABC\n" + "null,BCD\n" +
-      "null,CDE\n" + "null,DEF\n" + "null,EFG\n" + "null,FGH\n" + "null,GHI\n" + "null,HIJ\n" +
-      "null,IJK\n" + "null,JKL\n" + "null,KLM"
-
-    val results = tEnv.sql(sqlQuery).toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testLeftOuterJoin(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    tEnv.getConfig.setNullCheck(true)
-
-    val sqlQuery = "SELECT c, g FROM Table5 LEFT OUTER JOIN Table3 ON b = e"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    tEnv.sql(sqlQuery).toDataSet[Row].collect()
-
-    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" +
-      "null,Hallo Welt wie\n" + "null,Hallo Welt wie gehts?\n" + "null,ABC\n" + "null,BCD\n" +
-      "null,CDE\n" + "null,DEF\n" + "null,EFG\n" + "null,FGH\n" + "null,GHI\n" + "null,HIJ\n" +
-      "null,IJK\n" + "null,JKL\n" + "null,KLM"
-    val results = tEnv.sql(sqlQuery).toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testRightOuterJoin(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    tEnv.getConfig.setNullCheck(true)
-
-    val sqlQuery = "SELECT c, g FROM Table3 RIGHT OUTER JOIN Table5 ON b = e"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-    tEnv.registerTable("Table3", ds1)
-    tEnv.registerTable("Table5", ds2)
-
-    tEnv.sql(sqlQuery).toDataSet[Row].collect()
-
-    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" +
-      "null,Hallo Welt wie\n" + "null,Hallo Welt wie gehts?\n" + "null,ABC\n" + "null,BCD\n" +
-      "null,CDE\n" + "null,DEF\n" + "null,EFG\n" + "null,FGH\n" + "null,GHI\n" + "null,HIJ\n" +
-      "null,IJK\n" + "null,JKL\n" + "null,KLM"
-    val results = tEnv.sql(sqlQuery).toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testCrossJoin(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val table1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a1, 'a2, 'a3)
-    val table2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('b1, 'b2, 'b3)
-    tEnv.registerTable("A", table1)
-    tEnv.registerTable("B", table2)
-
-    val sqlQuery = "SELECT a1, b1 FROM A CROSS JOIN B"
-    tEnv.sql(sqlQuery).count
-  }
-
-  @Test
-  def testCrossJoinWithLeftSingleRowInput(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val table = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a1, 'a2, 'a3)
-    tEnv.registerTable("A", table)
-
-    val sqlQuery2 = "SELECT * FROM (SELECT count(*) FROM A) CROSS JOIN A"
-    val expected =
-      "3,1,1,Hi\n" +
-      "3,2,2,Hello\n" +
-      "3,3,2,Hello world"
-    val result = tEnv.sql(sqlQuery2).collect()
-    TestBaseUtils.compareResultAsText(result.asJava, expected)
-  }
-
-  @Test
-  def testCrossJoinWithRightSingleRowInput(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val table = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a1, 'a2, 'a3)
-    tEnv.registerTable("A", table)
-
-    val sqlQuery1 = "SELECT * FROM A CROSS JOIN (SELECT count(*) FROM A)"
-    val expected =
-      "1,1,Hi,3\n" +
-      "2,2,Hello,3\n" +
-      "3,2,Hello world,3"
-    val result = tEnv.sql(sqlQuery1).collect()
-    TestBaseUtils.compareResultAsText(result.asJava, expected)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsITCase.scala
deleted file mode 100644
index 42bd6e8..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsITCase.scala
+++ /dev/null
@@ -1,266 +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.api.scala.batch.sql
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.util.Random
-
-@RunWith(classOf[Parameterized])
-class SetOperatorsITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testUnionAll(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c FROM t1 UNION ALL (SELECT f FROM t2)"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
-    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env)
-    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
-    tEnv.registerDataSet("t2", ds2, 'd, 'e, 'f)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hi\n" + "Hello\n" + "Hello world\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testUnion(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c FROM t1 UNION (SELECT f FROM t2)"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
-    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env)
-    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
-    tEnv.registerDataSet("t2", ds2, 'd, 'e, 'f)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hi\n" + "Hello\n" + "Hello world\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testUnionWithFilter(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c FROM (" +
-      "SELECT * FROM t1 UNION ALL (SELECT a, b, c FROM t2))" +
-      "WHERE b < 2"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env)
-    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
-    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'd, 'c, 'e)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hi\n" + "Hallo\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testUnionWithAggregation(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT count(c) FROM (" +
-      "SELECT * FROM t1 UNION ALL (SELECT a, b, c FROM t2))"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env)
-    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
-    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'd, 'c, 'e)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "18"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testExcept(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c FROM t1 EXCEPT (SELECT c FROM t2)"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
-    val ds2 = env.fromElements((1, 1L, "Hi"))
-    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
-    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'c)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hello\n" + "Hello world\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  @Ignore
-  // calcite sql parser doesn't support EXCEPT ALL
-  def testExceptAll(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c FROM t1 EXCEPT ALL SELECT c FROM t2"
-
-    val data1 = new mutable.MutableList[Int]
-    data1 += (1, 1, 1, 2, 2)
-    val data2 = new mutable.MutableList[Int]
-    data2 += (1, 2, 2, 3)
-    val ds1 = env.fromCollection(data1)
-    val ds2 = env.fromCollection(data2)
-
-    tEnv.registerDataSet("t1", ds1, 'c)
-    tEnv.registerDataSet("t2", ds2, 'c)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1\n1"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testExceptWithFilter(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c FROM (" +
-      "SELECT * FROM t1 EXCEPT (SELECT a, b, c FROM t2))" +
-      "WHERE b < 2"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env)
-    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
-    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'd, 'c, 'e)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hi\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testIntersect(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c FROM t1 INTERSECT SELECT c FROM t2"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
-    val data = new mutable.MutableList[(Int, Long, String)]
-    data.+=((1, 1L, "Hi"))
-    data.+=((2, 2L, "Hello"))
-    data.+=((2, 2L, "Hello"))
-    data.+=((3, 2L, "Hello world!"))
-    val ds2 = env.fromCollection(Random.shuffle(data))
-
-    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
-    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'c)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hi\n" + "Hello\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  @Ignore
-  // calcite sql parser doesn't support INTERSECT ALL
-  def testIntersectAll(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c FROM t1 INTERSECT ALL SELECT c FROM t2"
-
-    val data1 = new mutable.MutableList[Int]
-    data1 += (1, 1, 1, 2, 2)
-    val data2 = new mutable.MutableList[Int]
-    data2 += (1, 2, 2, 3)
-    val ds1 = env.fromCollection(data1)
-    val ds2 = env.fromCollection(data2)
-
-    tEnv.registerDataSet("t1", ds1, 'c)
-    tEnv.registerDataSet("t2", ds2, 'c)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1\n2\n2"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testIntersectWithFilter(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT c FROM ((SELECT * FROM t1) INTERSECT (SELECT * FROM t2)) WHERE a > 1"
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
-    val ds2 = CollectionDataSets.get3TupleDataSet(env)
-
-    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
-    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'c)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hello\n" + "Hello world\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsTest.scala
deleted file mode 100644
index d0c0400..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SetOperatorsTest.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.api.scala.batch.sql
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.utils.TableTestBase
-import org.apache.flink.api.table.utils.TableTestUtil._
-import org.junit.Test
-
-class SetOperatorsTest extends TableTestBase {
-
-  @Test
-  def testExists(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Long, Int, String)]("A", 'a_long, 'a_int, 'a_string)
-    util.addTable[(Long, Int, String)]("B", 'b_long, 'b_int, 'b_string)
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      binaryNode(
-        "DataSetJoin",
-        batchTableNode(0),
-        unaryNode(
-          "DataSetAggregate",
-          unaryNode(
-            "DataSetCalc",
-            binaryNode(
-              "DataSetJoin",
-              unaryNode(
-                "DataSetCalc",
-                batchTableNode(1),
-                term("select", "b_long")
-              ),
-              unaryNode(
-                "DataSetAggregate",
-                unaryNode(
-                  "DataSetCalc",
-                  batchTableNode(0),
-                  term("select", "a_long")
-                ),
-                term("groupBy", "a_long"),
-                term("select", "a_long")
-              ),
-              term("where", "=(a_long, b_long)"),
-              term("join", "b_long", "a_long"),
-              term("joinType", "InnerJoin")
-            ),
-            term("select", "true AS $f0", "a_long")
-          ),
-          term("groupBy", "a_long"),
-          term("select", "a_long", "MIN($f0) AS $f1")
-        ),
-        term("where", "=(a_long, a_long0)"),
-        term("join", "a_long", "a_int", "a_string", "a_long0", "$f1"),
-        term("joinType", "InnerJoin")
-      ),
-      term("select", "a_int", "a_string")
-    )
-
-    util.verifySql(
-      "SELECT a_int, a_string FROM A WHERE EXISTS(SELECT * FROM B WHERE a_long = b_long)",
-      expected
-    )
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SingleRowJoinTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SingleRowJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SingleRowJoinTest.scala
deleted file mode 100644
index 49f61af..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SingleRowJoinTest.scala
+++ /dev/null
@@ -1,152 +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.api.scala.batch.sql
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.utils.TableTestBase
-import org.apache.flink.api.table.utils.TableTestUtil._
-import org.junit.Test
-
-class SingleRowJoinTest extends TableTestBase {
-
-  @Test
-  def testSingleRowEquiJoin(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, String)]("A", 'a1, 'a2)
-
-    val query =
-      "SELECT a1, a2 " +
-      "FROM A, (SELECT count(a1) AS cnt FROM A) " +
-      "WHERE a1 = cnt"
-
-    val expected =
-      unaryNode(
-        "DataSetCalc",
-        binaryNode(
-          "DataSetSingleRowJoin",
-          batchTableNode(0),
-          unaryNode(
-            "DataSetAggregate",
-            unaryNode(
-              "DataSetUnion",
-              unaryNode(
-                "DataSetValues",
-                unaryNode(
-                  "DataSetCalc",
-                  batchTableNode(0),
-                  term("select", "a1")
-                ),
-                tuples(List(null)),
-                term("values", "a1")
-              ),
-              term("union","a1")
-            ),
-            term("select", "COUNT(a1) AS cnt")
-          ),
-          term("where", "=(CAST(a1), cnt)"),
-          term("join", "a1", "a2", "cnt"),
-          term("joinType", "NestedLoopJoin")
-        ),
-        term("select", "a1", "a2")
-      )
-
-    util.verifySql(query, expected)
-  }
-
-  @Test
-  def testSingleRowNotEquiJoin(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, String)]("A", 'a1, 'a2)
-
-    val query =
-      "SELECT a1, a2 " +
-      "FROM A, (SELECT count(a1) AS cnt FROM A) " +
-      "WHERE a1 < cnt"
-
-    val expected =
-      unaryNode(
-        "DataSetCalc",
-        binaryNode(
-          "DataSetSingleRowJoin",
-          batchTableNode(0),
-          unaryNode(
-            "DataSetAggregate",
-            unaryNode(
-              "DataSetUnion",
-              unaryNode(
-                "DataSetValues",
-                unaryNode(
-                  "DataSetCalc",
-                  batchTableNode(0),
-                  term("select", "a1")
-                ),
-                tuples(List(null)),
-                term("values", "a1")
-              ),
-              term("union", "a1")
-            ),
-            term("select", "COUNT(a1) AS cnt")
-          ),
-          term("where", "<(a1, cnt)"),
-          term("join", "a1", "a2", "cnt"),
-          term("joinType", "NestedLoopJoin")
-        ),
-        term("select", "a1", "a2")
-      )
-
-    util.verifySql(query, expected)
-  }
-
-  @Test
-  def testSingleRowJoinWithComplexPredicate(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, Long)]("A", 'a1, 'a2)
-    util.addTable[(Int, Long)]("B", 'b1, 'b2)
-
-    val query =
-      "SELECT a1, a2, b1, b2 " +
-        "FROM A, (SELECT min(b1) AS b1, max(b2) AS b2 FROM B) " +
-        "WHERE a1 < b1 AND a2 = b2"
-
-    val expected = binaryNode(
-      "DataSetSingleRowJoin",
-      batchTableNode(0),
-      unaryNode(
-        "DataSetAggregate",
-        unaryNode(
-          "DataSetUnion",
-          unaryNode(
-            "DataSetValues",
-            batchTableNode(1),
-            tuples(List(null, null)),
-            term("values", "b1", "b2")
-          ),
-          term("union","b1","b2")
-        ),
-        term("select", "MIN(b1) AS b1", "MAX(b2) AS b2")
-      ),
-      term("where", "AND(<(a1, b1)", "=(a2, b2))"),
-      term("join", "a1", "a2", "b1", "b2"),
-      term("joinType", "NestedLoopJoin")
-    )
-
-    util.verifySql(query, expected)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala
deleted file mode 100644
index b94cd00..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/SortITCase.scala
+++ /dev/null
@@ -1,153 +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.api.scala.batch.sql
-
-import org.apache.flink.api.scala.ExecutionEnvironment
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.batch.utils.SortTestUtils._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala._
-import org.apache.flink.api.table.{TableEnvironment, TableException}
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class SortITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testOrderByMultipleFieldsWithSql(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable ORDER BY _1 DESC, _2 DESC"
-
-    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
-      (- x.productElement(0).asInstanceOf[Int], - x.productElement(1).asInstanceOf[Long]))
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds)
-
-    val expected = sortExpectedly(tupleDataSetStrings)
-    val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
-
-    val result = results
-      .filterNot(_.isEmpty)
-      .sortBy(_.head)(Ordering.by(f=> f.toString))
-      .reduceLeft(_ ++ _)
-
-    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
-  }
-
-  @Test
-  def testOrderByWithOffset(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable ORDER BY _1 DESC OFFSET 2 ROWS"
-
-    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
-      - x.productElement(0).asInstanceOf[Int] )
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds)
-
-    val expected = sortExpectedly(tupleDataSetStrings, 2, 21)
-    val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
-
-    val result = results.
-      filterNot(_.isEmpty)
-      .sortBy(_.head)(Ordering.by(f=> f.toString))
-      .reduceLeft(_ ++ _)
-
-    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
-  }
-
-  @Test
-  def testOrderByWithOffsetAndFetch(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable ORDER BY _1 OFFSET 2 ROWS FETCH NEXT 5 ROWS ONLY"
-
-    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
-      x.productElement(0).asInstanceOf[Int] )
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds)
-
-    val expected = sortExpectedly(tupleDataSetStrings, 2, 7)
-    val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
-
-    val result = results
-      .filterNot(_.isEmpty)
-      .sortBy(_.head)(Ordering.by(f=> f.toString))
-      .reduceLeft(_ ++ _)
-
-    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
-  }
-
-  @Test
-  def testOrderByLimit(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable ORDER BY _2, _1 LIMIT 5"
-
-    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
-      (x.productElement(1).asInstanceOf[Long], x.productElement(0).asInstanceOf[Int]) )
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds)
-
-    val expected = sortExpectedly(tupleDataSetStrings, 0, 5)
-    val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
-
-    val result = results
-      .filterNot(_.isEmpty)
-      .sortBy(_.head)(Ordering.by(f=> f.toString))
-      .reduceLeft(_ ++ _)
-
-    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testLimitWithoutOrder(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable LIMIT 5"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds)
-
-    tEnv.sql(sqlQuery).toDataSet[Row].collect()
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/TableWithSQLITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/TableWithSQLITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/TableWithSQLITCase.scala
deleted file mode 100644
index d41f3e0..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/TableWithSQLITCase.scala
+++ /dev/null
@@ -1,118 +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.api.scala.batch.sql
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class TableWithSQLITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testSQLTable(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT * FROM MyTable WHERE a > 9"
-
-    val result = tEnv.sql(sqlQuery).select('a.avg, 'b.sum, 'c.count)
-
-    val expected = "15,65,12"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testTableSQLTable(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val t1 = ds.filter('a > 9)
-
-    tEnv.registerTable("MyTable", t1)
-
-    val sqlQuery = "SELECT avg(a) as a1, sum(b) as b1, count(c) as c1 FROM MyTable"
-
-    val result = tEnv.sql(sqlQuery).select('a1 + 1, 'b1 - 5, 'c1)
-
-    val expected = "16,60,12"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testMultipleSQLQueries(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    tEnv.registerTable("MyTable", t)
-
-    val sqlQuery = "SELECT a as aa FROM MyTable WHERE b = 6"
-    val result1 = tEnv.sql(sqlQuery)
-    tEnv.registerTable("ResTable", result1)
-
-    val sqlQuery2 = "SELECT count(aa) FROM ResTable"
-    val result2 = tEnv.sql(sqlQuery2)
-
-    val expected = "6"
-    val results = result2.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testSelectWithCompositeType(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT MyTable.a2, MyTable.a1._2 FROM MyTable"
-
-    val ds = env.fromElements(((12, true), "Hello")).toTable(tEnv).as('a1, 'a2)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "Hello,true\n"
-
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/UserDefinedTableFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/UserDefinedTableFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/UserDefinedTableFunctionTest.scala
deleted file mode 100644
index 245f117..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/UserDefinedTableFunctionTest.scala
+++ /dev/null
@@ -1,238 +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.api.scala.batch.sql
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.utils.{HierarchyTableFunction, PojoTableFunc, TableFunc2}
-import org.apache.flink.api.table.utils._
-import org.apache.flink.api.table.utils.TableTestUtil._
-import org.junit.Test
-
-class UserDefinedTableFunctionTest extends TableTestBase {
-
-  @Test
-  def testCrossJoin(): Unit = {
-    val util = batchTestUtil()
-    val func1 = new TableFunc1
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    util.addFunction("func1", func1)
-
-    val sqlQuery = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(c)) AS T(s)"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", "func1($cor0.c)"),
-        term("function", func1.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "f0 AS s")
-    )
-
-    util.verifySql(sqlQuery, expected)
-
-    // test overloading
-
-    val sqlQuery2 = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(c, '$')) AS T(s)"
-
-    val expected2 = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", "func1($cor0.c, '$')"),
-        term("function", func1.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "f0 AS s")
-    )
-
-    util.verifySql(sqlQuery2, expected2)
-  }
-
-  @Test
-  def testLeftOuterJoin(): Unit = {
-    val util = batchTestUtil()
-    val func1 = new TableFunc1
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    util.addFunction("func1", func1)
-
-    val sqlQuery = "SELECT c, s FROM MyTable LEFT JOIN LATERAL TABLE(func1(c)) AS T(s) ON TRUE"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", "func1($cor0.c)"),
-        term("function", func1.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
-        term("joinType", "LEFT")
-      ),
-      term("select", "c", "f0 AS s")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testCustomType(): Unit = {
-    val util = batchTestUtil()
-    val func2 = new TableFunc2
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    util.addFunction("func2", func2)
-
-    val sqlQuery = "SELECT c, name, len FROM MyTable, LATERAL TABLE(func2(c)) AS T(name, len)"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", "func2($cor0.c)"),
-        term("function", func2.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
-               "VARCHAR(2147483647) f0, INTEGER f1)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "f0 AS name", "f1 AS len")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testHierarchyType(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = new HierarchyTableFunction
-    util.addFunction("hierarchy", function)
-
-    val sqlQuery = "SELECT c, T.* FROM MyTable, LATERAL TABLE(hierarchy(c)) AS T(name, adult, len)"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", "hierarchy($cor0.c)"),
-        term("function", function.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," +
-               " VARCHAR(2147483647) f0, BOOLEAN f1, INTEGER f2)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "f0 AS name", "f1 AS adult", "f2 AS len")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testPojoType(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = new PojoTableFunc
-    util.addFunction("pojo", function)
-
-    val sqlQuery = "SELECT c, name, age FROM MyTable, LATERAL TABLE(pojo(c))"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", "pojo($cor0.c)"),
-        term("function", function.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," +
-               " INTEGER age, VARCHAR(2147483647) name)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "name", "age")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testFilter(): Unit = {
-    val util = batchTestUtil()
-    val func2 = new TableFunc2
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    util.addFunction("func2", func2)
-
-    val sqlQuery = "SELECT c, name, len FROM MyTable, LATERAL TABLE(func2(c)) AS T(name, len) " +
-      "WHERE len > 2"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", "func2($cor0.c)"),
-        term("function", func2.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
-               "VARCHAR(2147483647) f0, INTEGER f1)"),
-        term("joinType", "INNER"),
-        term("condition", ">($1, 2)")
-      ),
-      term("select", "c", "f0 AS name", "f1 AS len")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-
-  @Test
-  def testScalarFunction(): Unit = {
-    val util = batchTestUtil()
-    val func1 = new TableFunc1
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    util.addFunction("func1", func1)
-
-    val sqlQuery = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(SUBSTRING(c, 2))) AS T(s)"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", "func1(SUBSTRING($cor0.c, 2))"),
-        term("function", func1.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "f0 AS s")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala
deleted file mode 100644
index 3f4e1e5..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/AggregationsITCase.scala
+++ /dev/null
@@ -1,405 +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.api.scala.batch.table
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.{TableEnvironment, ValidationException}
-import org.apache.flink.types.Row
-import org.apache.flink.examples.scala.WordCountTable.{WC => MyWC}
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class AggregationsITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testAggregationTypes(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-      .select('_1.sum, '_1.min, '_1.max, '_1.count, '_1.avg)
-
-    val results = t.toDataSet[Row].collect()
-    val expected = "231,1,21,21,11"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testAggregationOnNonExistingField(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-      // Must fail. Field 'foo does not exist.
-      .select('foo.avg)
-  }
-
-  @Test
-  def testWorkingAggregationDataTypes(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = env.fromElements(
-      (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"),
-      (2: Byte, 2: Short, 2, 2L, 2.0f, 2.0d, "Ciao")).toTable(tEnv)
-      .select('_1.avg, '_2.avg, '_3.avg, '_4.avg, '_5.avg, '_6.avg, '_7.count)
-
-    val expected = "1,1,1,1,1.5,1.5,2"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testProjection(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = env.fromElements(
-      (1: Byte, 1: Short),
-      (2: Byte, 2: Short)).toTable(tEnv)
-      .select('_1.avg, '_1.sum, '_1.count, '_2.avg, '_2.sum)
-
-    val expected = "1,3,2,1,3"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testAggregationWithArithmetic(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = env.fromElements((1f, "Hello"), (2f, "Ciao")).toTable(tEnv)
-      .select(('_1 + 2).avg + 2, '_2.count + 5)
-
-    val expected = "5.5,7"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testAggregationWithTwoCount(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = env.fromElements((1f, "Hello"), (2f, "Ciao")).toTable(tEnv)
-      .select('_1.count, '_2.count)
-
-    val expected = "2,2"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testAggregationAfterProjection(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = env.fromElements(
-      (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"),
-      (2: Byte, 2: Short, 2, 2L, 2.0f, 2.0d, "Ciao")).toTable(tEnv)
-      .select('_1, '_2, '_3)
-      .select('_1.avg, '_2.sum, '_3.count)
-
-    val expected = "1,3,2"
-    val result = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(result.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testNonWorkingAggregationDataTypes(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = env.fromElements(("Hello", 1)).toTable(tEnv)
-      // Must fail. Field '_1 is not a numeric type.
-      .select('_1.sum)
-
-    t.collect()
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testNoNestedAggregations(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = env.fromElements(("Hello", 1)).toTable(tEnv)
-      // Must fail. Sum aggregation can not be chained.
-      .select('_2.sum.sum)
-  }
-
-  @Test
-  def testSQLStyleAggregations(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      .select(
-        """Sum( a) as a1, a.sum as a2,
-          |Min (a) as b1, a.min as b2,
-          |Max (a ) as c1, a.max as c2,
-          |Avg ( a ) as d1, a.avg as d2,
-          |Count(a) as e1, a.count as e2
-        """.stripMargin)
-
-    val expected = "231,231,1,1,21,21,11,11,21,21"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testPojoAggregation(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val input = env.fromElements(
-      MyWC("hello", 1),
-      MyWC("hello", 1),
-      MyWC("ciao", 1),
-      MyWC("hola", 1),
-      MyWC("hola", 1))
-    val expr = input.toTable(tEnv)
-    val result = expr
-      .groupBy('word)
-      .select('word, 'frequency.sum as 'frequency)
-      .filter('frequency === 2)
-      .toDataSet[MyWC]
-
-    val mappedResult = result.map(w => (w.word, w.frequency * 10)).collect()
-    val expected = "(hello,20)\n" + "(hola,20)"
-    TestBaseUtils.compareResultAsText(mappedResult.asJava, expected)
-  }
-
-  @Test
-  def testDistinct(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val distinct = ds.select('b).distinct()
-
-    val expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"
-    val results = distinct.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testDistinctAfterAggregate(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
-    val distinct = ds.groupBy('a, 'e).select('e).distinct()
-
-    val expected = "1\n" + "2\n" + "3\n"
-    val results = distinct.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testGroupingOnNonExistentField(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      // must fail. '_foo not a valid field
-      .groupBy('_foo)
-      .select('a.avg)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testGroupingInvalidSelection(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      .groupBy('a, 'b)
-      // must fail. 'c is not a grouping key or aggregation
-      .select('c)
-  }
-
-  @Test
-  def testGroupedAggregate(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      .groupBy('b)
-      .select('b, 'a.sum)
-
-    val expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testGroupingKeyForwardIfNotUsed(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      .groupBy('b)
-      .select('a.sum)
-
-    val expected = "1\n" + "5\n" + "15\n" + "34\n" + "65\n" + "111\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testGroupNoAggregation(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env)
-      .toTable(tEnv, 'a, 'b, 'c)
-      .groupBy('b)
-      .select('a.sum as 'd, 'b)
-      .groupBy('b, 'd)
-      .select('b)
-
-    val expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testGroupedAggregateWithLongKeys(): Unit = {
-    // This uses very long keys to force serialized comparison.
-    // With short keys, the normalized key is sufficient.
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = env.fromElements(
-      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2),
-      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2),
-      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2),
-      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2),
-      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2),
-      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2),
-      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2),
-      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2),
-      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2))
-      .rebalance().setParallelism(2).toTable(tEnv, 'a, 'b, 'c)
-      .groupBy('a, 'b)
-      .select('c.sum)
-
-    val expected = "10\n" + "8\n"
-    val results = ds.collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testGroupedAggregateWithConstant1(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      .select('a, 4 as 'four, 'b)
-      .groupBy('four, 'a)
-      .select('four, 'b.sum)
-
-    val expected = "4,2\n" + "4,3\n" + "4,5\n" + "4,5\n" + "4,5\n" + "4,6\n" +
-      "4,6\n" + "4,6\n" + "4,3\n" + "4,4\n" + "4,6\n" + "4,1\n" + "4,4\n" +
-      "4,4\n" + "4,5\n" + "4,6\n" + "4,2\n" + "4,3\n" + "4,4\n" + "4,5\n" + "4,6\n"
-    val results = t.toDataSet[Row].collect()
-
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testGroupedAggregateWithConstant2(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-        .select('b, 4 as 'four, 'a)
-        .groupBy('b, 'four)
-        .select('four, 'a.sum)
-
-    val expected = "4,1\n" + "4,5\n" + "4,15\n" + "4,34\n" + "4,65\n" + "4,111\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testGroupedAggregateWithExpression(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
-        .groupBy('e, 'b % 3)
-        .select('c.min, 'e, 'a.avg, 'd.count)
-
-    val expected = "0,1,1,1\n" + "3,2,3,3\n" + "7,1,4,2\n" + "14,2,5,1\n" +
-        "5,3,4,2\n" + "2,1,3,2\n" + "1,2,3,3\n" + "12,3,5,1"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testGroupedAggregateWithFilter(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      .groupBy('b)
-      .select('b, 'a.sum)
-      .where('b === 2)
-
-    val expected = "2,5\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala
deleted file mode 100644
index b011462..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/CalcITCase.scala
+++ /dev/null
@@ -1,438 +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.api.scala.batch.table
-
-import java.sql.{Date, Time, Timestamp}
-import java.util
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.expressions.Literal
-import org.apache.flink.api.table.{TableEnvironment, TableException, ValidationException}
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit.Assert._
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class CalcITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testSimpleSelectAll(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).select('_1, '_2, '_3)
-
-    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
-      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
-      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testSimpleSelectAllWithAs(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('a, 'b, 'c)
-
-    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
-      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
-      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testSimpleSelectWithNaming(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-      .select('_1 as 'a, '_2 as 'b, '_1 as 'c)
-      .select('a, 'b)
-
-    val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
-      "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
-      "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testSimpleSelectRenameAll(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-      .select('_1 as 'a, '_2 as 'b, '_3 as 'c)
-      .select('a, 'b)
-
-    val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
-      "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
-      "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testSelectInvalidFieldFields(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      // must fail. Field 'foo does not exist
-      .select('a, 'foo)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testSelectAmbiguousRenaming(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      // must fail. 'a and 'b are both renamed to 'foo
-      .select('a + 1 as 'foo, 'b + 2 as 'foo).toDataSet[Row].print()
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testSelectAmbiguousRenaming2(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      // must fail. 'a and 'b are both renamed to 'a
-      .select('a, 'b as 'a).toDataSet[Row].print()
-  }
-
-  @Test
-  def testSelectStar(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('*)
-
-    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
-      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
-      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testAliasStarException(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    try {
-      CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, '*, 'b, 'c)
-      fail("TableException expected")
-    } catch {
-      case _: TableException => //ignore
-    }
-
-    try {
-      CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-        .select('_1 as '*, '_2 as 'b, '_1 as 'c)
-      fail("ValidationException expected")
-    } catch {
-      case _: ValidationException => //ignore
-    }
-
-    try {
-      CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('*, 'b, 'c)
-      fail("ValidationException expected")
-    } catch {
-      case _: ValidationException => //ignore
-    }
-
-    try {
-      CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('*, 'b)
-      fail("ValidationException expected")
-    } catch {
-      case _: ValidationException => //ignore
-    }
-  }
-
-  @Test
-  def testAllRejectingFilter(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter( Literal(false) )
-
-    val expected = "\n"
-    val results = filterDs.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testAllPassingFilter(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter( Literal(true) )
-    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, " +
-      "how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + "7,4," +
-      "Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + "11,5," +
-      "Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5," +
-      "Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19," +
-      "6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
-    val results = filterDs.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testFilterOnStringTupleField(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val filterDs = ds.filter( 'c.like("%world%") )
-
-    val expected = "3,2,Hello world\n" + "4,3,Hello world, how are you?\n"
-    val results = filterDs.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testFilterOnIntegerTupleField(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter( 'a % 2 === 0 )
-
-    val expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" +
-      "6,3,Luke Skywalker\n" + "8,4," + "Comment#2\n" + "10,4,Comment#4\n" +
-      "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," +
-      "Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n"
-    val results = filterDs.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testNotEquals(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter( 'a % 2 !== 0)
-    val expected = "1,1,Hi\n" + "3,2,Hello world\n" +
-      "5,3,I am fine.\n" + "7,4,Comment#1\n" + "9,4,Comment#3\n" +
-      "11,5,Comment#5\n" + "13,5,Comment#7\n" + "15,5,Comment#9\n" +
-      "17,6,Comment#11\n" + "19,6,Comment#13\n" + "21,6,Comment#15\n"
-    val results = filterDs.collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testDisjunctivePredicate(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter( 'a < 2 || 'a > 20)
-    val expected = "1,1,Hi\n" + "21,6,Comment#15\n"
-    val results = filterDs.collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testConsecutiveFilters(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter('a % 2 !== 0).filter('b % 2 === 0)
-    val expected = "3,2,Hello world\n" + "7,4,Comment#1\n" +
-      "9,4,Comment#3\n" + "17,6,Comment#11\n" +
-      "19,6,Comment#13\n" + "21,6,Comment#15\n"
-    val results = filterDs.collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testFilterBasicType(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.getStringDataSet(env)
-
-    val filterDs = ds.toTable(tEnv, 'a).filter( 'a.like("H%") )
-
-    val expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n"
-    val results = filterDs.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testFilterOnCustomType(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.getCustomTypeDataSet(env)
-    val filterDs = ds.toTable(tEnv, 'myInt as 'i, 'myLong as 'l, 'myString as 's)
-      .filter( 's.like("%a%") )
-
-    val expected = "3,3,Hello world, how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n"
-    val results = filterDs.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testFilterInvalidFieldName(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-
-    // must fail. Field 'foo does not exist
-    ds.filter( 'foo === 2 )
-  }
-
-  @Test
-  def testSimpleCalc(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-        .select('_1, '_2, '_3)
-        .where('_1 < 7)
-        .select('_1, '_3)
-
-    val expected = "1,Hi\n" + "2,Hello\n" + "3,Hello world\n" +
-      "4,Hello world, how are you?\n" + "5,I am fine.\n" + "6,Luke Skywalker\n"
-      val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testCalcWithTwoFilters(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-        .select('_1, '_2, '_3)
-        .where('_1 < 7 && '_2 === 3)
-        .select('_1, '_3)
-        .where('_1 === 4)
-        .select('_1)
-
-    val expected = "4\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testCalcWithAggregation(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-        .select('_1, '_2, '_3)
-        .where('_1 < 15)
-        .groupBy('_2)
-        .select('_1.min, '_2.count as 'cnt)
-        .where('cnt > 3)
-
-    val expected = "7,4\n" + "11,4\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testCalcJoin(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.select('a, 'b).join(ds2).where('b === 'e).select('a, 'b, 'd, 'e, 'f)
-      .where('b > 1).select('a, 'd).where('d === 2)
-
-    val expected = "2,2\n" + "3,2\n"
-    val results = joinT.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testAdvancedDataTypes(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = env
-      .fromElements((
-        BigDecimal("78.454654654654654").bigDecimal,
-        BigDecimal("4E+9999").bigDecimal,
-        Date.valueOf("1984-07-12"),
-        Time.valueOf("14:34:24"),
-        Timestamp.valueOf("1984-07-12 14:34:24")))
-      .toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
-      .select('a, 'b, 'c, 'd, 'e, BigDecimal("11.2"), BigDecimal("11.2").bigDecimal,
-        Date.valueOf("1984-07-12"), Time.valueOf("14:34:24"),
-        Timestamp.valueOf("1984-07-12 14:34:24"))
-
-    val expected = "78.454654654654654,4E+9999,1984-07-12,14:34:24,1984-07-12 14:34:24.0," +
-      "11.2,11.2,1984-07-12,14:34:24,1984-07-12 14:34:24.0"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-}
-
-object CalcITCase {
-
-  @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
-  def parameters(): util.Collection[Array[java.lang.Object]] = {
-    Seq[Array[AnyRef]](
-      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT),
-      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.NO_NULL)).asJava
-  }
-}


[13/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/JoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/JoinITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/JoinITCase.java
new file mode 100644
index 0000000..a916998
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/JoinITCase.java
@@ -0,0 +1,207 @@
+/*
+ * 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.java.batch.table;
+
+import java.util.List;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.types.Row;
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+
+@RunWith(Parameterized.class)
+public class JoinITCase extends TableProgramsTestBase {
+
+	public JoinITCase(TestExecutionMode mode, TableConfigMode configMode){
+		super(mode, configMode);
+	}
+
+	@Test
+	public void testJoin() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
+		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
+
+		Table result = in1.join(in2).where("b === e").select("c, g");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testJoinWithFilter() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
+		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
+
+		Table result = in1.join(in2).where("b === e && b < 2").select("c, g");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "Hi,Hallo\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testJoinWithJoinFilter() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
+		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
+
+		Table result = in1.join(in2).where("b === e && a < 6 && h < b").select("c, g");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "Hello world, how are you?,Hallo Welt wie\n" +
+				"I am fine.,Hallo Welt wie\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testJoinWithMultipleKeys() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
+		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
+
+		Table result = in1.join(in2).where("a === d && b === h").select("c, g");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" +
+				"Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testJoinNonExistingKey() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
+		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
+
+		// Must fail. Field foo does not exist.
+		in1.join(in2).where("foo === e").select("c, g");
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testJoinWithNonMatchingKeyTypes() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
+		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
+
+		Table result = in1.join(in2)
+			// Must fail. Types of join fields are not compatible (Integer and String)
+			.where("a === g").select("c, g");
+
+		tableEnv.toDataSet(result, Row.class).collect();
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testJoinWithAmbiguousFields() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
+		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, c");
+
+		// Must fail. Join input have overlapping field names.
+		in1.join(in2).where("a === d").select("c, g");
+	}
+
+	@Test
+	public void testJoinWithAggregation() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
+		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
+
+		Table result = in1
+				.join(in2).where("a === d").select("g.count");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "6";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testJoinTablesFromDifferentEnvs() {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tEnv1 = TableEnvironment.getTableEnvironment(env);
+		BatchTableEnvironment tEnv2 = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		Table in1 = tEnv1.fromDataSet(ds1, "a, b, c");
+		Table in2 = tEnv2.fromDataSet(ds2, "d, e, f, g, h");
+
+		// Must fail. Tables are bound to different TableEnvironments.
+		in1.join(in2).where("a === d").select("g.count");
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/SqlITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/SqlITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/SqlITCase.java
new file mode 100644
index 0000000..7c01d2b
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/sql/SqlITCase.java
@@ -0,0 +1,121 @@
+/*
+ * 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.java.stream.sql;
+
+import org.apache.flink.table.api.java.StreamTableEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.types.Row;
+import org.apache.flink.table.api.scala.stream.utils.StreamITCase;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
+import org.apache.flink.table.api.java.stream.utils.StreamTestData;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class SqlITCase extends StreamingMultipleProgramsTestBase {
+
+	@Test
+	public void testSelect() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+		StreamITCase.clear();
+
+		DataStream<Tuple3<Integer, Long, String>> ds = StreamTestData.getSmall3TupleDataSet(env);
+		Table in = tableEnv.fromDataStream(ds, "a,b,c");
+		tableEnv.registerTable("MyTable", in);
+
+		String sqlQuery = "SELECT * FROM MyTable";
+		Table result = tableEnv.sql(sqlQuery);
+
+		DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
+		resultSet.addSink(new StreamITCase.StringSink());
+		env.execute();
+
+		List<String> expected = new ArrayList<>();
+		expected.add("1,1,Hi");
+		expected.add("2,2,Hello");
+		expected.add("3,2,Hello world");
+
+		StreamITCase.compareWithList(expected);
+	}
+
+	@Test
+	public void testFilter() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+		StreamITCase.clear();
+
+		DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env);
+		tableEnv.registerDataStream("MyTable", ds, "a, b, c, d, e");
+
+		String sqlQuery = "SELECT a, b, e FROM MyTable WHERE c < 4";
+		Table result = tableEnv.sql(sqlQuery);
+
+		DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
+		resultSet.addSink(new StreamITCase.StringSink());
+		env.execute();
+
+		List<String> expected = new ArrayList<>();
+		expected.add("1,1,1");
+		expected.add("2,2,2");
+		expected.add("2,3,1");
+		expected.add("3,4,2");
+
+		StreamITCase.compareWithList(expected);
+	}
+
+	@Test
+	public void testUnion() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+		StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+		StreamITCase.clear();
+
+		DataStream<Tuple3<Integer, Long, String>> ds1 = StreamTestData.getSmall3TupleDataSet(env);
+		Table t1 = tableEnv.fromDataStream(ds1, "a,b,c");
+		tableEnv.registerTable("T1", t1);
+
+		DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds2 = StreamTestData.get5TupleDataStream(env);
+		tableEnv.registerDataStream("T2", ds2, "a, b, d, c, e");
+
+		String sqlQuery = "SELECT * FROM T1 " +
+							"UNION ALL " +
+							"(SELECT a, b, c FROM T2 WHERE a	< 3)";
+		Table result = tableEnv.sql(sqlQuery);
+
+		DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
+		resultSet.addSink(new StreamITCase.StringSink());
+		env.execute();
+
+		List<String> expected = new ArrayList<>();
+		expected.add("1,1,Hi");
+		expected.add("2,2,Hello");
+		expected.add("3,2,Hello world");
+		expected.add("1,1,Hallo");
+		expected.add("2,2,Hallo Welt");
+		expected.add("2,3,Hallo Welt wie");
+
+		StreamITCase.compareWithList(expected);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/utils/StreamTestData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/utils/StreamTestData.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/utils/StreamTestData.java
new file mode 100644
index 0000000..139801f
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/stream/utils/StreamTestData.java
@@ -0,0 +1,64 @@
+/*
+ * 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.java.stream.utils;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class StreamTestData {
+
+	public static DataStream<Tuple3<Integer, Long, String>> getSmall3TupleDataSet(StreamExecutionEnvironment env) {
+
+		List<Tuple3<Integer, Long, String>> data = new ArrayList<>();
+		data.add(new Tuple3<>(1, 1L, "Hi"));
+		data.add(new Tuple3<>(2, 2L, "Hello"));
+		data.add(new Tuple3<>(3, 2L, "Hello world"));
+
+		Collections.shuffle(data);
+
+		return env.fromCollection(data);
+	}
+
+	public static DataStream<Tuple5<Integer, Long, Integer, String, Long>> get5TupleDataStream(StreamExecutionEnvironment env) {
+
+		List<Tuple5<Integer, Long, Integer, String, Long>> data = new ArrayList<>();
+		data.add(new Tuple5<>(1, 1L, 0, "Hallo", 1L));
+		data.add(new Tuple5<>(2, 2L, 1, "Hallo Welt", 2L));
+		data.add(new Tuple5<>(2, 3L, 2, "Hallo Welt wie", 1L));
+		data.add(new Tuple5<>(3, 4L, 3, "Hallo Welt wie gehts?", 2L));
+		data.add(new Tuple5<>(3, 5L, 4, "ABC", 2L));
+		data.add(new Tuple5<>(3, 6L, 5, "BCD", 3L));
+		data.add(new Tuple5<>(4, 7L, 6, "CDE", 2L));
+		data.add(new Tuple5<>(4, 8L, 7, "DEF", 1L));
+		data.add(new Tuple5<>(4, 9L, 8, "EFG", 1L));
+		data.add(new Tuple5<>(4, 10L, 9, "FGH", 2L));
+		data.add(new Tuple5<>(5, 11L, 10, "GHI", 1L));
+		data.add(new Tuple5<>(5, 12L, 11, "HIJ", 3L));
+		data.add(new Tuple5<>(5, 13L, 12, "IJK", 3L));
+		data.add(new Tuple5<>(5, 15L, 14, "KLM", 2L));
+		data.add(new Tuple5<>(5, 14L, 13, "JKL", 2L));
+		return env.fromCollection(data);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ExplainTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ExplainTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ExplainTest.scala
deleted file mode 100644
index 9d00dda..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ExplainTest.scala
+++ /dev/null
@@ -1,122 +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.api.scala.batch
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.junit.Assert.assertEquals
-import org.junit._
-
-class ExplainTest
-  extends MultipleProgramsTestBase(MultipleProgramsTestBase.TestExecutionMode.CLUSTER) {
-
-  val testFilePath = ExplainTest.this.getClass.getResource("/").getFile
-
-  @Test
-  def testFilterWithoutExtended() : Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val table = env.fromElements((1, "hello"))
-      .toTable(tEnv, 'a, 'b)
-      .filter("a % 2 = 0")
-
-    val result = tEnv.explain(table).replaceAll("\\r\\n", "\n")
-    val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testFilter0.out").mkString.replaceAll("\\r\\n", "\n")
-    assertEquals(result, source)
-  }
-
-  @Test
-  def testFilterWithExtended() : Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val table = env.fromElements((1, "hello"))
-      .toTable(tEnv, 'a, 'b)
-      .filter("a % 2 = 0")
-
-    val result = tEnv.explain(table, true).replaceAll("\\r\\n", "\n")
-    val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testFilter1.out").mkString.replaceAll("\\r\\n", "\n")
-    assertEquals(result, source)
-  }
-
-  @Test
-  def testJoinWithoutExtended() : Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val table1 = env.fromElements((1, "hello")).toTable(tEnv, 'a, 'b)
-    val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'c, 'd)
-    val table = table1.join(table2).where("b = d").select("a, c")
-
-    val result = tEnv.explain(table).replaceAll("\\r\\n", "\n")
-    val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testJoin0.out").mkString.replaceAll("\\r\\n", "\n")
-    assertEquals(source, result)
-  }
-
-  @Test
-  def testJoinWithExtended() : Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val table1 = env.fromElements((1, "hello")).toTable(tEnv, 'a, 'b)
-    val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'c, 'd)
-    val table = table1.join(table2).where("b = d").select("a, c")
-
-    val result = tEnv.explain(table, true).replaceAll("\\r\\n", "\n")
-    val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testJoin1.out").mkString.replaceAll("\\r\\n", "\n")
-    assertEquals(source, result)
-  }
-
-  @Test
-  def testUnionWithoutExtended() : Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val table1 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
-    val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
-    val table = table1.unionAll(table2)
-
-    val result = tEnv.explain(table).replaceAll("\\r\\n", "\n")
-    val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testUnion0.out").mkString.replaceAll("\\r\\n", "\n")
-    assertEquals(result, source)
-  }
-
-  @Test
-  def testUnionWithExtended() : Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val table1 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
-    val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
-    val table = table1.unionAll(table2)
-
-    val result = tEnv.explain(table, true).replaceAll("\\r\\n", "\n")
-    val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testUnion1.out").mkString.replaceAll("\\r\\n", "\n")
-    assertEquals(result, source)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ProjectableTableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ProjectableTableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ProjectableTableSourceITCase.scala
deleted file mode 100644
index ddea3ba..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/ProjectableTableSourceITCase.scala
+++ /dev/null
@@ -1,146 +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.api.scala.batch
-
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.java.{DataSet => JavaSet, ExecutionEnvironment => JavaExecEnv}
-import org.apache.flink.api.scala.ExecutionEnvironment
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.sources.{BatchTableSource, ProjectableTableSource}
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.apache.flink.types.Row
-import org.junit.{Before, Test}
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class ProjectableTableSourceITCase(mode: TestExecutionMode,
-  configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  private val tableName = "MyTable"
-  private var tableEnv: BatchTableEnvironment = null
-
-  @Before
-  def initTableEnv(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    tableEnv = TableEnvironment.getTableEnvironment(env, config)
-    tableEnv.registerTableSource(tableName, new TestProjectableTableSource)
-  }
-
-  @Test
-  def testTableAPI(): Unit = {
-    val results = tableEnv
-                  .scan(tableName)
-                  .where("amount < 4")
-                  .select("id, name")
-                  .collect()
-
-    val expected = Seq(
-      "0,Record_0", "1,Record_1", "2,Record_2", "3,Record_3", "16,Record_16",
-      "17,Record_17", "18,Record_18", "19,Record_19", "32,Record_32").mkString("\n")
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-
-  @Test
-  def testSQL(): Unit = {
-    val results = tableEnv
-                  .sql(s"select id, name from $tableName where amount < 4 ")
-                  .collect()
-
-    val expected = Seq(
-      "0,Record_0", "1,Record_1", "2,Record_2", "3,Record_3", "16,Record_16",
-      "17,Record_17", "18,Record_18", "19,Record_19", "32,Record_32").mkString("\n")
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-}
-
-class TestProjectableTableSource(
-  fieldTypes: Array[TypeInformation[_]],
-  fieldNames: Array[String])
-  extends BatchTableSource[Row] with ProjectableTableSource[Row] {
-
-  def this() = this(
-    fieldTypes = Array(
-      BasicTypeInfo.STRING_TYPE_INFO,
-      BasicTypeInfo.LONG_TYPE_INFO,
-      BasicTypeInfo.INT_TYPE_INFO,
-      BasicTypeInfo.DOUBLE_TYPE_INFO),
-    fieldNames = Array[String]("name", "id", "amount", "price")
-  )
-
-  /** Returns the data of the table as a [[org.apache.flink.api.java.DataSet]]. */
-  override def getDataSet(execEnv: JavaExecEnv): JavaSet[Row] = {
-    execEnv.fromCollection(generateDynamicCollection(33, fieldNames).asJava, getReturnType)
-  }
-
-  /** Returns the types of the table fields. */
-  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
-
-  /** Returns the names of the table fields. */
-  override def getFieldsNames: Array[String] = fieldNames
-
-  /** Returns the [[TypeInformation]] for the return type. */
-  override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*)
-
-  /** Returns the number of fields of the table. */
-  override def getNumberOfFields: Int = fieldNames.length
-
-  override def projectFields(fields: Array[Int]): TestProjectableTableSource = {
-    val projectedFieldTypes = new Array[TypeInformation[_]](fields.length)
-    val projectedFieldNames = new Array[String](fields.length)
-
-    fields.zipWithIndex.foreach(f => {
-      projectedFieldTypes(f._2) = fieldTypes(f._1)
-      projectedFieldNames(f._2) = fieldNames(f._1)
-    })
-    new TestProjectableTableSource(projectedFieldTypes, projectedFieldNames)
-  }
-
-  private def generateDynamicCollection(num: Int, fieldNames: Array[String]): Seq[Row] = {
-    for {cnt <- 0 until num}
-      yield {
-        val row = new Row(fieldNames.length)
-        fieldNames.zipWithIndex.foreach(
-          f =>
-            f._1 match {
-              case "name" =>
-                row.setField(f._2, "Record_" + cnt)
-              case "id" =>
-                row.setField(f._2, cnt.toLong)
-              case "amount" =>
-                row.setField(f._2, cnt.toInt % 16)
-              case "price" =>
-                row.setField(f._2, cnt.toDouble / 3)
-              case _ =>
-                throw new IllegalArgumentException(s"unknown field name $f._1")
-            }
-        )
-        row
-      }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala
deleted file mode 100644
index b7c8bc0..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableEnvironmentITCase.scala
+++ /dev/null
@@ -1,273 +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.api.scala.batch
-
-import java.util
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.{TableEnvironment, TableException, ValidationException}
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class TableEnvironmentITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testSimpleRegister(): Unit = {
-
-    val tableName = "MyTable"
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet(tableName, ds)
-    val t = tEnv.scan(tableName).select('_1, '_2, '_3)
-
-    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
-      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
-      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testRegisterWithFields(): Unit = {
-
-    val tableName = "MyTable"
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet(tableName, ds, 'a, 'b, 'c)
-    val t = tEnv.scan(tableName).select('a, 'b)
-
-    val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" +
-      "7,4\n" + "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" +
-      "15,5\n" + "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testRegisterExistingDataSet(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds1)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env)
-    // Must fail. Name is already in use.
-    tEnv.registerDataSet("MyTable", ds2)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testScanUnregisteredTable(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    // Must fail. No table registered under that name.
-    tEnv.scan("someTable")
-  }
-
-  @Test
-  def testTableRegister(): Unit = {
-
-    val tableName = "MyTable"
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    tEnv.registerTable(tableName, t)
-
-    val regT = tEnv.scan(tableName).select('a, 'b).filter('a > 8)
-
-    val expected = "9,4\n" + "10,4\n" +
-      "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" +
-      "15,5\n" + "16,6\n" + "17,6\n" + "18,6\n" +
-      "19,6\n" + "20,6\n" + "21,6\n"
-
-    val results = regT.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testRegisterExistingTable(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-    tEnv.registerTable("MyTable", t1)
-    val t2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv)
-    // Must fail. Name is already in use.
-    tEnv.registerDataSet("MyTable", t2)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testRegisterTableFromOtherEnv(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv1 = TableEnvironment.getTableEnvironment(env, config)
-    val tEnv2 = TableEnvironment.getTableEnvironment(env, config)
-
-    val t1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv1)
-    // Must fail. Table is bound to different TableEnvironment.
-    tEnv2.registerTable("MyTable", t1)
-  }
-
-  @Test
-  def testToTable(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val t = CollectionDataSets.get3TupleDataSet(env)
-      .toTable(tEnv, 'a, 'b, 'c)
-      .select('a, 'b, 'c)
-
-    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
-      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
-      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testToTableFromCaseClass(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val data = List(
-      SomeCaseClass("Peter", 28, 4000.00, "Sales"),
-      SomeCaseClass("Anna", 56, 10000.00, "Engineering"),
-      SomeCaseClass("Lucy", 42, 6000.00, "HR"))
-
-    val t =  env.fromCollection(data)
-      .toTable(tEnv, 'a, 'b, 'c, 'd)
-      .select('a, 'b, 'c, 'd)
-
-    val expected: String =
-      "Peter,28,4000.0,Sales\n" +
-      "Anna,56,10000.0,Engineering\n" +
-      "Lucy,42,6000.0,HR\n"
-    val results = t.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testToTableFromAndToCaseClass(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val data = List(
-      SomeCaseClass("Peter", 28, 4000.00, "Sales"),
-      SomeCaseClass("Anna", 56, 10000.00, "Engineering"),
-      SomeCaseClass("Lucy", 42, 6000.00, "HR"))
-
-    val t =  env.fromCollection(data)
-      .toTable(tEnv, 'a, 'b, 'c, 'd)
-      .select('a, 'b, 'c, 'd)
-
-    val expected: String =
-      "SomeCaseClass(Peter,28,4000.0,Sales)\n" +
-      "SomeCaseClass(Anna,56,10000.0,Engineering)\n" +
-      "SomeCaseClass(Lucy,42,6000.0,HR)\n"
-    val results = t.toDataSet[SomeCaseClass].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testToTableWithToFewFields(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    CollectionDataSets.get3TupleDataSet(env)
-      // Must fail. Number of fields does not match.
-      .toTable(tEnv, 'a, 'b)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testToTableWithToManyFields(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    CollectionDataSets.get3TupleDataSet(env)
-      // Must fail. Number of fields does not match.
-      .toTable(tEnv, 'a, 'b, 'c, 'd)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testToTableWithAmbiguousFields(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    CollectionDataSets.get3TupleDataSet(env)
-      // Must fail. Field names not unique.
-      .toTable(tEnv, 'a, 'b, 'b)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testToTableWithNonFieldReference1(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    // Must fail. as() can only have field references
-    CollectionDataSets.get3TupleDataSet(env)
-      .toTable(tEnv, 'a + 1, 'b, 'c)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testToTableWithNonFieldReference2(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    // Must fail. as() can only have field references
-    CollectionDataSets.get3TupleDataSet(env)
-      .toTable(tEnv, 'a as 'foo, 'b, 'c)
-  }
-}
-
-object TableEnvironmentITCase {
-
-  @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
-  def parameters(): util.Collection[Array[java.lang.Object]] = {
-    Seq[Array[AnyRef]](
-      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT),
-      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.EFFICIENT)).asJava
-  }
-}
-
-case class SomeCaseClass(name: String, age: Int, salary: Double, department: String) {
-  def this() { this("", 0, 0.0, "") }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala
deleted file mode 100644
index d7e99d4..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSinkITCase.scala
+++ /dev/null
@@ -1,71 +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.api.scala.batch
-
-import java.io.File
-
-import org.apache.flink.api.scala.{ExecutionEnvironment, _}
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.api.table.sinks.CsvTableSink
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit.Test
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-
-@RunWith(classOf[Parameterized])
-class TableSinkITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testBatchTableSink(): Unit = {
-
-    val tmpFile = File.createTempFile("flink-table-sink-test", ".tmp")
-    tmpFile.deleteOnExit()
-    val path = tmpFile.toURI.toString
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    env.setParallelism(4)
-
-    val input = CollectionDataSets.get3TupleDataSet(env)
-      .map(x => x).setParallelism(4) // increase DOP to 4
-
-    val results = input.toTable(tEnv, 'a, 'b, 'c)
-      .where('a < 5 || 'a > 17)
-      .select('c, 'b)
-      .writeToSink(new CsvTableSink(path, fieldDelim = "|"))
-
-    env.execute()
-
-    val expected = Seq(
-      "Hi|1", "Hello|2", "Hello world|2", "Hello world, how are you?|3",
-      "Comment#12|6", "Comment#13|6", "Comment#14|6", "Comment#15|6").mkString("\n")
-
-    TestBaseUtils.compareResultsByLinesInMemory(expected, path)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala
deleted file mode 100644
index b5c8ada..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/TableSourceITCase.scala
+++ /dev/null
@@ -1,176 +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.api.scala.batch
-
-import java.io.{File, FileOutputStream, OutputStreamWriter}
-
-import org.apache.flink.api.common.io.GenericInputFormat
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.{DataSet => JavaSet, ExecutionEnvironment => JavaExecEnv}
-import org.apache.flink.api.scala.ExecutionEnvironment
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.sources.{BatchTableSource, CsvTableSource}
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit.Test
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class TableSourceITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testBatchTableSourceTableAPI(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    tEnv.registerTableSource("MyTestTable", new TestBatchTableSource())
-    val results = tEnv
-      .scan("MyTestTable")
-      .where('amount < 4)
-      .select('amount * 'id, 'name)
-      .collect()
-
-    val expected = Seq(
-      "0,Record_0", "0,Record_16", "0,Record_32", "1,Record_1", "17,Record_17",
-      "36,Record_18", "4,Record_2", "57,Record_19", "9,Record_3").mkString("\n")
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testBatchTableSourceSQL(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    tEnv.registerTableSource("MyTestTable", new TestBatchTableSource())
-    val results = tEnv.sql(
-      "SELECT amount * id, name FROM MyTestTable WHERE amount < 4").collect()
-
-    val expected = Seq(
-      "0,Record_0", "0,Record_16", "0,Record_32", "1,Record_1", "17,Record_17",
-      "36,Record_18", "4,Record_2", "57,Record_19", "9,Record_3").mkString("\n")
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testCsvTableSource(): Unit = {
-
-    val csvRecords = Seq(
-      "First#Id#Score#Last",
-      "Mike#1#12.3#Smith",
-      "Bob#2#45.6#Taylor",
-      "Sam#3#7.89#Miller",
-      "Peter#4#0.12#Smith",
-      "% Just a comment",
-      "Liz#5#34.5#Williams",
-      "Sally#6#6.78#Miller",
-      "Alice#7#90.1#Smith",
-      "Kelly#8#2.34#Williams"
-    )
-
-    val tempFile = File.createTempFile("csv-test", "tmp")
-    tempFile.deleteOnExit()
-    val tmpWriter = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8")
-    tmpWriter.write(csvRecords.mkString("$"))
-    tmpWriter.close()
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val csvTable = new CsvTableSource(
-      tempFile.getAbsolutePath,
-      Array("first", "id", "score", "last"),
-      Array(
-        BasicTypeInfo.STRING_TYPE_INFO,
-        BasicTypeInfo.INT_TYPE_INFO,
-        BasicTypeInfo.DOUBLE_TYPE_INFO,
-        BasicTypeInfo.STRING_TYPE_INFO
-      ),
-      fieldDelim = "#",
-      rowDelim = "$",
-      ignoreFirstLine = true,
-      ignoreComments = "%"
-    )
-
-    tEnv.registerTableSource("csvTable", csvTable)
-    val results = tEnv.sql(
-      "SELECT last, sum(score), max(id) FROM csvTable GROUP BY last").collect()
-
-    val expected = Seq(
-      "Smith,102.52,7",
-      "Taylor,45.6,2",
-      "Miller,14.67,6",
-      "Williams,36.84,8").mkString("\n")
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-}
-
-class TestBatchTableSource extends BatchTableSource[Row] {
-
-  val fieldTypes: Array[TypeInformation[_]] = Array(
-    BasicTypeInfo.STRING_TYPE_INFO,
-    BasicTypeInfo.LONG_TYPE_INFO,
-    BasicTypeInfo.INT_TYPE_INFO
-  )
-
-  /** Returns the data of the table as a [[org.apache.flink.api.java.DataSet]]. */
-  override def getDataSet(execEnv: JavaExecEnv): JavaSet[Row] = {
-    execEnv.createInput(new GeneratingInputFormat(33), getReturnType).setParallelism(1)
-  }
-
-  /** Returns the types of the table fields. */
-  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
-
-  /** Returns the names of the table fields. */
-  override def getFieldsNames: Array[String] = Array("name", "id", "amount")
-
-  /** Returns the [[TypeInformation]] for the return type. */
-  override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*)
-
-  /** Returns the number of fields of the table. */
-  override def getNumberOfFields: Int = 3
-}
-
-class GeneratingInputFormat(val num: Int) extends GenericInputFormat[Row] {
-
-  var cnt = 0L
-
-  override def reachedEnd(): Boolean = cnt >= num
-
-  override def nextRecord(reuse: Row): Row = {
-    reuse.setField(0, s"Record_$cnt")
-    reuse.setField(1, cnt)
-    reuse.setField(2, (cnt % 16).toInt)
-    cnt += 1
-    reuse
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala
deleted file mode 100644
index d5d46ba..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/AggregationsITCase.scala
+++ /dev/null
@@ -1,301 +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.api.scala.batch.sql
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.table.{TableException, TableEnvironment}
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class AggregationsITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testAggregationTypes(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT sum(_1), min(_1), max(_1), count(_1), avg(_1) FROM MyTable"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "231,1,21,21,11"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testTableAggregation(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT sum(_1) FROM MyTable"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "231"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testDataSetAggregation(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT sum(_1) FROM MyTable"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "231"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testWorkingAggregationDataTypes(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery =
-      "SELECT avg(_1), avg(_2), avg(_3), avg(_4), avg(_5), avg(_6), count(_7), " +
-      "  sum(CAST(_6 AS DECIMAL))" +
-      "FROM MyTable"
-
-    val ds = env.fromElements(
-      (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"),
-      (2: Byte, 2: Short, 2, 2L, 2.0f, 2.0d, "Ciao"))
-    tEnv.registerDataSet("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1,1,1,1,1.5,1.5,2,3.0"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testTableWorkingAggregationDataTypes(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT avg(a), avg(b), avg(c), avg(d), avg(e), avg(f), count(g)" +
-      "FROM MyTable"
-
-    val ds = env.fromElements(
-      (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"),
-      (2: Byte, 2: Short, 2, 2L, 2.0f, 2.0d, "Ciao")).toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'f, 'g)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1,1,1,1,1.5,1.5,2"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testTableProjection(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT avg(a), sum(a), count(a), avg(b), sum(b) " +
-      "FROM MyTable"
-
-    val ds = env.fromElements((1: Byte, 1: Short), (2: Byte, 2: Short)).toTable(tEnv, 'a, 'b)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1,3,2,1,3"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testTableAggregationWithArithmetic(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT avg(a + 2) + 2, count(b) + 5 " +
-      "FROM MyTable"
-
-    val ds = env.fromElements((1f, "Hello"), (2f, "Ciao")).toTable(tEnv, 'a, 'b)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "5.5,7"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testAggregationWithTwoCount(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT count(_1), count(_2) FROM MyTable"
-
-    val ds = env.fromElements((1f, "Hello"), (2f, "Ciao")).toTable(tEnv)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "2,2"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-
-  @Test
-  def testAggregationAfterProjection(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM " +
-      "(SELECT _1 as a, _2 as b, _3 as c FROM MyTable)"
-
-    val ds = env.fromElements(
-      (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"),
-      (2: Byte, 2: Short, 2, 2L, 2.0f, 2.0d, "Ciao")).toTable(tEnv)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1,3,2"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testDistinctAggregate(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT sum(_1) as a, count(distinct _3) as b FROM MyTable"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds)
-
-    // must fail. distinct aggregates are not supported
-    tEnv.sql(sqlQuery).toDataSet[Row]
-  }
-
-  @Test(expected = classOf[TableException])
-  def testGroupedDistinctAggregate(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT _2, avg(distinct _1) as a, count(_3) as b FROM MyTable GROUP BY _2"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds)
-
-    // must fail. distinct aggregates are not supported
-    tEnv.sql(sqlQuery).toDataSet[Row]
-  }
-
-  @Test(expected = classOf[TableException])
-  def testGroupingSetAggregate(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT _2, _3, avg(_1) as a FROM MyTable GROUP BY GROUPING SETS (_2, _3)"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds)
-
-    // must fail. grouping sets are not supported
-    tEnv.sql(sqlQuery).toDataSet[Row]
-  }
-
-  @Test
-  def testAggregateEmptyDataSets(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT avg(a), sum(a), count(b) " +
-      "FROM MyTable where a = 4 group by a"
-
-    val sqlQuery2 = "SELECT avg(a), sum(a), count(b) " +
-      "FROM MyTable where a = 4"
-
-    val sqlQuery3 = "SELECT avg(a), sum(a), count(b) " +
-      "FROM MyTable"
-
-    val ds = env.fromElements(
-      (1: Byte, 1: Short),
-      (2: Byte, 2: Short))
-      .toTable(tEnv, 'a, 'b)
-
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-    val result2 = tEnv.sql(sqlQuery2)
-    val result3 = tEnv.sql(sqlQuery3)
-
-    val results = result.toDataSet[Row].collect()
-    val expected = Seq.empty
-    val results2 =  result2.toDataSet[Row].collect()
-    val expected2 = "null,null,0"
-    val results3 = result3.toDataSet[Row].collect()
-    val expected3 = "1,3,2"
-
-    assert(results.equals(expected),
-      "Empty result is expected for grouped set, but actual: " + results)
-    TestBaseUtils.compareResultAsText(results2.asJava, expected2)
-    TestBaseUtils.compareResultAsText(results3.asJava, expected3)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala
deleted file mode 100644
index 5037469..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/sql/CalcITCase.scala
+++ /dev/null
@@ -1,329 +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.api.scala.batch.sql
-
-
-import java.sql.{Date, Time, Timestamp}
-import java.util
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.sql.FilterITCase.MyHashCode
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.functions.ScalarFunction
-import org.apache.flink.api.table.{TableEnvironment, ValidationException}
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class CalcITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testSelectStarFromTable(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
-      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
-      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
-
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testSelectStarFromDataSet(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
-      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
-      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
-
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testSimpleSelectAll(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT a, b, c FROM MyTable"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
-      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
-      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
-
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testSelectWithNaming(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT _1 as a, _2 as b FROM MyTable"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
-      "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
-      "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"
-
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidFields(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT a, foo FROM MyTable"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("MyTable", ds)
-
-    tEnv.sql(sqlQuery)
-  }
-
-  @Test
-  def testAllRejectingFilter(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable WHERE false"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testAllPassingFilter(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable WHERE true"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, " +
-      "how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + "7,4," +
-      "Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + "11,5," +
-      "Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5," +
-      "Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19," +
-      "6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testFilterOnString(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable WHERE c LIKE '%world%'"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "3,2,Hello world\n" + "4,3,Hello world, how are you?\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testFilterOnInteger(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable WHERE MOD(a,2)=0"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" +
-      "6,3,Luke Skywalker\n" + "8,4," + "Comment#2\n" + "10,4,Comment#4\n" +
-      "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," +
-      "Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testDisjunctivePredicate(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable WHERE a < 2 OR a > 20"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1,1,Hi\n" + "21,6,Comment#15\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testFilterWithAnd(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT * FROM MyTable WHERE MOD(a,2)<>0 AND MOD(b,2)=0"
-
-    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("MyTable", ds)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "3,2,Hello world\n" + "7,4,Comment#1\n" +
-      "9,4,Comment#3\n" + "17,6,Comment#11\n" +
-      "19,6,Comment#13\n" + "21,6,Comment#15\n"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testAdvancedDataTypes(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val sqlQuery = "SELECT a, b, c, DATE '1984-07-12', TIME '14:34:24', " +
-      "TIMESTAMP '1984-07-12 14:34:24' FROM MyTable"
-
-    val ds = env.fromElements((
-      Date.valueOf("1984-07-12"),
-      Time.valueOf("14:34:24"),
-      Timestamp.valueOf("1984-07-12 14:34:24")))
-    tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c)
-
-    val result = tEnv.sql(sqlQuery)
-
-    val expected = "1984-07-12,14:34:24,1984-07-12 14:34:24.0," +
-      "1984-07-12,14:34:24,1984-07-12 14:34:24.0"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testUserDefinedScalarFunction(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    tEnv.registerFunction("hashCode",
-      new org.apache.flink.api.java.batch.table.CalcITCase.OldHashCode)
-    tEnv.registerFunction("hashCode", MyHashCode)
-
-    val ds = env.fromElements("a", "b", "c")
-    tEnv.registerDataSet("MyTable", ds, 'text)
-
-    val result = tEnv.sql("SELECT hashCode(text) FROM MyTable")
-
-    val expected = "97\n98\n99"
-    val results = result.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-}
-
-object FilterITCase {
-  object MyHashCode extends ScalarFunction {
-    def eval(s: String): Int = s.hashCode()
-  }
-}
-
-object CalcITCase {
-
-  @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
-  def parameters(): util.Collection[Array[java.lang.Object]] = {
-    Seq[Array[AnyRef]](
-      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT),
-      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.NO_NULL)).asJava
-  }
-}


[28/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..4e43001
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
@@ -0,0 +1,352 @@
+/*
+ * 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.util.concurrent.atomic.AtomicInteger
+
+import org.apache.calcite.plan.RelOptPlanner.CannotPlanException
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.sql2rel.RelDecorrelator
+import org.apache.calcite.tools.{Programs, RuleSet}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.TypeExtractor
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+import org.apache.flink.table.calcite.FlinkPlannerImpl
+import org.apache.flink.table.explain.PlanJsonParser
+import org.apache.flink.table.expressions.Expression
+import org.apache.flink.table.plan.logical.{CatalogNode, LogicalRelNode}
+import org.apache.flink.table.plan.nodes.datastream.{DataStreamConvention, DataStreamRel}
+import org.apache.flink.table.plan.rules.FlinkRuleSets
+import org.apache.flink.table.plan.schema.{DataStreamTable, TableSourceTable}
+import org.apache.flink.table.sinks.{StreamTableSink, TableSink}
+import org.apache.flink.table.sources.StreamTableSource
+import org.apache.flink.types.Row
+
+/**
+  * The base class for stream TableEnvironments.
+  *
+  * A TableEnvironment can be used to:
+  * - convert [[DataStream]] to a [[Table]]
+  * - register a [[DataStream]] as a table in the catalog
+  * - register a [[Table]] in the catalog
+  * - scan a registered table to obtain a [[Table]]
+  * - specify a SQL query on registered tables to obtain a [[Table]]
+  * - convert a [[Table]] into a [[DataStream]]
+  *
+  * @param execEnv The [[StreamExecutionEnvironment]] which is wrapped in this
+  *                [[StreamTableEnvironment]].
+  * @param config The [[TableConfig]] of this [[StreamTableEnvironment]].
+  */
+abstract class StreamTableEnvironment(
+    private[flink] val execEnv: StreamExecutionEnvironment,
+    config: TableConfig)
+  extends TableEnvironment(config) {
+
+  // a counter for unique table names
+  private val nameCntr: AtomicInteger = new AtomicInteger(0)
+
+  // the naming pattern for internally registered tables.
+  private val internalNamePattern = "^_DataStreamTable_[0-9]+$".r
+
+  /**
+    * Checks if the chosen table name is valid.
+    *
+    * @param name The table name to check.
+    */
+  override protected def checkValidTableName(name: String): Unit = {
+    val m = internalNamePattern.findFirstIn(name)
+    m match {
+      case Some(_) =>
+        throw new TableException(s"Illegal Table name. " +
+          s"Please choose a name that does not contain the pattern $internalNamePattern")
+      case None =>
+    }
+  }
+
+  /** Returns a unique table name according to the internal naming pattern. */
+  protected def createUniqueTableName(): String = "_DataStreamTable_" + nameCntr.getAndIncrement()
+
+  /**
+    * Returns field names and field positions for a given [[TypeInformation]].
+    *
+    * Field names are automatically extracted for
+    * [[org.apache.flink.api.common.typeutils.CompositeType]].
+    * The method fails if inputType is not a
+    * [[org.apache.flink.api.common.typeutils.CompositeType]].
+    *
+    * @param inputType The TypeInformation extract the field names and positions from.
+    * @tparam A The type of the TypeInformation.
+    * @return A tuple of two arrays holding the field names and corresponding field positions.
+    */
+  override protected[flink] def getFieldInfo[A](inputType: TypeInformation[A])
+    : (Array[String], Array[Int]) = {
+    val fieldInfo = super.getFieldInfo(inputType)
+    if (fieldInfo._1.contains("rowtime")) {
+      throw new TableException("'rowtime' ia a reserved field name in stream environment.")
+    }
+    fieldInfo
+  }
+
+  /**
+    * Returns field names and field positions for a given [[TypeInformation]] and [[Array]] of
+    * [[Expression]].
+    *
+    * @param inputType The [[TypeInformation]] against which the [[Expression]]s are evaluated.
+    * @param exprs     The expressions that define the field names.
+    * @tparam A The type of the TypeInformation.
+    * @return A tuple of two arrays holding the field names and corresponding field positions.
+    */
+  override protected[flink] def getFieldInfo[A](
+      inputType: TypeInformation[A],
+      exprs: Array[Expression])
+    : (Array[String], Array[Int]) = {
+    val fieldInfo = super.getFieldInfo(inputType, exprs)
+    if (fieldInfo._1.contains("rowtime")) {
+      throw new TableException("'rowtime' is a reserved field name in stream environment.")
+    }
+    fieldInfo
+  }
+
+  /**
+    * Ingests a registered table and returns the resulting [[Table]].
+    *
+    * The table to ingest must be registered in the [[TableEnvironment]]'s catalog.
+    *
+    * @param tableName The name of the table to ingest.
+    * @throws ValidationException if no table is registered under the given name.
+    * @return The ingested table.
+    */
+  @throws[ValidationException]
+  def ingest(tableName: String): Table = {
+
+    if (isRegistered(tableName)) {
+      new Table(this, CatalogNode(tableName, getRowType(tableName)))
+    }
+    else {
+      throw new ValidationException(s"Table \'$tableName\' was not found in the registry.")
+    }
+  }
+
+  /**
+    * Registers an external [[StreamTableSource]] in this [[TableEnvironment]]'s catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * @param name        The name under which the [[StreamTableSource]] is registered.
+    * @param tableSource The [[org.apache.flink.table.sources.StreamTableSource]] to register.
+    */
+  def registerTableSource(name: String, tableSource: StreamTableSource[_]): Unit = {
+
+    checkValidTableName(name)
+    registerTableInternal(name, new TableSourceTable(tableSource))
+  }
+
+  /**
+    * Evaluates a SQL query on registered tables and retrieves the result as a [[Table]].
+    *
+    * All tables referenced by the query must be registered in the TableEnvironment.
+    *
+    * @param query The SQL query to evaluate.
+    * @return The result of the query as Table.
+    */
+  override def sql(query: String): Table = {
+
+    val planner = new FlinkPlannerImpl(getFrameworkConfig, getPlanner, getTypeFactory)
+    // parse the sql query
+    val parsed = planner.parse(query)
+    // validate the sql query
+    val validated = planner.validate(parsed)
+    // transform to a relational tree
+    val relational = planner.rel(validated)
+
+    new Table(this, LogicalRelNode(relational.rel))
+  }
+
+  /**
+    * Writes a [[Table]] to a [[TableSink]].
+    *
+    * Internally, the [[Table]] is translated into a [[DataStream]] and handed over to the
+    * [[TableSink]] to write it.
+    *
+    * @param table The [[Table]] to write.
+    * @param sink The [[TableSink]] to write the [[Table]] to.
+    * @tparam T The expected type of the [[DataStream]] which represents the [[Table]].
+    */
+  override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = {
+
+    sink match {
+      case streamSink: StreamTableSink[T] =>
+        val outputType = sink.getOutputType
+        // translate the Table into a DataStream and provide the type that the TableSink expects.
+        val result: DataStream[T] = translate(table)(outputType)
+        // Give the DataSet to the TableSink to emit it.
+        streamSink.emitDataStream(result)
+      case _ =>
+        throw new TableException("StreamTableSink required to emit streaming Table")
+    }
+  }
+
+  /**
+    * Registers a [[DataStream]] as a table under a given name in the [[TableEnvironment]]'s
+    * catalog.
+    *
+    * @param name The name under which the table is registered in the catalog.
+    * @param dataStream The [[DataStream]] to register as table in the catalog.
+    * @tparam T the type of the [[DataStream]].
+    */
+  protected def registerDataStreamInternal[T](
+    name: String,
+    dataStream: DataStream[T]): Unit = {
+
+    val (fieldNames, fieldIndexes) = getFieldInfo[T](dataStream.getType)
+    val dataStreamTable = new DataStreamTable[T](
+      dataStream,
+      fieldIndexes,
+      fieldNames
+    )
+    registerTableInternal(name, dataStreamTable)
+  }
+
+  /**
+    * Registers a [[DataStream]] as a table under a given name with field names as specified by
+    * field expressions in the [[TableEnvironment]]'s catalog.
+    *
+    * @param name The name under which the table is registered in the catalog.
+    * @param dataStream The [[DataStream]] to register as table in the catalog.
+    * @param fields The field expressions to define the field names of the table.
+    * @tparam T The type of the [[DataStream]].
+    */
+  protected def registerDataStreamInternal[T](
+    name: String,
+    dataStream: DataStream[T],
+    fields: Array[Expression]): Unit = {
+
+    val (fieldNames, fieldIndexes) = getFieldInfo[T](dataStream.getType, fields.toArray)
+    val dataStreamTable = new DataStreamTable[T](
+      dataStream,
+      fieldIndexes.toArray,
+      fieldNames.toArray
+    )
+    registerTableInternal(name, dataStreamTable)
+  }
+
+  /**
+    * Returns the built-in rules that are defined by the environment.
+    */
+  protected def getBuiltInRuleSet: RuleSet = FlinkRuleSets.DATASTREAM_OPT_RULES
+
+  /**
+    * Generates the optimized [[RelNode]] tree from the original relational node tree.
+    *
+    * @param relNode The root node of the relational expression tree.
+    * @return The optimized [[RelNode]] tree
+    */
+  private[flink] def optimize(relNode: RelNode): RelNode = {
+    // decorrelate
+    val decorPlan = RelDecorrelator.decorrelateQuery(relNode)
+
+    // optimize the logical Flink plan
+    val optProgram = Programs.ofRules(getRuleSet)
+    val flinkOutputProps = relNode.getTraitSet.replace(DataStreamConvention.INSTANCE).simplify()
+
+    val dataStreamPlan = try {
+      optProgram.run(getPlanner, decorPlan, flinkOutputProps)
+    }
+    catch {
+      case e: CannotPlanException =>
+        throw TableException(
+          s"Cannot generate a valid execution plan for the given query: \n\n" +
+            s"${RelOptUtil.toString(relNode)}\n" +
+            s"This exception indicates that the query uses an unsupported SQL feature.\n" +
+            s"Please check the documentation for the set of currently supported SQL features.", e)
+    }
+    dataStreamPlan
+  }
+
+
+  /**
+    * Translates a [[Table]] into a [[DataStream]].
+    *
+    * The transformation involves optimizing the relational expression tree as defined by
+    * Table API calls and / or SQL queries and generating corresponding [[DataStream]] operators.
+    *
+    * @param table The root node of the relational expression tree.
+    * @param tpe The [[TypeInformation]] of the resulting [[DataStream]].
+    * @tparam A The type of the resulting [[DataStream]].
+    * @return The [[DataStream]] that corresponds to the translated [[Table]].
+    */
+  protected def translate[A](table: Table)(implicit tpe: TypeInformation[A]): DataStream[A] = {
+    val dataStreamPlan = optimize(table.getRelNode)
+    translate(dataStreamPlan)
+  }
+
+  /**
+    * Translates a logical [[RelNode]] into a [[DataStream]].
+    *
+    * @param logicalPlan The root node of the relational expression tree.
+    * @param tpe         The [[TypeInformation]] of the resulting [[DataStream]].
+    * @tparam A The type of the resulting [[DataStream]].
+    * @return The [[DataStream]] that corresponds to the translated [[Table]].
+    */
+  protected def translate[A]
+      (logicalPlan: RelNode)(implicit tpe: TypeInformation[A]): DataStream[A] = {
+
+    validateType(tpe)
+
+    logicalPlan match {
+      case node: DataStreamRel =>
+        node.translateToPlan(
+          this,
+          Some(tpe.asInstanceOf[TypeInformation[Any]])
+        ).asInstanceOf[DataStream[A]]
+      case _ => ???
+    }
+  }
+
+  /**
+    * Returns the AST of the specified Table API and SQL queries and the execution plan to compute
+    * the result of the given [[Table]].
+    *
+    * @param table The table for which the AST and execution plan will be returned.
+    */
+  def explain(table: Table): String = {
+    val ast = table.getRelNode
+    val optimizedPlan = optimize(ast)
+    val dataStream = translate[Row](optimizedPlan)(TypeExtractor.createTypeInfo(classOf[Row]))
+
+    val env = dataStream.getExecutionEnvironment
+    val jsonSqlPlan = env.getExecutionPlan
+
+    val sqlPlan = PlanJsonParser.getSqlExecutionPlan(jsonSqlPlan, false)
+
+    s"== Abstract Syntax Tree ==" +
+        System.lineSeparator +
+        s"${RelOptUtil.toString(ast)}" +
+        System.lineSeparator +
+        s"== Optimized Logical Plan ==" +
+        System.lineSeparator +
+        s"${RelOptUtil.toString(optimizedPlan)}" +
+        System.lineSeparator +
+        s"== Physical Execution Plan ==" +
+        System.lineSeparator +
+        s"$sqlPlan"
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
new file mode 100644
index 0000000..a8876a8
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableConfig.scala
@@ -0,0 +1,109 @@
+/*
+ * 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.util.TimeZone
+
+import org.apache.flink.table.calcite.CalciteConfig
+
+/**
+ * A config to define the runtime behavior of the Table API.
+ */
+class TableConfig {
+
+  /**
+   * Defines the timezone for date/time/timestamp conversions.
+   */
+  private var timeZone: TimeZone = TimeZone.getTimeZone("UTC")
+
+  /**
+   * Defines if all fields need to be checked for NULL first.
+   */
+  private var nullCheck: Boolean = true
+
+  /**
+    * Defines if efficient types (such as Tuple types or Atomic types)
+    * should be used within operators where possible.
+    */
+  private var efficientTypeUsage = false
+
+  /**
+    * Defines the configuration of Calcite for Table API and SQL queries.
+    */
+  private var calciteConfig = CalciteConfig.DEFAULT
+
+  /**
+   * Sets the timezone for date/time/timestamp conversions.
+   */
+  def setTimeZone(timeZone: TimeZone): Unit = {
+    require(timeZone != null, "timeZone must not be null.")
+    this.timeZone = timeZone
+  }
+
+  /**
+   * Returns the timezone for date/time/timestamp conversions.
+   */
+  def getTimeZone = timeZone
+
+  /**
+   * Returns the NULL check. If enabled, all fields need to be checked for NULL first.
+   */
+  def getNullCheck = nullCheck
+
+  /**
+   * Sets the NULL check. If enabled, all fields need to be checked for NULL first.
+   */
+  def setNullCheck(nullCheck: Boolean): Unit = {
+    this.nullCheck = nullCheck
+  }
+
+  /**
+    * Returns the usage of efficient types. If enabled, efficient types (such as Tuple types
+    * or Atomic types) are used within operators where possible.
+    *
+    * NOTE: Currently, this is an experimental feature.
+    */
+  def getEfficientTypeUsage = efficientTypeUsage
+
+  /**
+    * Sets the usage of efficient types. If enabled, efficient types (such as Tuple types
+    * or Atomic types) are used within operators where possible.
+    *
+    * NOTE: Currently, this is an experimental feature.
+    */
+  def setEfficientTypeUsage(efficientTypeUsage: Boolean): Unit = {
+    this.efficientTypeUsage = efficientTypeUsage
+  }
+
+  /**
+    * Returns the current configuration of Calcite for Table API and SQL queries.
+    */
+  def getCalciteConfig: CalciteConfig = calciteConfig
+
+  /**
+    * Sets the configuration of Calcite for Table API and SQL queries.
+    * Changing the configuration has no effect after the first query has been defined.
+    */
+  def setCalciteConfig(calciteConfig: CalciteConfig): Unit = {
+    this.calciteConfig = calciteConfig
+  }
+}
+
+object TableConfig {
+  def DEFAULT = new TableConfig()
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..769008f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
@@ -0,0 +1,538 @@
+/*
+ * 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.util.concurrent.atomic.AtomicInteger
+import _root_.java.lang.reflect.Modifier
+
+import org.apache.calcite.config.Lex
+import org.apache.calcite.plan.RelOptPlanner
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.schema.SchemaPlus
+import org.apache.calcite.schema.impl.AbstractTable
+import org.apache.calcite.sql.SqlOperatorTable
+import org.apache.calcite.sql.parser.SqlParser
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable
+import org.apache.calcite.tools.{FrameworkConfig, Frameworks, RuleSet, RuleSets}
+import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo}
+import org.apache.flink.api.java.{ExecutionEnvironment => JavaBatchExecEnv}
+import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
+import org.apache.flink.api.scala.{ExecutionEnvironment => ScalaBatchExecEnv}
+import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaStreamExecEnv}
+import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => ScalaStreamExecEnv}
+import org.apache.flink.table.api.java.{BatchTableEnvironment => JavaBatchTableEnv, StreamTableEnvironment => JavaStreamTableEnv}
+import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTableEnv, StreamTableEnvironment => ScalaStreamTableEnv}
+import org.apache.flink.table.calcite.{FlinkRelBuilder, FlinkTypeFactory, FlinkTypeSystem}
+import org.apache.flink.table.codegen.ExpressionReducer
+import org.apache.flink.table.expressions.{Alias, Expression, UnresolvedFieldReference}
+import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{checkForInstantiation, checkNotSingleton, createScalarSqlFunction, createTableSqlFunctions}
+import org.apache.flink.table.functions.{ScalarFunction, TableFunction}
+import org.apache.flink.table.plan.cost.DataSetCostFactory
+import org.apache.flink.table.plan.schema.RelTable
+import org.apache.flink.table.sinks.TableSink
+import org.apache.flink.table.validate.FunctionCatalog
+
+import _root_.scala.collection.JavaConverters._
+
+/**
+  * The abstract base class for batch and stream TableEnvironments.
+  *
+  * @param config The configuration of the TableEnvironment
+  */
+abstract class TableEnvironment(val config: TableConfig) {
+
+  // the catalog to hold all registered and translated tables
+  private val tables: SchemaPlus = Frameworks.createRootSchema(true)
+
+  // Table API/SQL function catalog
+  private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns
+
+  // the configuration to create a Calcite planner
+  private lazy val frameworkConfig: FrameworkConfig = Frameworks
+    .newConfigBuilder
+    .defaultSchema(tables)
+    .parserConfig(getSqlParserConfig)
+    .costFactory(new DataSetCostFactory)
+    .typeSystem(new FlinkTypeSystem)
+    .operatorTable(getSqlOperatorTable)
+    // set the executor to evaluate constant expressions
+    .executor(new ExpressionReducer(config))
+    .build
+
+  // the builder for Calcite RelNodes, Calcite's representation of a relational expression tree.
+  protected lazy val relBuilder: FlinkRelBuilder = FlinkRelBuilder.create(frameworkConfig)
+
+  // the planner instance used to optimize queries of this TableEnvironment
+  private lazy val planner: RelOptPlanner = relBuilder.getPlanner
+
+  private lazy val typeFactory: FlinkTypeFactory = relBuilder.getTypeFactory
+
+  // a counter for unique attribute names
+  private val attrNameCntr: AtomicInteger = new AtomicInteger(0)
+
+  /** Returns the table config to define the runtime behavior of the Table API. */
+  def getConfig = config
+
+  /**
+    * Returns the operator table for this environment including a custom Calcite configuration.
+    */
+  protected def getSqlOperatorTable: SqlOperatorTable = {
+    val calciteConfig = config.getCalciteConfig
+    calciteConfig.getSqlOperatorTable match {
+
+      case None =>
+        functionCatalog.getSqlOperatorTable
+
+      case Some(table) =>
+        if (calciteConfig.replacesSqlOperatorTable) {
+          table
+        } else {
+          ChainedSqlOperatorTable.of(functionCatalog.getSqlOperatorTable, table)
+        }
+    }
+  }
+
+  /**
+    * Returns the rule set for this environment including a custom Calcite configuration.
+    */
+  protected def getRuleSet: RuleSet = {
+    val calciteConfig = config.getCalciteConfig
+    calciteConfig.getRuleSet match {
+
+      case None =>
+        getBuiltInRuleSet
+
+      case Some(ruleSet) =>
+        if (calciteConfig.replacesRuleSet) {
+          ruleSet
+        } else {
+          RuleSets.ofList((getBuiltInRuleSet.asScala ++ ruleSet.asScala).asJava)
+        }
+    }
+  }
+
+  /**
+    * Returns the SQL parser config for this environment including a custom Calcite configuration.
+    */
+  protected def getSqlParserConfig: SqlParser.Config = {
+    val calciteConfig = config.getCalciteConfig
+    calciteConfig.getSqlParserConfig match {
+
+      case None =>
+        // we use Java lex because back ticks are easier than double quotes in programming
+        // and cases are preserved
+        SqlParser
+          .configBuilder()
+          .setLex(Lex.JAVA)
+          .build()
+
+      case Some(sqlParserConfig) =>
+        sqlParserConfig
+    }
+  }
+
+  /**
+    * Returns the built-in rules that are defined by the environment.
+    */
+  protected def getBuiltInRuleSet: RuleSet
+
+  /**
+    * Registers a [[ScalarFunction]] under a unique name. Replaces already existing
+    * user-defined functions under this name.
+    */
+  def registerFunction(name: String, function: ScalarFunction): Unit = {
+    // check if class could be instantiated
+    checkForInstantiation(function.getClass)
+
+    // register in Table API
+    functionCatalog.registerFunction(name, function.getClass)
+
+    // register in SQL API
+    functionCatalog.registerSqlFunction(createScalarSqlFunction(name, function, typeFactory))
+  }
+
+  /**
+    * Registers a [[TableFunction]] under a unique name. Replaces already existing
+    * user-defined functions under this name.
+    */
+  private[flink] def registerTableFunctionInternal[T: TypeInformation](
+    name: String, function: TableFunction[T]): Unit = {
+    // check if class not Scala object
+    checkNotSingleton(function.getClass)
+    // check if class could be instantiated
+    checkForInstantiation(function.getClass)
+
+    val typeInfo: TypeInformation[_] = if (function.getResultType != null) {
+      function.getResultType
+    } else {
+      implicitly[TypeInformation[T]]
+    }
+
+    // register in Table API
+    functionCatalog.registerFunction(name, function.getClass)
+
+    // register in SQL API
+    val sqlFunctions = createTableSqlFunctions(name, function, typeInfo, typeFactory)
+    functionCatalog.registerSqlFunctions(sqlFunctions)
+  }
+
+  /**
+    * Registers a [[Table]] under a unique name in the TableEnvironment's catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * @param name The name under which the table is registered.
+    * @param table The table to register.
+    */
+  def registerTable(name: String, table: Table): Unit = {
+
+    // check that table belongs to this table environment
+    if (table.tableEnv != this) {
+      throw new TableException(
+        "Only tables that belong to this TableEnvironment can be registered.")
+    }
+
+    checkValidTableName(name)
+    val tableTable = new RelTable(table.getRelNode)
+    registerTableInternal(name, tableTable)
+  }
+
+  /**
+    * Replaces a registered Table with another Table under the same name.
+    * We use this method to replace a [[org.apache.flink.table.plan.schema.DataStreamTable]]
+    * with a [[org.apache.calcite.schema.TranslatableTable]].
+    *
+    * @param name Name of the table to replace.
+    * @param table The table that replaces the previous table.
+    */
+  protected def replaceRegisteredTable(name: String, table: AbstractTable): Unit = {
+
+    if (isRegistered(name)) {
+      tables.add(name, table)
+    } else {
+      throw new TableException(s"Table \'$name\' is not registered.")
+    }
+  }
+
+  /**
+    * Evaluates a SQL query on registered tables and retrieves the result as a [[Table]].
+    *
+    * All tables referenced by the query must be registered in the TableEnvironment.
+    *
+    * @param query The SQL query to evaluate.
+    * @return The result of the query as Table.
+    */
+  def sql(query: String): Table
+
+  /**
+    * Writes a [[Table]] to a [[TableSink]].
+    *
+    * @param table The [[Table]] to write.
+    * @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
+
+  /**
+    * Registers a Calcite [[AbstractTable]] in the TableEnvironment's catalog.
+    *
+    * @param name The name under which the table is registered.
+    * @param table The table to register in the catalog
+    * @throws TableException if another table is registered under the provided name.
+    */
+  @throws[TableException]
+  protected def registerTableInternal(name: String, table: AbstractTable): Unit = {
+
+    if (isRegistered(name)) {
+      throw new TableException(s"Table \'$name\' already exists. " +
+        s"Please, choose a different name.")
+    } else {
+      tables.add(name, table)
+    }
+  }
+
+  /**
+    * Checks if the chosen table name is valid.
+    *
+    * @param name The table name to check.
+    */
+  protected def checkValidTableName(name: String): Unit
+
+  /**
+    * Checks if a table is registered under the given name.
+    *
+    * @param name The table name to check.
+    * @return true, if a table is registered under the name, false otherwise.
+    */
+  protected def isRegistered(name: String): Boolean = {
+    tables.getTableNames.contains(name)
+  }
+
+  protected def getRowType(name: String): RelDataType = {
+    tables.getTable(name).getRowType(typeFactory)
+  }
+
+  /** Returns a unique temporary attribute name. */
+  private[flink] def createUniqueAttributeName(): String = {
+    "TMP_" + attrNameCntr.getAndIncrement()
+  }
+
+  /** Returns the [[FlinkRelBuilder]] of this TableEnvironment. */
+  private[flink] def getRelBuilder: FlinkRelBuilder = {
+    relBuilder
+  }
+
+  /** Returns the Calcite [[org.apache.calcite.plan.RelOptPlanner]] of this TableEnvironment. */
+  private[flink] def getPlanner: RelOptPlanner = {
+    planner
+  }
+
+  /** Returns the [[FlinkTypeFactory]] of this TableEnvironment. */
+  private[flink] def getTypeFactory: FlinkTypeFactory = {
+    typeFactory
+  }
+
+  private[flink] def getFunctionCatalog: FunctionCatalog = {
+    functionCatalog
+  }
+
+  /** Returns the Calcite [[FrameworkConfig]] of this TableEnvironment. */
+  private[flink] def getFrameworkConfig: FrameworkConfig = {
+    frameworkConfig
+  }
+
+  protected def validateType(typeInfo: TypeInformation[_]): Unit = {
+    val clazz = typeInfo.getTypeClass
+    if ((clazz.isMemberClass && !Modifier.isStatic(clazz.getModifiers)) ||
+        !Modifier.isPublic(clazz.getModifiers) ||
+        clazz.getCanonicalName == null) {
+      throw TableException(s"Class '$clazz' described in type information '$typeInfo' must be " +
+        s"static and globally accessible.")
+    }
+  }
+
+  /**
+    * Returns field names and field positions for a given [[TypeInformation]].
+    *
+    * Field names are automatically extracted for
+    * [[org.apache.flink.api.common.typeutils.CompositeType]].
+    * The method fails if inputType is not a
+    * [[org.apache.flink.api.common.typeutils.CompositeType]].
+    *
+    * @param inputType The TypeInformation extract the field names and positions from.
+    * @tparam A The type of the TypeInformation.
+    * @return A tuple of two arrays holding the field names and corresponding field positions.
+    */
+  protected[flink] def getFieldInfo[A](inputType: TypeInformation[A]):
+      (Array[String], Array[Int]) =
+  {
+    validateType(inputType)
+
+    val fieldNames: Array[String] = inputType match {
+      case t: TupleTypeInfo[A] => t.getFieldNames
+      case c: CaseClassTypeInfo[A] => c.getFieldNames
+      case p: PojoTypeInfo[A] => p.getFieldNames
+      case r: RowTypeInfo => r.getFieldNames
+      case tpe =>
+        throw new TableException(s"Type $tpe lacks explicit field naming")
+    }
+    val fieldIndexes = fieldNames.indices.toArray
+
+    if (fieldNames.contains("*")) {
+      throw new TableException("Field name can not be '*'.")
+    }
+
+    (fieldNames, fieldIndexes)
+  }
+
+  /**
+    * Returns field names and field positions for a given [[TypeInformation]] and [[Array]] of
+    * [[Expression]].
+    *
+    * @param inputType The [[TypeInformation]] against which the [[Expression]]s are evaluated.
+    * @param exprs The expressions that define the field names.
+    * @tparam A The type of the TypeInformation.
+    * @return A tuple of two arrays holding the field names and corresponding field positions.
+    */
+  protected[flink] def getFieldInfo[A](
+    inputType: TypeInformation[A],
+    exprs: Array[Expression]): (Array[String], Array[Int]) = {
+
+    validateType(inputType)
+
+    val indexedNames: Array[(Int, String)] = inputType match {
+      case a: AtomicType[A] =>
+        if (exprs.length != 1) {
+          throw new TableException("Table of atomic type can only have a single field.")
+        }
+        exprs.map {
+          case UnresolvedFieldReference(name) => (0, name)
+          case _ => throw new TableException("Field reference expression expected.")
+        }
+      case t: TupleTypeInfo[A] =>
+        exprs.zipWithIndex.map {
+          case (UnresolvedFieldReference(name), idx) => (idx, name)
+          case (Alias(UnresolvedFieldReference(origName), name, _), _) =>
+            val idx = t.getFieldIndex(origName)
+            if (idx < 0) {
+              throw new TableException(s"$origName is not a field of type $t")
+            }
+            (idx, name)
+          case _ => throw new TableException(
+            "Field reference expression or alias on field expression expected.")
+        }
+      case c: CaseClassTypeInfo[A] =>
+        exprs.zipWithIndex.map {
+          case (UnresolvedFieldReference(name), idx) => (idx, name)
+          case (Alias(UnresolvedFieldReference(origName), name, _), _) =>
+            val idx = c.getFieldIndex(origName)
+            if (idx < 0) {
+              throw new TableException(s"$origName is not a field of type $c")
+            }
+            (idx, name)
+          case _ => throw new TableException(
+            "Field reference expression or alias on field expression expected.")
+        }
+      case p: PojoTypeInfo[A] =>
+        exprs.map {
+          case (UnresolvedFieldReference(name)) =>
+            val idx = p.getFieldIndex(name)
+            if (idx < 0) {
+              throw new TableException(s"$name is not a field of type $p")
+            }
+            (idx, name)
+          case Alias(UnresolvedFieldReference(origName), name, _) =>
+            val idx = p.getFieldIndex(origName)
+            if (idx < 0) {
+              throw new TableException(s"$origName is not a field of type $p")
+            }
+            (idx, name)
+          case _ => throw new TableException(
+            "Field reference expression or alias on field expression expected.")
+        }
+      case tpe => throw new TableException(
+        s"Source of type $tpe cannot be converted into Table.")
+    }
+
+    val (fieldIndexes, fieldNames) = indexedNames.unzip
+
+    if (fieldNames.contains("*")) {
+      throw new TableException("Field name can not be '*'.")
+    }
+
+    (fieldNames.toArray, fieldIndexes.toArray)
+  }
+
+}
+
+/**
+  * Object to instantiate a [[TableEnvironment]] depending on the batch or stream execution
+  * environment.
+  */
+object TableEnvironment {
+
+  /**
+    * Returns a [[JavaBatchTableEnv]] for a Java [[JavaBatchExecEnv]].
+    *
+    * @param executionEnvironment The Java batch ExecutionEnvironment.
+    */
+  def getTableEnvironment(executionEnvironment: JavaBatchExecEnv): JavaBatchTableEnv = {
+    new JavaBatchTableEnv(executionEnvironment, new TableConfig())
+  }
+
+  /**
+    * Returns a [[JavaBatchTableEnv]] for a Java [[JavaBatchExecEnv]] and a given [[TableConfig]].
+    *
+    * @param executionEnvironment The Java batch ExecutionEnvironment.
+    * @param tableConfig The TableConfig for the new TableEnvironment.
+    */
+  def getTableEnvironment(
+    executionEnvironment: JavaBatchExecEnv,
+    tableConfig: TableConfig): JavaBatchTableEnv = {
+
+    new JavaBatchTableEnv(executionEnvironment, tableConfig)
+  }
+
+  /**
+    * Returns a [[ScalaBatchTableEnv]] for a Scala [[ScalaBatchExecEnv]].
+    *
+    * @param executionEnvironment The Scala batch ExecutionEnvironment.
+    */
+  def getTableEnvironment(executionEnvironment: ScalaBatchExecEnv): ScalaBatchTableEnv = {
+    new ScalaBatchTableEnv(executionEnvironment, new TableConfig())
+  }
+
+  /**
+    * Returns a [[ScalaBatchTableEnv]] for a Scala [[ScalaBatchExecEnv]] and a given
+    * [[TableConfig]].
+    *
+    * @param executionEnvironment The Scala batch ExecutionEnvironment.
+    * @param tableConfig The TableConfig for the new TableEnvironment.
+    */
+  def getTableEnvironment(
+    executionEnvironment: ScalaBatchExecEnv,
+    tableConfig: TableConfig): ScalaBatchTableEnv = {
+
+    new ScalaBatchTableEnv(executionEnvironment, tableConfig)
+  }
+
+  /**
+    * Returns a [[JavaStreamTableEnv]] for a Java [[JavaStreamExecEnv]].
+    *
+    * @param executionEnvironment The Java StreamExecutionEnvironment.
+    */
+  def getTableEnvironment(executionEnvironment: JavaStreamExecEnv): JavaStreamTableEnv = {
+    new JavaStreamTableEnv(executionEnvironment, new TableConfig())
+  }
+
+  /**
+    * Returns a [[JavaStreamTableEnv]] for a Java [[JavaStreamExecEnv]] and a given [[TableConfig]].
+    *
+    * @param executionEnvironment The Java StreamExecutionEnvironment.
+    * @param tableConfig The TableConfig for the new TableEnvironment.
+    */
+  def getTableEnvironment(
+    executionEnvironment: JavaStreamExecEnv,
+    tableConfig: TableConfig): JavaStreamTableEnv = {
+
+    new JavaStreamTableEnv(executionEnvironment, tableConfig)
+  }
+
+  /**
+    * Returns a [[ScalaStreamTableEnv]] for a Scala stream [[ScalaStreamExecEnv]].
+    *
+    * @param executionEnvironment The Scala StreamExecutionEnvironment.
+    */
+  def getTableEnvironment(executionEnvironment: ScalaStreamExecEnv): ScalaStreamTableEnv = {
+    new ScalaStreamTableEnv(executionEnvironment, new TableConfig())
+  }
+
+  /**
+    * Returns a [[ScalaStreamTableEnv]] for a Scala stream [[ScalaStreamExecEnv]].
+    *
+    * @param executionEnvironment The Scala StreamExecutionEnvironment.
+    * @param tableConfig The TableConfig for the new TableEnvironment.
+    */
+  def getTableEnvironment(
+    executionEnvironment: ScalaStreamExecEnv,
+    tableConfig: TableConfig): ScalaStreamTableEnv = {
+
+    new ScalaStreamTableEnv(executionEnvironment, tableConfig)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala
new file mode 100644
index 0000000..939cb67
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/Types.scala
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api
+
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo}
+import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
+
+/**
+  * This class enumerates all supported types of the Table API.
+  */
+object Types {
+
+  val STRING = BasicTypeInfo.STRING_TYPE_INFO
+  val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO
+
+  val BYTE = BasicTypeInfo.BYTE_TYPE_INFO
+  val SHORT = BasicTypeInfo.SHORT_TYPE_INFO
+  val INT = BasicTypeInfo.INT_TYPE_INFO
+  val LONG = BasicTypeInfo.LONG_TYPE_INFO
+  val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO
+  val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO
+  val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO
+
+  val DATE = SqlTimeTypeInfo.DATE
+  val TIME = SqlTimeTypeInfo.TIME
+  val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP
+  val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS
+  val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala
new file mode 100644
index 0000000..7ca45a6
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/exceptions.scala
@@ -0,0 +1,73 @@
+/*
+ * 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
+
+/**
+  * Exception for all errors occurring during expression parsing.
+  */
+case class ExpressionParserException(msg: String) extends RuntimeException(msg)
+
+/**
+  * Exception for all errors occurring during sql parsing.
+  */
+case class SqlParserException(
+    msg: String,
+    cause: Throwable)
+  extends RuntimeException(msg, cause) {
+
+  def this(msg: String) = this(msg, null)
+
+}
+
+/**
+  * General Exception for all errors during table handling.
+  */
+case class TableException(
+    msg: String,
+    cause: Throwable)
+  extends RuntimeException(msg, cause) {
+
+  def this(msg: String) = this(msg, null)
+
+}
+
+object TableException {
+  def apply(msg: String): TableException = new TableException(msg)
+}
+
+/**
+  * Exception for all errors occurring during validation phase.
+  */
+case class ValidationException(
+    msg: String,
+    cause: Throwable)
+  extends RuntimeException(msg, cause) {
+
+  def this(msg: String) = this(msg, null)
+
+}
+
+object ValidationException {
+  def apply(msg: String): ValidationException = new ValidationException(msg)
+}
+
+/**
+  * Exception for unwanted method calling on unresolved expression.
+  */
+case class UnresolvedException(msg: String) extends RuntimeException(msg)

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala
new file mode 100644
index 0000000..15e842e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/BatchTableEnvironment.scala
@@ -0,0 +1,181 @@
+/*
+ * 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.java
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.TypeExtractor
+import org.apache.flink.api.java.{DataSet, ExecutionEnvironment}
+import org.apache.flink.table.expressions.ExpressionParser
+import org.apache.flink.table.api._
+import org.apache.flink.table.functions.TableFunction
+
+/**
+  * The [[TableEnvironment]] for a Java batch [[DataSet]]
+  * [[ExecutionEnvironment]].
+  *
+  * A TableEnvironment can be used to:
+  * - convert a [[DataSet]] to a [[Table]]
+  * - register a [[DataSet]] in the [[TableEnvironment]]'s catalog
+  * - register a [[Table]] in the [[TableEnvironment]]'s catalog
+  * - scan a registered table to obtain a [[Table]]
+  * - specify a SQL query on registered tables to obtain a [[Table]]
+  * - convert a [[Table]] into a [[DataSet]]
+  * - explain the AST and execution plan of a [[Table]]
+  *
+  * @param execEnv The Java batch [[ExecutionEnvironment]] of the TableEnvironment.
+  * @param config The configuration of the TableEnvironment.
+  */
+class BatchTableEnvironment(
+    execEnv: ExecutionEnvironment,
+    config: TableConfig)
+  extends org.apache.flink.table.api.BatchTableEnvironment(execEnv, config) {
+
+  /**
+    * Converts the given [[DataSet]] into a [[Table]].
+    *
+    * The field names of the [[Table]] are automatically derived from the type of the [[DataSet]].
+    *
+    * @param dataSet The [[DataSet]] to be converted.
+    * @tparam T The type of the [[DataSet]].
+    * @return The converted [[Table]].
+    */
+  def fromDataSet[T](dataSet: DataSet[T]): Table = {
+
+    val name = createUniqueTableName()
+    registerDataSetInternal(name, dataSet)
+    scan(name)
+  }
+
+  /**
+    * Converts the given [[DataSet]] into a [[Table]] with specified field names.
+    *
+    * Example:
+    *
+    * {{{
+    *   DataSet<Tuple2<String, Long>> set = ...
+    *   Table tab = tableEnv.fromDataSet(set, "a, b")
+    * }}}
+    *
+    * @param dataSet The [[DataSet]] to be converted.
+    * @param fields The field names of the resulting [[Table]].
+    * @tparam T The type of the [[DataSet]].
+    * @return The converted [[Table]].
+    */
+  def fromDataSet[T](dataSet: DataSet[T], fields: String): Table = {
+    val exprs = ExpressionParser
+      .parseExpressionList(fields)
+      .toArray
+
+    val name = createUniqueTableName()
+    registerDataSetInternal(name, dataSet, exprs)
+    scan(name)
+  }
+
+  /**
+    * Registers the given [[DataSet]] as table in the
+    * [[TableEnvironment]]'s catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * The field names of the [[Table]] are automatically derived from the type of the [[DataSet]].
+    *
+    * @param name The name under which the [[DataSet]] is registered in the catalog.
+    * @param dataSet The [[DataSet]] to register.
+    * @tparam T The type of the [[DataSet]] to register.
+    */
+  def registerDataSet[T](name: String, dataSet: DataSet[T]): Unit = {
+
+    checkValidTableName(name)
+    registerDataSetInternal(name, dataSet)
+  }
+
+  /**
+    * Registers the given [[DataSet]] as table with specified field names in the
+    * [[TableEnvironment]]'s catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * Example:
+    *
+    * {{{
+    *   DataSet<Tuple2<String, Long>> set = ...
+    *   tableEnv.registerDataSet("myTable", set, "a, b")
+    * }}}
+    *
+    * @param name The name under which the [[DataSet]] is registered in the catalog.
+    * @param dataSet The [[DataSet]] to register.
+    * @param fields The field names of the registered table.
+    * @tparam T The type of the [[DataSet]] to register.
+    */
+  def registerDataSet[T](name: String, dataSet: DataSet[T], fields: String): Unit = {
+    val exprs = ExpressionParser
+      .parseExpressionList(fields)
+      .toArray
+
+    checkValidTableName(name)
+    registerDataSetInternal(name, dataSet, exprs)
+  }
+
+  /**
+    * Converts the given [[Table]] into a [[DataSet]] of a specified type.
+    *
+    * The fields of the [[Table]] are mapped to [[DataSet]] fields as follows:
+    * - [[Row]] and [[org.apache.flink.api.java.tuple.Tuple]]
+    * types: Fields are mapped by position, field types must match.
+    * - POJO [[DataSet]] 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 [[DataSet]].
+    * @tparam T The type of the resulting [[DataSet]].
+    * @return The converted [[DataSet]].
+    */
+  def toDataSet[T](table: Table, clazz: Class[T]): DataSet[T] = {
+    translate[T](table)(TypeExtractor.createTypeInfo(clazz))
+  }
+
+  /**
+    * Converts the given [[Table]] into a [[DataSet]] of a specified type.
+    *
+    * The fields of the [[Table]] are mapped to [[DataSet]] fields as follows:
+    * - [[Row]] and [[org.apache.flink.api.java.tuple.Tuple]]
+    * types: Fields are mapped by position, field types must match.
+    * - POJO [[DataSet]] 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 resulting [[DataSet]].
+    * @tparam T The type of the resulting [[DataSet]].
+    * @return The converted [[DataSet]].
+    */
+  def toDataSet[T](table: Table, typeInfo: TypeInformation[T]): DataSet[T] = {
+    translate[T](table)(typeInfo)
+  }
+
+  /**
+    * Registers a [[TableFunction]] under a unique name in the TableEnvironment's catalog.
+    * Registered functions can be referenced in Table API and SQL queries.
+    *
+    * @param name The name under which the function is registered.
+    * @param tf The TableFunction to register.
+    * @tparam T The type of the output row.
+    */
+  def registerFunction[T](name: String, tf: TableFunction[T]): Unit = {
+    implicit val typeInfo: TypeInformation[T] = TypeExtractor
+      .createTypeInfo(tf, classOf[TableFunction[_]], tf.getClass, 0)
+      .asInstanceOf[TypeInformation[T]]
+
+    registerTableFunctionInternal[T](name, tf)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..428dcae
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/StreamTableEnvironment.scala
@@ -0,0 +1,183 @@
+/*
+ * 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.java
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.TypeExtractor
+import org.apache.flink.table.api._
+import org.apache.flink.table.functions.TableFunction
+import org.apache.flink.table.expressions.ExpressionParser
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+
+/**
+  * The [[TableEnvironment]] for a Java [[StreamExecutionEnvironment]].
+  *
+  * A TableEnvironment can be used to:
+  * - convert a [[DataStream]] to a [[Table]]
+  * - register a [[DataStream]] in the [[TableEnvironment]]'s catalog
+  * - register a [[Table]] in the [[TableEnvironment]]'s catalog
+  * - scan a registered table to obtain a [[Table]]
+  * - specify a SQL query on registered tables to obtain a [[Table]]
+  * - convert a [[Table]] into a [[DataStream]]
+  * - explain the AST and execution plan of a [[Table]]
+  *
+  * @param execEnv The Java [[StreamExecutionEnvironment]] of the TableEnvironment.
+  * @param config The configuration of the TableEnvironment.
+  */
+class StreamTableEnvironment(
+    execEnv: StreamExecutionEnvironment,
+    config: TableConfig)
+  extends org.apache.flink.table.api.StreamTableEnvironment(execEnv, config) {
+
+  /**
+    * Converts the given [[DataStream]] into a [[Table]].
+    *
+    * The field names of the [[Table]] are automatically derived from the type of the
+    * [[DataStream]].
+    *
+    * @param dataStream The [[DataStream]] to be converted.
+    * @tparam T The type of the [[DataStream]].
+    * @return The converted [[Table]].
+    */
+  def fromDataStream[T](dataStream: DataStream[T]): Table = {
+
+    val name = createUniqueTableName()
+    registerDataStreamInternal(name, dataStream)
+    ingest(name)
+  }
+
+  /**
+    * Converts the given [[DataStream]] into a [[Table]] with specified field names.
+    *
+    * Example:
+    *
+    * {{{
+    *   DataStream<Tuple2<String, Long>> stream = ...
+    *   Table tab = tableEnv.fromDataStream(stream, "a, b")
+    * }}}
+    *
+    * @param dataStream The [[DataStream]] to be converted.
+    * @param fields The field names of the resulting [[Table]].
+    * @tparam T The type of the [[DataStream]].
+    * @return The converted [[Table]].
+    */
+  def fromDataStream[T](dataStream: DataStream[T], fields: String): Table = {
+    val exprs = ExpressionParser
+      .parseExpressionList(fields)
+      .toArray
+
+    val name = createUniqueTableName()
+    registerDataStreamInternal(name, dataStream, exprs)
+    ingest(name)
+  }
+
+  /**
+    * Registers the given [[DataStream]] as table in the
+    * [[TableEnvironment]]'s catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * The field names of the [[Table]] are automatically derived
+    * from the type of the [[DataStream]].
+    *
+    * @param name The name under which the [[DataStream]] is registered in the catalog.
+    * @param dataStream The [[DataStream]] to register.
+    * @tparam T The type of the [[DataStream]] to register.
+    */
+  def registerDataStream[T](name: String, dataStream: DataStream[T]): Unit = {
+
+    checkValidTableName(name)
+    registerDataStreamInternal(name, dataStream)
+  }
+
+  /**
+    * Registers the given [[DataStream]] as table with specified field names in the
+    * [[TableEnvironment]]'s catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * Example:
+    *
+    * {{{
+    *   DataStream<Tuple2<String, Long>> set = ...
+    *   tableEnv.registerDataStream("myTable", set, "a, b")
+    * }}}
+    *
+    * @param name The name under which the [[DataStream]] is registered in the catalog.
+    * @param dataStream The [[DataStream]] to register.
+    * @param fields The field names of the registered table.
+    * @tparam T The type of the [[DataStream]] to register.
+    */
+  def registerDataStream[T](name: String, dataStream: DataStream[T], fields: String): Unit = {
+    val exprs = ExpressionParser
+      .parseExpressionList(fields)
+      .toArray
+
+    checkValidTableName(name)
+    registerDataStreamInternal(name, dataStream, exprs)
+  }
+
+  /**
+    * Converts the given [[Table]] into a [[DataStream]] of a specified type.
+    *
+    * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows:
+    * - [[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]].
+    * @tparam T The type of the resulting [[DataStream]].
+    * @return The converted [[DataStream]].
+    */
+  def toDataStream[T](table: Table, clazz: Class[T]): DataStream[T] = {
+    translate[T](table)(TypeExtractor.createTypeInfo(clazz))
+  }
+
+  /**
+    * Converts the given [[Table]] into a [[DataStream]] of a specified type.
+    *
+    * The fields of the [[Table]] are mapped to [[DataStream]] fields as follows:
+    * - [[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] = {
+    translate[T](table)(typeInfo)
+  }
+
+  /**
+    * Registers a [[TableFunction]] under a unique name in the TableEnvironment's catalog.
+    * Registered functions can be referenced in Table API and SQL queries.
+    *
+    * @param name The name under which the function is registered.
+    * @param tf The TableFunction to register.
+    * @tparam T The type of the output row.
+    */
+  def registerFunction[T](name: String, tf: TableFunction[T]): Unit = {
+    implicit val typeInfo: TypeInformation[T] = TypeExtractor
+      .createTypeInfo(tf, classOf[TableFunction[_]], tf.getClass, 0)
+      .asInstanceOf[TypeInformation[T]]
+
+    registerTableFunctionInternal[T](name, tf)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/groupWindows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/groupWindows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/groupWindows.scala
new file mode 100644
index 0000000..9c82e9b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/java/groupWindows.scala
@@ -0,0 +1,84 @@
+/*
+ * 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.java
+
+import org.apache.flink.table.api.{SessionWindow, SlideWithSize, TumblingWindow}
+
+/**
+  * Helper class for creating a tumbling window. Tumbling windows are consecutive, non-overlapping
+  * windows of a specified fixed length. For example, a tumbling window of 5 minutes size groups
+  * elements in 5 minutes intervals.
+  */
+object Tumble {
+
+  /**
+    * Creates a tumbling window. Tumbling windows are consecutive, non-overlapping
+    * windows of a specified fixed length. For example, a tumbling window of 5 minutes size groups
+    * elements in 5 minutes intervals.
+    *
+    * @param size the size of the window as time or row-count interval.
+    * @return a tumbling window
+    */
+  def over(size: String): TumblingWindow = new TumblingWindow(size)
+}
+
+/**
+  * Helper class for creating a sliding window. Sliding windows have a fixed size and slide by
+  * a specified slide interval. If the slide interval is smaller than the window size, sliding
+  * windows are overlapping. Thus, an element can be assigned to multiple windows.
+  *
+  * For example, a sliding window of size 15 minutes with 5 minutes sliding interval groups elements
+  * of 15 minutes and evaluates every five minutes. Each element is contained in three consecutive
+  * window evaluations.
+  */
+object Slide {
+
+  /**
+    * Creates a sliding window. Sliding windows have a fixed size and slide by
+    * a specified slide interval. If the slide interval is smaller than the window size, sliding
+    * windows are overlapping. Thus, an element can be assigned to multiple windows.
+    *
+    * For example, a sliding window of size 15 minutes with 5 minutes sliding interval groups
+    * elements of 15 minutes and evaluates every five minutes. Each element is contained in three
+    * consecutive window evaluations.
+    *
+    * @param size the size of the window as time or row-count interval
+    * @return a partially specified sliding window
+    */
+  def over(size: String): SlideWithSize = new SlideWithSize(size)
+}
+
+/**
+  * Helper class for creating a session window. The boundary of session windows are defined by
+  * intervals of inactivity, i.e., a session window is closes if no event appears for a defined
+  * gap period.
+  */
+object Session {
+
+  /**
+    * Creates a session window. The boundary of session windows are defined by
+    * intervals of inactivity, i.e., a session window is closes if no event appears for a defined
+    * gap period.
+    *
+    * @param gap specifies how long (as interval of milliseconds) to wait for new data before
+    *            closing the session window.
+    * @return a session window
+    */
+  def withGap(gap: String): SessionWindow = new SessionWindow(gap)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/package.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/package.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/package.scala
new file mode 100644
index 0000000..e16e1a8
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/package.scala
@@ -0,0 +1,34 @@
+/*
+ * 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
+
+/**
+ * == Table API ==
+ *
+ * This package contains the API of the Table API. It can be used with Flink Streaming
+ * and Flink Batch. From Scala as well as from Java.
+ *
+ * When using the Table API, as user creates a [[org.apache.flink.table.api.Table]] from
+ * a DataSet or DataStream. On this relational operations can be performed. A table can also
+ * be converted back to a DataSet or DataStream.
+ *
+ * Packages [[org.apache.flink.table.api.scala]] and [[org.apache.flink.table.api.java]] contain
+ * the language specific part of the API. Refer to these packages for documentation on how
+ * the Table API can be used in Java and Scala.
+ */
+package object api

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala
new file mode 100644
index 0000000..3ae8c31
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/BatchTableEnvironment.scala
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api._
+import org.apache.flink.table.expressions.Expression
+import org.apache.flink.table.functions.TableFunction
+
+import _root_.scala.reflect.ClassTag
+
+/**
+  * The [[TableEnvironment]] for a Scala batch [[DataSet]]
+  * [[ExecutionEnvironment]].
+  *
+  * A TableEnvironment can be used to:
+  * - convert a [[DataSet]] to a [[Table]]
+  * - register a [[DataSet]] in the [[TableEnvironment]]'s catalog
+  * - register a [[Table]] in the [[TableEnvironment]]'s catalog
+  * - scan a registered table to obtain a [[Table]]
+  * - specify a SQL query on registered tables to obtain a [[Table]]
+  * - convert a [[Table]] into a [[DataSet]]
+  * - explain the AST and execution plan of a [[Table]]
+  *
+  * @param execEnv The Scala batch [[ExecutionEnvironment]] of the TableEnvironment.
+  * @param config The configuration of the TableEnvironment.
+  */
+class BatchTableEnvironment(
+    execEnv: ExecutionEnvironment,
+    config: TableConfig)
+  extends org.apache.flink.table.api.BatchTableEnvironment(execEnv.getJavaEnv, config) {
+
+  /**
+    * Converts the given [[DataSet]] into a [[Table]].
+    *
+    * The field names of the [[Table]] are automatically derived from the type of the [[DataSet]].
+    *
+    * @param dataSet The [[DataSet]] to be converted.
+    * @tparam T The type of the [[DataSet]].
+    * @return The converted [[Table]].
+    */
+  def fromDataSet[T](dataSet: DataSet[T]): Table = {
+
+    val name = createUniqueTableName()
+    registerDataSetInternal(name, dataSet.javaSet)
+    scan(name)
+  }
+
+  /**
+    * Converts the given [[DataSet]] into a [[Table]] with specified field names.
+    *
+    * Example:
+    *
+    * {{{
+    *   val set: DataSet[(String, Long)] = ...
+    *   val tab: Table = tableEnv.fromDataSet(set, 'a, 'b)
+    * }}}
+    *
+    * @param dataSet The [[DataSet]] to be converted.
+    * @param fields The field names of the resulting [[Table]].
+    * @tparam T The type of the [[DataSet]].
+    * @return The converted [[Table]].
+    */
+  def fromDataSet[T](dataSet: DataSet[T], fields: Expression*): Table = {
+
+    val name = createUniqueTableName()
+    registerDataSetInternal(name, dataSet.javaSet, fields.toArray)
+    scan(name)
+  }
+
+  /**
+    * Registers the given [[DataSet]] as table in the
+    * [[TableEnvironment]]'s catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * The field names of the [[Table]] are automatically derived from the type of the [[DataSet]].
+    *
+    * @param name The name under which the [[DataSet]] is registered in the catalog.
+    * @param dataSet The [[DataSet]] to register.
+    * @tparam T The type of the [[DataSet]] to register.
+    */
+  def registerDataSet[T](name: String, dataSet: DataSet[T]): Unit = {
+
+    checkValidTableName(name)
+    registerDataSetInternal(name, dataSet.javaSet)
+  }
+
+  /**
+    * Registers the given [[DataSet]] as table with specified field names in the
+    * [[TableEnvironment]]'s catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * Example:
+    *
+    * {{{
+    *   val set: DataSet[(String, Long)] = ...
+    *   tableEnv.registerDataSet("myTable", set, 'a, 'b)
+    * }}}
+    *
+    * @param name The name under which the [[DataSet]] is registered in the catalog.
+    * @param dataSet The [[DataSet]] to register.
+    * @param fields The field names of the registered table.
+    * @tparam T The type of the [[DataSet]] to register.
+    */
+  def registerDataSet[T](name: String, dataSet: DataSet[T], fields: Expression*): Unit = {
+
+    checkValidTableName(name)
+    registerDataSetInternal(name, dataSet.javaSet, fields.toArray)
+  }
+
+  /**
+    * Converts the given [[Table]] into a [[DataSet]] of a specified type.
+    *
+    * The fields of the [[Table]] are mapped to [[DataSet]] 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 [[DataSet]] types: Fields are mapped by field name, field types must match.
+    *
+    * @param table The [[Table]] to convert.
+    * @tparam T The type of the resulting [[DataSet]].
+    * @return The converted [[DataSet]].
+    */
+  def toDataSet[T: TypeInformation](table: Table): DataSet[T] = {
+    wrap[T](translate(table))(ClassTag.AnyRef.asInstanceOf[ClassTag[T]])
+  }
+
+  /**
+    * Registers a [[TableFunction]] under a unique name in the TableEnvironment's catalog.
+    * Registered functions can be referenced in Table API and SQL queries.
+    *
+    * @param name The name under which the function is registered.
+    * @param tf The TableFunction to register.
+    * @tparam T The type of the output row.
+    */
+  def registerFunction[T: TypeInformation](name: String, tf: TableFunction[T]): Unit = {
+    registerTableFunctionInternal(name, tf)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/DataSetConversions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/DataSetConversions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/DataSetConversions.scala
new file mode 100644
index 0000000..4b92bdb
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/DataSetConversions.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.Table
+import org.apache.flink.table.expressions.Expression
+
+/**
+  * Holds methods to convert a [[DataSet]] into a [[Table]].
+  *
+  * @param dataSet The [[DataSet]] to convert.
+  * @param inputType The [[TypeInformation]] for the type of the [[DataSet]].
+  * @tparam T The type of the [[DataSet]].
+  */
+class DataSetConversions[T](dataSet: DataSet[T], inputType: TypeInformation[T]) {
+
+  /**
+    * Converts the [[DataSet]] into a [[Table]].
+    *
+    * The field name of the new [[Table]] can be specified like this:
+    *
+    * {{{
+    *   val env = ExecutionEnvironment.getExecutionEnvironment
+    *   val tEnv = TableEnvironment.getTableEnvironment(env)
+    *
+    *   val set: DataSet[(String, Int)] = ...
+    *   val table = set.toTable(tEnv, 'name, 'amount)
+    * }}}
+    *
+    * If not explicitly specified, field names are automatically extracted from the type of
+    * the [[DataSet]].
+    *
+    * @param tableEnv The [[BatchTableEnvironment]] in which the new [[Table]] is created.
+    * @param fields The field names of the new [[Table]] (optional).
+    * @return The resulting [[Table]].
+    */
+  def toTable(tableEnv: BatchTableEnvironment, fields: Expression*): Table = {
+    if (fields.isEmpty) {
+      tableEnv.fromDataSet(dataSet)
+    } else {
+      tableEnv.fromDataSet(dataSet, fields: _*)
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/DataStreamConversions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/DataStreamConversions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/DataStreamConversions.scala
new file mode 100644
index 0000000..89f7627
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/DataStreamConversions.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.Table
+import org.apache.flink.table.expressions.Expression
+import org.apache.flink.streaming.api.scala.DataStream
+
+/**
+  * Holds methods to convert a [[DataStream]] into a [[Table]].
+  *
+  * @param dataStream The [[DataStream]] to convert.
+  * @param inputType The [[TypeInformation]] for the type of the [[DataStream]].
+  * @tparam T The type of the [[DataStream]].
+  */
+class DataStreamConversions[T](dataStream: DataStream[T], inputType: TypeInformation[T]) {
+
+  /**
+    * Converts the [[DataStream]] into a [[Table]].
+    *
+    * The field name of the new [[Table]] can be specified like this:
+    *
+    * {{{
+    *   val env = StreamExecutionEnvironment.getExecutionEnvironment
+    *   val tEnv = TableEnvironment.getTableEnvironment(env)
+    *
+    *   val stream: DataStream[(String, Int)] = ...
+    *   val table = stream.toTable(tEnv, 'name, 'amount)
+    * }}}
+    *
+    * If not explicitly specified, field names are automatically extracted from the type of
+    * the [[DataStream]].
+    *
+    * @param tableEnv The [[StreamTableEnvironment]] in which the new [[Table]] is created.
+    * @param fields The field names of the new [[Table]] (optional).
+    * @return The resulting [[Table]].
+    */
+  def toTable(tableEnv: StreamTableEnvironment, fields: Expression*): Table = {
+    if (fields.isEmpty) {
+      tableEnv.fromDataStream(dataStream)
+    } else {
+      tableEnv.fromDataStream(dataStream, fields:_*)
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..1e6749e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/StreamTableEnvironment.scala
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.{TableEnvironment, Table, TableConfig}
+import org.apache.flink.table.functions.TableFunction
+import org.apache.flink.table.expressions.Expression
+import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
+import org.apache.flink.streaming.api.scala.asScalaStream
+
+/**
+  * The [[TableEnvironment]] for a Scala [[StreamExecutionEnvironment]].
+  *
+  * A TableEnvironment can be used to:
+  * - convert a [[DataStream]] to a [[Table]]
+  * - register a [[DataStream]] in the [[TableEnvironment]]'s catalog
+  * - register a [[Table]] in the [[TableEnvironment]]'s catalog
+  * - scan a registered table to obtain a [[Table]]
+  * - specify a SQL query on registered tables to obtain a [[Table]]
+  * - convert a [[Table]] into a [[DataStream]]
+  * - explain the AST and execution plan of a [[Table]]
+  *
+  * @param execEnv The Scala [[StreamExecutionEnvironment]] of the TableEnvironment.
+  * @param config The configuration of the TableEnvironment.
+  */
+class StreamTableEnvironment(
+    execEnv: StreamExecutionEnvironment,
+    config: TableConfig)
+  extends org.apache.flink.table.api.StreamTableEnvironment(
+    execEnv.getWrappedStreamExecutionEnvironment,
+    config) {
+
+  /**
+    * Converts the given [[DataStream]] into a [[Table]].
+    *
+    * The field names of the [[Table]] are automatically derived from the type of the
+    * [[DataStream]].
+    *
+    * @param dataStream The [[DataStream]] to be converted.
+    * @tparam T The type of the [[DataStream]].
+    * @return The converted [[Table]].
+    */
+  def fromDataStream[T](dataStream: DataStream[T]): Table = {
+
+    val name = createUniqueTableName()
+    registerDataStreamInternal(name, dataStream.javaStream)
+    ingest(name)
+  }
+
+  /**
+    * Converts the given [[DataStream]] into a [[Table]] with specified field names.
+    *
+    * Example:
+    *
+    * {{{
+    *   val stream: DataStream[(String, Long)] = ...
+    *   val tab: Table = tableEnv.fromDataStream(stream, 'a, 'b)
+    * }}}
+    *
+    * @param dataStream The [[DataStream]] to be converted.
+    * @param fields The field names of the resulting [[Table]].
+    * @tparam T The type of the [[DataStream]].
+    * @return The converted [[Table]].
+    */
+  def fromDataStream[T](dataStream: DataStream[T], fields: Expression*): Table = {
+
+    val name = createUniqueTableName()
+    registerDataStreamInternal(name, dataStream.javaStream, fields.toArray)
+    ingest(name)
+  }
+
+  /**
+    * Registers the given [[DataStream]] as table in the
+    * [[TableEnvironment]]'s catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * The field names of the [[Table]] are automatically derived
+    * from the type of the [[DataStream]].
+    *
+    * @param name The name under which the [[DataStream]] is registered in the catalog.
+    * @param dataStream The [[DataStream]] to register.
+    * @tparam T The type of the [[DataStream]] to register.
+    */
+  def registerDataStream[T](name: String, dataStream: DataStream[T]): Unit = {
+
+    checkValidTableName(name)
+    registerDataStreamInternal(name, dataStream.javaStream)
+  }
+
+  /**
+    * Registers the given [[DataStream]] as table with specified field names in the
+    * [[TableEnvironment]]'s catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * Example:
+    *
+    * {{{
+    *   val set: DataStream[(String, Long)] = ...
+    *   tableEnv.registerDataStream("myTable", set, 'a, 'b)
+    * }}}
+    *
+    * @param name The name under which the [[DataStream]] is registered in the catalog.
+    * @param dataStream The [[DataStream]] to register.
+    * @param fields The field names of the registered table.
+    * @tparam T The type of the [[DataStream]] to register.
+    */
+  def registerDataStream[T](name: String, dataStream: DataStream[T], fields: Expression*): Unit = {
+
+    checkValidTableName(name)
+    registerDataStreamInternal(name, dataStream.javaStream, fields.toArray)
+  }
+
+  /**
+    * Converts the given [[Table]] into a [[DataStream]] of a specified type.
+    *
+    * 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.
+    * @tparam T The type of the resulting [[DataStream]].
+    * @return The converted [[DataStream]].
+    */
+  def toDataStream[T: TypeInformation](table: Table): DataStream[T] = {
+    asScalaStream(translate(table))
+  }
+
+  /**
+    * Registers a [[TableFunction]] under a unique name in the TableEnvironment's catalog.
+    * Registered functions can be referenced in SQL queries.
+    *
+    * @param name The name under which the function is registered.
+    * @param tf The TableFunction to register
+    */
+  def registerFunction[T: TypeInformation](name: String, tf: TableFunction[T]): Unit = {
+    registerTableFunctionInternal(name, tf)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..2a0d571
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/TableConversions.scala
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.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.scala.{BatchTableEnvironment => ScalaBatchTableEnv}
+import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnv}
+
+/**
+  * Holds methods to convert a [[Table]] into a [[DataSet]] or a [[DataStream]].
+  *
+  * @param table The table to convert.
+  */
+class TableConversions(table: Table) {
+
+  /** Converts the [[Table]] to a [[DataSet]] of the specified type. */
+  def toDataSet[T: TypeInformation]: DataSet[T] = {
+
+    table.tableEnv match {
+      case tEnv: ScalaBatchTableEnv =>
+        tEnv.toDataSet(table)
+      case _ =>
+        throw new TableException(
+          "Only tables that originate from Scala DataSets can be converted to Scala DataSets.")
+    }
+  }
+
+  /** Converts the [[Table]] to a [[DataStream]] of the specified type. */
+  def toDataStream[T: TypeInformation]: DataStream[T] = {
+
+    table.tableEnv match {
+      case tEnv: ScalaStreamTableEnv =>
+        tEnv.toDataStream(table)
+      case _ =>
+        throw new TableException(
+          "Only tables that originate from Scala DataStreams " +
+            "can be converted to Scala DataStreams.")
+    }
+  }
+
+}
+


[02/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala
new file mode 100644
index 0000000..098feba
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarOperatorsTest.scala
@@ -0,0 +1,220 @@
+/*
+ * 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.expressions
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{Types, ValidationException}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.utils.ExpressionTestBase
+import org.junit.Test
+
+class ScalarOperatorsTest extends ExpressionTestBase {
+
+  @Test
+  def testCasting(): Unit = {
+    // test casting
+    // * -> String
+    testTableApi('f2.cast(Types.STRING), "f2.cast(STRING)", "1")
+    testTableApi('f5.cast(Types.STRING), "f5.cast(STRING)", "1.0")
+    testTableApi('f3.cast(Types.STRING), "f3.cast(STRING)", "1")
+    testTableApi('f6.cast(Types.STRING), "f6.cast(STRING)", "true")
+    // NUMERIC TYPE -> Boolean
+    testTableApi('f2.cast(Types.BOOLEAN), "f2.cast(BOOLEAN)", "true")
+    testTableApi('f7.cast(Types.BOOLEAN), "f7.cast(BOOLEAN)", "false")
+    testTableApi('f3.cast(Types.BOOLEAN), "f3.cast(BOOLEAN)", "true")
+    // NUMERIC TYPE -> NUMERIC TYPE
+    testTableApi('f2.cast(Types.DOUBLE), "f2.cast(DOUBLE)", "1.0")
+    testTableApi('f7.cast(Types.INT), "f7.cast(INT)", "0")
+    testTableApi('f3.cast(Types.SHORT), "f3.cast(SHORT)", "1")
+    // Boolean -> NUMERIC TYPE
+    testTableApi('f6.cast(Types.DOUBLE), "f6.cast(DOUBLE)", "1.0")
+    // identity casting
+    testTableApi('f2.cast(Types.INT), "f2.cast(INT)", "1")
+    testTableApi('f7.cast(Types.DOUBLE), "f7.cast(DOUBLE)", "0.0")
+    testTableApi('f3.cast(Types.LONG), "f3.cast(LONG)", "1")
+    testTableApi('f6.cast(Types.BOOLEAN), "f6.cast(BOOLEAN)", "true")
+    // String -> BASIC TYPE (not String, Date, Void, Character)
+    testTableApi('f2.cast(Types.BYTE), "f2.cast(BYTE)", "1")
+    testTableApi('f2.cast(Types.SHORT), "f2.cast(SHORT)", "1")
+    testTableApi('f2.cast(Types.INT), "f2.cast(INT)", "1")
+    testTableApi('f2.cast(Types.LONG), "f2.cast(LONG)", "1")
+    testTableApi('f3.cast(Types.DOUBLE), "f3.cast(DOUBLE)", "1.0")
+    testTableApi('f3.cast(Types.FLOAT), "f3.cast(FLOAT)", "1.0")
+    testTableApi('f5.cast(Types.BOOLEAN), "f5.cast(BOOLEAN)", "true")
+
+    // numeric auto cast in arithmetic
+    testTableApi('f0 + 1, "f0 + 1", "2")
+    testTableApi('f1 + 1, "f1 + 1", "2")
+    testTableApi('f2 + 1L, "f2 + 1L", "2")
+    testTableApi('f3 + 1.0f, "f3 + 1.0f", "2.0")
+    testTableApi('f3 + 1.0d, "f3 + 1.0d", "2.0")
+    testTableApi('f5 + 1, "f5 + 1", "2.0")
+    testTableApi('f3 + 1.0d, "f3 + 1.0d", "2.0")
+    testTableApi('f4 + 'f0, "f4 + f0", "2.0")
+
+    // numeric auto cast in comparison
+    testTableApi(
+      'f0 > 0 && 'f1 > 0 && 'f2 > 0L && 'f4 > 0.0f && 'f5 > 0.0d  && 'f3 > 0,
+      "f0 > 0 && f1 > 0 && f2 > 0L && f4 > 0.0f && f5 > 0.0d  && f3 > 0",
+      "true")
+  }
+
+  @Test
+  def testArithmetic(): Unit = {
+    // math arthmetic
+    testTableApi('f8 - 5, "f8 - 5", "0")
+    testTableApi('f8 + 5, "f8 + 5", "10")
+    testTableApi('f8 / 2, "f8 / 2", "2")
+    testTableApi('f8 * 2, "f8 * 2", "10")
+    testTableApi('f8 % 2, "f8 % 2", "1")
+    testTableApi(-'f8, "-f8", "-5")
+    testTableApi(3.toExpr + 'f8, "3 + f8", "8")
+
+    // boolean arithmetic
+    testTableApi('f6 && true, "f6 && true", "true")
+    testTableApi('f6 && false, "f6 && false", "false")
+    testTableApi('f6 || false, "f6 || false", "true")
+    testTableApi(!'f6, "!f6", "false")
+
+    // comparison
+    testTableApi('f8 > 'f2, "f8 > f2", "true")
+    testTableApi('f8 >= 'f8, "f8 >= f8", "true")
+    testTableApi('f8 < 'f2, "f8 < f2", "false")
+    testTableApi('f8.isNull, "f8.isNull", "false")
+    testTableApi('f8.isNotNull, "f8.isNotNull", "true")
+    testTableApi(12.toExpr <= 'f8, "12 <= f8", "false")
+
+    // string arithmetic
+    testTableApi(42.toExpr + 'f10 + 'f9, "42 + f10 + f9", "42String10")
+    testTableApi('f10 + 'f9, "f10 + f9", "String10")
+  }
+
+  @Test
+  def testOtherExpressions(): Unit = {
+    // null
+    testAllApis(Null(Types.INT), "Null(INT)", "CAST(NULL AS INT)", "null")
+    testAllApis(
+      Null(Types.STRING) === "",
+      "Null(STRING) === ''",
+      "CAST(NULL AS VARCHAR) = ''",
+      "null")
+
+    // if
+    testTableApi(('f6 && true).?("true", "false"), "(f6 && true).?('true', 'false')", "true")
+    testTableApi(false.?("true", "false"), "false.?('true', 'false')", "false")
+    testTableApi(
+      true.?(true.?(true.?(10, 4), 4), 4),
+      "true.?(true.?(true.?(10, 4), 4), 4)",
+      "10")
+    testTableApi(true, "?((f6 && true), 'true', 'false')", "true")
+    testSqlApi("CASE 11 WHEN 1 THEN 'a' ELSE 'b' END", "b")
+    testSqlApi("CASE 2 WHEN 1 THEN 'a' ELSE 'b' END", "b")
+    testSqlApi(
+      "CASE 1 WHEN 1, 2 THEN '1 or 2' WHEN 2 THEN 'not possible' WHEN 3, 2 THEN '3' " +
+      "ELSE 'none of the above' END",
+      "1 or 2           ")
+    testSqlApi("CASE WHEN 'a'='a' THEN 1 END", "1")
+    testSqlApi("CASE 2 WHEN 1 THEN 'a' WHEN 2 THEN 'bcd' END", "bcd")
+    testSqlApi("CASE f2 WHEN 1 THEN 11 WHEN 2 THEN 4 ELSE NULL END", "11")
+    testSqlApi("CASE f7 WHEN 1 THEN 11 WHEN 2 THEN 4 ELSE NULL END", "null")
+    testSqlApi("CASE 42 WHEN 1 THEN 'a' WHEN 2 THEN 'bcd' END", "null")
+    testSqlApi("CASE 1 WHEN 1 THEN true WHEN 2 THEN false ELSE NULL END", "true")
+
+    // case insensitive as
+    testTableApi(5 as 'test, "5 As test", "5")
+
+    // complex expressions
+    testTableApi('f0.isNull.isNull, "f0.isNull().isNull", "false")
+    testTableApi(
+      'f8.abs() + 'f8.abs().abs().abs().abs(),
+      "f8.abs() + f8.abs().abs().abs().abs()",
+      "10")
+    testTableApi(
+      'f8.cast(Types.STRING) + 'f8.cast(Types.STRING),
+      "f8.cast(STRING) + f8.cast(STRING)",
+      "55")
+    testTableApi('f8.isNull.cast(Types.INT), "CAST(ISNULL(f8), INT)", "0")
+    testTableApi(
+      'f8.cast(Types.INT).abs().isNull === false,
+      "ISNULL(CAST(f8, INT).abs()) === false",
+      "true")
+    testTableApi(
+      (((true === true) || false).cast(Types.STRING) + "X ").trim(),
+      "((((true) === true) || false).cast(STRING) + 'X ').trim",
+      "trueX")
+    testTableApi(12.isNull, "12.isNull", "false")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testIfInvalidTypesScala(): Unit = {
+    testTableApi(('f6 && true).?(5, "false"), "FAIL", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testIfInvalidTypesJava(): Unit = {
+    testTableApi("FAIL", "(f8 && true).?(5, 'false')", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidStringComparison1(): Unit = {
+    testTableApi("w" === 4, "FAIL", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidStringComparison2(): Unit = {
+    testTableApi("w" > 4.toExpr, "FAIL", "FAIL")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  def testData = {
+    val testData = new Row(11)
+    testData.setField(0, 1: Byte)
+    testData.setField(1, 1: Short)
+    testData.setField(2, 1)
+    testData.setField(3, 1L)
+    testData.setField(4, 1.0f)
+    testData.setField(5, 1.0d)
+    testData.setField(6, true)
+    testData.setField(7, 0.0d)
+    testData.setField(8, 5)
+    testData.setField(9, 10)
+    testData.setField(10, "String")
+    testData
+  }
+
+  def typeInfo = {
+    new RowTypeInfo(
+      Types.BYTE,
+      Types.SHORT,
+      Types.INT,
+      Types.LONG,
+      Types.FLOAT,
+      Types.DOUBLE,
+      Types.BOOLEAN,
+      Types.DOUBLE,
+      Types.INT,
+      Types.INT,
+      Types.STRING
+      ).asInstanceOf[TypeInformation[Any]]
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/SqlExpressionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/SqlExpressionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/SqlExpressionTest.scala
new file mode 100644
index 0000000..e0f45d4
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/SqlExpressionTest.scala
@@ -0,0 +1,170 @@
+/*
+ * 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.expressions
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.types.Row
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.expressions.utils.ExpressionTestBase
+import org.junit.Test
+
+/**
+  * Tests all SQL expressions that are currently supported according to the documentation.
+  * This tests should be kept in sync with the documentation to reduce confusion due to the
+  * large amount of SQL functions.
+  *
+  * The tests do not test every parameter combination of a function.
+  * They are rather a function existence test and simple functional test.
+  *
+  * The tests are split up and ordered like the sections in the documentation.
+  */
+class SqlExpressionTest extends ExpressionTestBase {
+
+  @Test
+  def testComparisonFunctions(): Unit = {
+    testSqlApi("1 = 1", "true")
+    testSqlApi("1 <> 1", "false")
+    testSqlApi("5 > 2", "true")
+    testSqlApi("2 >= 2", "true")
+    testSqlApi("5 < 2", "false")
+    testSqlApi("2 <= 2", "true")
+    testSqlApi("1 IS NULL", "false")
+    testSqlApi("1 IS NOT NULL", "true")
+    testSqlApi("NULLIF(1,1) IS DISTINCT FROM NULLIF(1,1)", "false")
+    testSqlApi("NULLIF(1,1) IS NOT DISTINCT FROM NULLIF(1,1)", "true")
+    testSqlApi("NULLIF(1,1) IS NOT DISTINCT FROM NULLIF(1,1)", "true")
+    testSqlApi("12 BETWEEN 11 AND 13", "true")
+    testSqlApi("12 BETWEEN ASYMMETRIC 13 AND 11", "false")
+    testSqlApi("12 BETWEEN SYMMETRIC 13 AND 11", "true")
+    testSqlApi("12 NOT BETWEEN 11 AND 13", "false")
+    testSqlApi("12 NOT BETWEEN ASYMMETRIC 13 AND 11", "true")
+    testSqlApi("12 NOT BETWEEN SYMMETRIC 13 AND 11", "false")
+    testSqlApi("'TEST' LIKE '%EST'", "true")
+    //testSqlApi("'%EST' LIKE '.%EST' ESCAPE '.'", "true") // TODO
+    testSqlApi("'TEST' NOT LIKE '%EST'", "false")
+    //testSqlApi("'%EST' NOT LIKE '.%EST' ESCAPE '.'", "false") // TODO
+    testSqlApi("'TEST' SIMILAR TO '.EST'", "true")
+    //testSqlApi("'TEST' SIMILAR TO ':.EST' ESCAPE ':'", "true") // TODO
+    testSqlApi("'TEST' NOT SIMILAR TO '.EST'", "false")
+    //testSqlApi("'TEST' NOT SIMILAR TO ':.EST' ESCAPE ':'", "false") // TODO
+    testSqlApi("'TEST' IN ('west', 'TEST', 'rest')", "true")
+    testSqlApi("'TEST' IN ('west', 'rest')", "false")
+    testSqlApi("'TEST' NOT IN ('west', 'TEST', 'rest')", "false")
+    testSqlApi("'TEST' NOT IN ('west', 'rest')", "true")
+
+    // sub-query functions are not listed here
+  }
+
+  @Test
+  def testLogicalFunctions(): Unit = {
+    testSqlApi("TRUE OR FALSE", "true")
+    testSqlApi("TRUE AND FALSE", "false")
+    testSqlApi("NOT TRUE", "false")
+    testSqlApi("TRUE IS FALSE", "false")
+    testSqlApi("TRUE IS NOT FALSE", "true")
+    testSqlApi("TRUE IS TRUE", "true")
+    testSqlApi("TRUE IS NOT TRUE", "false")
+    testSqlApi("NULLIF(TRUE,TRUE) IS UNKNOWN", "true")
+    testSqlApi("NULLIF(TRUE,TRUE) IS NOT UNKNOWN", "false")
+  }
+
+  @Test
+  def testArithmeticFunctions(): Unit = {
+    testSqlApi("+5", "5")
+    testSqlApi("-5", "-5")
+    testSqlApi("5+5", "10")
+    testSqlApi("5-5", "0")
+    testSqlApi("5*5", "25")
+    testSqlApi("5/5", "1")
+    testSqlApi("POWER(5, 5)", "3125.0")
+    testSqlApi("ABS(-5)", "5")
+    testSqlApi("MOD(-26, 5)", "-1")
+    testSqlApi("SQRT(4)", "2.0")
+    testSqlApi("LN(1)", "0.0")
+    testSqlApi("LOG10(1)", "0.0")
+    testSqlApi("EXP(0)", "1.0")
+    testSqlApi("CEIL(2.5)", "3")
+    testSqlApi("FLOOR(2.5)", "2")
+  }
+
+  @Test
+  def testStringFunctions(): Unit = {
+    testSqlApi("'test' || 'string'", "teststring")
+    testSqlApi("CHAR_LENGTH('string')", "6")
+    testSqlApi("CHARACTER_LENGTH('string')", "6")
+    testSqlApi("UPPER('string')", "STRING")
+    testSqlApi("LOWER('STRING')", "string")
+    testSqlApi("POSITION('STR' IN 'STRING')", "1")
+    testSqlApi("TRIM(BOTH ' STRING ')", "STRING")
+    testSqlApi("TRIM(LEADING 'x' FROM 'xxxxSTRINGxxxx')", "STRINGxxxx")
+    testSqlApi("TRIM(TRAILING 'x' FROM 'xxxxSTRINGxxxx')", "xxxxSTRING")
+    testSqlApi(
+      "OVERLAY('This is a old string' PLACING 'new' FROM 11 FOR 3)",
+      "This is a new string")
+    testSqlApi("SUBSTRING('hello world', 2)", "ello world")
+    testSqlApi("SUBSTRING('hello world', 2, 3)", "ell")
+    testSqlApi("INITCAP('hello world')", "Hello World")
+  }
+
+  @Test
+  def testConditionalFunctions(): Unit = {
+    testSqlApi("CASE 2 WHEN 1, 2 THEN 2 ELSE 3 END", "2")
+    testSqlApi("CASE WHEN 1 = 2 THEN 2 WHEN 1 = 1 THEN 3 ELSE 3 END", "3")
+    testSqlApi("NULLIF(1, 1)", "null")
+    testSqlApi("COALESCE(NULL, 5)", "5")
+  }
+
+  @Test
+  def testTypeConversionFunctions(): Unit = {
+    testSqlApi("CAST(2 AS DOUBLE)", "2.0")
+  }
+
+  @Test
+  def testValueConstructorFunctions(): Unit = {
+    // TODO we need a special code path that flattens ROW types
+    // testSqlApi("ROW('hello world', 12)", "hello world") // test base only returns field 0
+    // testSqlApi("('hello world', 12)", "hello world") // test base only returns field 0
+    testSqlApi("ARRAY[TRUE, FALSE][2]", "false")
+    testSqlApi("ARRAY[TRUE, TRUE]", "[true, true]")
+  }
+
+  @Test
+  def testDateTimeFunctions(): Unit = {
+    testSqlApi("DATE '1990-10-14'", "1990-10-14")
+    testSqlApi("TIME '12:12:12'", "12:12:12")
+    testSqlApi("TIMESTAMP '1990-10-14 12:12:12.123'", "1990-10-14 12:12:12.123")
+    testSqlApi("INTERVAL '10 00:00:00.004' DAY TO SECOND", "+10 00:00:00.004")
+    testSqlApi("INTERVAL '10 00:12' DAY TO MINUTE", "+10 00:12:00.000")
+    testSqlApi("INTERVAL '2-10' YEAR TO MONTH", "+2-10")
+    testSqlApi("EXTRACT(DAY FROM DATE '1990-12-01')", "1")
+    testSqlApi("EXTRACT(DAY FROM INTERVAL '19 12:10:10.123' DAY TO SECOND(3))", "19")
+    testSqlApi("QUARTER(DATE '2016-04-12')", "2")
+  }
+
+  @Test
+  def testArrayFunctions(): Unit = {
+    testSqlApi("CARDINALITY(ARRAY[TRUE, TRUE, FALSE])", "3")
+    testSqlApi("ELEMENT(ARRAY['HELLO WORLD'])", "HELLO WORLD")
+  }
+
+  override def testData: Any = new Row(0)
+
+  override def typeInfo: TypeInformation[Any] =
+    new RowTypeInfo().asInstanceOf[TypeInformation[Any]]
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/TemporalTypesTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/TemporalTypesTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/TemporalTypesTest.scala
new file mode 100644
index 0000000..840bec1
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/TemporalTypesTest.scala
@@ -0,0 +1,573 @@
+/*
+ * 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.expressions
+
+import java.sql.{Date, Time, Timestamp}
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.utils.ExpressionTestBase
+import org.junit.Test
+
+class TemporalTypesTest extends ExpressionTestBase {
+
+  @Test
+  def testTimePointLiterals(): Unit = {
+    testAllApis(
+      "1990-10-14".toDate,
+      "'1990-10-14'.toDate",
+      "DATE '1990-10-14'",
+      "1990-10-14")
+
+    testTableApi(
+      Date.valueOf("2040-09-11"),
+      "'2040-09-11'.toDate",
+      "2040-09-11")
+
+    testAllApis(
+      "1500-04-30".cast(Types.DATE),
+      "'1500-04-30'.cast(DATE)",
+      "CAST('1500-04-30' AS DATE)",
+      "1500-04-30")
+
+    testAllApis(
+      "15:45:59".toTime,
+      "'15:45:59'.toTime",
+      "TIME '15:45:59'",
+      "15:45:59")
+
+    testTableApi(
+      Time.valueOf("00:00:00"),
+      "'00:00:00'.toTime",
+      "00:00:00")
+
+    testAllApis(
+      "1:30:00".cast(Types.TIME),
+      "'1:30:00'.cast(TIME)",
+      "CAST('1:30:00' AS TIME)",
+      "01:30:00")
+
+    testAllApis(
+      "1990-10-14 23:00:00.123".toTimestamp,
+      "'1990-10-14 23:00:00.123'.toTimestamp",
+      "TIMESTAMP '1990-10-14 23:00:00.123'",
+      "1990-10-14 23:00:00.123")
+
+    testTableApi(
+      Timestamp.valueOf("2040-09-11 00:00:00.000"),
+      "'2040-09-11 00:00:00.000'.toTimestamp",
+      "2040-09-11 00:00:00.0")
+
+    testAllApis(
+      "1500-04-30 12:00:00".cast(Types.TIMESTAMP),
+      "'1500-04-30 12:00:00'.cast(TIMESTAMP)",
+      "CAST('1500-04-30 12:00:00' AS TIMESTAMP)",
+      "1500-04-30 12:00:00.0")
+  }
+
+  @Test
+  def testTimeIntervalLiterals(): Unit = {
+    testAllApis(
+      1.year,
+      "1.year",
+      "INTERVAL '1' YEAR",
+      "+1-00")
+
+    testAllApis(
+      1.month,
+      "1.month",
+      "INTERVAL '1' MONTH",
+      "+0-01")
+
+    testAllApis(
+      12.days,
+      "12.days",
+      "INTERVAL '12' DAY",
+      "+12 00:00:00.000")
+
+    testAllApis(
+      1.hour,
+      "1.hour",
+      "INTERVAL '1' HOUR",
+      "+0 01:00:00.000")
+
+    testAllApis(
+      3.minutes,
+      "3.minutes",
+      "INTERVAL '3' MINUTE",
+      "+0 00:03:00.000")
+
+    testAllApis(
+      3.seconds,
+      "3.seconds",
+      "INTERVAL '3' SECOND",
+      "+0 00:00:03.000")
+
+    testAllApis(
+      3.millis,
+      "3.millis",
+      "INTERVAL '0.003' SECOND",
+      "+0 00:00:00.003")
+  }
+
+  @Test
+  def testTimePointInput(): Unit = {
+    testAllApis(
+      'f0,
+      "f0",
+      "f0",
+      "1990-10-14")
+
+    testAllApis(
+      'f1,
+      "f1",
+      "f1",
+      "10:20:45")
+
+    testAllApis(
+      'f2,
+      "f2",
+      "f2",
+      "1990-10-14 10:20:45.123")
+  }
+
+  @Test
+  def testTimeIntervalInput(): Unit = {
+    testAllApis(
+      'f9,
+      "f9",
+      "f9",
+      "+2-00")
+
+    testAllApis(
+      'f10,
+      "f10",
+      "f10",
+      "+0 00:00:12.000")
+  }
+
+  @Test
+  def testTimePointCasting(): Unit = {
+    testAllApis(
+      'f0.cast(Types.TIMESTAMP),
+      "f0.cast(TIMESTAMP)",
+      "CAST(f0 AS TIMESTAMP)",
+      "1990-10-14 00:00:00.0")
+
+    testAllApis(
+      'f1.cast(Types.TIMESTAMP),
+      "f1.cast(TIMESTAMP)",
+      "CAST(f1 AS TIMESTAMP)",
+      "1970-01-01 10:20:45.0")
+
+    testAllApis(
+      'f2.cast(Types.DATE),
+      "f2.cast(DATE)",
+      "CAST(f2 AS DATE)",
+      "1990-10-14")
+
+    testAllApis(
+      'f2.cast(Types.TIME),
+      "f2.cast(TIME)",
+      "CAST(f2 AS TIME)",
+      "10:20:45")
+
+    testAllApis(
+      'f2.cast(Types.TIME),
+      "f2.cast(TIME)",
+      "CAST(f2 AS TIME)",
+      "10:20:45")
+
+    testTableApi(
+      'f7.cast(Types.DATE),
+      "f7.cast(DATE)",
+      "2002-11-09")
+
+    testTableApi(
+      'f7.cast(Types.DATE).cast(Types.INT),
+      "f7.cast(DATE).cast(INT)",
+      "12000")
+
+    testTableApi(
+      'f7.cast(Types.TIME),
+      "f7.cast(TIME)",
+      "00:00:12")
+
+    testTableApi(
+      'f7.cast(Types.TIME).cast(Types.INT),
+      "f7.cast(TIME).cast(INT)",
+      "12000")
+
+    testTableApi(
+      'f8.cast(Types.TIMESTAMP),
+      "f8.cast(TIMESTAMP)",
+      "2016-06-27 07:23:33.0")
+
+    testTableApi(
+      'f8.cast(Types.TIMESTAMP).cast(Types.LONG),
+      "f8.cast(TIMESTAMP).cast(LONG)",
+      "1467012213000")
+  }
+
+  @Test
+  def testTimeIntervalCasting(): Unit = {
+    testTableApi(
+      'f7.cast(Types.INTERVAL_MONTHS),
+      "f7.cast(INTERVAL_MONTHS)",
+      "+1000-00")
+
+    testTableApi(
+      'f8.cast(Types.INTERVAL_MILLIS),
+      "f8.cast(INTERVAL_MILLIS)",
+      "+16979 07:23:33.000")
+  }
+
+  @Test
+  def testTimePointComparison(): Unit = {
+    testAllApis(
+      'f0 < 'f3,
+      "f0 < f3",
+      "f0 < f3",
+      "false")
+
+    testAllApis(
+      'f0 < 'f4,
+      "f0 < f4",
+      "f0 < f4",
+      "true")
+
+    testAllApis(
+      'f1 < 'f5,
+      "f1 < f5",
+      "f1 < f5",
+      "false")
+
+    testAllApis(
+      'f0.cast(Types.TIMESTAMP) !== 'f2,
+      "f0.cast(TIMESTAMP) !== f2",
+      "CAST(f0 AS TIMESTAMP) <> f2",
+      "true")
+
+    testAllApis(
+      'f0.cast(Types.TIMESTAMP) === 'f6,
+      "f0.cast(TIMESTAMP) === f6",
+      "CAST(f0 AS TIMESTAMP) = f6",
+      "true")
+  }
+
+  @Test
+  def testTimeIntervalArithmetic(): Unit = {
+
+    // interval months comparison
+
+    testAllApis(
+      12.months < 24.months,
+      "12.months < 24.months",
+      "INTERVAL '12' MONTH < INTERVAL '24' MONTH",
+      "true")
+
+    testAllApis(
+      8.years === 8.years,
+      "8.years === 8.years",
+      "INTERVAL '8' YEAR = INTERVAL '8' YEAR",
+      "true")
+
+    // interval millis comparison
+
+    testAllApis(
+      8.millis > 10.millis,
+      "8.millis > 10.millis",
+      "INTERVAL '0.008' SECOND > INTERVAL '0.010' SECOND",
+      "false")
+
+    testAllApis(
+      8.millis === 8.millis,
+      "8.millis === 8.millis",
+      "INTERVAL '0.008' SECOND = INTERVAL '0.008' SECOND",
+      "true")
+
+    // interval months addition/subtraction
+
+    testAllApis(
+      8.years + 10.months,
+      "8.years + 10.months",
+      "INTERVAL '8' YEAR + INTERVAL '10' MONTH",
+      "+8-10")
+
+    testAllApis(
+      2.years - 12.months,
+      "2.years - 12.months",
+      "INTERVAL '2' YEAR - INTERVAL '12' MONTH",
+      "+1-00")
+
+    testAllApis(
+      -2.years,
+      "-2.years",
+      "-INTERVAL '2' YEAR",
+      "-2-00")
+
+    // interval millis addition/subtraction
+
+    testAllApis(
+      8.hours + 10.minutes + 12.seconds + 5.millis,
+      "8.hours + 10.minutes + 12.seconds + 5.millis",
+      "INTERVAL '8' HOUR + INTERVAL '10' MINUTE + INTERVAL '12.005' SECOND",
+      "+0 08:10:12.005")
+
+    testAllApis(
+      1.minute - 10.seconds,
+      "1.minute - 10.seconds",
+      "INTERVAL '1' MINUTE - INTERVAL '10' SECOND",
+      "+0 00:00:50.000")
+
+    testAllApis(
+      -10.seconds,
+      "-10.seconds",
+      "-INTERVAL '10' SECOND",
+      "-0 00:00:10.000")
+
+    // addition to date
+
+    // interval millis
+    testAllApis(
+      'f0 + 2.days,
+      "f0 + 2.days",
+      "f0 + INTERVAL '2' DAY",
+      "1990-10-16")
+
+    // interval millis
+    testAllApis(
+      30.days + 'f0,
+      "30.days + f0",
+      "INTERVAL '30' DAY + f0",
+      "1990-11-13")
+
+    // interval months
+    testAllApis(
+      'f0 + 2.months,
+      "f0 + 2.months",
+      "f0 + INTERVAL '2' MONTH",
+      "1990-12-14")
+
+    // interval months
+    testAllApis(
+      2.months + 'f0,
+      "2.months + f0",
+      "INTERVAL '2' MONTH + f0",
+      "1990-12-14")
+
+    // addition to time
+
+    // interval millis
+    testAllApis(
+      'f1 + 12.hours,
+      "f1 + 12.hours",
+      "f1 + INTERVAL '12' HOUR",
+      "22:20:45")
+
+    // interval millis
+    testAllApis(
+      12.hours + 'f1,
+      "12.hours + f1",
+      "INTERVAL '12' HOUR + f1",
+      "22:20:45")
+
+    // addition to timestamp
+
+    // interval millis
+    testAllApis(
+      'f2 + 10.days + 4.millis,
+      "f2 + 10.days + 4.millis",
+      "f2 + INTERVAL '10 00:00:00.004' DAY TO SECOND",
+      "1990-10-24 10:20:45.127")
+
+    // interval millis
+    testAllApis(
+      10.days + 'f2 + 4.millis,
+      "10.days + f2 + 4.millis",
+      "INTERVAL '10 00:00:00.004' DAY TO SECOND + f2",
+      "1990-10-24 10:20:45.127")
+
+    // interval months
+    testAllApis(
+      'f2 + 10.years,
+      "f2 + 10.years",
+      "f2 + INTERVAL '10' YEAR",
+      "2000-10-14 10:20:45.123")
+
+    // interval months
+    testAllApis(
+      10.years + 'f2,
+      "10.years + f2",
+      "INTERVAL '10' YEAR + f2",
+      "2000-10-14 10:20:45.123")
+
+    // subtraction from date
+
+    // interval millis
+    testAllApis(
+      'f0 - 2.days,
+      "f0 - 2.days",
+      "f0 - INTERVAL '2' DAY",
+      "1990-10-12")
+
+    // interval millis
+    testAllApis(
+      -30.days + 'f0,
+      "-30.days + f0",
+      "INTERVAL '-30' DAY + f0",
+      "1990-09-14")
+
+    // interval months
+    testAllApis(
+      'f0 - 2.months,
+      "f0 - 2.months",
+      "f0 - INTERVAL '2' MONTH",
+      "1990-08-14")
+
+    // interval months
+    testAllApis(
+      -2.months + 'f0,
+      "-2.months + f0",
+      "-INTERVAL '2' MONTH + f0",
+      "1990-08-14")
+
+    // subtraction from time
+
+    // interval millis
+    testAllApis(
+      'f1 - 12.hours,
+      "f1 - 12.hours",
+      "f1 - INTERVAL '12' HOUR",
+      "22:20:45")
+
+    // interval millis
+    testAllApis(
+      -12.hours + 'f1,
+      "-12.hours + f1",
+      "INTERVAL '-12' HOUR + f1",
+      "22:20:45")
+
+    // subtraction from timestamp
+
+    // interval millis
+    testAllApis(
+      'f2 - 10.days - 4.millis,
+      "f2 - 10.days - 4.millis",
+      "f2 - INTERVAL '10 00:00:00.004' DAY TO SECOND",
+      "1990-10-04 10:20:45.119")
+
+    // interval millis
+    testAllApis(
+      -10.days + 'f2 - 4.millis,
+      "-10.days + f2 - 4.millis",
+      "INTERVAL '-10 00:00:00.004' DAY TO SECOND + f2",
+      "1990-10-04 10:20:45.119")
+
+    // interval months
+    testAllApis(
+      'f2 - 10.years,
+      "f2 - 10.years",
+      "f2 - INTERVAL '10' YEAR",
+      "1980-10-14 10:20:45.123")
+
+    // interval months
+    testAllApis(
+      -10.years + 'f2,
+      "-10.years + f2",
+      "INTERVAL '-10' YEAR + f2",
+      "1980-10-14 10:20:45.123")
+
+    // casting
+
+    testAllApis(
+      -'f9.cast(Types.INTERVAL_MONTHS),
+      "-f9.cast(INTERVAL_MONTHS)",
+      "-CAST(f9 AS INTERVAL YEAR)",
+      "-2-00")
+
+    testAllApis(
+      -'f10.cast(Types.INTERVAL_MILLIS),
+      "-f10.cast(INTERVAL_MILLIS)",
+      "-CAST(f10 AS INTERVAL SECOND)",
+      "-0 00:00:12.000")
+
+    // addition/subtraction of interval millis and interval months
+
+    testAllApis(
+      'f0 + 2.days + 1.month,
+      "f0 + 2.days + 1.month",
+      "f0 + INTERVAL '2' DAY + INTERVAL '1' MONTH",
+      "1990-11-16")
+
+    testAllApis(
+      'f0 - 2.days - 1.month,
+      "f0 - 2.days - 1.month",
+      "f0 - INTERVAL '2' DAY - INTERVAL '1' MONTH",
+      "1990-09-12")
+
+    testAllApis(
+      'f2 + 2.days + 1.month,
+      "f2 + 2.days + 1.month",
+      "f2 + INTERVAL '2' DAY + INTERVAL '1' MONTH",
+      "1990-11-16 10:20:45.123")
+
+    testAllApis(
+      'f2 - 2.days - 1.month,
+      "f2 - 2.days - 1.month",
+      "f2 - INTERVAL '2' DAY - INTERVAL '1' MONTH",
+      "1990-09-12 10:20:45.123")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  def testData = {
+    val testData = new Row(11)
+    testData.setField(0, Date.valueOf("1990-10-14"))
+    testData.setField(1, Time.valueOf("10:20:45"))
+    testData.setField(2, Timestamp.valueOf("1990-10-14 10:20:45.123"))
+    testData.setField(3, Date.valueOf("1990-10-13"))
+    testData.setField(4, Date.valueOf("1990-10-15"))
+    testData.setField(5, Time.valueOf("00:00:00"))
+    testData.setField(6, Timestamp.valueOf("1990-10-14 00:00:00.0"))
+    testData.setField(7, 12000)
+    testData.setField(8, 1467012213000L)
+    testData.setField(9, 24)
+    testData.setField(10, 12000L)
+    testData
+  }
+
+  def typeInfo = {
+    new RowTypeInfo(
+      Types.DATE,
+      Types.TIME,
+      Types.TIMESTAMP,
+      Types.DATE,
+      Types.DATE,
+      Types.TIME,
+      Types.TIMESTAMP,
+      Types.INT,
+      Types.LONG,
+      Types.INTERVAL_MONTHS,
+      Types.INTERVAL_MILLIS).asInstanceOf[TypeInformation[Any]]
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala
new file mode 100644
index 0000000..26bbd44
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/UserDefinedScalarFunctionTest.scala
@@ -0,0 +1,228 @@
+/*
+ * 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.expressions
+
+import java.sql.{Date, Time, Timestamp}
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.utils._
+import org.apache.flink.table.functions.ScalarFunction
+import org.junit.Test
+
+class UserDefinedScalarFunctionTest extends ExpressionTestBase {
+
+  @Test
+  def testParameters(): Unit = {
+    testAllApis(
+      Func0('f0),
+      "Func0(f0)",
+      "Func0(f0)",
+      "42")
+
+    testAllApis(
+      Func1('f0),
+      "Func1(f0)",
+      "Func1(f0)",
+      "43")
+
+    testAllApis(
+      Func2('f0, 'f1, 'f3),
+      "Func2(f0, f1, f3)",
+      "Func2(f0, f1, f3)",
+      "42 and Test and SimplePojo(Bob,36)")
+
+    testAllApis(
+      Func0(123),
+      "Func0(123)",
+      "Func0(123)",
+      "123")
+
+    testAllApis(
+      Func6('f4, 'f5, 'f6),
+      "Func6(f4, f5, f6)",
+      "Func6(f4, f5, f6)",
+      "(1990-10-14,12:10:10,1990-10-14 12:10:10.0)")
+  }
+
+  @Test
+  def testNullableParameters(): Unit = {
+    testAllApis(
+      Func3(Null(INT_TYPE_INFO), Null(STRING_TYPE_INFO)),
+      "Func3(Null(INT), Null(STRING))",
+      "Func3(NULL, NULL)",
+      "null and null")
+
+    testAllApis(
+      Func3(Null(INT_TYPE_INFO), "Test"),
+      "Func3(Null(INT), 'Test')",
+      "Func3(NULL, 'Test')",
+      "null and Test")
+
+    testAllApis(
+      Func3(42, Null(STRING_TYPE_INFO)),
+      "Func3(42, Null(STRING))",
+      "Func3(42, NULL)",
+      "42 and null")
+
+    testAllApis(
+      Func0(Null(INT_TYPE_INFO)),
+      "Func0(Null(INT))",
+      "Func0(NULL)",
+      "-1")
+  }
+
+  @Test
+  def testResults(): Unit = {
+    testAllApis(
+      Func4(),
+      "Func4()",
+      "Func4()",
+      "null")
+
+    testAllApis(
+      Func5(),
+      "Func5()",
+      "Func5()",
+      "-1")
+  }
+
+  @Test
+  def testNesting(): Unit = {
+    testAllApis(
+      Func0(Func0('f0)),
+      "Func0(Func0(f0))",
+      "Func0(Func0(f0))",
+      "42")
+
+    testAllApis(
+      Func0(Func0('f0)),
+      "Func0(Func0(f0))",
+      "Func0(Func0(f0))",
+      "42")
+
+    testAllApis(
+      Func7(Func7(Func7(1, 1), Func7(1, 1)), Func7(Func7(1, 1), Func7(1, 1))),
+      "Func7(Func7(Func7(1, 1), Func7(1, 1)), Func7(Func7(1, 1), Func7(1, 1)))",
+      "Func7(Func7(Func7(1, 1), Func7(1, 1)), Func7(Func7(1, 1), Func7(1, 1)))",
+      "8")
+  }
+
+  @Test
+  def testOverloadedParameters(): Unit = {
+    testAllApis(
+      Func8(1),
+      "Func8(1)",
+      "Func8(1)",
+      "a")
+
+    testAllApis(
+      Func8(1, 1),
+      "Func8(1, 1)",
+      "Func8(1, 1)",
+      "b")
+
+    testAllApis(
+      Func8("a", "a"),
+      "Func8('a', 'a')",
+      "Func8('a', 'a')",
+      "c")
+  }
+
+  @Test
+  def testTimePointsOnPrimitives(): Unit = {
+    testAllApis(
+      Func9('f4, 'f5, 'f6),
+      "Func9(f4, f5, f6)",
+      "Func9(f4, f5, f6)",
+      "7591 and 43810000 and 655906210000")
+
+    testAllApis(
+      Func10('f6),
+      "Func10(f6)",
+      "Func10(f6)",
+      "1990-10-14 12:10:10.0")
+  }
+
+  @Test
+  def testTimeIntervalsOnPrimitives(): Unit = {
+    testAllApis(
+      Func11('f7, 'f8),
+      "Func11(f7, f8)",
+      "Func11(f7, f8)",
+      "12 and 1000")
+
+    testAllApis(
+      Func12('f8),
+      "Func12(f8)",
+      "Func12(f8)",
+      "+0 00:00:01.000")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  override def testData: Any = {
+    val testData = new Row(9)
+    testData.setField(0, 42)
+    testData.setField(1, "Test")
+    testData.setField(2, null)
+    testData.setField(3, SimplePojo("Bob", 36))
+    testData.setField(4, Date.valueOf("1990-10-14"))
+    testData.setField(5, Time.valueOf("12:10:10"))
+    testData.setField(6, Timestamp.valueOf("1990-10-14 12:10:10"))
+    testData.setField(7, 12)
+    testData.setField(8, 1000L)
+    testData
+  }
+
+  override def typeInfo: TypeInformation[Any] = {
+    new RowTypeInfo(
+      Types.INT,
+      Types.STRING,
+      Types.BOOLEAN,
+      TypeInformation.of(classOf[SimplePojo]),
+      Types.DATE,
+      Types.TIME,
+      Types.TIMESTAMP,
+      Types.INTERVAL_MONTHS,
+      Types.INTERVAL_MILLIS
+    ).asInstanceOf[TypeInformation[Any]]
+  }
+
+  override def functions: Map[String, ScalarFunction] = Map(
+    "Func0" -> Func0,
+    "Func1" -> Func1,
+    "Func2" -> Func2,
+    "Func3" -> Func3,
+    "Func4" -> Func4,
+    "Func5" -> Func5,
+    "Func6" -> Func6,
+    "Func7" -> Func7,
+    "Func8" -> Func8,
+    "Func9" -> Func9,
+    "Func10" -> Func10,
+    "Func11" -> Func11,
+    "Func12" -> Func12
+  )
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala
new file mode 100644
index 0000000..8555632
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/ExpressionTestBase.scala
@@ -0,0 +1,218 @@
+/*
+ * 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.expressions.utils
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import org.apache.calcite.sql2rel.RelDecorrelator
+import org.apache.calcite.tools.{Programs, RelBuilder}
+import org.apache.flink.api.common.functions.{Function, MapFunction}
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.{DataSet => JDataSet}
+import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{BatchTableEnvironment, TableConfig, TableEnvironment}
+import org.apache.flink.table.calcite.FlinkPlannerImpl
+import org.apache.flink.table.codegen.{CodeGenerator, Compiler, GeneratedFunction}
+import org.apache.flink.table.expressions.{Expression, ExpressionParser}
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.plan.nodes.dataset.{DataSetCalc, DataSetConvention}
+import org.apache.flink.table.plan.rules.FlinkRuleSets
+import org.junit.Assert._
+import org.junit.{After, Before}
+import org.mockito.Mockito._
+
+import scala.collection.mutable
+
+/**
+  * Base test class for expression tests.
+  */
+abstract class ExpressionTestBase {
+
+  private val testExprs = mutable.ArrayBuffer[(RexNode, String)]()
+
+  // setup test utils
+  private val tableName = "testTable"
+  private val context = prepareContext(typeInfo)
+  private val planner = new FlinkPlannerImpl(
+    context._2.getFrameworkConfig,
+    context._2.getPlanner,
+    context._2.getTypeFactory)
+  private val optProgram = Programs.ofRules(FlinkRuleSets.DATASET_OPT_RULES)
+
+  private def prepareContext(typeInfo: TypeInformation[Any]): (RelBuilder, TableEnvironment) = {
+    // create DataSetTable
+    val dataSetMock = mock(classOf[DataSet[Any]])
+    val jDataSetMock = mock(classOf[JDataSet[Any]])
+    when(dataSetMock.javaSet).thenReturn(jDataSetMock)
+    when(jDataSetMock.getType).thenReturn(typeInfo)
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    tEnv.registerDataSet(tableName, dataSetMock)
+    functions.foreach(f => tEnv.registerFunction(f._1, f._2))
+
+    // prepare RelBuilder
+    val relBuilder = tEnv.getRelBuilder
+    relBuilder.scan(tableName)
+
+    (relBuilder, tEnv)
+  }
+
+  def testData: Any
+
+  def typeInfo: TypeInformation[Any]
+
+  def functions: Map[String, ScalarFunction] = Map()
+
+  @Before
+  def resetTestExprs() = {
+    testExprs.clear()
+  }
+
+  @After
+  def evaluateExprs() = {
+    val relBuilder = context._1
+    val config = new TableConfig()
+    val generator = new CodeGenerator(config, false, typeInfo)
+
+    // cast expressions to String
+    val stringTestExprs = testExprs.map(expr => relBuilder.cast(expr._1, VARCHAR)).toSeq
+
+    // generate code
+    val resultType = new RowTypeInfo(Seq.fill(testExprs.size)(STRING_TYPE_INFO): _*)
+    val genExpr = generator.generateResultExpression(
+      resultType,
+      resultType.getFieldNames,
+      stringTestExprs)
+
+    val bodyCode =
+      s"""
+        |${genExpr.code}
+        |return ${genExpr.resultTerm};
+        |""".stripMargin
+
+    val genFunc = generator.generateFunction[MapFunction[Any, String]](
+      "TestFunction",
+      classOf[MapFunction[Any, String]],
+      bodyCode,
+      resultType.asInstanceOf[TypeInformation[Any]])
+
+    // compile and evaluate
+    val clazz = new TestCompiler[MapFunction[Any, String]]().compile(genFunc)
+    val mapper = clazz.newInstance()
+    val result = mapper.map(testData).asInstanceOf[Row]
+
+    // compare
+    testExprs
+      .zipWithIndex
+      .foreach {
+        case ((expr, expected), index) =>
+          val actual = result.getField(index)
+          assertEquals(
+            s"Wrong result for: $expr",
+            expected,
+            if (actual == null) "null" else actual)
+      }
+  }
+
+  private def addSqlTestExpr(sqlExpr: String, expected: String): Unit = {
+    // create RelNode from SQL expression
+    val parsed = planner.parse(s"SELECT $sqlExpr FROM $tableName")
+    val validated = planner.validate(parsed)
+    val converted = planner.rel(validated).rel
+
+    // create DataSetCalc
+    val decorPlan = RelDecorrelator.decorrelateQuery(converted)
+    val flinkOutputProps = converted.getTraitSet.replace(DataSetConvention.INSTANCE).simplify()
+    val dataSetCalc = optProgram.run(context._2.getPlanner, decorPlan, flinkOutputProps)
+
+    // extract RexNode
+    val calcProgram = dataSetCalc
+     .asInstanceOf[DataSetCalc]
+     .calcProgram
+    val expanded = calcProgram.expandLocalRef(calcProgram.getProjectList.get(0))
+
+    testExprs += ((expanded, expected))
+  }
+
+  private def addTableApiTestExpr(tableApiExpr: Expression, expected: String): Unit = {
+    // create RelNode from Table API expression
+    val env = context._2
+    val converted = env
+      .asInstanceOf[BatchTableEnvironment]
+      .scan(tableName)
+      .select(tableApiExpr)
+      .getRelNode
+
+    // create DataSetCalc
+    val decorPlan = RelDecorrelator.decorrelateQuery(converted)
+    val flinkOutputProps = converted.getTraitSet.replace(DataSetConvention.INSTANCE).simplify()
+    val dataSetCalc = optProgram.run(context._2.getPlanner, decorPlan, flinkOutputProps)
+
+    // extract RexNode
+    val calcProgram = dataSetCalc
+     .asInstanceOf[DataSetCalc]
+     .calcProgram
+    val expanded = calcProgram.expandLocalRef(calcProgram.getProjectList.get(0))
+
+    testExprs += ((expanded, expected))
+  }
+
+  private def addTableApiTestExpr(tableApiString: String, expected: String): Unit = {
+    addTableApiTestExpr(ExpressionParser.parseExpression(tableApiString), expected)
+  }
+
+  def testAllApis(
+      expr: Expression,
+      exprString: String,
+      sqlExpr: String,
+      expected: String)
+    : Unit = {
+    addTableApiTestExpr(expr, expected)
+    addTableApiTestExpr(exprString, expected)
+    addSqlTestExpr(sqlExpr, expected)
+  }
+
+  def testTableApi(
+      expr: Expression,
+      exprString: String,
+      expected: String)
+    : Unit = {
+    addTableApiTestExpr(expr, expected)
+    addTableApiTestExpr(exprString, expected)
+  }
+
+  def testSqlApi(
+      sqlExpr: String,
+      expected: String)
+    : Unit = {
+    addSqlTestExpr(sqlExpr, expected)
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  // TestCompiler that uses current class loader
+  class TestCompiler[T <: Function] extends Compiler[T] {
+    def compile(genFunc: GeneratedFunction[T]): Class[T] =
+      compile(getClass.getClassLoader, genFunc.name, genFunc.code)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/UserDefinedScalarFunctions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/UserDefinedScalarFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/UserDefinedScalarFunctions.scala
new file mode 100644
index 0000000..4e9b6d3
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/utils/UserDefinedScalarFunctions.scala
@@ -0,0 +1,121 @@
+/*
+ * 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.expressions.utils
+
+import java.sql.{Date, Time, Timestamp}
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.functions.ScalarFunction
+
+case class SimplePojo(name: String, age: Int)
+
+object Func0 extends ScalarFunction {
+  def eval(index: Int): Int = {
+    index
+  }
+}
+
+object Func1 extends ScalarFunction {
+  def eval(index: Integer): Integer = {
+    index + 1
+  }
+}
+
+object Func2 extends ScalarFunction {
+  def eval(index: Integer, str: String, pojo: SimplePojo): String = {
+    s"$index and $str and $pojo"
+  }
+}
+
+object Func3 extends ScalarFunction {
+  def eval(index: Integer, str: String): String = {
+    s"$index and $str"
+  }
+}
+
+object Func4 extends ScalarFunction {
+  def eval(): Integer = {
+    null
+  }
+}
+
+object Func5 extends ScalarFunction {
+  def eval(): Int = {
+    -1
+  }
+}
+
+object Func6 extends ScalarFunction {
+  def eval(date: Date, time: Time, timestamp: Timestamp): (Date, Time, Timestamp) = {
+    (date, time, timestamp)
+  }
+}
+
+object Func7 extends ScalarFunction {
+  def eval(a: Integer, b: Integer): Integer = {
+    a + b
+  }
+}
+
+object Func8 extends ScalarFunction {
+  def eval(a: Int): String = {
+    "a"
+  }
+
+  def eval(a: Int, b: Int): String = {
+    "b"
+  }
+
+  def eval(a: String, b: String): String = {
+    "c"
+  }
+}
+
+object Func9 extends ScalarFunction {
+  def eval(a: Int, b: Int, c: Long): String = {
+    s"$a and $b and $c"
+  }
+}
+
+object Func10 extends ScalarFunction {
+  def eval(c: Long): Long = {
+    c
+  }
+
+  override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = {
+    Types.TIMESTAMP
+  }
+}
+
+object Func11 extends ScalarFunction {
+  def eval(a: Int, b: Long): String = {
+    s"$a and $b"
+  }
+}
+
+object Func12 extends ScalarFunction {
+  def eval(a: Long): Long = {
+    a
+  }
+
+  override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = {
+    Types.INTERVAL_MILLIS
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/util/RexProgramProjectExtractorTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/util/RexProgramProjectExtractorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/util/RexProgramProjectExtractorTest.scala
new file mode 100644
index 0000000..f7385ac
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/util/RexProgramProjectExtractorTest.scala
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.util
+
+import java.math.BigDecimal
+
+import org.apache.calcite.adapter.java.JavaTypeFactory
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeSystem}
+import org.apache.calcite.sql.`type`.SqlTypeName.{BIGINT, DOUBLE, INTEGER, VARCHAR}
+import org.apache.calcite.rex.{RexBuilder, RexProgram, RexProgramBuilder}
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.flink.table.plan.rules.util.RexProgramProjectExtractor
+
+import scala.collection.JavaConverters._
+import RexProgramProjectExtractor._
+import org.junit.{Assert, Before, Test}
+
+/**
+  * This class is responsible for testing RexProgramProjectExtractor
+  */
+class RexProgramProjectExtractorTest {
+  private var typeFactory: JavaTypeFactory = null
+  private var rexBuilder: RexBuilder = null
+  private var allFieldTypes: Seq[RelDataType] = null
+  private val allFieldNames = List("name", "id", "amount", "price")
+
+  @Before
+  def setUp: Unit = {
+    typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT)
+    rexBuilder = new RexBuilder(typeFactory)
+    allFieldTypes = List(VARCHAR, BIGINT, INTEGER, DOUBLE).map(typeFactory.createSqlType(_))
+  }
+
+  @Test
+  def testExtractRefInputFields: Unit = {
+    val usedFields = extractRefInputFields(buildRexProgram)
+    Assert.assertArrayEquals(usedFields, Array(2, 3, 1))
+  }
+
+  @Test
+  def testRewriteRexProgram: Unit = {
+    val originRexProgram = buildRexProgram
+    Assert.assertTrue(extractExprStrList(originRexProgram).sameElements(Array(
+      "$0",
+      "$1",
+      "$2",
+      "$3",
+      "*($t2, $t3)",
+      "100",
+      "<($t4, $t5)",
+      "6",
+      ">($t1, $t7)",
+      "AND($t6, $t8)")))
+    // use amount, id, price fields to create a new RexProgram
+    val usedFields = Array(2, 3, 1)
+    val types = usedFields.map(allFieldTypes(_)).toList.asJava
+    val names = usedFields.map(allFieldNames(_)).toList.asJava
+    val inputRowType = typeFactory.createStructType(types, names)
+    val newRexProgram = rewriteRexProgram(originRexProgram, inputRowType, usedFields, rexBuilder)
+    Assert.assertTrue(extractExprStrList(newRexProgram).sameElements(Array(
+      "$0",
+      "$1",
+      "$2",
+      "*($t0, $t1)",
+      "100",
+      "<($t3, $t4)",
+      "6",
+      ">($t2, $t6)",
+      "AND($t5, $t7)")))
+  }
+
+  private def buildRexProgram: RexProgram = {
+    val types = allFieldTypes.asJava
+    val names = allFieldNames.asJava
+    val inputRowType = typeFactory.createStructType(types, names)
+    val builder = new RexProgramBuilder(inputRowType, rexBuilder)
+    val t0 = rexBuilder.makeInputRef(types.get(2), 2)
+    val t1 = rexBuilder.makeInputRef(types.get(1), 1)
+    val t2 = rexBuilder.makeInputRef(types.get(3), 3)
+    val t3 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, t0, t2))
+    val t4 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L))
+    val t5 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(6L))
+    // project: amount, amount * price
+    builder.addProject(t0, "amount")
+    builder.addProject(t3, "total")
+    // condition: amount * price < 100 and id > 6
+    val t6 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t3, t4))
+    val t7 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t5))
+    val t8 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(t6, t7).asJava))
+    builder.addCondition(t8)
+    builder.getProgram
+  }
+
+  /**
+    * extract all expression string list from input RexProgram expression lists
+    *
+    * @param rexProgram input RexProgram instance to analyze
+    * @return all expression string list of input RexProgram expression lists
+    */
+  private def extractExprStrList(rexProgram: RexProgram) = {
+    rexProgram.getExprList.asScala.map(_.toString)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/AggregateTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/AggregateTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/AggregateTestBase.scala
new file mode 100644
index 0000000..0ca101d
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/AggregateTestBase.scala
@@ -0,0 +1,111 @@
+/*
+ * 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.math.BigDecimal
+import org.apache.flink.types.Row
+
+import org.junit.Test
+import org.junit.Assert.assertEquals
+
+abstract class AggregateTestBase[T] {
+
+  private val offset = 2
+  private val rowArity: Int = offset + aggregator.intermediateDataType.length
+
+  def inputValueSets: Seq[Seq[_]]
+
+  def expectedResults: Seq[T]
+
+  def aggregator: Aggregate[T]
+
+  private def createAggregator(): Aggregate[T] = {
+    val agg = aggregator
+    agg.setAggOffsetInRow(offset)
+    agg
+  }
+
+  private def createRow(): Row = {
+    new Row(rowArity)
+  }
+
+  @Test
+  def testAggregate(): Unit = {
+
+    // iterate over input sets
+    for((vals, expected) <- inputValueSets.zip(expectedResults)) {
+
+      // prepare mapper
+      val rows: Seq[Row] = prepare(vals)
+
+      val result = if (aggregator.supportPartial) {
+        // test with combiner
+        val (firstVals, secondVals) = rows.splitAt(rows.length / 2)
+        val combined = partialAgg(firstVals) :: partialAgg(secondVals) :: Nil
+        finalAgg(combined)
+
+      } else {
+        // test without combiner
+        finalAgg(rows)
+      }
+
+      (expected, result) match {
+        case (e: BigDecimal, r: BigDecimal) =>
+          // BigDecimal.equals() value and scale but we are only interested in value.
+          assert(e.compareTo(r) == 0)
+        case _ =>
+          assertEquals(expected, result)
+      }
+    }
+  }
+
+  private def prepare(vals: Seq[_]): Seq[Row] = {
+
+    val agg = createAggregator()
+
+    vals.map { v =>
+      val row = createRow()
+      agg.prepare(v, row)
+      row
+    }
+  }
+
+  private def partialAgg(rows: Seq[Row]): Row = {
+
+    val agg = createAggregator()
+    val aggBuf = createRow()
+
+    agg.initiate(aggBuf)
+    rows.foreach(v => agg.merge(v, aggBuf))
+
+    aggBuf
+  }
+
+  private def finalAgg(rows: Seq[Row]): T = {
+
+    val agg = createAggregator()
+    val aggBuf = createRow()
+
+    agg.initiate(aggBuf)
+    rows.foreach(v => agg.merge(v, aggBuf))
+
+    agg.evaluate(partialAgg(rows))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/AvgAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/AvgAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/AvgAggregateTest.scala
new file mode 100644
index 0000000..a72d08b
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/AvgAggregateTest.scala
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.aggregate
+
+import java.math.BigDecimal
+
+abstract class AvgAggregateTestBase[T: Numeric] extends AggregateTestBase[T] {
+
+  private val numeric: Numeric[T] = implicitly[Numeric[T]]
+
+  def minVal: T
+  def maxVal: T
+
+  override def inputValueSets: Seq[Seq[T]] = Seq(
+    Seq(
+      minVal,
+      minVal,
+      null.asInstanceOf[T],
+      minVal,
+      minVal,
+      null.asInstanceOf[T],
+      minVal,
+      minVal,
+      minVal
+    ),
+    Seq(
+      maxVal,
+      maxVal,
+      null.asInstanceOf[T],
+      maxVal,
+      maxVal,
+      null.asInstanceOf[T],
+      maxVal,
+      maxVal,
+      maxVal
+    ),
+    Seq(
+      minVal,
+      maxVal,
+      null.asInstanceOf[T],
+      numeric.fromInt(0),
+      numeric.negate(maxVal),
+      numeric.negate(minVal),
+      null.asInstanceOf[T]
+    ),
+    Seq(
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T]
+    )
+  )
+
+  override def expectedResults: Seq[T] = Seq(
+    minVal,
+    maxVal,
+    numeric.fromInt(0),
+    null.asInstanceOf[T]
+  )
+}
+
+class ByteAvgAggregateTest extends AvgAggregateTestBase[Byte] {
+
+  override def minVal = (Byte.MinValue + 1).toByte
+  override def maxVal = (Byte.MaxValue - 1).toByte
+
+  override def aggregator = new ByteAvgAggregate()
+}
+
+class ShortAvgAggregateTest extends AvgAggregateTestBase[Short] {
+
+  override def minVal = (Short.MinValue + 1).toShort
+  override def maxVal = (Short.MaxValue - 1).toShort
+
+  override def aggregator = new ShortAvgAggregate()
+}
+
+class IntAvgAggregateTest extends AvgAggregateTestBase[Int] {
+
+  override def minVal = Int.MinValue + 1
+  override def maxVal = Int.MaxValue - 1
+
+  override def aggregator = new IntAvgAggregate()
+}
+
+class LongAvgAggregateTest extends AvgAggregateTestBase[Long] {
+
+  override def minVal = Long.MinValue + 1
+  override def maxVal = Long.MaxValue - 1
+
+  override def aggregator = new LongAvgAggregate()
+}
+
+class FloatAvgAggregateTest extends AvgAggregateTestBase[Float] {
+
+  override def minVal = Float.MinValue
+  override def maxVal = Float.MaxValue
+
+  override def aggregator = new FloatAvgAggregate()
+}
+
+class DoubleAvgAggregateTest extends AvgAggregateTestBase[Double] {
+
+  override def minVal = Float.MinValue
+  override def maxVal = Float.MaxValue
+
+  override def aggregator = new DoubleAvgAggregate()
+}
+
+class DecimalAvgAggregateTest extends AggregateTestBase[BigDecimal] {
+
+  override def inputValueSets: Seq[Seq[_]] = Seq(
+    Seq(
+      new BigDecimal("987654321000000"),
+      new BigDecimal("-0.000000000012345"),
+      null,
+      new BigDecimal("0.000000000012345"),
+      new BigDecimal("-987654321000000"),
+      null,
+      new BigDecimal("0")
+    ),
+    Seq(
+      null,
+      null,
+      null,
+      null
+    )
+  )
+
+  override def expectedResults: Seq[BigDecimal] = Seq(
+    BigDecimal.ZERO,
+    null
+  )
+
+  override def aggregator: Aggregate[BigDecimal] = new DecimalAvgAggregate()
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/CountAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/CountAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/CountAggregateTest.scala
new file mode 100644
index 0000000..55f73b4
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/CountAggregateTest.scala
@@ -0,0 +1,31 @@
+/*
+ * 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
+
+class CountAggregateTest extends AggregateTestBase[Long] {
+
+  override def inputValueSets: Seq[Seq[_]] = Seq(
+    Seq("a", "b", null, "c", null, "d", "e", null, "f"),
+    Seq(null, null, null, null, null, null)
+  )
+
+  override def expectedResults: Seq[Long] = Seq(6L, 0L)
+
+  override def aggregator: Aggregate[Long] = new CountAggregate()
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/MaxAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/MaxAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/MaxAggregateTest.scala
new file mode 100644
index 0000000..1bf879d
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/MaxAggregateTest.scala
@@ -0,0 +1,177 @@
+/*
+ * 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.math.BigDecimal
+
+abstract class MaxAggregateTestBase[T: Numeric] extends AggregateTestBase[T] {
+
+  private val numeric: Numeric[T] = implicitly[Numeric[T]]
+
+  def minVal: T
+  def maxVal: T
+
+  override def inputValueSets: Seq[Seq[T]] = Seq(
+    Seq(
+      numeric.fromInt(1),
+      null.asInstanceOf[T],
+      maxVal,
+      numeric.fromInt(-99),
+      numeric.fromInt(3),
+      numeric.fromInt(56),
+      numeric.fromInt(0),
+      minVal,
+      numeric.fromInt(-20),
+      numeric.fromInt(17),
+      null.asInstanceOf[T]
+    ),
+    Seq(
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T]
+    )
+  )
+
+  override def expectedResults: Seq[T] = Seq(
+    maxVal,
+    null.asInstanceOf[T]
+  )
+}
+
+class ByteMaxAggregateTest extends MaxAggregateTestBase[Byte] {
+
+  override def minVal = (Byte.MinValue + 1).toByte
+  override def maxVal = (Byte.MaxValue - 1).toByte
+
+  override def aggregator: Aggregate[Byte] = new ByteMaxAggregate()
+}
+
+class ShortMaxAggregateTest extends MaxAggregateTestBase[Short] {
+
+  override def minVal = (Short.MinValue + 1).toShort
+  override def maxVal = (Short.MaxValue - 1).toShort
+
+  override def aggregator: Aggregate[Short] = new ShortMaxAggregate()
+}
+
+class IntMaxAggregateTest extends MaxAggregateTestBase[Int] {
+
+  override def minVal = Int.MinValue + 1
+  override def maxVal = Int.MaxValue - 1
+
+  override def aggregator: Aggregate[Int] = new IntMaxAggregate()
+}
+
+class LongMaxAggregateTest extends MaxAggregateTestBase[Long] {
+
+  override def minVal = Long.MinValue + 1
+  override def maxVal = Long.MaxValue - 1
+
+  override def aggregator: Aggregate[Long] = new LongMaxAggregate()
+}
+
+class FloatMaxAggregateTest extends MaxAggregateTestBase[Float] {
+
+  override def minVal = Float.MinValue / 2
+  override def maxVal = Float.MaxValue / 2
+
+  override def aggregator: Aggregate[Float] = new FloatMaxAggregate()
+}
+
+class DoubleMaxAggregateTest extends MaxAggregateTestBase[Double] {
+
+  override def minVal = Double.MinValue / 2
+  override def maxVal = Double.MaxValue / 2
+
+  override def aggregator: Aggregate[Double] = new DoubleMaxAggregate()
+}
+
+class BooleanMaxAggregateTest extends AggregateTestBase[Boolean] {
+
+  override def inputValueSets: Seq[Seq[Boolean]] = Seq(
+    Seq(
+      false,
+      false,
+      false
+    ),
+    Seq(
+      true,
+      true,
+      true
+    ),
+    Seq(
+      true,
+      false,
+      null.asInstanceOf[Boolean],
+      true,
+      false,
+      true,
+      null.asInstanceOf[Boolean]
+    ),
+    Seq(
+      null.asInstanceOf[Boolean],
+      null.asInstanceOf[Boolean],
+      null.asInstanceOf[Boolean]
+    )
+  )
+
+  override def expectedResults: Seq[Boolean] = Seq(
+    false,
+    true,
+    true,
+    null.asInstanceOf[Boolean]
+  )
+
+  override def aggregator: Aggregate[Boolean] = new BooleanMaxAggregate()
+}
+
+class DecimalMaxAggregateTest extends AggregateTestBase[BigDecimal] {
+
+  override def inputValueSets: Seq[Seq[_]] = Seq(
+    Seq(
+      new BigDecimal("1"),
+      new BigDecimal("1000.000001"),
+      new BigDecimal("-1"),
+      new BigDecimal("-999.998999"),
+      null,
+      new BigDecimal("0"),
+      new BigDecimal("-999.999"),
+      null,
+      new BigDecimal("999.999")
+    ),
+    Seq(
+      null,
+      null,
+      null,
+      null,
+      null
+    )
+  )
+
+  override def expectedResults: Seq[BigDecimal] = Seq(
+    new BigDecimal("1000.000001"),
+    null
+  )
+
+  override def aggregator: Aggregate[BigDecimal] = new DecimalMaxAggregate()
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/MinAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/MinAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/MinAggregateTest.scala
new file mode 100644
index 0000000..3e2404d
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/MinAggregateTest.scala
@@ -0,0 +1,177 @@
+/*
+ * 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.math.BigDecimal
+
+abstract class MinAggregateTestBase[T: Numeric] extends AggregateTestBase[T] {
+
+  private val numeric: Numeric[T] = implicitly[Numeric[T]]
+
+  def minVal: T
+  def maxVal: T
+
+  override def inputValueSets: Seq[Seq[T]] = Seq(
+    Seq(
+      numeric.fromInt(1),
+      null.asInstanceOf[T],
+      maxVal,
+      numeric.fromInt(-99),
+      numeric.fromInt(3),
+      numeric.fromInt(56),
+      numeric.fromInt(0),
+      minVal,
+      numeric.fromInt(-20),
+      numeric.fromInt(17),
+      null.asInstanceOf[T]
+    ),
+    Seq(
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T]
+    )
+  )
+
+  override def expectedResults: Seq[T] = Seq(
+    minVal,
+    null.asInstanceOf[T]
+  )
+}
+
+class ByteMinAggregateTest extends MinAggregateTestBase[Byte] {
+
+  override def minVal = (Byte.MinValue + 1).toByte
+  override def maxVal = (Byte.MaxValue - 1).toByte
+
+  override def aggregator: Aggregate[Byte] = new ByteMinAggregate()
+}
+
+class ShortMinAggregateTest extends MinAggregateTestBase[Short] {
+
+  override def minVal = (Short.MinValue + 1).toShort
+  override def maxVal = (Short.MaxValue - 1).toShort
+
+  override def aggregator: Aggregate[Short] = new ShortMinAggregate()
+}
+
+class IntMinAggregateTest extends MinAggregateTestBase[Int] {
+
+  override def minVal = Int.MinValue + 1
+  override def maxVal = Int.MaxValue - 1
+
+  override def aggregator: Aggregate[Int] = new IntMinAggregate()
+}
+
+class LongMinAggregateTest extends MinAggregateTestBase[Long] {
+
+  override def minVal = Long.MinValue + 1
+  override def maxVal = Long.MaxValue - 1
+
+  override def aggregator: Aggregate[Long] = new LongMinAggregate()
+}
+
+class FloatMinAggregateTest extends MinAggregateTestBase[Float] {
+
+  override def minVal = Float.MinValue / 2
+  override def maxVal = Float.MaxValue / 2
+
+  override def aggregator: Aggregate[Float] = new FloatMinAggregate()
+}
+
+class DoubleMinAggregateTest extends MinAggregateTestBase[Double] {
+
+  override def minVal = Double.MinValue / 2
+  override def maxVal = Double.MaxValue / 2
+
+  override def aggregator: Aggregate[Double] = new DoubleMinAggregate()
+}
+
+class BooleanMinAggregateTest extends AggregateTestBase[Boolean] {
+
+  override def inputValueSets: Seq[Seq[Boolean]] = Seq(
+    Seq(
+      false,
+      false,
+      false
+    ),
+    Seq(
+      true,
+      true,
+      true
+    ),
+    Seq(
+      true,
+      false,
+      null.asInstanceOf[Boolean],
+      true,
+      false,
+      true,
+      null.asInstanceOf[Boolean]
+    ),
+    Seq(
+      null.asInstanceOf[Boolean],
+      null.asInstanceOf[Boolean],
+      null.asInstanceOf[Boolean]
+    )
+  )
+
+  override def expectedResults: Seq[Boolean] = Seq(
+    false,
+    true,
+    false,
+    null.asInstanceOf[Boolean]
+  )
+
+  override def aggregator: Aggregate[Boolean] = new BooleanMinAggregate()
+}
+
+class DecimalMinAggregateTest extends AggregateTestBase[BigDecimal] {
+
+  override def inputValueSets: Seq[Seq[_]] = Seq(
+    Seq(
+      new BigDecimal("1"),
+      new BigDecimal("1000"),
+      new BigDecimal("-1"),
+      new BigDecimal("-999.998999"),
+      null,
+      new BigDecimal("0"),
+      new BigDecimal("-999.999"),
+      null,
+      new BigDecimal("999.999")
+    ),
+    Seq(
+      null,
+      null,
+      null,
+      null,
+      null
+    )
+  )
+
+  override def expectedResults: Seq[BigDecimal] = Seq(
+    new BigDecimal("-999.999"),
+    null
+  )
+
+  override def aggregator: Aggregate[BigDecimal] = new DecimalMinAggregate()
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/SumAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/SumAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/SumAggregateTest.scala
new file mode 100644
index 0000000..c085334
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/aggregate/SumAggregateTest.scala
@@ -0,0 +1,137 @@
+/*
+ * 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.math.BigDecimal
+
+abstract class SumAggregateTestBase[T: Numeric] extends AggregateTestBase[T] {
+
+  private val numeric: Numeric[T] = implicitly[Numeric[T]]
+
+  def maxVal: T
+  private val minVal = numeric.negate(maxVal)
+
+  override def inputValueSets: Seq[Seq[T]] = Seq(
+    Seq(
+      minVal,
+      numeric.fromInt(1),
+      null.asInstanceOf[T],
+      numeric.fromInt(2),
+      numeric.fromInt(3),
+      numeric.fromInt(4),
+      numeric.fromInt(5),
+      numeric.fromInt(-10),
+      numeric.fromInt(-20),
+      numeric.fromInt(17),
+      null.asInstanceOf[T],
+      maxVal
+    ),
+    Seq(
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T],
+      null.asInstanceOf[T]
+    )
+  )
+
+  override def expectedResults: Seq[T] = Seq(
+    numeric.fromInt(2),
+    null.asInstanceOf[T]
+  )
+}
+
+class ByteSumAggregateTest extends SumAggregateTestBase[Byte] {
+
+  override def maxVal = (Byte.MaxValue / 2).toByte
+
+  override def aggregator: Aggregate[Byte] = new ByteSumAggregate
+}
+
+class ShortSumAggregateTest extends SumAggregateTestBase[Short] {
+
+  override def maxVal = (Short.MaxValue / 2).toShort
+
+  override def aggregator: Aggregate[Short] = new ShortSumAggregate
+}
+
+class IntSumAggregateTest extends SumAggregateTestBase[Int] {
+
+  override def maxVal = Int.MaxValue / 2
+
+  override def aggregator: Aggregate[Int] = new IntSumAggregate
+}
+
+class LongSumAggregateTest extends SumAggregateTestBase[Long] {
+
+  override def maxVal = Long.MaxValue / 2
+
+  override def aggregator: Aggregate[Long] = new LongSumAggregate
+}
+
+class FloatSumAggregateTest extends SumAggregateTestBase[Float] {
+
+  override def maxVal = 12345.6789f
+
+  override def aggregator: Aggregate[Float] = new FloatSumAggregate
+}
+
+class DoubleSumAggregateTest extends SumAggregateTestBase[Double] {
+
+  override def maxVal = 12345.6789d
+
+  override def aggregator: Aggregate[Double] = new DoubleSumAggregate
+}
+
+class DecimalSumAggregateTest extends AggregateTestBase[BigDecimal] {
+
+  override def inputValueSets: Seq[Seq[_]] = Seq(
+    Seq(
+      new BigDecimal("1"),
+      new BigDecimal("2"),
+      new BigDecimal("3"),
+      null,
+      new BigDecimal("0"),
+      new BigDecimal("-1000"),
+      new BigDecimal("0.000000000002"),
+      new BigDecimal("1000"),
+      new BigDecimal("-0.000000000001"),
+      new BigDecimal("999.999"),
+      null,
+      new BigDecimal("4"),
+      new BigDecimal("-999.999"),
+      null
+    ),
+    Seq(
+      null,
+      null,
+      null,
+      null,
+      null
+    )
+  )
+
+  override def expectedResults: Seq[BigDecimal] = Seq(
+    new BigDecimal("10.000000000001"),
+    null
+  )
+
+  override def aggregator: Aggregate[BigDecimal] = new DecimalSumAggregate()
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/dataset/DataSetCorrelateITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/dataset/DataSetCorrelateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/dataset/DataSetCorrelateITCase.scala
new file mode 100644
index 0000000..550669e
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/dataset/DataSetCorrelateITCase.scala
@@ -0,0 +1,177 @@
+/*
+ * 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.dataset
+
+import org.apache.flink.api.scala._
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.utils._
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+@RunWith(classOf[Parameterized])
+class DataSetCorrelateITCase(
+  mode: TestExecutionMode,
+  configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testCrossJoin(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
+    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
+
+    val func1 = new TableFunc1
+    val result = in.join(func1('c) as 's).select('c, 's).toDataSet[Row]
+    val results = result.collect()
+    val expected = "Jack#22,Jack\n" + "Jack#22,22\n" + "John#19,John\n" + "John#19,19\n" +
+      "Anna#44,Anna\n" + "Anna#44,44\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+
+    // with overloading
+    val result2 = in.join(func1('c, "$") as 's).select('c, 's).toDataSet[Row]
+    val results2 = result2.collect()
+    val expected2 = "Jack#22,$Jack\n" + "Jack#22,$22\n" + "John#19,$John\n" +
+      "John#19,$19\n" + "Anna#44,$Anna\n" + "Anna#44,$44\n"
+    TestBaseUtils.compareResultAsText(results2.asJava, expected2)
+  }
+
+  @Test
+  def testLeftOuterJoin(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
+    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
+
+    val func2 = new TableFunc2
+    val result = in.leftOuterJoin(func2('c) as ('s, 'l)).select('c, 's, 'l).toDataSet[Row]
+    val results = result.collect()
+    val expected = "Jack#22,Jack,4\n" + "Jack#22,22,2\n" + "John#19,John,4\n" +
+      "John#19,19,2\n" + "Anna#44,Anna,4\n" + "Anna#44,44,2\n" + "nosharp,null,null"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testWithFilter(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
+    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
+    val func0 = new TableFunc0
+
+    val result = in
+      .join(func0('c) as ('name, 'age))
+      .select('c, 'name, 'age)
+      .filter('age > 20)
+      .toDataSet[Row]
+
+    val results = result.collect()
+    val expected = "Jack#22,Jack,22\n" + "Anna#44,Anna,44\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testCustomReturnType(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
+    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
+    val func2 = new TableFunc2
+
+    val result = in
+      .join(func2('c) as ('name, 'len))
+      .select('c, 'name, 'len)
+      .toDataSet[Row]
+
+    val results = result.collect()
+    val expected = "Jack#22,Jack,4\n" + "Jack#22,22,2\n" + "John#19,John,4\n" +
+      "John#19,19,2\n" + "Anna#44,Anna,4\n" + "Anna#44,44,2\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testHierarchyType(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
+    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
+
+    val hierarchy = new HierarchyTableFunction
+    val result = in
+      .join(hierarchy('c) as ('name, 'adult, 'len))
+      .select('c, 'name, 'adult, 'len)
+      .toDataSet[Row]
+
+    val results = result.collect()
+    val expected = "Jack#22,Jack,true,22\n" + "John#19,John,false,19\n" +
+      "Anna#44,Anna,true,44\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testPojoType(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
+    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
+
+    val pojo = new PojoTableFunc()
+    val result = in
+      .join(pojo('c))
+      .select('c, 'name, 'age)
+      .toDataSet[Row]
+
+    val results = result.collect()
+    val expected = "Jack#22,Jack,22\n" + "John#19,John,19\n" + "Anna#44,Anna,44\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testUDTFWithScalarFunction(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
+    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
+    val func1 = new TableFunc1
+
+    val result = in
+      .join(func1('c.substring(2)) as 's)
+      .select('c, 's)
+      .toDataSet[Row]
+
+    val results = result.collect()
+    val expected = "Jack#22,ack\n" + "Jack#22,22\n" + "John#19,ohn\n" + "John#19,19\n" +
+      "Anna#44,nna\n" + "Anna#44,44\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  private def testData(
+      env: ExecutionEnvironment)
+    : DataSet[(Int, Long, String)] = {
+
+    val data = new mutable.MutableList[(Int, Long, String)]
+    data.+=((1, 1L, "Jack#22"))
+    data.+=((2, 2L, "John#19"))
+    data.+=((3, 2L, "Anna#44"))
+    data.+=((4, 3L, "nosharp"))
+    env.fromCollection(data)
+  }
+}


[19/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
new file mode 100644
index 0000000..9902486
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamAggregate.scala
@@ -0,0 +1,359 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+import org.apache.flink.table.calcite.{FlinkRelBuilder, FlinkTypeFactory}
+import FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.plan.nodes.FlinkAggregate
+import org.apache.flink.table.plan.nodes.datastream.DataStreamAggregate._
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo, TypeConverter}
+import org.apache.flink.streaming.api.datastream.{AllWindowedStream, DataStream, KeyedStream, WindowedStream}
+import org.apache.flink.streaming.api.windowing.assigners._
+import org.apache.flink.streaming.api.windowing.time.Time
+import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow}
+import org.apache.flink.table.api.StreamTableEnvironment
+
+import scala.collection.JavaConverters._
+
+class DataStreamAggregate(
+    window: LogicalWindow,
+    namedProperties: Seq[NamedWindowProperty],
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputNode: RelNode,
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    rowRelDataType: RelDataType,
+    inputType: RelDataType,
+    grouping: Array[Int])
+  extends SingleRel(cluster, traitSet, inputNode)
+  with FlinkAggregate
+  with DataStreamRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataStreamAggregate(
+      window,
+      namedProperties,
+      cluster,
+      traitSet,
+      inputs.get(0),
+      namedAggregates,
+      getRowType,
+      inputType,
+      grouping)
+  }
+
+  override def toString: String = {
+    s"Aggregate(${
+      if (!grouping.isEmpty) {
+        s"groupBy: (${groupingToString(inputType, grouping)}), "
+      } else {
+        ""
+      }
+    }window: ($window), " +
+      s"select: (${
+        aggregationToString(
+          inputType,
+          grouping,
+          getRowType,
+          namedAggregates,
+          namedProperties)
+      }))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty)
+      .item("window", window)
+      .item("select", aggregationToString(
+        inputType,
+        grouping,
+        getRowType,
+        namedAggregates,
+        namedProperties))
+  }
+
+  override def translateToPlan(
+    tableEnv: StreamTableEnvironment,
+    expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {
+
+    val config = tableEnv.getConfig
+    val groupingKeys = grouping.indices.toArray
+    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(
+      tableEnv,
+      // tell the input operator that this operator currently only supports Rows as input
+      Some(TypeConverter.DEFAULT_ROW_TYPE))
+
+    // get the output types
+    val fieldTypes: Array[TypeInformation[_]] =
+      getRowType.getFieldList.asScala
+      .map(field => FlinkTypeFactory.toTypeInfo(field.getType))
+      .toArray
+
+    val rowTypeInfo = new RowTypeInfo(fieldTypes: _*)
+
+    val aggString = aggregationToString(
+      inputType,
+      grouping,
+      getRowType,
+      namedAggregates,
+      namedProperties)
+
+    val prepareOpName = s"prepare select: ($aggString)"
+    val keyedAggOpName = s"groupBy: (${groupingToString(inputType, grouping)}), " +
+      s"window: ($window), " +
+      s"select: ($aggString)"
+    val nonKeyedAggOpName = s"window: ($window), select: ($aggString)"
+
+    val mapFunction = AggregateUtil.createPrepareMapFunction(
+      namedAggregates,
+      grouping,
+      inputType)
+
+    val mappedInput = inputDS.map(mapFunction).name(prepareOpName)
+
+    val result: DataStream[Any] = {
+      // check whether all aggregates support partial aggregate
+      if (AggregateUtil.doAllSupportPartialAggregation(
+            namedAggregates.map(_.getKey),
+            inputType,
+            grouping.length)) {
+        // do Incremental Aggregation
+        val reduceFunction = AggregateUtil.createIncrementalAggregateReduceFunction(
+          namedAggregates,
+          inputType,
+          getRowType,
+          grouping)
+        // grouped / keyed aggregation
+        if (groupingKeys.length > 0) {
+          val windowFunction = AggregateUtil.createWindowIncrementalAggregationFunction(
+            window,
+            namedAggregates,
+            inputType,
+            rowRelDataType,
+            grouping,
+            namedProperties)
+
+          val keyedStream = mappedInput.keyBy(groupingKeys: _*)
+          val windowedStream =
+            createKeyedWindowedStream(window, keyedStream)
+            .asInstanceOf[WindowedStream[Row, Tuple, DataStreamWindow]]
+
+          windowedStream
+          .apply(reduceFunction, windowFunction)
+          .returns(rowTypeInfo)
+          .name(keyedAggOpName)
+          .asInstanceOf[DataStream[Any]]
+        }
+        // global / non-keyed aggregation
+        else {
+          val windowFunction = AggregateUtil.createAllWindowIncrementalAggregationFunction(
+            window,
+            namedAggregates,
+            inputType,
+            rowRelDataType,
+            grouping,
+            namedProperties)
+
+          val windowedStream =
+            createNonKeyedWindowedStream(window, mappedInput)
+            .asInstanceOf[AllWindowedStream[Row, DataStreamWindow]]
+
+          windowedStream
+          .apply(reduceFunction, windowFunction)
+          .returns(rowTypeInfo)
+          .name(nonKeyedAggOpName)
+          .asInstanceOf[DataStream[Any]]
+        }
+      }
+      else {
+        // do non-Incremental Aggregation
+        // grouped / keyed aggregation
+        if (groupingKeys.length > 0) {
+
+          val windowFunction = AggregateUtil.createWindowAggregationFunction(
+            window,
+            namedAggregates,
+            inputType,
+            rowRelDataType,
+            grouping,
+            namedProperties)
+
+          val keyedStream = mappedInput.keyBy(groupingKeys: _*)
+          val windowedStream =
+            createKeyedWindowedStream(window, keyedStream)
+            .asInstanceOf[WindowedStream[Row, Tuple, DataStreamWindow]]
+
+          windowedStream
+          .apply(windowFunction)
+          .returns(rowTypeInfo)
+          .name(keyedAggOpName)
+          .asInstanceOf[DataStream[Any]]
+        }
+        // global / non-keyed aggregation
+        else {
+          val windowFunction = AggregateUtil.createAllWindowAggregationFunction(
+            window,
+            namedAggregates,
+            inputType,
+            rowRelDataType,
+            grouping,
+            namedProperties)
+
+          val windowedStream =
+            createNonKeyedWindowedStream(window, mappedInput)
+            .asInstanceOf[AllWindowedStream[Row, DataStreamWindow]]
+
+          windowedStream
+          .apply(windowFunction)
+          .returns(rowTypeInfo)
+          .name(nonKeyedAggOpName)
+          .asInstanceOf[DataStream[Any]]
+        }
+      }
+    }
+    // if the expected type is not a Row, inject a mapper to convert to the expected type
+    expectedType match {
+      case Some(typeInfo) if typeInfo.getTypeClass != classOf[Row] =>
+        val mapName = s"convert: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
+        result.map(getConversionMapper(
+          config = config,
+          nullableInput = false,
+          inputType = rowTypeInfo.asInstanceOf[TypeInformation[Any]],
+          expectedType = expectedType.get,
+          conversionOperatorName = "DataStreamAggregateConversion",
+          fieldNames = getRowType.getFieldNames.asScala
+        ))
+          .name(mapName)
+      case _ => result
+    }
+  }
+}
+object DataStreamAggregate {
+
+
+  private def createKeyedWindowedStream(groupWindow: LogicalWindow, stream: KeyedStream[Row, Tuple])
+    : WindowedStream[Row, Tuple, _ <: DataStreamWindow] = groupWindow match {
+
+    case ProcessingTimeTumblingGroupWindow(_, size) if isTimeInterval(size.resultType) =>
+      stream.window(TumblingProcessingTimeWindows.of(asTime(size)))
+
+    case ProcessingTimeTumblingGroupWindow(_, size) =>
+      stream.countWindow(asCount(size))
+
+    case EventTimeTumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
+      stream.window(TumblingEventTimeWindows.of(asTime(size)))
+
+    case EventTimeTumblingGroupWindow(_, _, size) =>
+      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
+      // before applying the  windowing logic. Otherwise, this would be the same as a
+      // ProcessingTimeTumblingGroupWindow
+      throw new UnsupportedOperationException("Event-time grouping windows on row intervals are " +
+        "currently not supported.")
+
+    case ProcessingTimeSlidingGroupWindow(_, size, slide) if isTimeInterval(size.resultType) =>
+      stream.window(SlidingProcessingTimeWindows.of(asTime(size), asTime(slide)))
+
+    case ProcessingTimeSlidingGroupWindow(_, size, slide) =>
+      stream.countWindow(asCount(size), asCount(slide))
+
+    case EventTimeSlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) =>
+      stream.window(SlidingEventTimeWindows.of(asTime(size), asTime(slide)))
+
+    case EventTimeSlidingGroupWindow(_, _, size, slide) =>
+      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
+      // before applying the  windowing logic. Otherwise, this would be the same as a
+      // ProcessingTimeTumblingGroupWindow
+      throw new UnsupportedOperationException("Event-time grouping windows on row intervals are " +
+        "currently not supported.")
+
+    case ProcessingTimeSessionGroupWindow(_, gap: Expression) =>
+      stream.window(ProcessingTimeSessionWindows.withGap(asTime(gap)))
+
+    case EventTimeSessionGroupWindow(_, _, gap) =>
+      stream.window(EventTimeSessionWindows.withGap(asTime(gap)))
+  }
+
+  private def createNonKeyedWindowedStream(groupWindow: LogicalWindow, stream: DataStream[Row])
+    : AllWindowedStream[Row, _ <: DataStreamWindow] = groupWindow match {
+
+    case ProcessingTimeTumblingGroupWindow(_, size) if isTimeInterval(size.resultType) =>
+      stream.windowAll(TumblingProcessingTimeWindows.of(asTime(size)))
+
+    case ProcessingTimeTumblingGroupWindow(_, size) =>
+      stream.countWindowAll(asCount(size))
+
+    case EventTimeTumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
+      stream.windowAll(TumblingEventTimeWindows.of(asTime(size)))
+
+    case EventTimeTumblingGroupWindow(_, _, size) =>
+      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
+      // before applying the  windowing logic. Otherwise, this would be the same as a
+      // ProcessingTimeTumblingGroupWindow
+      throw new UnsupportedOperationException("Event-time grouping windows on row intervals are " +
+        "currently not supported.")
+
+    case ProcessingTimeSlidingGroupWindow(_, size, slide) if isTimeInterval(size.resultType) =>
+      stream.windowAll(SlidingProcessingTimeWindows.of(asTime(size), asTime(slide)))
+
+    case ProcessingTimeSlidingGroupWindow(_, size, slide) =>
+      stream.countWindowAll(asCount(size), asCount(slide))
+
+    case EventTimeSlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) =>
+      stream.windowAll(SlidingEventTimeWindows.of(asTime(size), asTime(slide)))
+
+    case EventTimeSlidingGroupWindow(_, _, size, slide) =>
+      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
+      // before applying the  windowing logic. Otherwise, this would be the same as a
+      // ProcessingTimeTumblingGroupWindow
+      throw new UnsupportedOperationException("Event-time grouping windows on row intervals are " +
+        "currently not supported.")
+
+    case ProcessingTimeSessionGroupWindow(_, gap) =>
+      stream.windowAll(ProcessingTimeSessionWindows.withGap(asTime(gap)))
+
+    case EventTimeSessionGroupWindow(_, _, gap) =>
+      stream.windowAll(EventTimeSessionWindows.withGap(asTime(gap)))
+  }
+
+  def asTime(expr: Expression): Time = expr match {
+    case Literal(value: Long, TimeIntervalTypeInfo.INTERVAL_MILLIS) => Time.milliseconds(value)
+    case _ => throw new IllegalArgumentException()
+  }
+
+  def asCount(expr: Expression): Long = expr match {
+    case Literal(value: Long, RowIntervalTypeInfo.INTERVAL_ROWS) => value
+    case _ => throw new IllegalArgumentException()
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..774c17b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCalc.scala
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.calcite.rex.RexProgram
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.plan.nodes.FlinkCalc
+import org.apache.flink.table.typeutils.TypeConverter._
+import org.apache.flink.api.common.functions.FlatMapFunction
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.StreamTableEnvironment
+
+/**
+  * Flink RelNode which matches along with FlatMapOperator.
+  *
+  */
+class DataStreamCalc(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    input: RelNode,
+    rowRelDataType: RelDataType,
+    calcProgram: RexProgram,
+    ruleDescription: String)
+  extends SingleRel(cluster, traitSet, input)
+  with FlinkCalc
+  with DataStreamRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataStreamCalc(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      getRowType,
+      calcProgram,
+      ruleDescription
+    )
+  }
+
+  override def toString: String = calcToString(calcProgram, getExpressionString)
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .item("select", selectionToString(calcProgram, getExpressionString))
+      .itemIf("where",
+        conditionToString(calcProgram, getExpressionString),
+        calcProgram.getCondition != null)
+  }
+
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {
+
+    val config = tableEnv.getConfig
+
+    val inputDataStream = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+
+    val returnType = determineReturnType(
+      getRowType,
+      expectedType,
+      config.getNullCheck,
+      config.getEfficientTypeUsage)
+
+    val generator = new CodeGenerator(config, false, inputDataStream.getType)
+
+    val body = functionBody(
+      generator,
+      inputDataStream.getType,
+      getRowType,
+      calcProgram,
+      config,
+      expectedType)
+
+    val genFunction = generator.generateFunction(
+      ruleDescription,
+      classOf[FlatMapFunction[Any, Any]],
+      body,
+      returnType)
+
+    val mapFunc = calcMapFunction(genFunction)
+    inputDataStream.flatMap(mapFunc).name(calcOpName(calcProgram, getExpressionString))
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamConvention.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamConvention.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamConvention.scala
new file mode 100644
index 0000000..9525d1f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamConvention.scala
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+
+class DataStreamConvention extends Convention {
+
+  override def toString: String = getName
+
+  override def useAbstractConvertersForConversion(
+    fromTraits: RelTraitSet,
+    toTraits: RelTraitSet): Boolean = false
+
+  override def canConvertConvention(toConvention: Convention): Boolean = false
+
+  def getInterface: Class[_] = classOf[DataStreamRel]
+
+  def getName: String = "DATASTREAM"
+
+  def getTraitDef: RelTraitDef[_ <: RelTrait] = ConventionTraitDef.INSTANCE
+
+  def satisfies(`trait`: RelTrait): Boolean = this eq `trait`
+
+  def register(planner: RelOptPlanner): Unit = { }
+}
+
+object DataStreamConvention {
+
+  val INSTANCE = new DataStreamConvention
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..a2d167b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamCorrelate.scala
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan
+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.table.codegen.CodeGenerator
+import org.apache.flink.table.functions.utils.TableSqlFunction
+import org.apache.flink.table.plan.nodes.FlinkCorrelate
+import org.apache.flink.table.typeutils.TypeConverter._
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.StreamTableEnvironment
+
+/**
+  * Flink RelNode which matches along with join a user defined table function.
+  */
+class DataStreamCorrelate(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputNode: RelNode,
+    scan: LogicalTableFunctionScan,
+    condition: Option[RexNode],
+    relRowType: RelDataType,
+    joinRowType: RelDataType,
+    joinType: SemiJoinType,
+    ruleDescription: String)
+  extends SingleRel(cluster, traitSet, inputNode)
+  with FlinkCorrelate
+  with DataStreamRel {
+
+  override def deriveRowType() = relRowType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataStreamCorrelate(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      scan,
+      condition,
+      relRowType,
+      joinRowType,
+      joinType,
+      ruleDescription)
+  }
+
+  override def toString: String = {
+    val rexCall = scan.getCall.asInstanceOf[RexCall]
+    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
+    correlateToString(rexCall, sqlFunction)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    val rexCall = scan.getCall.asInstanceOf[RexCall]
+    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
+    super.explainTerms(pw)
+      .item("invocation", scan.getCall)
+      .item("function", sqlFunction.getTableFunction.getClass.getCanonicalName)
+      .item("rowType", relRowType)
+      .item("joinType", joinType)
+      .itemIf("condition", condition.orNull, condition.isDefined)
+  }
+
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      expectedType: Option[TypeInformation[Any]])
+    : DataStream[Any] = {
+
+    val config = tableEnv.getConfig
+    val returnType = determineReturnType(
+      getRowType,
+      expectedType,
+      config.getNullCheck,
+      config.getEfficientTypeUsage)
+
+    // we do not need to specify input type
+    val inputDS = inputNode.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+
+    val funcRel = scan.asInstanceOf[LogicalTableFunctionScan]
+    val rexCall = funcRel.getCall.asInstanceOf[RexCall]
+    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
+    val pojoFieldMapping = sqlFunction.getPojoFieldMapping
+    val udtfTypeInfo = sqlFunction.getRowTypeInfo.asInstanceOf[TypeInformation[Any]]
+
+    val generator = new CodeGenerator(
+      config,
+      false,
+      inputDS.getType,
+      Some(udtfTypeInfo),
+      None,
+      Some(pojoFieldMapping))
+
+    val body = functionBody(
+      generator,
+      udtfTypeInfo,
+      getRowType,
+      rexCall,
+      condition,
+      config,
+      joinType,
+      expectedType)
+
+    val genFunction = generator.generateFunction(
+      ruleDescription,
+      classOf[FlatMapFunction[Any, Any]],
+      body,
+      returnType)
+
+    val mapFunc = correlateMapFunction(genFunction)
+
+    inputDS.flatMap(mapFunc).name(correlateOpName(rexCall, sqlFunction, relRowType))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..16427b8
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamRel.scala
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.rel.RelNode
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.plan.nodes.FlinkRel
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.StreamTableEnvironment
+
+trait DataStreamRel extends RelNode with FlinkRel {
+
+  /**
+    * Translates the FlinkRelNode into a Flink operator.
+    *
+    * @param tableEnv The [[StreamTableEnvironment]] of the translated Table.
+    * @param expectedType specifies the type the Flink operator should return. The type must
+    *                     have the same arity as the result. For instance, if the
+    *                     expected type is a RowTypeInfo this method will return a DataSet of
+    *                     type Row. If the expected type is Tuple2, the operator will return
+    *                     a Tuple2 if possible. Row otherwise.
+    * @return DataStream of type expectedType or RowTypeInfo
+    */
+  def translateToPlan(
+    tableEnv: StreamTableEnvironment,
+    expectedType: Option[TypeInformation[Any]] = None) : DataStream[Any]
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..2d5ec09
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamScan.scala
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.plan.schema.DataStreamTable
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.StreamTableEnvironment
+
+/**
+  * Flink RelNode which matches along with DataStreamSource.
+  * It ensures that types without deterministic field order (e.g. POJOs) are not part of
+  * the plan translation.
+  */
+class DataStreamScan(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    table: RelOptTable,
+    rowRelDataType: RelDataType)
+  extends StreamScan(cluster, traitSet, table) {
+
+  val dataStreamTable: DataStreamTable[Any] = getTable.unwrap(classOf[DataStreamTable[Any]])
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataStreamScan(
+      cluster,
+      traitSet,
+      getTable,
+      getRowType
+    )
+  }
+
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {
+
+    val config = tableEnv.getConfig
+    val inputDataStream: DataStream[Any] = dataStreamTable.dataStream
+
+    convertToExpectedType(inputDataStream, dataStreamTable, expectedType, config)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..beb15d2
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamUnion.scala
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.StreamTableEnvironment
+
+import scala.collection.JavaConverters._
+
+/**
+  * Flink RelNode which matches along with Union.
+  *
+  */
+class DataStreamUnion(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    leftNode: RelNode,
+    rightNode: RelNode,
+    rowRelDataType: RelDataType)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+  with DataStreamRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataStreamUnion(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      inputs.get(1),
+      getRowType
+    )
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw).item("union", unionSelectionToString)
+  }
+
+  override def toString = {
+    s"Union(union: (${getRowType.getFieldNames.asScala.toList.mkString(", ")}))"
+  }
+
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {
+
+    val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+    leftDataSet.union(rightDataSet)
+  }
+
+  private def unionSelectionToString: String = {
+    getRowType.getFieldNames.asScala.toList.mkString(", ")
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..f2a3d72
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamValues.scala
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import com.google.common.collect.ImmutableList
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.Values
+import org.apache.calcite.rex.RexLiteral
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.runtime.io.ValuesInputFormat
+import org.apache.flink.table.typeutils.TypeConverter._
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.StreamTableEnvironment
+
+import scala.collection.JavaConverters._
+
+/**
+  * DataStream RelNode for LogicalValues.
+  */
+class DataStreamValues(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    rowRelDataType: RelDataType,
+    tuples: ImmutableList[ImmutableList[RexLiteral]],
+    ruleDescription: String)
+  extends Values(cluster, rowRelDataType, tuples, traitSet)
+  with DataStreamRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataStreamValues(
+      cluster,
+      traitSet,
+      getRowType,
+      getTuples,
+      ruleDescription
+    )
+  }
+
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      expectedType: Option[TypeInformation[Any]])
+    : DataStream[Any] = {
+
+    val config = tableEnv.getConfig
+
+    val returnType = determineReturnType(
+      getRowType,
+      expectedType,
+      config.getNullCheck,
+      config.getEfficientTypeUsage)
+
+    val generator = new CodeGenerator(config)
+
+    // generate code for every record
+    val generatedRecords = getTuples.asScala.map { r =>
+      generator.generateResultExpression(
+        returnType,
+        getRowType.getFieldNames.asScala,
+        r.asScala)
+    }
+
+    // generate input format
+    val generatedFunction = generator.generateValuesInputFormat(
+      ruleDescription,
+      generatedRecords.map(_.code),
+      returnType)
+
+    val inputFormat = new ValuesInputFormat[Any](
+      generatedFunction.name,
+      generatedFunction.code,
+      generatedFunction.returnType)
+
+    tableEnv.execEnv.createInput(inputFormat, returnType).asInstanceOf[DataStream[Any]]
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
new file mode 100644
index 0000000..ddac958
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamScan.scala
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.TableScan
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.PojoTypeInfo
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.plan.schema.FlinkTable
+import org.apache.flink.table.runtime.MapRunner
+import org.apache.flink.table.typeutils.TypeConverter.determineReturnType
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.TableConfig
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+abstract class StreamScan(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    table: RelOptTable)
+  extends TableScan(cluster, traitSet, table)
+  with DataStreamRel {
+
+  protected def convertToExpectedType(
+      input: DataStream[Any],
+      flinkTable: FlinkTable[_],
+      expectedType: Option[TypeInformation[Any]],
+      config: TableConfig): DataStream[Any] = {
+
+    val inputType = input.getType
+
+    expectedType match {
+
+      // special case:
+      // if efficient type usage is enabled and no expected type is set
+      // we can simply forward the DataSet to the next operator.
+      // however, we cannot forward PojoTypes as their fields don't have an order
+      case None if config.getEfficientTypeUsage && !inputType.isInstanceOf[PojoTypeInfo[_]] =>
+        input
+
+      case _ =>
+        val determinedType = determineReturnType(
+          getRowType,
+          expectedType,
+          config.getNullCheck,
+          config.getEfficientTypeUsage)
+
+        // conversion
+        if (determinedType != inputType) {
+          val generator = new CodeGenerator(
+            config,
+            nullableInput = false,
+            input.getType,
+            flinkTable.fieldIndexes)
+
+          val conversion = generator.generateConverterResultExpression(
+            determinedType,
+            getRowType.getFieldNames)
+
+          val body =
+            s"""
+               |${conversion.code}
+               |return ${conversion.resultTerm};
+               |""".stripMargin
+
+          val genFunction = generator.generateFunction(
+            "DataSetSourceConversion",
+            classOf[MapFunction[Any, Any]],
+            body,
+            determinedType)
+
+          val mapFunc = new MapRunner[Any, Any](
+            genFunction.name,
+            genFunction.code,
+            genFunction.returnType)
+
+          val opName = s"from: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
+
+          input.map(mapFunc).name(opName)
+        }
+        // no conversion necessary, forward
+        else {
+          input
+        }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..f86a54b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/StreamTableSourceScan.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.datastream
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.RelNode
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.schema.TableSourceTable
+import org.apache.flink.table.sources.StreamTableSource
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.StreamTableEnvironment
+
+/** Flink RelNode to read data from an external source defined by a [[StreamTableSource]]. */
+class StreamTableSourceScan(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    table: RelOptTable,
+    tableSource: StreamTableSource[_])
+  extends StreamScan(cluster, traitSet, table) {
+
+  override def deriveRowType() = {
+    val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+    flinkTypeFactory.buildRowDataType(tableSource.getFieldsNames, tableSource.getFieldTypes)
+  }
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new StreamTableSourceScan(
+      cluster,
+      traitSet,
+      getTable,
+      tableSource
+    )
+  }
+
+  override def translateToPlan(
+      tableEnv: StreamTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {
+
+    val config = tableEnv.getConfig
+    val inputDataStream: DataStream[Any] = tableSource
+      .getDataStream(tableEnv.execEnv).asInstanceOf[DataStream[Any]]
+
+    convertToExpectedType(inputDataStream, new TableSourceTable(tableSource), expectedType, config)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/EnumerableToLogicalTableScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/EnumerableToLogicalTableScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/EnumerableToLogicalTableScan.scala
new file mode 100644
index 0000000..86b8a23
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/EnumerableToLogicalTableScan.scala
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules
+
+import org.apache.calcite.plan.RelOptRule.{any, operand}
+import org.apache.calcite.adapter.enumerable.EnumerableTableScan
+import org.apache.calcite.plan.{RelOptRuleCall, RelOptRule, RelOptRuleOperand}
+import org.apache.calcite.rel.logical.LogicalTableScan
+
+/**
+ * Rule that converts an EnumerableTableScan into a LogicalTableScan.
+ * We need this rule because Calcite creates an EnumerableTableScan
+ * when parsing a SQL query. We convert it into a LogicalTableScan
+ * so we can merge the optimization process with any plan that might be created
+ * by the Table API.
+ */
+class EnumerableToLogicalTableScan(
+    operand: RelOptRuleOperand,
+    description: String) extends RelOptRule(operand, description) {
+
+  override def onMatch(call: RelOptRuleCall): Unit = {
+    val oldRel = call.rel(0).asInstanceOf[EnumerableTableScan]
+    val table = oldRel.getTable
+    val newRel = LogicalTableScan.create(oldRel.getCluster, table)
+    call.transformTo(newRel)
+  }
+}
+
+object EnumerableToLogicalTableScan {
+  val INSTANCE = new EnumerableToLogicalTableScan(
+      operand(classOf[EnumerableTableScan], any),
+    "EnumerableToLogicalTableScan")
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
new file mode 100644
index 0000000..bcd12a4
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules
+
+import org.apache.calcite.rel.rules._
+import org.apache.calcite.tools.{RuleSets, RuleSet}
+import org.apache.flink.table.plan.rules.dataSet._
+import org.apache.flink.table.plan.rules.datastream._
+import org.apache.flink.table.plan.rules.datastream.{DataStreamCalcRule, DataStreamScanRule, DataStreamUnionRule}
+
+object FlinkRuleSets {
+
+  /**
+    * RuleSet to optimize plans for batch / DataSet execution
+    */
+  val DATASET_OPT_RULES: RuleSet = RuleSets.ofList(
+
+    // convert a logical table scan to a relational expression
+    TableScanRule.INSTANCE,
+    EnumerableToLogicalTableScan.INSTANCE,
+
+    // push a filter into a join
+    FilterJoinRule.FILTER_ON_JOIN,
+    // push filter into the children of a join
+    FilterJoinRule.JOIN,
+    // push filter through an aggregation
+    FilterAggregateTransposeRule.INSTANCE,
+
+    // aggregation and projection rules
+    AggregateProjectMergeRule.INSTANCE,
+    AggregateProjectPullUpConstantsRule.INSTANCE,
+    // push a projection past a filter or vice versa
+    ProjectFilterTransposeRule.INSTANCE,
+    FilterProjectTransposeRule.INSTANCE,
+    // push a projection to the children of a join
+    ProjectJoinTransposeRule.INSTANCE,
+    // remove identity project
+    ProjectRemoveRule.INSTANCE,
+    // reorder sort and projection
+    SortProjectTransposeRule.INSTANCE,
+    ProjectSortTransposeRule.INSTANCE,
+
+    // join rules
+    JoinPushExpressionsRule.INSTANCE,
+
+    // remove union with only a single child
+    UnionEliminatorRule.INSTANCE,
+    // convert non-all union into all-union + distinct
+    UnionToDistinctRule.INSTANCE,
+
+    // remove aggregation if it does not aggregate and input is already distinct
+    AggregateRemoveRule.INSTANCE,
+    // push aggregate through join
+    AggregateJoinTransposeRule.EXTENDED,
+    // aggregate union rule
+    AggregateUnionAggregateRule.INSTANCE,
+
+    // remove unnecessary sort rule
+    SortRemoveRule.INSTANCE,
+
+    // simplify expressions rules
+    ReduceExpressionsRule.FILTER_INSTANCE,
+    ReduceExpressionsRule.PROJECT_INSTANCE,
+    ReduceExpressionsRule.CALC_INSTANCE,
+    ReduceExpressionsRule.JOIN_INSTANCE,
+
+    // prune empty results rules
+    PruneEmptyRules.AGGREGATE_INSTANCE,
+    PruneEmptyRules.FILTER_INSTANCE,
+    PruneEmptyRules.JOIN_LEFT_INSTANCE,
+    PruneEmptyRules.JOIN_RIGHT_INSTANCE,
+    PruneEmptyRules.PROJECT_INSTANCE,
+    PruneEmptyRules.SORT_INSTANCE,
+    PruneEmptyRules.UNION_INSTANCE,
+
+    // calc rules
+    FilterCalcMergeRule.INSTANCE,
+    ProjectCalcMergeRule.INSTANCE,
+    FilterToCalcRule.INSTANCE,
+    ProjectToCalcRule.INSTANCE,
+    CalcMergeRule.INSTANCE,
+
+    // translate to Flink DataSet nodes
+    DataSetAggregateRule.INSTANCE,
+    DataSetAggregateWithNullValuesRule.INSTANCE,
+    DataSetCalcRule.INSTANCE,
+    DataSetJoinRule.INSTANCE,
+    DataSetSingleRowJoinRule.INSTANCE,
+    DataSetScanRule.INSTANCE,
+    DataSetUnionRule.INSTANCE,
+    DataSetIntersectRule.INSTANCE,
+    DataSetMinusRule.INSTANCE,
+    DataSetSortRule.INSTANCE,
+    DataSetValuesRule.INSTANCE,
+    DataSetCorrelateRule.INSTANCE,
+    BatchTableSourceScanRule.INSTANCE,
+    // project pushdown optimization
+    PushProjectIntoBatchTableSourceScanRule.INSTANCE
+  )
+
+  /**
+  * RuleSet to optimize plans for stream / DataStream execution
+  */
+  val DATASTREAM_OPT_RULES: RuleSet = RuleSets.ofList(
+
+      // convert a logical table scan to a relational expression
+      TableScanRule.INSTANCE,
+      EnumerableToLogicalTableScan.INSTANCE,
+
+      // calc rules
+      FilterToCalcRule.INSTANCE,
+      ProjectToCalcRule.INSTANCE,
+      FilterCalcMergeRule.INSTANCE,
+      ProjectCalcMergeRule.INSTANCE,
+      CalcMergeRule.INSTANCE,
+
+      // prune empty results rules
+      PruneEmptyRules.FILTER_INSTANCE,
+      PruneEmptyRules.PROJECT_INSTANCE,
+      PruneEmptyRules.UNION_INSTANCE,
+
+      // push and merge projection rules
+      ProjectFilterTransposeRule.INSTANCE,
+      FilterProjectTransposeRule.INSTANCE,
+      ProjectRemoveRule.INSTANCE,
+
+      // simplify expressions rules
+      ReduceExpressionsRule.FILTER_INSTANCE,
+      ReduceExpressionsRule.PROJECT_INSTANCE,
+      ReduceExpressionsRule.CALC_INSTANCE,
+
+      // merge and push unions rules
+      UnionEliminatorRule.INSTANCE,
+
+      // translate to DataStream nodes
+      DataStreamAggregateRule.INSTANCE,
+      DataStreamCalcRule.INSTANCE,
+      DataStreamScanRule.INSTANCE,
+      DataStreamUnionRule.INSTANCE,
+      DataStreamValuesRule.INSTANCE,
+      DataStreamCorrelateRule.INSTANCE,
+      StreamTableSourceScanRule.INSTANCE
+  )
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala
new file mode 100644
index 0000000..d699585
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/BatchTableSourceScanRule.scala
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.logical.LogicalTableScan
+import org.apache.flink.table.plan.nodes.dataset.{BatchTableSourceScan, DataSetConvention}
+import org.apache.flink.table.plan.schema.TableSourceTable
+import org.apache.flink.table.sources.BatchTableSource
+
+/** Rule to convert a [[LogicalTableScan]] into a [[BatchTableSourceScan]]. */
+class BatchTableSourceScanRule
+  extends ConverterRule(
+      classOf[LogicalTableScan],
+      Convention.NONE,
+      DataSetConvention.INSTANCE,
+      "BatchTableSourceScanRule")
+  {
+
+  /** Rule must only match if TableScan targets a [[BatchTableSource]] */
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val scan: TableScan = call.rel(0).asInstanceOf[TableScan]
+    val dataSetTable = scan.getTable.unwrap(classOf[TableSourceTable])
+    dataSetTable match {
+      case tst: TableSourceTable =>
+        tst.tableSource match {
+          case _: BatchTableSource[_] =>
+            true
+          case _ =>
+            false
+        }
+      case _ =>
+        false
+    }
+  }
+
+  def convert(rel: RelNode): RelNode = {
+    val scan: TableScan = rel.asInstanceOf[TableScan]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+
+    val tableSource = scan.getTable.unwrap(classOf[TableSourceTable]).tableSource
+      .asInstanceOf[BatchTableSource[_]]
+    new BatchTableSourceScan(
+      rel.getCluster,
+      traitSet,
+      scan.getTable,
+      tableSource
+    )
+  }
+}
+
+object BatchTableSourceScanRule {
+  val INSTANCE: RelOptRule = new BatchTableSourceScanRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateRule.scala
new file mode 100644
index 0000000..d634a6c
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateRule.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.plan.rules.dataSet
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalAggregate
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.plan.nodes.dataset.{DataSetAggregate, DataSetConvention}
+
+import scala.collection.JavaConversions._
+
+class DataSetAggregateRule
+  extends ConverterRule(
+      classOf[LogicalAggregate],
+      Convention.NONE,
+      DataSetConvention.INSTANCE,
+      "DataSetAggregateRule")
+  {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val agg: LogicalAggregate = call.rel(0).asInstanceOf[LogicalAggregate]
+
+    //for non grouped agg sets should attach null row to source data
+    //need apply DataSetAggregateWithNullValuesRule
+    if (agg.getGroupSet.isEmpty) {
+      return false
+    }
+
+    // check if we have distinct aggregates
+    val distinctAggs = agg.getAggCallList.exists(_.isDistinct)
+    if (distinctAggs) {
+      throw TableException("DISTINCT aggregates are currently not supported.")
+    }
+
+    // check if we have grouping sets
+    val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet
+    if (groupSets || agg.indicator) {
+      throw TableException("GROUPING SETS are currently not supported.")
+    }
+
+    !distinctAggs && !groupSets && !agg.indicator
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+    val agg: LogicalAggregate = rel.asInstanceOf[LogicalAggregate]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+    val convInput: RelNode = RelOptRule.convert(agg.getInput, DataSetConvention.INSTANCE)
+
+    new DataSetAggregate(
+      rel.getCluster,
+      traitSet,
+      convInput,
+      agg.getNamedAggCalls,
+      rel.getRowType,
+      agg.getInput.getRowType,
+      agg.getGroupSet.toArray)
+    }
+  }
+
+object DataSetAggregateRule {
+  val INSTANCE: RelOptRule = new DataSetAggregateRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala
new file mode 100644
index 0000000..b708af4
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan._
+
+import scala.collection.JavaConversions._
+import com.google.common.collect.ImmutableList
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalUnion, LogicalValues}
+import org.apache.calcite.rex.RexLiteral
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.plan.nodes.dataset.{DataSetAggregate, DataSetConvention}
+
+/**
+  * Rule for insert [[org.apache.flink.types.Row]] with null records into a [[DataSetAggregate]]
+  * Rule apply for non grouped aggregate query
+  */
+class DataSetAggregateWithNullValuesRule
+  extends ConverterRule(
+    classOf[LogicalAggregate],
+    Convention.NONE,
+    DataSetConvention.INSTANCE,
+    "DataSetAggregateWithNullValuesRule")
+{
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val agg: LogicalAggregate = call.rel(0).asInstanceOf[LogicalAggregate]
+
+    //for grouped agg sets shouldn't attach of null row
+    //need apply other rules. e.g. [[DataSetAggregateRule]]
+    if (!agg.getGroupSet.isEmpty) {
+      return false
+    }
+
+    // check if we have distinct aggregates
+    val distinctAggs = agg.getAggCallList.exists(_.isDistinct)
+    if (distinctAggs) {
+      throw TableException("DISTINCT aggregates are currently not supported.")
+    }
+
+    // check if we have grouping sets
+    val groupSets = agg.getGroupSets.size() == 0 || agg.getGroupSets.get(0) != agg.getGroupSet
+    if (groupSets || agg.indicator) {
+      throw TableException("GROUPING SETS are currently not supported.")
+    }
+    !distinctAggs && !groupSets && !agg.indicator
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+    val agg: LogicalAggregate = rel.asInstanceOf[LogicalAggregate]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+    val cluster: RelOptCluster = rel.getCluster
+
+    val fieldTypes = agg.getInput.getRowType.getFieldList.map(_.getType)
+    val nullLiterals: ImmutableList[ImmutableList[RexLiteral]] =
+      ImmutableList.of(ImmutableList.copyOf[RexLiteral](
+        for (fieldType <- fieldTypes)
+          yield {
+            cluster.getRexBuilder.
+              makeLiteral(null, fieldType, false).asInstanceOf[RexLiteral]
+          }))
+
+    val logicalValues = LogicalValues.create(cluster, agg.getInput.getRowType, nullLiterals)
+    val logicalUnion = LogicalUnion.create(List(logicalValues, agg.getInput), true)
+
+    new DataSetAggregate(
+      cluster,
+      traitSet,
+      RelOptRule.convert(logicalUnion, DataSetConvention.INSTANCE),
+      agg.getNamedAggCalls,
+      rel.getRowType,
+      agg.getInput.getRowType,
+      agg.getGroupSet.toArray
+    )
+  }
+}
+
+object DataSetAggregateWithNullValuesRule {
+  val INSTANCE: RelOptRule = new DataSetAggregateWithNullValuesRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCalcRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCalcRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCalcRule.scala
new file mode 100644
index 0000000..1d100fa
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCalcRule.scala
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalCalc
+import org.apache.flink.table.plan.nodes.dataset.{DataSetCalc, DataSetConvention}
+
+class DataSetCalcRule
+  extends ConverterRule(
+      classOf[LogicalCalc],
+      Convention.NONE,
+      DataSetConvention.INSTANCE,
+      "DataSetCalcRule")
+  {
+
+    def convert(rel: RelNode): RelNode = {
+      val calc: LogicalCalc = rel.asInstanceOf[LogicalCalc]
+      val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+      val convInput: RelNode = RelOptRule.convert(calc.getInput, DataSetConvention.INSTANCE)
+
+      new DataSetCalc(
+        rel.getCluster,
+        traitSet,
+        convInput,
+        rel.getRowType,
+        calc.getProgram,
+        description)
+    }
+  }
+
+object DataSetCalcRule {
+  val INSTANCE: RelOptRule = new DataSetCalcRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCorrelateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCorrelateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCorrelateRule.scala
new file mode 100644
index 0000000..819bcae
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetCorrelateRule.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.plan.rules.dataSet
+
+import org.apache.calcite.plan.volcano.RelSubset
+import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.{LogicalFilter, LogicalCorrelate, LogicalTableFunctionScan}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.plan.nodes.dataset.{DataSetConvention, DataSetCorrelate}
+
+/**
+  * Rule to convert a LogicalCorrelate into a DataSetCorrelate.
+  */
+class DataSetCorrelateRule
+  extends ConverterRule(
+      classOf[LogicalCorrelate],
+      Convention.NONE,
+      DataSetConvention.INSTANCE,
+      "DataSetCorrelateRule") {
+
+    override def matches(call: RelOptRuleCall): Boolean = {
+      val join: LogicalCorrelate = call.rel(0).asInstanceOf[LogicalCorrelate]
+      val right = join.getRight.asInstanceOf[RelSubset].getOriginal
+
+
+      right match {
+        // right node is a table function
+        case scan: LogicalTableFunctionScan => true
+        // a filter is pushed above the table function
+        case filter: LogicalFilter =>
+          filter
+            .getInput.asInstanceOf[RelSubset]
+            .getOriginal
+            .isInstanceOf[LogicalTableFunctionScan]
+        case _ => false
+      }
+    }
+
+    override def convert(rel: RelNode): RelNode = {
+      val join: LogicalCorrelate = rel.asInstanceOf[LogicalCorrelate]
+      val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+      val convInput: RelNode = RelOptRule.convert(join.getInput(0), DataSetConvention.INSTANCE)
+      val right: RelNode = join.getInput(1)
+
+      def convertToCorrelate(relNode: RelNode, condition: Option[RexNode]): DataSetCorrelate = {
+        relNode match {
+          case rel: RelSubset =>
+            convertToCorrelate(rel.getRelList.get(0), condition)
+
+          case filter: LogicalFilter =>
+            convertToCorrelate(
+              filter.getInput.asInstanceOf[RelSubset].getOriginal,
+              Some(filter.getCondition))
+
+          case scan: LogicalTableFunctionScan =>
+            new DataSetCorrelate(
+              rel.getCluster,
+              traitSet,
+              convInput,
+              scan,
+              condition,
+              rel.getRowType,
+              join.getRowType,
+              join.getJoinType,
+              description)
+        }
+      }
+      convertToCorrelate(right, None)
+    }
+  }
+
+object DataSetCorrelateRule {
+  val INSTANCE: RelOptRule = new DataSetCorrelateRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetIntersectRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetIntersectRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetIntersectRule.scala
new file mode 100644
index 0000000..d158f34
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetIntersectRule.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalIntersect
+import org.apache.flink.table.plan.nodes.dataset.{DataSetIntersect, DataSetConvention}
+
+class DataSetIntersectRule
+  extends ConverterRule(
+    classOf[LogicalIntersect],
+    Convention.NONE,
+    DataSetConvention.INSTANCE,
+    "DataSetIntersectRule")
+{
+
+  def convert(rel: RelNode): RelNode = {
+
+    val intersect: LogicalIntersect = rel.asInstanceOf[LogicalIntersect]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+    val convLeft: RelNode = RelOptRule.convert(intersect.getInput(0), DataSetConvention.INSTANCE)
+    val convRight: RelNode = RelOptRule.convert(intersect.getInput(1), DataSetConvention.INSTANCE)
+
+    new DataSetIntersect(
+      rel.getCluster,
+      traitSet,
+      convLeft,
+      convRight,
+      rel.getRowType,
+      intersect.all)
+  }
+}
+
+object DataSetIntersectRule {
+  val INSTANCE: RelOptRule = new DataSetIntersectRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetJoinRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetJoinRule.scala
new file mode 100644
index 0000000..2874198
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetJoinRule.scala
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalJoin
+
+import org.apache.flink.table.plan.nodes.dataset.{DataSetJoin, DataSetConvention}
+
+import scala.collection.JavaConversions._
+
+class DataSetJoinRule
+  extends ConverterRule(
+      classOf[LogicalJoin],
+      Convention.NONE,
+      DataSetConvention.INSTANCE,
+      "DataSetJoinRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val join: LogicalJoin = call.rel(0).asInstanceOf[LogicalJoin]
+
+    val joinInfo = join.analyzeCondition
+
+    // joins require an equi-condition or a conjunctive predicate with at least one equi-condition
+    !joinInfo.pairs().isEmpty
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+
+    val join: LogicalJoin = rel.asInstanceOf[LogicalJoin]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+    val convLeft: RelNode = RelOptRule.convert(join.getInput(0), DataSetConvention.INSTANCE)
+    val convRight: RelNode = RelOptRule.convert(join.getInput(1), DataSetConvention.INSTANCE)
+    val joinInfo = join.analyzeCondition
+
+    new DataSetJoin(
+      rel.getCluster,
+      traitSet,
+      convLeft,
+      convRight,
+      rel.getRowType,
+      join.getCondition,
+      join.getRowType,
+      joinInfo,
+      joinInfo.pairs.toList,
+      join.getJoinType,
+      null,
+      description)
+  }
+
+}
+
+object DataSetJoinRule {
+  val INSTANCE: RelOptRule = new DataSetJoinRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetMinusRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetMinusRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetMinusRule.scala
new file mode 100644
index 0000000..7172596
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetMinusRule.scala
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalMinus
+import org.apache.calcite.rel.rules.UnionToDistinctRule
+import org.apache.flink.table.plan.nodes.dataset.{DataSetConvention, DataSetMinus}
+
+class DataSetMinusRule
+  extends ConverterRule(
+    classOf[LogicalMinus],
+    Convention.NONE,
+    DataSetConvention.INSTANCE,
+    "DataSetMinusRule")
+{
+
+  def convert(rel: RelNode): RelNode = {
+
+    val minus: LogicalMinus = rel.asInstanceOf[LogicalMinus]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+    val convLeft: RelNode = RelOptRule.convert(minus.getInput(0), DataSetConvention.INSTANCE)
+    val convRight: RelNode = RelOptRule.convert(minus.getInput(1), DataSetConvention.INSTANCE)
+
+    new DataSetMinus(
+      rel.getCluster,
+      traitSet,
+      convLeft,
+      convRight,
+      rel.getRowType,
+      minus.all)
+  }
+}
+
+object DataSetMinusRule {
+  val INSTANCE: RelOptRule = new DataSetMinusRule
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetScanRule.scala
new file mode 100644
index 0000000..9d593aa
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetScanRule.scala
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.{RelOptRuleCall, Convention, RelOptRule, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.logical.LogicalTableScan
+import org.apache.flink.table.plan.nodes.dataset.{DataSetConvention, DataSetScan}
+import org.apache.flink.table.plan.schema.DataSetTable
+
+class DataSetScanRule
+  extends ConverterRule(
+      classOf[LogicalTableScan],
+      Convention.NONE,
+      DataSetConvention.INSTANCE,
+      "DataSetScanRule")
+  {
+
+  /**
+   * If the input is not a DataSetTable, we want the TableScanRule to match instead
+   */
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val scan: TableScan = call.rel(0).asInstanceOf[TableScan]
+    val dataSetTable = scan.getTable.unwrap(classOf[DataSetTable[Any]])
+    dataSetTable match {
+      case _: DataSetTable[Any] =>
+        true
+      case _ =>
+        false
+    }
+  }
+
+  def convert(rel: RelNode): RelNode = {
+    val scan: TableScan = rel.asInstanceOf[TableScan]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+
+    new DataSetScan(
+      rel.getCluster,
+      traitSet,
+      scan.getTable,
+      rel.getRowType
+    )
+  }
+}
+
+object DataSetScanRule {
+  val INSTANCE: RelOptRule = new DataSetScanRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala
new file mode 100644
index 0000000..1f5c91a
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.volcano.RelSubset
+import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalJoin}
+import org.apache.flink.table.plan.nodes.dataset.{DataSetConvention, DataSetSingleRowJoin}
+
+class DataSetSingleRowJoinRule
+  extends ConverterRule(
+      classOf[LogicalJoin],
+      Convention.NONE,
+      DataSetConvention.INSTANCE,
+      "DataSetSingleRowCrossRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val join = call.rel(0).asInstanceOf[LogicalJoin]
+
+    if (isInnerJoin(join)) {
+      isGlobalAggregation(join.getRight.asInstanceOf[RelSubset].getOriginal) ||
+        isGlobalAggregation(join.getLeft.asInstanceOf[RelSubset].getOriginal)
+    } else {
+      false
+    }
+  }
+
+  private def isInnerJoin(join: LogicalJoin) = {
+    join.getJoinType == JoinRelType.INNER
+  }
+
+  private def isGlobalAggregation(node: RelNode) = {
+    node.isInstanceOf[LogicalAggregate] &&
+      isSingleRow(node.asInstanceOf[LogicalAggregate])
+  }
+
+  private def isSingleRow(agg: LogicalAggregate) = {
+    agg.getGroupSet.isEmpty
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+    val join = rel.asInstanceOf[LogicalJoin]
+    val traitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+    val dataSetLeftNode = RelOptRule.convert(join.getLeft, DataSetConvention.INSTANCE)
+    val dataSetRightNode = RelOptRule.convert(join.getRight, DataSetConvention.INSTANCE)
+    val leftIsSingle = isGlobalAggregation(join.getLeft.asInstanceOf[RelSubset].getOriginal)
+
+    new DataSetSingleRowJoin(
+      rel.getCluster,
+      traitSet,
+      dataSetLeftNode,
+      dataSetRightNode,
+      leftIsSingle,
+      rel.getRowType,
+      join.getCondition,
+      join.getRowType,
+      description)
+  }
+}
+
+object DataSetSingleRowJoinRule {
+  val INSTANCE: RelOptRule = new DataSetSingleRowJoinRule
+}


[46/47] flink git commit: [FLINK-3848] [table] follow-up: Refactor TableSource tests.

Posted by fh...@apache.org.
[FLINK-3848] [table] follow-up: Refactor TableSource tests.


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

Branch: refs/heads/master
Commit: 22af6cf57918af645a0bd8282bf1a8e51606f7f0
Parents: ef575e8
Author: Fabian Hueske <fh...@apache.org>
Authored: Fri Dec 16 11:19:42 2016 +0100
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Dec 16 16:41:21 2016 +0100

----------------------------------------------------------------------
 .../table/api/java/batch/TableSourceITCase.java |  68 +++-----
 .../batch/ProjectableTableSourceITCase.scala    | 146 -----------------
 .../batch/ProjectableTableSourceTest.scala      | 155 -------------------
 .../api/scala/batch/TableSourceITCase.scala     |  99 ++----------
 .../table/api/scala/batch/TableSourceTest.scala | 155 +++++++++++++++++++
 .../api/scala/stream/TableSourceITCase.scala    | 127 ++-------------
 6 files changed, 199 insertions(+), 551 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/22af6cf5/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableSourceITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableSourceITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableSourceITCase.java
index c822efb..e5777f2 100644
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableSourceITCase.java
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableSourceITCase.java
@@ -22,10 +22,9 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.DataSet;
 import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.table.utils.CommonTestData;
 import org.apache.flink.types.Row;
-import org.apache.flink.api.java.typeutils.RowTypeInfo;
 import org.apache.flink.table.api.java.BatchTableEnvironment;
-import org.apache.flink.table.api.scala.batch.GeneratingInputFormat;
 import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase;
 import org.apache.flink.table.api.Table;
 import org.apache.flink.table.api.TableEnvironment;
@@ -45,20 +44,27 @@ public class TableSourceITCase extends TableProgramsTestBase {
 
 	@Test
 	public void testBatchTableSourceTableAPI() throws Exception {
+
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+		BatchTableSource csvTable = CommonTestData.getCsvTableSource();
 
-		tableEnv.registerTableSource("MyTable", new TestBatchTableSource());
+		tableEnv.registerTableSource("persons", csvTable);
 
-		Table result = tableEnv.scan("MyTable")
-			.where("amount < 4")
-			.select("amount * id, name");
+		Table result = tableEnv.scan("persons")
+			.select("id, first, last, score");
 
 		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
 		List<Row> results = resultSet.collect();
 
-		String expected = "0,Record_0\n" + "0,Record_16\n" + "0,Record_32\n" + "1,Record_1\n" +
-			"17,Record_17\n" + "36,Record_18\n" + "4,Record_2\n" + "57,Record_19\n" + "9,Record_3\n";
+		String expected = "1,Mike,Smith,12.3\n" +
+			"2,Bob,Taylor,45.6\n" +
+			"3,Sam,Miller,7.89\n" +
+			"4,Peter,Smith,0.12\n" +
+			"5,Liz,Williams,34.5\n" +
+			"6,Sally,Miller,6.78\n" +
+			"7,Alice,Smith,90.1\n" +
+			"8,Kelly,Williams,2.34\n";
 
 		compareResultAsText(results, expected);
 	}
@@ -67,53 +73,23 @@ public class TableSourceITCase extends TableProgramsTestBase {
 	public void testBatchTableSourceSQL() throws Exception {
 		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
 		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+		BatchTableSource csvTable = CommonTestData.getCsvTableSource();
 
-		tableEnv.registerTableSource("MyTable", new TestBatchTableSource());
+		tableEnv.registerTableSource("persons", csvTable);
 
 		Table result = tableEnv
-			.sql("SELECT amount * id, name FROM MyTable WHERE amount < 4");
+			.sql("SELECT last, FLOOR(id), score * 2 FROM persons WHERE score < 20");
 
 		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
 		List<Row> results = resultSet.collect();
 
-		String expected = "0,Record_0\n" + "0,Record_16\n" + "0,Record_32\n" + "1,Record_1\n" +
-			"17,Record_17\n" + "36,Record_18\n" + "4,Record_2\n" + "57,Record_19\n" + "9,Record_3\n";
+		String expected = "Smith,1,24.6\n" +
+			"Miller,3,15.78\n" +
+			"Smith,4,0.24\n" +
+			"Miller,6,13.56\n" +
+			"Williams,8,4.68\n";
 
 		compareResultAsText(results, expected);
 	}
 
-	public static class TestBatchTableSource implements BatchTableSource<Row> {
-
-		private TypeInformation<?>[] fieldTypes = new TypeInformation<?>[] {
-			BasicTypeInfo.STRING_TYPE_INFO,
-			BasicTypeInfo.LONG_TYPE_INFO,
-			BasicTypeInfo.INT_TYPE_INFO
-		};
-
-		@Override
-		public DataSet<Row> getDataSet(ExecutionEnvironment execEnv) {
-			return execEnv.createInput(new GeneratingInputFormat(33), getReturnType()).setParallelism(1);
-		}
-
-		@Override
-		public int getNumberOfFields() {
-			return 3;
-		}
-
-		@Override
-		public String[] getFieldsNames() {
-			return new String[]{"name", "id", "amount"};
-		}
-
-		@Override
-		public TypeInformation<?>[] getFieldTypes() {
-			return fieldTypes;
-		}
-
-		@Override
-		public TypeInformation<Row> getReturnType() {
-			return new RowTypeInfo(fieldTypes);
-		}
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/22af6cf5/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala
deleted file mode 100644
index 37407c8..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.api.scala.batch
-
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.{DataSet => JavaSet, ExecutionEnvironment => JavaExecEnv}
-import org.apache.flink.api.scala.ExecutionEnvironment
-import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.table.api.scala._
-import org.apache.flink.table.sources.{BatchTableSource, ProjectableTableSource}
-import org.apache.flink.table.api.TableEnvironment
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.apache.flink.types.Row
-import org.junit.{Before, Test}
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class ProjectableTableSourceITCase(mode: TestExecutionMode,
-  configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  private val tableName = "MyTable"
-  private var tableEnv: BatchTableEnvironment = null
-
-  @Before
-  def initTableEnv(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    tableEnv = TableEnvironment.getTableEnvironment(env, config)
-    tableEnv.registerTableSource(tableName, new TestProjectableTableSource)
-  }
-
-  @Test
-  def testTableAPI(): Unit = {
-    val results = tableEnv
-                  .scan(tableName)
-                  .where("amount < 4")
-                  .select("id, name")
-                  .collect()
-
-    val expected = Seq(
-      "0,Record_0", "1,Record_1", "2,Record_2", "3,Record_3", "16,Record_16",
-      "17,Record_17", "18,Record_18", "19,Record_19", "32,Record_32").mkString("\n")
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-
-  @Test
-  def testSQL(): Unit = {
-    val results = tableEnv
-                  .sql(s"select id, name from $tableName where amount < 4 ")
-                  .collect()
-
-    val expected = Seq(
-      "0,Record_0", "1,Record_1", "2,Record_2", "3,Record_3", "16,Record_16",
-      "17,Record_17", "18,Record_18", "19,Record_19", "32,Record_32").mkString("\n")
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-}
-
-class TestProjectableTableSource(
-  fieldTypes: Array[TypeInformation[_]],
-  fieldNames: Array[String])
-  extends BatchTableSource[Row] with ProjectableTableSource[Row] {
-
-  def this() = this(
-    fieldTypes = Array(
-      BasicTypeInfo.STRING_TYPE_INFO,
-      BasicTypeInfo.LONG_TYPE_INFO,
-      BasicTypeInfo.INT_TYPE_INFO,
-      BasicTypeInfo.DOUBLE_TYPE_INFO),
-    fieldNames = Array[String]("name", "id", "amount", "price")
-  )
-
-  /** Returns the data of the table as a [[org.apache.flink.api.java.DataSet]]. */
-  override def getDataSet(execEnv: JavaExecEnv): JavaSet[Row] = {
-    execEnv.fromCollection(generateDynamicCollection(33, fieldNames).asJava, getReturnType)
-  }
-
-  /** Returns the types of the table fields. */
-  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
-
-  /** Returns the names of the table fields. */
-  override def getFieldsNames: Array[String] = fieldNames
-
-  /** Returns the [[TypeInformation]] for the return type. */
-  override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*)
-
-  /** Returns the number of fields of the table. */
-  override def getNumberOfFields: Int = fieldNames.length
-
-  override def projectFields(fields: Array[Int]): TestProjectableTableSource = {
-    val projectedFieldTypes = new Array[TypeInformation[_]](fields.length)
-    val projectedFieldNames = new Array[String](fields.length)
-
-    fields.zipWithIndex.foreach(f => {
-      projectedFieldTypes(f._2) = fieldTypes(f._1)
-      projectedFieldNames(f._2) = fieldNames(f._1)
-    })
-    new TestProjectableTableSource(projectedFieldTypes, projectedFieldNames)
-  }
-
-  private def generateDynamicCollection(num: Int, fieldNames: Array[String]): Seq[Row] = {
-    for {cnt <- 0 until num}
-      yield {
-        val row = new Row(fieldNames.length)
-        fieldNames.zipWithIndex.foreach(
-          f =>
-            f._1 match {
-              case "name" =>
-                row.setField(f._2, "Record_" + cnt)
-              case "id" =>
-                row.setField(f._2, cnt.toLong)
-              case "amount" =>
-                row.setField(f._2, cnt.toInt % 16)
-              case "price" =>
-                row.setField(f._2, cnt.toDouble / 3)
-              case _ =>
-                throw new IllegalArgumentException(s"unknown field name $f._1")
-            }
-        )
-        row
-      }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/22af6cf5/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceTest.scala
deleted file mode 100644
index b3097cf..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceTest.scala
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.api.scala.batch
-
-import org.apache.flink.table.api.scala._
-import org.apache.flink.table.sources.CsvTableSource
-import org.apache.flink.table.utils.{CommonTestData, TableTestBase}
-import org.apache.flink.table.utils.TableTestUtil._
-import org.junit.Test
-
-class ProjectableTableSourceTest extends TableTestBase {
-
-  private val projectedFields: Array[String] = Array("last", "id", "score")
-  private val noCalcFields: Array[String] = Array("id", "score", "first")
-
-  @Test
-  def testBatchProjectableSourceScanPlanTableApi(): Unit = {
-    val (csvTable, tableName) = tableSource
-    val util = batchTestUtil()
-    val tEnv = util.tEnv
-
-    tEnv.registerTableSource(tableName, csvTable)
-
-    val result = tEnv
-      .scan(tableName)
-      .select('last.upperCase(), 'id.floor(), 'score * 2)
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      sourceBatchTableNode(tableName, projectedFields),
-      term("select", "UPPER(last) AS _c0", "FLOOR(id) AS _c1", "*(score, 2) AS _c2")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testBatchProjectableSourceScanPlanSQL(): Unit = {
-    val (csvTable, tableName) = tableSource
-    val util = batchTestUtil()
-
-    util.tEnv.registerTableSource(tableName, csvTable)
-
-    val sqlQuery = s"SELECT last, floor(id), score * 2 FROM $tableName"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      sourceBatchTableNode(tableName, projectedFields),
-      term("select", "last", "FLOOR(id) AS EXPR$1", "*(score, 2) AS EXPR$2")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testBatchProjectableSourceScanNoIdentityCalc(): Unit = {
-    val (csvTable, tableName) = tableSource
-    val util = batchTestUtil()
-    val tEnv = util.tEnv
-
-    tEnv.registerTableSource(tableName, csvTable)
-
-    val result = tEnv
-      .scan(tableName)
-      .select('id, 'score, 'first)
-
-    val expected = sourceBatchTableNode(tableName, noCalcFields)
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testStreamProjectableSourceScanPlanTableApi(): Unit = {
-    val (csvTable, tableName) = tableSource
-    val util = streamTestUtil()
-    val tEnv = util.tEnv
-
-    tEnv.registerTableSource(tableName, csvTable)
-
-    val result = tEnv
-      .ingest(tableName)
-      .select('last, 'id.floor(), 'score * 2)
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      sourceStreamTableNode(tableName, projectedFields),
-      term("select", "last", "FLOOR(id) AS _c1", "*(score, 2) AS _c2")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testStreamProjectableSourceScanPlanSQL(): Unit = {
-    val (csvTable, tableName) = tableSource
-    val util = streamTestUtil()
-
-    util.tEnv.registerTableSource(tableName, csvTable)
-
-    val sqlQuery = s"SELECT last, floor(id), score * 2 FROM $tableName"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      sourceStreamTableNode(tableName, projectedFields),
-      term("select", "last", "FLOOR(id) AS EXPR$1", "*(score, 2) AS EXPR$2")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testStreamProjectableSourceScanNoIdentityCalc(): Unit = {
-    val (csvTable, tableName) = tableSource
-    val util = streamTestUtil()
-    val tEnv = util.tEnv
-
-    tEnv.registerTableSource(tableName, csvTable)
-
-    val result = tEnv
-      .ingest(tableName)
-      .select('id, 'score, 'first)
-
-    val expected = sourceStreamTableNode(tableName, noCalcFields)
-    util.verifyTable(result, expected)
-  }
-
-  def tableSource: (CsvTableSource, String) = {
-    val csvTable = CommonTestData.getCsvTableSource
-    val tableName = "csvTable"
-    (csvTable, tableName)
-  }
-
-  def sourceBatchTableNode(sourceName: String, fields: Array[String]): String = {
-    s"BatchTableSourceScan(table=[[$sourceName]], fields=[${fields.mkString(", ")}])"
-  }
-
-  def sourceStreamTableNode(sourceName: String, fields: Array[String] ): String = {
-    s"StreamTableSourceScan(table=[[$sourceName]], fields=[${fields.mkString(", ")}])"
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/22af6cf5/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
index e324aad..a9218ac 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
@@ -18,16 +18,10 @@
 
 package org.apache.flink.table.api.scala.batch
 
-import org.apache.flink.api.common.io.GenericInputFormat
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.{DataSet => JavaSet, ExecutionEnvironment => JavaExecEnv}
 import org.apache.flink.api.scala.ExecutionEnvironment
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.types.Row
 import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
 import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.sources.BatchTableSource
 import org.apache.flink.table.api.TableEnvironment
 import org.apache.flink.table.utils.CommonTestData
 import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
@@ -45,41 +39,6 @@ class TableSourceITCase(
   extends TableProgramsTestBase(mode, configMode) {
 
   @Test
-  def testBatchTableSourceTableAPI(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    tEnv.registerTableSource("MyTestTable", new TestBatchTableSource())
-    val results = tEnv
-      .scan("MyTestTable")
-      .where('amount < 4)
-      .select('amount * 'id, 'name)
-      .collect()
-
-    val expected = Seq(
-      "0,Record_0", "0,Record_16", "0,Record_32", "1,Record_1", "17,Record_17",
-      "36,Record_18", "4,Record_2", "57,Record_19", "9,Record_3").mkString("\n")
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testBatchTableSourceSQL(): Unit = {
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    tEnv.registerTableSource("MyTestTable", new TestBatchTableSource())
-    val results = tEnv.sql(
-      "SELECT amount * id, name FROM MyTestTable WHERE amount < 4").collect()
-
-    val expected = Seq(
-      "0,Record_0", "0,Record_16", "0,Record_32", "1,Record_1", "17,Record_17",
-      "36,Record_18", "4,Record_2", "57,Record_19", "9,Record_3").mkString("\n")
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
   def testCsvTableSource(): Unit = {
 
     val csvTable = CommonTestData.getCsvTableSource
@@ -89,13 +48,17 @@ class TableSourceITCase(
 
     tEnv.registerTableSource("csvTable", csvTable)
     val results = tEnv.sql(
-      "SELECT last, sum(score), max(id) FROM csvTable GROUP BY last").collect()
+      "SELECT id, first, last, score FROM csvTable").collect()
 
     val expected = Seq(
-      "Smith,102.52,7",
-      "Taylor,45.6,2",
-      "Miller,14.67,6",
-      "Williams,36.84,8").mkString("\n")
+      "1,Mike,Smith,12.3",
+      "2,Bob,Taylor,45.6",
+      "3,Sam,Miller,7.89",
+      "4,Peter,Smith,0.12",
+      "5,Liz,Williams,34.5",
+      "6,Sally,Miller,6.78",
+      "7,Alice,Smith,90.1",
+      "8,Kelly,Williams,2.34").mkString("\n")
     TestBaseUtils.compareResultAsText(results.asJava, expected)
   }
 
@@ -110,59 +73,17 @@ class TableSourceITCase(
 
     val results = tEnv
       .scan("csvTable")
+      .where('score < 20)
       .select('last, 'id.floor(), 'score * 2)
       .collect()
 
     val expected = Seq(
       "Smith,1,24.6",
-      "Taylor,2,91.2",
       "Miller,3,15.78",
       "Smith,4,0.24",
-      "Williams,5,69.0",
       "Miller,6,13.56",
-      "Smith,7,180.2",
       "Williams,8,4.68").mkString("\n")
     TestBaseUtils.compareResultAsText(results.asJava, expected)
   }
-}
-
-class TestBatchTableSource extends BatchTableSource[Row] {
-
-  val fieldTypes: Array[TypeInformation[_]] = Array(
-    BasicTypeInfo.STRING_TYPE_INFO,
-    BasicTypeInfo.LONG_TYPE_INFO,
-    BasicTypeInfo.INT_TYPE_INFO
-  )
-
-  /** Returns the data of the table as a [[org.apache.flink.api.java.DataSet]]. */
-  override def getDataSet(execEnv: JavaExecEnv): JavaSet[Row] = {
-    execEnv.createInput(new GeneratingInputFormat(33), getReturnType).setParallelism(1)
-  }
 
-  /** Returns the types of the table fields. */
-  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
-
-  /** Returns the names of the table fields. */
-  override def getFieldsNames: Array[String] = Array("name", "id", "amount")
-
-  /** Returns the [[TypeInformation]] for the return type. */
-  override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*)
-
-  /** Returns the number of fields of the table. */
-  override def getNumberOfFields: Int = 3
-}
-
-class GeneratingInputFormat(val num: Int) extends GenericInputFormat[Row] {
-
-  var cnt = 0L
-
-  override def reachedEnd(): Boolean = cnt >= num
-
-  override def nextRecord(reuse: Row): Row = {
-    reuse.setField(0, s"Record_$cnt")
-    reuse.setField(1, cnt)
-    reuse.setField(2, (cnt % 16).toInt)
-    cnt += 1
-    reuse
-  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/22af6cf5/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceTest.scala
new file mode 100644
index 0000000..55da42a
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceTest.scala
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch
+
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.sources.CsvTableSource
+import org.apache.flink.table.utils.{CommonTestData, TableTestBase}
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+class TableSourceTest extends TableTestBase {
+
+  private val projectedFields: Array[String] = Array("last", "id", "score")
+  private val noCalcFields: Array[String] = Array("id", "score", "first")
+
+  @Test
+  def testBatchProjectableSourceScanPlanTableApi(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = batchTestUtil()
+    val tEnv = util.tEnv
+
+    tEnv.registerTableSource(tableName, csvTable)
+
+    val result = tEnv
+      .scan(tableName)
+      .select('last.upperCase(), 'id.floor(), 'score * 2)
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      sourceBatchTableNode(tableName, projectedFields),
+      term("select", "UPPER(last) AS _c0", "FLOOR(id) AS _c1", "*(score, 2) AS _c2")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testBatchProjectableSourceScanPlanSQL(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = batchTestUtil()
+
+    util.tEnv.registerTableSource(tableName, csvTable)
+
+    val sqlQuery = s"SELECT last, floor(id), score * 2 FROM $tableName"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      sourceBatchTableNode(tableName, projectedFields),
+      term("select", "last", "FLOOR(id) AS EXPR$1", "*(score, 2) AS EXPR$2")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testBatchProjectableSourceScanNoIdentityCalc(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = batchTestUtil()
+    val tEnv = util.tEnv
+
+    tEnv.registerTableSource(tableName, csvTable)
+
+    val result = tEnv
+      .scan(tableName)
+      .select('id, 'score, 'first)
+
+    val expected = sourceBatchTableNode(tableName, noCalcFields)
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testStreamProjectableSourceScanPlanTableApi(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = streamTestUtil()
+    val tEnv = util.tEnv
+
+    tEnv.registerTableSource(tableName, csvTable)
+
+    val result = tEnv
+      .ingest(tableName)
+      .select('last, 'id.floor(), 'score * 2)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      sourceStreamTableNode(tableName, projectedFields),
+      term("select", "last", "FLOOR(id) AS _c1", "*(score, 2) AS _c2")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testStreamProjectableSourceScanPlanSQL(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = streamTestUtil()
+
+    util.tEnv.registerTableSource(tableName, csvTable)
+
+    val sqlQuery = s"SELECT last, floor(id), score * 2 FROM $tableName"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      sourceStreamTableNode(tableName, projectedFields),
+      term("select", "last", "FLOOR(id) AS EXPR$1", "*(score, 2) AS EXPR$2")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testStreamProjectableSourceScanNoIdentityCalc(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = streamTestUtil()
+    val tEnv = util.tEnv
+
+    tEnv.registerTableSource(tableName, csvTable)
+
+    val result = tEnv
+      .ingest(tableName)
+      .select('id, 'score, 'first)
+
+    val expected = sourceStreamTableNode(tableName, noCalcFields)
+    util.verifyTable(result, expected)
+  }
+
+  def tableSource: (CsvTableSource, String) = {
+    val csvTable = CommonTestData.getCsvTableSource
+    val tableName = "csvTable"
+    (csvTable, tableName)
+  }
+
+  def sourceBatchTableNode(sourceName: String, fields: Array[String]): String = {
+    s"BatchTableSourceScan(table=[[$sourceName]], fields=[${fields.mkString(", ")}])"
+  }
+
+  def sourceStreamTableNode(sourceName: String, fields: Array[String] ): String = {
+    s"StreamTableSourceScan(table=[[$sourceName]], fields=[${fields.mkString(", ")}])"
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/22af6cf5/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
index 316f2f3..381bd5d 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
@@ -18,16 +18,9 @@
 
 package org.apache.flink.table.api.scala.stream
 
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
 import org.apache.flink.table.api.scala.stream.utils.StreamITCase
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.sources.StreamTableSource
 import org.apache.flink.api.scala._
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.streaming.api.environment
-import org.apache.flink.streaming.api.functions.source.SourceFunction
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
 import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
 import org.apache.flink.table.api.TableEnvironment
@@ -41,51 +34,6 @@ import scala.collection.mutable
 class TableSourceITCase extends StreamingMultipleProgramsTestBase {
 
   @Test
-  def testStreamTableSourceTableAPI(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    tEnv.registerTableSource("MyTestTable", new TestStreamTableSource(33))
-    tEnv.ingest("MyTestTable")
-      .where('amount < 4)
-      .select('amount * 'id, 'name)
-      .toDataStream[Row]
-      .addSink(new StreamITCase.StringSink)
-
-    env.execute()
-
-    val expected = mutable.MutableList(
-      "0,Record_0", "0,Record_16", "0,Record_32",
-      "1,Record_1", "17,Record_17", "36,Record_18",
-      "4,Record_2", "57,Record_19", "9,Record_3")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testStreamTableSourceSQL(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    tEnv.registerTableSource("MyTestTable", new TestStreamTableSource(33))
-    tEnv.sql(
-      "SELECT amount * id, name FROM MyTestTable WHERE amount < 4")
-      .toDataStream[Row]
-      .addSink(new StreamITCase.StringSink)
-
-    env.execute()
-
-    val expected = mutable.MutableList(
-      "0,Record_0", "0,Record_16", "0,Record_32",
-      "1,Record_1", "17,Record_17", "36,Record_18",
-      "4,Record_2", "57,Record_19", "9,Record_3")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
   def testCsvTableSourceSQL(): Unit = {
 
     val csvTable = CommonTestData.getCsvTableSource
@@ -94,18 +42,19 @@ class TableSourceITCase extends StreamingMultipleProgramsTestBase {
     val tEnv = TableEnvironment.getTableEnvironment(env)
     StreamITCase.testResults = mutable.MutableList()
 
-    tEnv.registerTableSource("csvTable", csvTable)
+    tEnv.registerTableSource("persons", csvTable)
+
     tEnv.sql(
-      "SELECT last, score, id FROM csvTable WHERE id < 4 ")
+      "SELECT id, first, last, score FROM persons WHERE id < 4 ")
       .toDataStream[Row]
       .addSink(new StreamITCase.StringSink)
 
     env.execute()
 
     val expected = mutable.MutableList(
-      "Smith,12.3,1",
-      "Taylor,45.6,2",
-      "Miller,7.89,3")
+      "1,Mike,Smith,12.3",
+      "2,Bob,Taylor,45.6",
+      "3,Sam,Miller,7.89")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 
@@ -120,70 +69,18 @@ class TableSourceITCase extends StreamingMultipleProgramsTestBase {
 
     tEnv.registerTableSource("csvTable", csvTable)
     tEnv.ingest("csvTable")
-      .select('last, 'id.floor(), 'score * 2)
+      .where('id > 4)
+      .select('last, 'score * 2)
       .toDataStream[Row]
       .addSink(new StreamITCase.StringSink)
 
     env.execute()
 
     val expected = mutable.MutableList(
-      "Smith,1,24.6",
-      "Taylor,2,91.2",
-      "Miller,3,15.78",
-      "Smith,4,0.24",
-      "Williams,5,69.0",
-      "Miller,6,13.56",
-      "Smith,7,180.2",
-      "Williams,8,4.68")
+      "Williams,69.0",
+      "Miller,13.56",
+      "Smith,180.2",
+      "Williams,4.68")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
 }
-
-
-class TestStreamTableSource(val numRecords: Int) extends StreamTableSource[Row] {
-
-  val fieldTypes: Array[TypeInformation[_]] = Array(
-    BasicTypeInfo.STRING_TYPE_INFO,
-    BasicTypeInfo.LONG_TYPE_INFO,
-    BasicTypeInfo.INT_TYPE_INFO
-  )
-
-  /** Returns the data of the table as a [[DataStream]]. */
-  override def getDataStream(execEnv: environment.StreamExecutionEnvironment): DataStream[Row] = {
-    execEnv.addSource(new GeneratingSourceFunction(numRecords), getReturnType).setParallelism(1)
-  }
-
-  /** Returns the types of the table fields. */
-  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
-
-  /** Returns the names of the table fields. */
-  override def getFieldsNames: Array[String] = Array("name", "id", "amount")
-
-  /** Returns the [[TypeInformation]] for the return type. */
-  override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*)
-
-  /** Returns the number of fields of the table. */
-  override def getNumberOfFields: Int = 3
-}
-
-class GeneratingSourceFunction(val num: Long) extends SourceFunction[Row] {
-
-  var running = true
-
-  override def run(ctx: SourceContext[Row]): Unit = {
-    var cnt = 0L
-    while(running && cnt < num) {
-      val out = new Row(3)
-      out.setField(0, s"Record_$cnt")
-      out.setField(1, cnt)
-      out.setField(2, (cnt % 16).toInt)
-
-      ctx.collect(out)
-      cnt += 1
-    }
-  }
-
-  override def cancel(): Unit = {
-    running = false
-  }
-}


[40/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala
deleted file mode 100644
index 6d00ab3..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/BatchTableEnvironment.scala
+++ /dev/null
@@ -1,311 +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.api.table
-
-import java.util.concurrent.atomic.AtomicInteger
-
-import org.apache.calcite.plan.RelOptPlanner.CannotPlanException
-import org.apache.calcite.plan.RelOptUtil
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.sql2rel.RelDecorrelator
-import org.apache.calcite.tools.{Programs, RuleSet}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.io.DiscardingOutputFormat
-import org.apache.flink.api.java.typeutils.TypeExtractor
-import org.apache.flink.api.java.{DataSet, ExecutionEnvironment}
-import org.apache.flink.api.table.explain.PlanJsonParser
-import org.apache.flink.api.table.expressions.Expression
-import org.apache.flink.api.table.plan.logical.{CatalogNode, LogicalRelNode}
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetRel}
-import org.apache.flink.api.table.plan.rules.FlinkRuleSets
-import org.apache.flink.api.table.plan.schema.{DataSetTable, TableSourceTable}
-import org.apache.flink.api.table.sinks.{BatchTableSink, TableSink}
-import org.apache.flink.api.table.sources.BatchTableSource
-import org.apache.flink.types.Row
-
-/**
-  * The abstract base class for batch TableEnvironments.
-  *
-  * A TableEnvironment can be used to:
-  * - convert a [[DataSet]] to a [[Table]]
-  * - register a [[DataSet]] in the [[org.apache.flink.api.table.TableEnvironment]]'s catalog
-  * - register a [[Table]] in the [[org.apache.flink.api.table.TableEnvironment]]'s catalog
-  * - scan a registered table to obtain a [[Table]]
-  * - specify a SQL query on registered tables to obtain a [[Table]]
-  * - convert a [[Table]] into a [[DataSet]]
-  * - explain the AST and execution plan of a [[Table]]
-  *
-  * @param execEnv The [[ExecutionEnvironment]] which is wrapped in this [[BatchTableEnvironment]].
-  * @param config The [[TableConfig]] of this [[BatchTableEnvironment]].
-  */
-abstract class BatchTableEnvironment(
-    private[flink] val execEnv: ExecutionEnvironment,
-    config: TableConfig)
-  extends TableEnvironment(config) {
-
-  // a counter for unique table names.
-  private val nameCntr: AtomicInteger = new AtomicInteger(0)
-
-  // the naming pattern for internally registered tables.
-  private val internalNamePattern = "^_DataSetTable_[0-9]+$".r
-
-  /**
-    * Checks if the chosen table name is valid.
-    *
-    * @param name The table name to check.
-    */
-  override protected def checkValidTableName(name: String): Unit = {
-    val m = internalNamePattern.findFirstIn(name)
-    m match {
-      case Some(_) =>
-        throw new TableException(s"Illegal Table name. " +
-          s"Please choose a name that does not contain the pattern $internalNamePattern")
-      case None =>
-    }
-  }
-
-  /** Returns a unique table name according to the internal naming pattern. */
-  protected def createUniqueTableName(): String = "_DataSetTable_" + nameCntr.getAndIncrement()
-
-  /**
-    * Scans a registered table and returns the resulting [[Table]].
-    *
-    * The table to scan must be registered in the [[TableEnvironment]]'s catalog.
-    *
-    * @param tableName The name of the table to scan.
-    * @throws ValidationException if no table is registered under the given name.
-    * @return The scanned table.
-    */
-  @throws[ValidationException]
-  def scan(tableName: String): Table = {
-    if (isRegistered(tableName)) {
-      new Table(this, CatalogNode(tableName, getRowType(tableName)))
-    } else {
-      throw new TableException(s"Table \'$tableName\' was not found in the registry.")
-    }
-  }
-
-  /**
-    * Registers an external [[BatchTableSource]] in this [[TableEnvironment]]'s catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * @param name The name under which the [[BatchTableSource]] is registered.
-    * @param tableSource The [[BatchTableSource]] to register.
-    */
-  def registerTableSource(name: String, tableSource: BatchTableSource[_]): Unit = {
-
-    checkValidTableName(name)
-    registerTableInternal(name, new TableSourceTable(tableSource))
-  }
-
-  /**
-    * Evaluates a SQL query on registered tables and retrieves the result as a [[Table]].
-    *
-    * All tables referenced by the query must be registered in the TableEnvironment.
-    *
-    * @param query The SQL query to evaluate.
-    * @return The result of the query as Table.
-    */
-  override def sql(query: String): Table = {
-
-    val planner = new FlinkPlannerImpl(getFrameworkConfig, getPlanner, getTypeFactory)
-    // parse the sql query
-    val parsed = planner.parse(query)
-    // validate the sql query
-    val validated = planner.validate(parsed)
-    // transform to a relational tree
-    val relational = planner.rel(validated)
-
-    new Table(this, LogicalRelNode(relational.rel))
-  }
-
-  /**
-    * Writes a [[Table]] to a [[TableSink]].
-    *
-    * Internally, the [[Table]] is translated into a [[DataSet]] and handed over to the
-    * [[TableSink]] to write it.
-    *
-    * @param table The [[Table]] to write.
-    * @param sink The [[TableSink]] to write the [[Table]] to.
-    * @tparam T The expected type of the [[DataSet]] which represents the [[Table]].
-    */
-  override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = {
-
-    sink match {
-      case batchSink: BatchTableSink[T] =>
-        val outputType = sink.getOutputType
-        // translate the Table into a DataSet and provide the type that the TableSink expects.
-        val result: DataSet[T] = translate(table)(outputType)
-        // Give the DataSet to the TableSink to emit it.
-        batchSink.emitDataSet(result)
-      case _ =>
-        throw new TableException("BatchTableSink required to emit batch Table")
-    }
-  }
-
-  /**
-    * Returns the AST of the specified Table API and SQL queries and the execution plan to compute
-    * the result of the given [[Table]].
-    *
-    * @param table The table for which the AST and execution plan will be returned.
-    * @param extended Flag to include detailed optimizer estimates.
-    */
-  private[flink] def explain(table: Table, extended: Boolean): String = {
-    val ast = table.getRelNode
-    val optimizedPlan = optimize(ast)
-    val dataSet = translate[Row](optimizedPlan) (TypeExtractor.createTypeInfo(classOf[Row]))
-    dataSet.output(new DiscardingOutputFormat[Row])
-    val env = dataSet.getExecutionEnvironment
-    val jasonSqlPlan = env.getExecutionPlan
-    val sqlPlan = PlanJsonParser.getSqlExecutionPlan(jasonSqlPlan, extended)
-
-    s"== Abstract Syntax Tree ==" +
-        System.lineSeparator +
-        s"${RelOptUtil.toString(ast)}" +
-        System.lineSeparator +
-        s"== Optimized Logical Plan ==" +
-        System.lineSeparator +
-        s"${RelOptUtil.toString(optimizedPlan)}" +
-        System.lineSeparator +
-        s"== Physical Execution Plan ==" +
-        System.lineSeparator +
-        s"$sqlPlan"
-  }
-
-  /**
-    * Returns the AST of the specified Table API and SQL queries and the execution plan to compute
-    * the result of the given [[Table]].
-    *
-    * @param table The table for which the AST and execution plan will be returned.
-    */
-  def explain(table: Table): String = explain(table: Table, extended = false)
-
-  /**
-    * Registers a [[DataSet]] as a table under a given name in the [[TableEnvironment]]'s catalog.
-    *
-    * @param name The name under which the table is registered in the catalog.
-    * @param dataSet The [[DataSet]] to register as table in the catalog.
-    * @tparam T the type of the [[DataSet]].
-    */
-  protected def registerDataSetInternal[T](name: String, dataSet: DataSet[T]): Unit = {
-
-    val (fieldNames, fieldIndexes) = getFieldInfo[T](dataSet.getType)
-    val dataSetTable = new DataSetTable[T](
-      dataSet,
-      fieldIndexes,
-      fieldNames
-    )
-    registerTableInternal(name, dataSetTable)
-  }
-
-  /**
-    * Registers a [[DataSet]] as a table under a given name with field names as specified by
-    * field expressions in the [[TableEnvironment]]'s catalog.
-    *
-    * @param name The name under which the table is registered in the catalog.
-    * @param dataSet The [[DataSet]] to register as table in the catalog.
-    * @param fields The field expressions to define the field names of the table.
-    * @tparam T The type of the [[DataSet]].
-    */
-  protected def registerDataSetInternal[T](
-      name: String, dataSet: DataSet[T], fields: Array[Expression]): Unit = {
-
-    val (fieldNames, fieldIndexes) = getFieldInfo[T](dataSet.getType, fields)
-    val dataSetTable = new DataSetTable[T](dataSet, fieldIndexes, fieldNames)
-    registerTableInternal(name, dataSetTable)
-  }
-
-  /**
-    * Returns the built-in rules that are defined by the environment.
-    */
-  protected def getBuiltInRuleSet: RuleSet = FlinkRuleSets.DATASET_OPT_RULES
-
-  /**
-    * Generates the optimized [[RelNode]] tree from the original relational node tree.
-    *
-    * @param relNode The original [[RelNode]] tree
-    * @return The optimized [[RelNode]] tree
-    */
-  private[flink] def optimize(relNode: RelNode): RelNode = {
-
-    // decorrelate
-    val decorPlan = RelDecorrelator.decorrelateQuery(relNode)
-
-    // optimize the logical Flink plan
-    val optProgram = Programs.ofRules(getRuleSet)
-    val flinkOutputProps = relNode.getTraitSet.replace(DataSetConvention.INSTANCE).simplify()
-
-    val dataSetPlan = try {
-      optProgram.run(getPlanner, decorPlan, flinkOutputProps)
-    } catch {
-      case e: CannotPlanException =>
-        throw new TableException(
-          s"Cannot generate a valid execution plan for the given query: \n\n" +
-            s"${RelOptUtil.toString(relNode)}\n" +
-            s"This exception indicates that the query uses an unsupported SQL feature.\n" +
-            s"Please check the documentation for the set of currently supported SQL features.")
-      case t: TableException =>
-        throw new TableException(
-          s"Cannot generate a valid execution plan for the given query: \n\n" +
-            s"${RelOptUtil.toString(relNode)}\n" +
-            s"${t.msg}\n" +
-            s"Please check the documentation for the set of currently supported SQL features.")
-      case a: AssertionError =>
-        throw a.getCause
-    }
-    dataSetPlan
-  }
-
-  /**
-    * Translates a [[Table]] into a [[DataSet]].
-    *
-    * The transformation involves optimizing the relational expression tree as defined by
-    * Table API calls and / or SQL queries and generating corresponding [[DataSet]] operators.
-    *
-    * @param table The root node of the relational expression tree.
-    * @param tpe   The [[TypeInformation]] of the resulting [[DataSet]].
-    * @tparam A The type of the resulting [[DataSet]].
-    * @return The [[DataSet]] that corresponds to the translated [[Table]].
-    */
-  protected def translate[A](table: Table)(implicit tpe: TypeInformation[A]): DataSet[A] = {
-    val dataSetPlan = optimize(table.getRelNode)
-    translate(dataSetPlan)
-  }
-
-  /**
-    * Translates a logical [[RelNode]] into a [[DataSet]].
-    *
-    * @param logicalPlan The root node of the relational expression tree.
-    * @param tpe         The [[TypeInformation]] of the resulting [[DataSet]].
-    * @tparam A The type of the resulting [[DataSet]].
-    * @return The [[DataSet]] that corresponds to the translated [[Table]].
-    */
-  protected def translate[A](logicalPlan: RelNode)(implicit tpe: TypeInformation[A]): DataSet[A] = {
-    validateType(tpe)
-
-    logicalPlan match {
-      case node: DataSetRel =>
-        node.translateToPlan(
-          this,
-          Some(tpe.asInstanceOf[TypeInformation[Any]])
-        ).asInstanceOf[DataSet[A]]
-      case _ => ???
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/CalciteConfig.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/CalciteConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/CalciteConfig.scala
deleted file mode 100644
index 06b3edc..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/CalciteConfig.scala
+++ /dev/null
@@ -1,161 +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.api.table
-
-import org.apache.calcite.plan.RelOptRule
-import org.apache.calcite.sql.SqlOperatorTable
-import org.apache.calcite.sql.parser.SqlParser
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable
-import org.apache.calcite.tools.{RuleSets, RuleSet}
-import org.apache.flink.util.Preconditions
-
-import scala.collection.JavaConverters._
-
-/**
-  * Builder for creating a Calcite configuration.
-  */
-class CalciteConfigBuilder {
-  private var replaceRules: Boolean = false
-  private var ruleSets: List[RuleSet] = Nil
-
-  private var replaceOperatorTable: Boolean = false
-  private var operatorTables: List[SqlOperatorTable] = Nil
-
-  private var replaceSqlParserConfig: Option[SqlParser.Config] = None
-
-  /**
-    * Replaces the built-in rule set with the given rule set.
-    */
-  def replaceRuleSet(replaceRuleSet: RuleSet): CalciteConfigBuilder = {
-    Preconditions.checkNotNull(replaceRuleSet)
-    ruleSets = List(replaceRuleSet)
-    replaceRules = true
-    this
-  }
-
-  /**
-    * Appends the given rule set to the built-in rule set.
-    */
-  def addRuleSet(addedRuleSet: RuleSet): CalciteConfigBuilder = {
-    Preconditions.checkNotNull(addedRuleSet)
-    ruleSets = addedRuleSet :: ruleSets
-    this
-  }
-
-  /**
-    * Replaces the built-in SQL operator table with the given table.
-    */
-  def replaceSqlOperatorTable(replaceSqlOperatorTable: SqlOperatorTable): CalciteConfigBuilder = {
-    Preconditions.checkNotNull(replaceSqlOperatorTable)
-    operatorTables = List(replaceSqlOperatorTable)
-    replaceOperatorTable = true
-    this
-  }
-
-  /**
-    * Appends the given table to the built-in SQL operator table.
-    */
-  def addSqlOperatorTable(addedSqlOperatorTable: SqlOperatorTable): CalciteConfigBuilder = {
-    Preconditions.checkNotNull(addedSqlOperatorTable)
-    this.operatorTables = addedSqlOperatorTable :: this.operatorTables
-    this
-  }
-
-  /**
-    * Replaces the built-in SQL parser configuration with the given configuration.
-    */
-  def replaceSqlParserConfig(sqlParserConfig: SqlParser.Config): CalciteConfigBuilder = {
-    Preconditions.checkNotNull(sqlParserConfig)
-    replaceSqlParserConfig = Some(sqlParserConfig)
-    this
-  }
-
-  private class CalciteConfigImpl(
-      val getRuleSet: Option[RuleSet],
-      val replacesRuleSet: Boolean,
-      val getSqlOperatorTable: Option[SqlOperatorTable],
-      val replacesSqlOperatorTable: Boolean,
-      val getSqlParserConfig: Option[SqlParser.Config])
-    extends CalciteConfig
-
-  /**
-    * Builds a new [[CalciteConfig]].
-    */
-  def build(): CalciteConfig = new CalciteConfigImpl(
-        ruleSets match {
-      case Nil => None
-      case h :: Nil => Some(h)
-      case _ =>
-        // concat rule sets
-        val concatRules = ruleSets.foldLeft(Nil: Iterable[RelOptRule])( (c, r) => r.asScala ++ c)
-        Some(RuleSets.ofList(concatRules.asJava))
-    },
-    this.replaceRules,
-    operatorTables match {
-      case Nil => None
-      case h :: Nil => Some(h)
-      case _ =>
-        // chain operator tables
-        Some(operatorTables.reduce( (x, y) => ChainedSqlOperatorTable.of(x, y)))
-    },
-    this.replaceOperatorTable,
-    replaceSqlParserConfig)
-}
-
-/**
-  * Calcite configuration for defining a custom Calcite configuration for Table and SQL API.
-  */
-trait CalciteConfig {
-  /**
-    * Returns whether this configuration replaces the built-in rule set.
-    */
-  def replacesRuleSet: Boolean
-
-  /**
-    * Returns a custom rule set.
-    */
-  def getRuleSet: Option[RuleSet]
-
-  /**
-    * Returns whether this configuration replaces the built-in SQL operator table.
-    */
-  def replacesSqlOperatorTable: Boolean
-
-  /**
-    * Returns a custom SQL operator table.
-    */
-  def getSqlOperatorTable: Option[SqlOperatorTable]
-
-  /**
-    * Returns a custom SQL parser configuration.
-    */
-  def getSqlParserConfig: Option[SqlParser.Config]
-}
-
-object CalciteConfig {
-
-  val DEFAULT = createBuilder().build()
-
-  /**
-    * Creates a new builder for constructing a [[CalciteConfig]].
-    */
-  def createBuilder(): CalciteConfigBuilder = {
-    new CalciteConfigBuilder
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkCalciteSqlValidator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkCalciteSqlValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkCalciteSqlValidator.scala
deleted file mode 100644
index b1ccc09..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkCalciteSqlValidator.scala
+++ /dev/null
@@ -1,47 +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.api.table
-
-import org.apache.calcite.adapter.java.JavaTypeFactory
-import org.apache.calcite.prepare.CalciteCatalogReader
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.sql.{SqlInsert, SqlOperatorTable}
-import org.apache.calcite.sql.validate.{SqlValidatorImpl, SqlConformance}
-
-/**
- * This is a copy of Calcite's CalciteSqlValidator to use with [[FlinkPlannerImpl]].
- */
-class FlinkCalciteSqlValidator(
-    opTab: SqlOperatorTable,
-    catalogReader: CalciteCatalogReader,
-    typeFactory: JavaTypeFactory) extends SqlValidatorImpl(
-        opTab, catalogReader, typeFactory, SqlConformance.DEFAULT) {
-
-  override def getLogicalSourceRowType(
-      sourceRowType: RelDataType,
-      insert: SqlInsert): RelDataType = {
-    typeFactory.asInstanceOf[JavaTypeFactory].toSql(sourceRowType)
-  }
-
-  override def getLogicalTargetRowType(
-      targetRowType: RelDataType,
-      insert: SqlInsert): RelDataType = {
-    typeFactory.asInstanceOf[JavaTypeFactory].toSql(targetRowType)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkPlannerImpl.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkPlannerImpl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkPlannerImpl.scala
deleted file mode 100644
index 131cdc6..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkPlannerImpl.scala
+++ /dev/null
@@ -1,177 +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.api.table
-
-import java.util
-
-import com.google.common.collect.ImmutableList
-import org.apache.calcite.jdbc.CalciteSchema
-import org.apache.calcite.plan.RelOptTable.ViewExpander
-import org.apache.calcite.plan._
-import org.apache.calcite.prepare.CalciteCatalogReader
-import org.apache.calcite.rel.RelRoot
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rex.RexBuilder
-import org.apache.calcite.schema.SchemaPlus
-import org.apache.calcite.sql.parser.{SqlParseException => CSqlParseException, SqlParser}
-import org.apache.calcite.sql.validate.SqlValidator
-import org.apache.calcite.sql.{SqlNode, SqlOperatorTable}
-import org.apache.calcite.sql2rel.{RelDecorrelator, SqlRexConvertletTable, SqlToRelConverter}
-import org.apache.calcite.tools.{FrameworkConfig, RelConversionException}
-
-import scala.collection.JavaConversions._
-
-/**
-  * NOTE: this is heavily inspired by Calcite's PlannerImpl.
-  * We need it in order to share the planner between the Table API relational plans
-  * and the SQL relation plans that are created by the Calcite parser.
-  * The main difference is that we do not create a new RelOptPlanner in the ready() method.
-  */
-class FlinkPlannerImpl(
-    config: FrameworkConfig,
-    planner: RelOptPlanner,
-    typeFactory: FlinkTypeFactory) {
-
-  val operatorTable: SqlOperatorTable = config.getOperatorTable
-  /** Holds the trait definitions to be registered with planner. May be null. */
-  val traitDefs: ImmutableList[RelTraitDef[_ <: RelTrait]] = config.getTraitDefs
-  val parserConfig: SqlParser.Config = config.getParserConfig
-  val convertletTable: SqlRexConvertletTable = config.getConvertletTable
-  val defaultSchema: SchemaPlus = config.getDefaultSchema
-
-  var validator: FlinkCalciteSqlValidator = _
-  var validatedSqlNode: SqlNode = _
-  var root: RelRoot = _
-
-  private def ready() {
-    if (this.traitDefs != null) {
-      planner.clearRelTraitDefs()
-      for (traitDef <- this.traitDefs) {
-        planner.addRelTraitDef(traitDef)
-      }
-    }
-  }
-
-  def parse(sql: String): SqlNode = {
-    try {
-      ready()
-      val parser: SqlParser = SqlParser.create(sql, parserConfig)
-      val sqlNode: SqlNode = parser.parseStmt
-      sqlNode
-    } catch {
-      case e: CSqlParseException =>
-        throw SqlParserException(s"SQL parse failed. ${e.getMessage}", e)
-    }
-  }
-
-  def validate(sqlNode: SqlNode): SqlNode = {
-    validator = new FlinkCalciteSqlValidator(operatorTable, createCatalogReader, typeFactory)
-    validator.setIdentifierExpansion(true)
-    try {
-      validatedSqlNode = validator.validate(sqlNode)
-    }
-    catch {
-      case e: RuntimeException =>
-        throw new ValidationException(s"SQL validation failed. ${e.getMessage}", e)
-    }
-    validatedSqlNode
-  }
-
-  def rel(sql: SqlNode): RelRoot = {
-    try {
-      assert(validatedSqlNode != null)
-      val rexBuilder: RexBuilder = createRexBuilder
-      val cluster: RelOptCluster = RelOptCluster.create(planner, rexBuilder)
-      val config = SqlToRelConverter.configBuilder()
-        .withTrimUnusedFields(false).withConvertTableAccess(false).build()
-      val sqlToRelConverter: SqlToRelConverter = new SqlToRelConverter(
-        new ViewExpanderImpl, validator, createCatalogReader, cluster, convertletTable, config)
-      root = sqlToRelConverter.convertQuery(validatedSqlNode, false, true)
-      // we disable automatic flattening in order to let composite types pass without modification
-      // we might enable it again once Calcite has better support for structured types
-      // root = root.withRel(sqlToRelConverter.flattenTypes(root.rel, true))
-      root = root.withRel(RelDecorrelator.decorrelateQuery(root.rel))
-      root
-    } catch {
-      case e: RelConversionException => throw TableException(e.getMessage)
-    }
-  }
-
-  /** Implements [[org.apache.calcite.plan.RelOptTable.ViewExpander]]
-    * interface for [[org.apache.calcite.tools.Planner]]. */
-  class ViewExpanderImpl extends ViewExpander {
-
-    override def expandView(
-        rowType: RelDataType,
-        queryString: String,
-        schemaPath: util.List[String],
-        viewPath: util.List[String]): RelRoot = {
-
-      val parser: SqlParser = SqlParser.create(queryString, parserConfig)
-      var sqlNode: SqlNode = null
-      try {
-        sqlNode = parser.parseQuery
-      }
-      catch {
-        case e: CSqlParseException =>
-          throw SqlParserException(s"SQL parse failed. ${e.getMessage}", e)
-      }
-      val catalogReader: CalciteCatalogReader = createCatalogReader.withSchemaPath(schemaPath)
-      val validator: SqlValidator =
-        new FlinkCalciteSqlValidator(operatorTable, catalogReader, typeFactory)
-      validator.setIdentifierExpansion(true)
-      val validatedSqlNode: SqlNode = validator.validate(sqlNode)
-      val rexBuilder: RexBuilder = createRexBuilder
-      val cluster: RelOptCluster = RelOptCluster.create(planner, rexBuilder)
-      val config: SqlToRelConverter.Config = SqlToRelConverter.configBuilder
-        .withTrimUnusedFields(false).withConvertTableAccess(false).build
-      val sqlToRelConverter: SqlToRelConverter = new SqlToRelConverter(
-        new ViewExpanderImpl, validator, catalogReader, cluster, convertletTable, config)
-      root = sqlToRelConverter.convertQuery(validatedSqlNode, true, false)
-      root = root.withRel(sqlToRelConverter.flattenTypes(root.rel, true))
-      root = root.withRel(RelDecorrelator.decorrelateQuery(root.rel))
-      FlinkPlannerImpl.this.root
-    }
-  }
-
-  private def createCatalogReader: CalciteCatalogReader = {
-    val rootSchema: SchemaPlus = FlinkPlannerImpl.rootSchema(defaultSchema)
-    new CalciteCatalogReader(
-      CalciteSchema.from(rootSchema),
-      parserConfig.caseSensitive,
-      CalciteSchema.from(defaultSchema).path(null),
-      typeFactory)
-  }
-
-  private def createRexBuilder: RexBuilder = {
-    new RexBuilder(typeFactory)
-  }
-
-}
-
-object FlinkPlannerImpl {
-  private def rootSchema(schema: SchemaPlus): SchemaPlus = {
-    if (schema.getParentSchema == null) {
-      schema
-    }
-    else {
-      rootSchema(schema.getParentSchema)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkRelBuilder.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkRelBuilder.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkRelBuilder.scala
deleted file mode 100644
index 8508e53..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkRelBuilder.scala
+++ /dev/null
@@ -1,103 +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.api.table
-
-import java.util.Collections
-
-import org.apache.calcite.plan.volcano.VolcanoPlanner
-import java.lang.Iterable
-
-import org.apache.calcite.jdbc.CalciteSchema
-import org.apache.calcite.plan._
-import org.apache.calcite.prepare.CalciteCatalogReader
-import org.apache.calcite.rel.logical.LogicalAggregate
-import org.apache.calcite.rex.RexBuilder
-import org.apache.calcite.tools.RelBuilder.{AggCall, GroupKey}
-import org.apache.calcite.tools.{FrameworkConfig, RelBuilder}
-import org.apache.flink.api.table.FlinkRelBuilder.NamedWindowProperty
-import org.apache.flink.api.table.expressions.WindowProperty
-import org.apache.flink.api.table.plan.logical.LogicalWindow
-import org.apache.flink.api.table.plan.logical.rel.LogicalWindowAggregate
-
-/**
-  * Flink specific [[RelBuilder]] that changes the default type factory to a [[FlinkTypeFactory]].
-  */
-class FlinkRelBuilder(
-    context: Context,
-    relOptCluster: RelOptCluster,
-    relOptSchema: RelOptSchema)
-  extends RelBuilder(
-    context,
-    relOptCluster,
-    relOptSchema) {
-
-  def getPlanner: RelOptPlanner = cluster.getPlanner
-
-  def getCluster: RelOptCluster = relOptCluster
-
-  override def getTypeFactory: FlinkTypeFactory =
-    super.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-
-  def aggregate(
-      window: LogicalWindow,
-      groupKey: GroupKey,
-      namedProperties: Seq[NamedWindowProperty],
-      aggCalls: Iterable[AggCall])
-    : RelBuilder = {
-    // build logical aggregate
-    val aggregate = super.aggregate(groupKey, aggCalls).build().asInstanceOf[LogicalAggregate]
-
-    // build logical window aggregate from it
-    push(LogicalWindowAggregate.create(window, namedProperties, aggregate))
-    this
-  }
-
-}
-
-object FlinkRelBuilder {
-
-  def create(config: FrameworkConfig): FlinkRelBuilder = {
-
-    // create Flink type factory
-    val typeSystem = config.getTypeSystem
-    val typeFactory = new FlinkTypeFactory(typeSystem)
-
-    // create context instances with Flink type factory
-    val planner = new VolcanoPlanner(config.getCostFactory, Contexts.empty())
-    planner.setExecutor(config.getExecutor)
-    planner.addRelTraitDef(ConventionTraitDef.INSTANCE)
-    val cluster = RelOptCluster.create(planner, new RexBuilder(typeFactory))
-    val calciteSchema = CalciteSchema.from(config.getDefaultSchema)
-    val relOptSchema = new CalciteCatalogReader(
-      calciteSchema,
-      config.getParserConfig.caseSensitive(),
-      Collections.emptyList(),
-      typeFactory)
-
-    new FlinkRelBuilder(config.getContext, cluster, relOptSchema)
-  }
-
-  /**
-    * Information necessary to create a window aggregate.
-    *
-    * Similar to [[RelBuilder.AggCall]] or [[RelBuilder.GroupKey]].
-    */
-  case class NamedWindowProperty(name: String, property: WindowProperty)
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkTypeFactory.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkTypeFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkTypeFactory.scala
deleted file mode 100644
index 8dcd660..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkTypeFactory.scala
+++ /dev/null
@@ -1,212 +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.api.table
-
-import org.apache.calcite.avatica.util.TimeUnit
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl
-import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeSystem}
-import org.apache.calcite.sql.SqlIntervalQualifier
-import org.apache.calcite.sql.`type`.SqlTypeName
-import org.apache.calcite.sql.`type`.SqlTypeName._
-import org.apache.calcite.sql.parser.SqlParserPos
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.{NothingTypeInfo, PrimitiveArrayTypeInfo, SqlTimeTypeInfo, TypeInformation}
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
-import org.apache.flink.api.java.typeutils.ValueTypeInfo._
-import org.apache.flink.api.table.FlinkTypeFactory.typeInfoToSqlTypeName
-import org.apache.flink.api.table.plan.schema.{ArrayRelDataType, CompositeRelDataType, GenericRelDataType}
-import org.apache.flink.api.table.typeutils.TimeIntervalTypeInfo
-import org.apache.flink.api.table.typeutils.TypeCheckUtils.isSimple
-
-import scala.collection.mutable
-
-/**
-  * Flink specific type factory that represents the interface between Flink's [[TypeInformation]]
-  * and Calcite's [[RelDataType]].
-  */
-class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImpl(typeSystem) {
-
-  // NOTE: for future data types it might be necessary to
-  // override more methods of RelDataTypeFactoryImpl
-
-  private val seenTypes = mutable.HashMap[TypeInformation[_], RelDataType]()
-
-  def createTypeFromTypeInfo(typeInfo: TypeInformation[_]): RelDataType = {
-    // simple type can be converted to SQL types and vice versa
-    if (isSimple(typeInfo)) {
-      val sqlType = typeInfoToSqlTypeName(typeInfo)
-      sqlType match {
-
-        case INTERVAL_YEAR_MONTH =>
-          createSqlIntervalType(
-            new SqlIntervalQualifier(TimeUnit.YEAR, TimeUnit.MONTH, SqlParserPos.ZERO))
-
-        case INTERVAL_DAY_SECOND =>
-          createSqlIntervalType(
-            new SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.SECOND, SqlParserPos.ZERO))
-
-        case _ =>
-          createSqlType(sqlType)
-      }
-    }
-    // advanced types require specific RelDataType
-    // for storing the original TypeInformation
-    else {
-      seenTypes.getOrElseUpdate(typeInfo, canonize(createAdvancedType(typeInfo)))
-    }
-  }
-
-  /**
-    * Creates a struct type with the input fieldNames and input fieldTypes using FlinkTypeFactory
-    *
-    * @param fieldNames field names
-    * @param fieldTypes field types, every element is Flink's [[TypeInformation]]
-    * @return a struct type with the input fieldNames and input fieldTypes
-    */
-  def buildRowDataType(
-      fieldNames: Array[String],
-      fieldTypes: Array[TypeInformation[_]])
-    : RelDataType = {
-    val rowDataTypeBuilder = builder
-    fieldNames
-      .zip(fieldTypes)
-      .foreach { f =>
-        rowDataTypeBuilder.add(f._1, createTypeFromTypeInfo(f._2)).nullable(true)
-      }
-    rowDataTypeBuilder.build
-  }
-
-  override def createSqlType(typeName: SqlTypeName, precision: Int): RelDataType = {
-    // it might happen that inferred VARCHAR types overflow as we set them to Int.MaxValue
-    // always set those to default value
-    if (typeName == VARCHAR && precision < 0) {
-      createSqlType(typeName, getTypeSystem.getDefaultPrecision(typeName))
-    } else {
-      super.createSqlType(typeName, precision)
-    }
-  }
-
-  override def createArrayType(elementType: RelDataType, maxCardinality: Long): RelDataType =
-    new ArrayRelDataType(
-      ObjectArrayTypeInfo.getInfoFor(FlinkTypeFactory.toTypeInfo(elementType)),
-      elementType,
-      true)
-
-  private def createAdvancedType(typeInfo: TypeInformation[_]): RelDataType = typeInfo match {
-    case ct: CompositeType[_] =>
-      new CompositeRelDataType(ct, this)
-
-    case pa: PrimitiveArrayTypeInfo[_] =>
-      new ArrayRelDataType(pa, createTypeFromTypeInfo(pa.getComponentType), false)
-
-    case oa: ObjectArrayTypeInfo[_, _] =>
-      new ArrayRelDataType(oa, createTypeFromTypeInfo(oa.getComponentInfo), true)
-
-    case ti: TypeInformation[_] =>
-      new GenericRelDataType(typeInfo, getTypeSystem.asInstanceOf[FlinkTypeSystem])
-
-    case ti@_ =>
-      throw TableException(s"Unsupported type information: $ti")
-  }
-
-  override def createTypeWithNullability(
-    relDataType: RelDataType,
-    nullable: Boolean)
-  : RelDataType = relDataType match {
-    case composite: CompositeRelDataType =>
-      // at the moment we do not care about nullability
-      composite
-    case _ =>
-      super.createTypeWithNullability(relDataType, nullable)
-  }
-}
-
-object FlinkTypeFactory {
-
-  private def typeInfoToSqlTypeName(typeInfo: TypeInformation[_]): SqlTypeName = typeInfo match {
-      case BOOLEAN_TYPE_INFO => BOOLEAN
-      case BYTE_TYPE_INFO => TINYINT
-      case SHORT_TYPE_INFO => SMALLINT
-      case INT_TYPE_INFO => INTEGER
-      case LONG_TYPE_INFO => BIGINT
-      case FLOAT_TYPE_INFO => FLOAT
-      case DOUBLE_TYPE_INFO => DOUBLE
-      case STRING_TYPE_INFO => VARCHAR
-      case BIG_DEC_TYPE_INFO => DECIMAL
-
-      // temporal types
-      case SqlTimeTypeInfo.DATE => DATE
-      case SqlTimeTypeInfo.TIME => TIME
-      case SqlTimeTypeInfo.TIMESTAMP => TIMESTAMP
-      case TimeIntervalTypeInfo.INTERVAL_MONTHS => INTERVAL_YEAR_MONTH
-      case TimeIntervalTypeInfo.INTERVAL_MILLIS => INTERVAL_DAY_SECOND
-
-      case CHAR_TYPE_INFO | CHAR_VALUE_TYPE_INFO =>
-        throw TableException("Character type is not supported.")
-
-      case _@t =>
-        throw TableException(s"Type is not supported: $t")
-  }
-
-  def toTypeInfo(relDataType: RelDataType): TypeInformation[_] = relDataType.getSqlTypeName match {
-    case BOOLEAN => BOOLEAN_TYPE_INFO
-    case TINYINT => BYTE_TYPE_INFO
-    case SMALLINT => SHORT_TYPE_INFO
-    case INTEGER => INT_TYPE_INFO
-    case BIGINT => LONG_TYPE_INFO
-    case FLOAT => FLOAT_TYPE_INFO
-    case DOUBLE => DOUBLE_TYPE_INFO
-    case VARCHAR | CHAR => STRING_TYPE_INFO
-    case DECIMAL => BIG_DEC_TYPE_INFO
-
-    // temporal types
-    case DATE => SqlTimeTypeInfo.DATE
-    case TIME => SqlTimeTypeInfo.TIME
-    case TIMESTAMP => SqlTimeTypeInfo.TIMESTAMP
-    case typeName if YEAR_INTERVAL_TYPES.contains(typeName) => TimeIntervalTypeInfo.INTERVAL_MONTHS
-    case typeName if DAY_INTERVAL_TYPES.contains(typeName) => TimeIntervalTypeInfo.INTERVAL_MILLIS
-
-    case NULL =>
-      throw TableException("Type NULL is not supported. Null values must have a supported type.")
-
-    // symbol for special flags e.g. TRIM's BOTH, LEADING, TRAILING
-    // are represented as integer
-    case SYMBOL => INT_TYPE_INFO
-
-    // extract encapsulated TypeInformation
-    case ANY if relDataType.isInstanceOf[GenericRelDataType] =>
-      val genericRelDataType = relDataType.asInstanceOf[GenericRelDataType]
-      genericRelDataType.typeInfo
-
-    case ROW if relDataType.isInstanceOf[CompositeRelDataType] =>
-      val compositeRelDataType = relDataType.asInstanceOf[CompositeRelDataType]
-      compositeRelDataType.compositeType
-
-    // ROW and CURSOR for UDTF case, whose type info will never be used, just a placeholder
-    case ROW | CURSOR => new NothingTypeInfo
-
-    case ARRAY if relDataType.isInstanceOf[ArrayRelDataType] =>
-      val arrayRelDataType = relDataType.asInstanceOf[ArrayRelDataType]
-      arrayRelDataType.typeInfo
-
-    case _@t =>
-      throw TableException(s"Type is not supported: $t")
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkTypeSystem.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkTypeSystem.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkTypeSystem.scala
deleted file mode 100644
index 3222eee..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/FlinkTypeSystem.scala
+++ /dev/null
@@ -1,51 +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.api.table
-
-import org.apache.calcite.rel.`type`.RelDataTypeSystemImpl
-import org.apache.calcite.sql.`type`.SqlTypeName
-
-/**
-  * Custom type system for Flink.
-  */
-class FlinkTypeSystem extends RelDataTypeSystemImpl {
-
-  // we cannot use Int.MaxValue because of an overflow in Calcites type inference logic
-  // half should be enough for all use cases
-  override def getMaxNumericScale: Int = Int.MaxValue / 2
-
-  // we cannot use Int.MaxValue because of an overflow in Calcites type inference logic
-  // half should be enough for all use cases
-  override def getMaxNumericPrecision: Int = Int.MaxValue / 2
-
-  override def getDefaultPrecision(typeName: SqlTypeName): Int = typeName match {
-
-    // by default all VARCHARs can have the Java default length
-    case SqlTypeName.VARCHAR =>
-      Int.MaxValue
-
-    // we currenty support only timestamps with milliseconds precision
-    case SqlTypeName.TIMESTAMP =>
-      3
-
-    case _ =>
-      super.getDefaultPrecision(typeName)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala
deleted file mode 100644
index da20e07..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/StreamTableEnvironment.scala
+++ /dev/null
@@ -1,351 +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.api.table
-
-import java.util.concurrent.atomic.AtomicInteger
-
-import org.apache.calcite.plan.RelOptPlanner.CannotPlanException
-import org.apache.calcite.plan.RelOptUtil
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.sql2rel.RelDecorrelator
-import org.apache.calcite.tools.{Programs, RuleSet}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.TypeExtractor
-import org.apache.flink.api.table.explain.PlanJsonParser
-import org.apache.flink.api.table.expressions.Expression
-import org.apache.flink.api.table.plan.logical.{CatalogNode, LogicalRelNode}
-import org.apache.flink.api.table.plan.nodes.datastream.{DataStreamConvention, DataStreamRel}
-import org.apache.flink.api.table.plan.rules.FlinkRuleSets
-import org.apache.flink.api.table.plan.schema.{DataStreamTable, TableSourceTable}
-import org.apache.flink.api.table.sinks.{StreamTableSink, TableSink}
-import org.apache.flink.api.table.sources.StreamTableSource
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
-import org.apache.flink.types.Row
-
-/**
-  * The base class for stream TableEnvironments.
-  *
-  * A TableEnvironment can be used to:
-  * - convert [[DataStream]] to a [[Table]]
-  * - register a [[DataStream]] as a table in the catalog
-  * - register a [[Table]] in the catalog
-  * - scan a registered table to obtain a [[Table]]
-  * - specify a SQL query on registered tables to obtain a [[Table]]
-  * - convert a [[Table]] into a [[DataStream]]
-  *
-  * @param execEnv The [[StreamExecutionEnvironment]] which is wrapped in this
-  *                [[StreamTableEnvironment]].
-  * @param config The [[TableConfig]] of this [[StreamTableEnvironment]].
-  */
-abstract class StreamTableEnvironment(
-    private[flink] val execEnv: StreamExecutionEnvironment,
-    config: TableConfig)
-  extends TableEnvironment(config) {
-
-  // a counter for unique table names
-  private val nameCntr: AtomicInteger = new AtomicInteger(0)
-
-  // the naming pattern for internally registered tables.
-  private val internalNamePattern = "^_DataStreamTable_[0-9]+$".r
-
-  /**
-    * Checks if the chosen table name is valid.
-    *
-    * @param name The table name to check.
-    */
-  override protected def checkValidTableName(name: String): Unit = {
-    val m = internalNamePattern.findFirstIn(name)
-    m match {
-      case Some(_) =>
-        throw new TableException(s"Illegal Table name. " +
-          s"Please choose a name that does not contain the pattern $internalNamePattern")
-      case None =>
-    }
-  }
-
-  /** Returns a unique table name according to the internal naming pattern. */
-  protected def createUniqueTableName(): String = "_DataStreamTable_" + nameCntr.getAndIncrement()
-
-  /**
-    * Returns field names and field positions for a given [[TypeInformation]].
-    *
-    * Field names are automatically extracted for
-    * [[org.apache.flink.api.common.typeutils.CompositeType]].
-    * The method fails if inputType is not a
-    * [[org.apache.flink.api.common.typeutils.CompositeType]].
-    *
-    * @param inputType The TypeInformation extract the field names and positions from.
-    * @tparam A The type of the TypeInformation.
-    * @return A tuple of two arrays holding the field names and corresponding field positions.
-    */
-  override protected[flink] def getFieldInfo[A](inputType: TypeInformation[A])
-    : (Array[String], Array[Int]) = {
-    val fieldInfo = super.getFieldInfo(inputType)
-    if (fieldInfo._1.contains("rowtime")) {
-      throw new TableException("'rowtime' ia a reserved field name in stream environment.")
-    }
-    fieldInfo
-  }
-
-  /**
-    * Returns field names and field positions for a given [[TypeInformation]] and [[Array]] of
-    * [[Expression]].
-    *
-    * @param inputType The [[TypeInformation]] against which the [[Expression]]s are evaluated.
-    * @param exprs     The expressions that define the field names.
-    * @tparam A The type of the TypeInformation.
-    * @return A tuple of two arrays holding the field names and corresponding field positions.
-    */
-  override protected[flink] def getFieldInfo[A](
-      inputType: TypeInformation[A],
-      exprs: Array[Expression])
-    : (Array[String], Array[Int]) = {
-    val fieldInfo = super.getFieldInfo(inputType, exprs)
-    if (fieldInfo._1.contains("rowtime")) {
-      throw new TableException("'rowtime' is a reserved field name in stream environment.")
-    }
-    fieldInfo
-  }
-
-  /**
-    * Ingests a registered table and returns the resulting [[Table]].
-    *
-    * The table to ingest must be registered in the [[TableEnvironment]]'s catalog.
-    *
-    * @param tableName The name of the table to ingest.
-    * @throws ValidationException if no table is registered under the given name.
-    * @return The ingested table.
-    */
-  @throws[ValidationException]
-  def ingest(tableName: String): Table = {
-
-    if (isRegistered(tableName)) {
-      new Table(this, CatalogNode(tableName, getRowType(tableName)))
-    }
-    else {
-      throw new ValidationException(s"Table \'$tableName\' was not found in the registry.")
-    }
-  }
-
-  /**
-    * Registers an external [[StreamTableSource]] in this [[TableEnvironment]]'s catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * @param name        The name under which the [[StreamTableSource]] is registered.
-    * @param tableSource The [[org.apache.flink.api.table.sources.StreamTableSource]] to register.
-    */
-  def registerTableSource(name: String, tableSource: StreamTableSource[_]): Unit = {
-
-    checkValidTableName(name)
-    registerTableInternal(name, new TableSourceTable(tableSource))
-  }
-
-  /**
-    * Evaluates a SQL query on registered tables and retrieves the result as a [[Table]].
-    *
-    * All tables referenced by the query must be registered in the TableEnvironment.
-    *
-    * @param query The SQL query to evaluate.
-    * @return The result of the query as Table.
-    */
-  override def sql(query: String): Table = {
-
-    val planner = new FlinkPlannerImpl(getFrameworkConfig, getPlanner, getTypeFactory)
-    // parse the sql query
-    val parsed = planner.parse(query)
-    // validate the sql query
-    val validated = planner.validate(parsed)
-    // transform to a relational tree
-    val relational = planner.rel(validated)
-
-    new Table(this, LogicalRelNode(relational.rel))
-  }
-
-  /**
-    * Writes a [[Table]] to a [[TableSink]].
-    *
-    * Internally, the [[Table]] is translated into a [[DataStream]] and handed over to the
-    * [[TableSink]] to write it.
-    *
-    * @param table The [[Table]] to write.
-    * @param sink The [[TableSink]] to write the [[Table]] to.
-    * @tparam T The expected type of the [[DataStream]] which represents the [[Table]].
-    */
-  override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = {
-
-    sink match {
-      case streamSink: StreamTableSink[T] =>
-        val outputType = sink.getOutputType
-        // translate the Table into a DataStream and provide the type that the TableSink expects.
-        val result: DataStream[T] = translate(table)(outputType)
-        // Give the DataSet to the TableSink to emit it.
-        streamSink.emitDataStream(result)
-      case _ =>
-        throw new TableException("StreamTableSink required to emit streaming Table")
-    }
-  }
-
-  /**
-    * Registers a [[DataStream]] as a table under a given name in the [[TableEnvironment]]'s
-    * catalog.
-    *
-    * @param name The name under which the table is registered in the catalog.
-    * @param dataStream The [[DataStream]] to register as table in the catalog.
-    * @tparam T the type of the [[DataStream]].
-    */
-  protected def registerDataStreamInternal[T](
-    name: String,
-    dataStream: DataStream[T]): Unit = {
-
-    val (fieldNames, fieldIndexes) = getFieldInfo[T](dataStream.getType)
-    val dataStreamTable = new DataStreamTable[T](
-      dataStream,
-      fieldIndexes,
-      fieldNames
-    )
-    registerTableInternal(name, dataStreamTable)
-  }
-
-  /**
-    * Registers a [[DataStream]] as a table under a given name with field names as specified by
-    * field expressions in the [[TableEnvironment]]'s catalog.
-    *
-    * @param name The name under which the table is registered in the catalog.
-    * @param dataStream The [[DataStream]] to register as table in the catalog.
-    * @param fields The field expressions to define the field names of the table.
-    * @tparam T The type of the [[DataStream]].
-    */
-  protected def registerDataStreamInternal[T](
-    name: String,
-    dataStream: DataStream[T],
-    fields: Array[Expression]): Unit = {
-
-    val (fieldNames, fieldIndexes) = getFieldInfo[T](dataStream.getType, fields.toArray)
-    val dataStreamTable = new DataStreamTable[T](
-      dataStream,
-      fieldIndexes.toArray,
-      fieldNames.toArray
-    )
-    registerTableInternal(name, dataStreamTable)
-  }
-
-  /**
-    * Returns the built-in rules that are defined by the environment.
-    */
-  protected def getBuiltInRuleSet: RuleSet = FlinkRuleSets.DATASTREAM_OPT_RULES
-
-  /**
-    * Generates the optimized [[RelNode]] tree from the original relational node tree.
-    *
-    * @param relNode The root node of the relational expression tree.
-    * @return The optimized [[RelNode]] tree
-    */
-  private[flink] def optimize(relNode: RelNode): RelNode = {
-    // decorrelate
-    val decorPlan = RelDecorrelator.decorrelateQuery(relNode)
-
-    // optimize the logical Flink plan
-    val optProgram = Programs.ofRules(getRuleSet)
-    val flinkOutputProps = relNode.getTraitSet.replace(DataStreamConvention.INSTANCE).simplify()
-
-    val dataStreamPlan = try {
-      optProgram.run(getPlanner, decorPlan, flinkOutputProps)
-    }
-    catch {
-      case e: CannotPlanException =>
-        throw TableException(
-          s"Cannot generate a valid execution plan for the given query: \n\n" +
-            s"${RelOptUtil.toString(relNode)}\n" +
-            s"This exception indicates that the query uses an unsupported SQL feature.\n" +
-            s"Please check the documentation for the set of currently supported SQL features.", e)
-    }
-    dataStreamPlan
-  }
-
-
-  /**
-    * Translates a [[Table]] into a [[DataStream]].
-    *
-    * The transformation involves optimizing the relational expression tree as defined by
-    * Table API calls and / or SQL queries and generating corresponding [[DataStream]] operators.
-    *
-    * @param table The root node of the relational expression tree.
-    * @param tpe The [[TypeInformation]] of the resulting [[DataStream]].
-    * @tparam A The type of the resulting [[DataStream]].
-    * @return The [[DataStream]] that corresponds to the translated [[Table]].
-    */
-  protected def translate[A](table: Table)(implicit tpe: TypeInformation[A]): DataStream[A] = {
-    val dataStreamPlan = optimize(table.getRelNode)
-    translate(dataStreamPlan)
-  }
-
-  /**
-    * Translates a logical [[RelNode]] into a [[DataStream]].
-    *
-    * @param logicalPlan The root node of the relational expression tree.
-    * @param tpe         The [[TypeInformation]] of the resulting [[DataStream]].
-    * @tparam A The type of the resulting [[DataStream]].
-    * @return The [[DataStream]] that corresponds to the translated [[Table]].
-    */
-  protected def translate[A]
-      (logicalPlan: RelNode)(implicit tpe: TypeInformation[A]): DataStream[A] = {
-
-    validateType(tpe)
-
-    logicalPlan match {
-      case node: DataStreamRel =>
-        node.translateToPlan(
-          this,
-          Some(tpe.asInstanceOf[TypeInformation[Any]])
-        ).asInstanceOf[DataStream[A]]
-      case _ => ???
-    }
-  }
-
-  /**
-    * Returns the AST of the specified Table API and SQL queries and the execution plan to compute
-    * the result of the given [[Table]].
-    *
-    * @param table The table for which the AST and execution plan will be returned.
-    */
-  def explain(table: Table): String = {
-    val ast = table.getRelNode
-    val optimizedPlan = optimize(ast)
-    val dataStream = translate[Row](optimizedPlan)(TypeExtractor.createTypeInfo(classOf[Row]))
-
-    val env = dataStream.getExecutionEnvironment
-    val jsonSqlPlan = env.getExecutionPlan
-
-    val sqlPlan = PlanJsonParser.getSqlExecutionPlan(jsonSqlPlan, false)
-
-    s"== Abstract Syntax Tree ==" +
-        System.lineSeparator +
-        s"${RelOptUtil.toString(ast)}" +
-        System.lineSeparator +
-        s"== Optimized Logical Plan ==" +
-        System.lineSeparator +
-        s"${RelOptUtil.toString(optimizedPlan)}" +
-        System.lineSeparator +
-        s"== Physical Execution Plan ==" +
-        System.lineSeparator +
-        s"$sqlPlan"
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableConfig.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableConfig.scala
deleted file mode 100644
index 37d9cb5..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableConfig.scala
+++ /dev/null
@@ -1,107 +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.api.table
-
-import java.util.TimeZone
-
-/**
- * A config to define the runtime behavior of the Table API.
- */
-class TableConfig {
-
-  /**
-   * Defines the timezone for date/time/timestamp conversions.
-   */
-  private var timeZone: TimeZone = TimeZone.getTimeZone("UTC")
-
-  /**
-   * Defines if all fields need to be checked for NULL first.
-   */
-  private var nullCheck: Boolean = true
-
-  /**
-    * Defines if efficient types (such as Tuple types or Atomic types)
-    * should be used within operators where possible.
-    */
-  private var efficientTypeUsage = false
-
-  /**
-    * Defines the configuration of Calcite for Table API and SQL queries.
-    */
-  private var calciteConfig = CalciteConfig.DEFAULT
-
-  /**
-   * Sets the timezone for date/time/timestamp conversions.
-   */
-  def setTimeZone(timeZone: TimeZone): Unit = {
-    require(timeZone != null, "timeZone must not be null.")
-    this.timeZone = timeZone
-  }
-
-  /**
-   * Returns the timezone for date/time/timestamp conversions.
-   */
-  def getTimeZone = timeZone
-
-  /**
-   * Returns the NULL check. If enabled, all fields need to be checked for NULL first.
-   */
-  def getNullCheck = nullCheck
-
-  /**
-   * Sets the NULL check. If enabled, all fields need to be checked for NULL first.
-   */
-  def setNullCheck(nullCheck: Boolean): Unit = {
-    this.nullCheck = nullCheck
-  }
-
-  /**
-    * Returns the usage of efficient types. If enabled, efficient types (such as Tuple types
-    * or Atomic types) are used within operators where possible.
-    *
-    * NOTE: Currently, this is an experimental feature.
-    */
-  def getEfficientTypeUsage = efficientTypeUsage
-
-  /**
-    * Sets the usage of efficient types. If enabled, efficient types (such as Tuple types
-    * or Atomic types) are used within operators where possible.
-    *
-    * NOTE: Currently, this is an experimental feature.
-    */
-  def setEfficientTypeUsage(efficientTypeUsage: Boolean): Unit = {
-    this.efficientTypeUsage = efficientTypeUsage
-  }
-
-  /**
-    * Returns the current configuration of Calcite for Table API and SQL queries.
-    */
-  def getCalciteConfig: CalciteConfig = calciteConfig
-
-  /**
-    * Sets the configuration of Calcite for Table API and SQL queries.
-    * Changing the configuration has no effect after the first query has been defined.
-    */
-  def setCalciteConfig(calciteConfig: CalciteConfig): Unit = {
-    this.calciteConfig = calciteConfig
-  }
-}
-
-object TableConfig {
-  def DEFAULT = new TableConfig()
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala
deleted file mode 100644
index 07ea860..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/TableEnvironment.scala
+++ /dev/null
@@ -1,537 +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.api.table
-
-import java.lang.reflect.Modifier
-import java.util.concurrent.atomic.AtomicInteger
-
-import org.apache.calcite.config.Lex
-import org.apache.calcite.plan.RelOptPlanner
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.schema.SchemaPlus
-import org.apache.calcite.schema.impl.AbstractTable
-import org.apache.calcite.sql.SqlOperatorTable
-import org.apache.calcite.sql.parser.SqlParser
-import org.apache.calcite.sql.util.ChainedSqlOperatorTable
-import org.apache.calcite.tools.{FrameworkConfig, Frameworks, RuleSet, RuleSets}
-import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
-import org.apache.flink.api.java.table.{BatchTableEnvironment => JavaBatchTableEnv, StreamTableEnvironment => JavaStreamTableEnv}
-import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo}
-import org.apache.flink.api.java.{ExecutionEnvironment => JavaBatchExecEnv}
-import org.apache.flink.api.scala.table.{BatchTableEnvironment => ScalaBatchTableEnv, StreamTableEnvironment => ScalaStreamTableEnv}
-import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
-import org.apache.flink.api.scala.{ExecutionEnvironment => ScalaBatchExecEnv}
-import org.apache.flink.api.table.codegen.ExpressionReducer
-import org.apache.flink.api.table.expressions.{Alias, Expression, UnresolvedFieldReference}
-import org.apache.flink.api.table.functions.utils.UserDefinedFunctionUtils.{checkForInstantiation, checkNotSingleton, createScalarSqlFunction, createTableSqlFunctions}
-import org.apache.flink.api.table.functions.{ScalarFunction, TableFunction}
-import org.apache.flink.api.table.plan.cost.DataSetCostFactory
-import org.apache.flink.api.table.plan.schema.RelTable
-import org.apache.flink.api.table.sinks.TableSink
-import org.apache.flink.api.table.validate.FunctionCatalog
-import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaStreamExecEnv}
-import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => ScalaStreamExecEnv}
-
-import scala.collection.JavaConverters._
-
-/**
-  * The abstract base class for batch and stream TableEnvironments.
-  *
-  * @param config The configuration of the TableEnvironment
-  */
-abstract class TableEnvironment(val config: TableConfig) {
-
-  // the catalog to hold all registered and translated tables
-  private val tables: SchemaPlus = Frameworks.createRootSchema(true)
-
-  // Table API/SQL function catalog
-  private val functionCatalog: FunctionCatalog = FunctionCatalog.withBuiltIns
-
-  // the configuration to create a Calcite planner
-  private lazy val frameworkConfig: FrameworkConfig = Frameworks
-    .newConfigBuilder
-    .defaultSchema(tables)
-    .parserConfig(getSqlParserConfig)
-    .costFactory(new DataSetCostFactory)
-    .typeSystem(new FlinkTypeSystem)
-    .operatorTable(getSqlOperatorTable)
-    // set the executor to evaluate constant expressions
-    .executor(new ExpressionReducer(config))
-    .build
-
-  // the builder for Calcite RelNodes, Calcite's representation of a relational expression tree.
-  protected lazy val relBuilder: FlinkRelBuilder = FlinkRelBuilder.create(frameworkConfig)
-
-  // the planner instance used to optimize queries of this TableEnvironment
-  private lazy val planner: RelOptPlanner = relBuilder.getPlanner
-
-  private lazy val typeFactory: FlinkTypeFactory = relBuilder.getTypeFactory
-
-  // a counter for unique attribute names
-  private val attrNameCntr: AtomicInteger = new AtomicInteger(0)
-
-  /** Returns the table config to define the runtime behavior of the Table API. */
-  def getConfig = config
-
-  /**
-    * Returns the operator table for this environment including a custom Calcite configuration.
-    */
-  protected def getSqlOperatorTable: SqlOperatorTable = {
-    val calciteConfig = config.getCalciteConfig
-    calciteConfig.getSqlOperatorTable match {
-
-      case None =>
-        functionCatalog.getSqlOperatorTable
-
-      case Some(table) =>
-        if (calciteConfig.replacesSqlOperatorTable) {
-          table
-        } else {
-          ChainedSqlOperatorTable.of(functionCatalog.getSqlOperatorTable, table)
-        }
-    }
-  }
-
-  /**
-    * Returns the rule set for this environment including a custom Calcite configuration.
-    */
-  protected def getRuleSet: RuleSet = {
-    val calciteConfig = config.getCalciteConfig
-    calciteConfig.getRuleSet match {
-
-      case None =>
-        getBuiltInRuleSet
-
-      case Some(ruleSet) =>
-        if (calciteConfig.replacesRuleSet) {
-          ruleSet
-        } else {
-          RuleSets.ofList((getBuiltInRuleSet.asScala ++ ruleSet.asScala).asJava)
-        }
-    }
-  }
-
-  /**
-    * Returns the SQL parser config for this environment including a custom Calcite configuration.
-    */
-  protected def getSqlParserConfig: SqlParser.Config = {
-    val calciteConfig = config.getCalciteConfig
-    calciteConfig.getSqlParserConfig match {
-
-      case None =>
-        // we use Java lex because back ticks are easier than double quotes in programming
-        // and cases are preserved
-        SqlParser
-          .configBuilder()
-          .setLex(Lex.JAVA)
-          .build()
-
-      case Some(sqlParserConfig) =>
-        sqlParserConfig
-    }
-  }
-
-  /**
-    * Returns the built-in rules that are defined by the environment.
-    */
-  protected def getBuiltInRuleSet: RuleSet
-
-  /**
-    * Registers a [[ScalarFunction]] under a unique name. Replaces already existing
-    * user-defined functions under this name.
-    */
-  def registerFunction(name: String, function: ScalarFunction): Unit = {
-    // check if class could be instantiated
-    checkForInstantiation(function.getClass)
-
-    // register in Table API
-    functionCatalog.registerFunction(name, function.getClass)
-
-    // register in SQL API
-    functionCatalog.registerSqlFunction(createScalarSqlFunction(name, function, typeFactory))
-  }
-
-  /**
-    * Registers a [[TableFunction]] under a unique name. Replaces already existing
-    * user-defined functions under this name.
-    */
-  private[flink] def registerTableFunctionInternal[T: TypeInformation](
-    name: String, function: TableFunction[T]): Unit = {
-    // check if class not Scala object
-    checkNotSingleton(function.getClass)
-    // check if class could be instantiated
-    checkForInstantiation(function.getClass)
-
-    val typeInfo: TypeInformation[_] = if (function.getResultType != null) {
-      function.getResultType
-    } else {
-      implicitly[TypeInformation[T]]
-    }
-
-    // register in Table API
-    functionCatalog.registerFunction(name, function.getClass)
-
-    // register in SQL API
-    val sqlFunctions = createTableSqlFunctions(name, function, typeInfo, typeFactory)
-    functionCatalog.registerSqlFunctions(sqlFunctions)
-  }
-
-  /**
-    * Registers a [[Table]] under a unique name in the TableEnvironment's catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * @param name The name under which the table is registered.
-    * @param table The table to register.
-    */
-  def registerTable(name: String, table: Table): Unit = {
-
-    // check that table belongs to this table environment
-    if (table.tableEnv != this) {
-      throw new TableException(
-        "Only tables that belong to this TableEnvironment can be registered.")
-    }
-
-    checkValidTableName(name)
-    val tableTable = new RelTable(table.getRelNode)
-    registerTableInternal(name, tableTable)
-  }
-
-  /**
-    * Replaces a registered Table with another Table under the same name.
-    * We use this method to replace a [[org.apache.flink.api.table.plan.schema.DataStreamTable]]
-    * with a [[org.apache.calcite.schema.TranslatableTable]].
-    *
-    * @param name Name of the table to replace.
-    * @param table The table that replaces the previous table.
-    */
-  protected def replaceRegisteredTable(name: String, table: AbstractTable): Unit = {
-
-    if (isRegistered(name)) {
-      tables.add(name, table)
-    } else {
-      throw new TableException(s"Table \'$name\' is not registered.")
-    }
-  }
-
-  /**
-    * Evaluates a SQL query on registered tables and retrieves the result as a [[Table]].
-    *
-    * All tables referenced by the query must be registered in the TableEnvironment.
-    *
-    * @param query The SQL query to evaluate.
-    * @return The result of the query as Table.
-    */
-  def sql(query: String): Table
-
-  /**
-    * Writes a [[Table]] to a [[TableSink]].
-    *
-    * @param table The [[Table]] to write.
-    * @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
-
-  /**
-    * Registers a Calcite [[AbstractTable]] in the TableEnvironment's catalog.
-    *
-    * @param name The name under which the table is registered.
-    * @param table The table to register in the catalog
-    * @throws TableException if another table is registered under the provided name.
-    */
-  @throws[TableException]
-  protected def registerTableInternal(name: String, table: AbstractTable): Unit = {
-
-    if (isRegistered(name)) {
-      throw new TableException(s"Table \'$name\' already exists. " +
-        s"Please, choose a different name.")
-    } else {
-      tables.add(name, table)
-    }
-  }
-
-  /**
-    * Checks if the chosen table name is valid.
-    *
-    * @param name The table name to check.
-    */
-  protected def checkValidTableName(name: String): Unit
-
-  /**
-    * Checks if a table is registered under the given name.
-    *
-    * @param name The table name to check.
-    * @return true, if a table is registered under the name, false otherwise.
-    */
-  protected def isRegistered(name: String): Boolean = {
-    tables.getTableNames.contains(name)
-  }
-
-  protected def getRowType(name: String): RelDataType = {
-    tables.getTable(name).getRowType(typeFactory)
-  }
-
-  /** Returns a unique temporary attribute name. */
-  private[flink] def createUniqueAttributeName(): String = {
-    "TMP_" + attrNameCntr.getAndIncrement()
-  }
-
-  /** Returns the [[FlinkRelBuilder]] of this TableEnvironment. */
-  private[flink] def getRelBuilder: FlinkRelBuilder = {
-    relBuilder
-  }
-
-  /** Returns the Calcite [[org.apache.calcite.plan.RelOptPlanner]] of this TableEnvironment. */
-  private[flink] def getPlanner: RelOptPlanner = {
-    planner
-  }
-
-  /** Returns the [[FlinkTypeFactory]] of this TableEnvironment. */
-  private[flink] def getTypeFactory: FlinkTypeFactory = {
-    typeFactory
-  }
-
-  private[flink] def getFunctionCatalog: FunctionCatalog = {
-    functionCatalog
-  }
-
-  /** Returns the Calcite [[FrameworkConfig]] of this TableEnvironment. */
-  private[flink] def getFrameworkConfig: FrameworkConfig = {
-    frameworkConfig
-  }
-
-  protected def validateType(typeInfo: TypeInformation[_]): Unit = {
-    val clazz = typeInfo.getTypeClass
-    if ((clazz.isMemberClass && !Modifier.isStatic(clazz.getModifiers)) ||
-        !Modifier.isPublic(clazz.getModifiers) ||
-        clazz.getCanonicalName == null) {
-      throw TableException(s"Class '$clazz' described in type information '$typeInfo' must be " +
-        s"static and globally accessible.")
-    }
-  }
-
-  /**
-    * Returns field names and field positions for a given [[TypeInformation]].
-    *
-    * Field names are automatically extracted for
-    * [[org.apache.flink.api.common.typeutils.CompositeType]].
-    * The method fails if inputType is not a
-    * [[org.apache.flink.api.common.typeutils.CompositeType]].
-    *
-    * @param inputType The TypeInformation extract the field names and positions from.
-    * @tparam A The type of the TypeInformation.
-    * @return A tuple of two arrays holding the field names and corresponding field positions.
-    */
-  protected[flink] def getFieldInfo[A](inputType: TypeInformation[A]):
-      (Array[String], Array[Int]) =
-  {
-    validateType(inputType)
-
-    val fieldNames: Array[String] = inputType match {
-      case t: TupleTypeInfo[A] => t.getFieldNames
-      case c: CaseClassTypeInfo[A] => c.getFieldNames
-      case p: PojoTypeInfo[A] => p.getFieldNames
-      case r: RowTypeInfo => r.getFieldNames
-      case tpe =>
-        throw new TableException(s"Type $tpe lacks explicit field naming")
-    }
-    val fieldIndexes = fieldNames.indices.toArray
-
-    if (fieldNames.contains("*")) {
-      throw new TableException("Field name can not be '*'.")
-    }
-
-    (fieldNames, fieldIndexes)
-  }
-
-  /**
-    * Returns field names and field positions for a given [[TypeInformation]] and [[Array]] of
-    * [[Expression]].
-    *
-    * @param inputType The [[TypeInformation]] against which the [[Expression]]s are evaluated.
-    * @param exprs The expressions that define the field names.
-    * @tparam A The type of the TypeInformation.
-    * @return A tuple of two arrays holding the field names and corresponding field positions.
-    */
-  protected[flink] def getFieldInfo[A](
-    inputType: TypeInformation[A],
-    exprs: Array[Expression]): (Array[String], Array[Int]) = {
-
-    validateType(inputType)
-
-    val indexedNames: Array[(Int, String)] = inputType match {
-      case a: AtomicType[A] =>
-        if (exprs.length != 1) {
-          throw new TableException("Table of atomic type can only have a single field.")
-        }
-        exprs.map {
-          case UnresolvedFieldReference(name) => (0, name)
-          case _ => throw new TableException("Field reference expression expected.")
-        }
-      case t: TupleTypeInfo[A] =>
-        exprs.zipWithIndex.map {
-          case (UnresolvedFieldReference(name), idx) => (idx, name)
-          case (Alias(UnresolvedFieldReference(origName), name, _), _) =>
-            val idx = t.getFieldIndex(origName)
-            if (idx < 0) {
-              throw new TableException(s"$origName is not a field of type $t")
-            }
-            (idx, name)
-          case _ => throw new TableException(
-            "Field reference expression or alias on field expression expected.")
-        }
-      case c: CaseClassTypeInfo[A] =>
-        exprs.zipWithIndex.map {
-          case (UnresolvedFieldReference(name), idx) => (idx, name)
-          case (Alias(UnresolvedFieldReference(origName), name, _), _) =>
-            val idx = c.getFieldIndex(origName)
-            if (idx < 0) {
-              throw new TableException(s"$origName is not a field of type $c")
-            }
-            (idx, name)
-          case _ => throw new TableException(
-            "Field reference expression or alias on field expression expected.")
-        }
-      case p: PojoTypeInfo[A] =>
-        exprs.map {
-          case (UnresolvedFieldReference(name)) =>
-            val idx = p.getFieldIndex(name)
-            if (idx < 0) {
-              throw new TableException(s"$name is not a field of type $p")
-            }
-            (idx, name)
-          case Alias(UnresolvedFieldReference(origName), name, _) =>
-            val idx = p.getFieldIndex(origName)
-            if (idx < 0) {
-              throw new TableException(s"$origName is not a field of type $p")
-            }
-            (idx, name)
-          case _ => throw new TableException(
-            "Field reference expression or alias on field expression expected.")
-        }
-      case tpe => throw new TableException(
-        s"Source of type $tpe cannot be converted into Table.")
-    }
-
-    val (fieldIndexes, fieldNames) = indexedNames.unzip
-
-    if (fieldNames.contains("*")) {
-      throw new TableException("Field name can not be '*'.")
-    }
-
-    (fieldNames.toArray, fieldIndexes.toArray)
-  }
-
-}
-
-/**
-  * Object to instantiate a [[TableEnvironment]] depending on the batch or stream execution
-  * environment.
-  */
-object TableEnvironment {
-
-  /**
-    * Returns a [[JavaBatchTableEnv]] for a Java [[JavaBatchExecEnv]].
-    *
-    * @param executionEnvironment The Java batch ExecutionEnvironment.
-    */
-  def getTableEnvironment(executionEnvironment: JavaBatchExecEnv): JavaBatchTableEnv = {
-    new JavaBatchTableEnv(executionEnvironment, new TableConfig())
-  }
-
-  /**
-    * Returns a [[JavaBatchTableEnv]] for a Java [[JavaBatchExecEnv]] and a given [[TableConfig]].
-    *
-    * @param executionEnvironment The Java batch ExecutionEnvironment.
-    * @param tableConfig The TableConfig for the new TableEnvironment.
-    */
-  def getTableEnvironment(
-    executionEnvironment: JavaBatchExecEnv,
-    tableConfig: TableConfig): JavaBatchTableEnv = {
-
-    new JavaBatchTableEnv(executionEnvironment, tableConfig)
-  }
-
-  /**
-    * Returns a [[ScalaBatchTableEnv]] for a Scala [[ScalaBatchExecEnv]].
-    *
-    * @param executionEnvironment The Scala batch ExecutionEnvironment.
-    */
-  def getTableEnvironment(executionEnvironment: ScalaBatchExecEnv): ScalaBatchTableEnv = {
-    new ScalaBatchTableEnv(executionEnvironment, new TableConfig())
-  }
-
-  /**
-    * Returns a [[ScalaBatchTableEnv]] for a Scala [[ScalaBatchExecEnv]] and a given
-    * [[TableConfig]].
-    *
-    * @param executionEnvironment The Scala batch ExecutionEnvironment.
-    * @param tableConfig The TableConfig for the new TableEnvironment.
-    */
-  def getTableEnvironment(
-    executionEnvironment: ScalaBatchExecEnv,
-    tableConfig: TableConfig): ScalaBatchTableEnv = {
-
-    new ScalaBatchTableEnv(executionEnvironment, tableConfig)
-  }
-
-  /**
-    * Returns a [[JavaStreamTableEnv]] for a Java [[JavaStreamExecEnv]].
-    *
-    * @param executionEnvironment The Java StreamExecutionEnvironment.
-    */
-  def getTableEnvironment(executionEnvironment: JavaStreamExecEnv): JavaStreamTableEnv = {
-    new JavaStreamTableEnv(executionEnvironment, new TableConfig())
-  }
-
-  /**
-    * Returns a [[JavaStreamTableEnv]] for a Java [[JavaStreamExecEnv]] and a given [[TableConfig]].
-    *
-    * @param executionEnvironment The Java StreamExecutionEnvironment.
-    * @param tableConfig The TableConfig for the new TableEnvironment.
-    */
-  def getTableEnvironment(
-    executionEnvironment: JavaStreamExecEnv,
-    tableConfig: TableConfig): JavaStreamTableEnv = {
-
-    new JavaStreamTableEnv(executionEnvironment, tableConfig)
-  }
-
-  /**
-    * Returns a [[ScalaStreamTableEnv]] for a Scala stream [[ScalaStreamExecEnv]].
-    *
-    * @param executionEnvironment The Scala StreamExecutionEnvironment.
-    */
-  def getTableEnvironment(executionEnvironment: ScalaStreamExecEnv): ScalaStreamTableEnv = {
-    new ScalaStreamTableEnv(executionEnvironment, new TableConfig())
-  }
-
-  /**
-    * Returns a [[ScalaStreamTableEnv]] for a Scala stream [[ScalaStreamExecEnv]].
-    *
-    * @param executionEnvironment The Scala StreamExecutionEnvironment.
-    * @param tableConfig The TableConfig for the new TableEnvironment.
-    */
-  def getTableEnvironment(
-    executionEnvironment: ScalaStreamExecEnv,
-    tableConfig: TableConfig): ScalaStreamTableEnv = {
-
-    new ScalaStreamTableEnv(executionEnvironment, tableConfig)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Types.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Types.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Types.scala
deleted file mode 100644
index a988152..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/Types.scala
+++ /dev/null
@@ -1,45 +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.api.table
-
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo}
-import org.apache.flink.api.table.typeutils.TimeIntervalTypeInfo
-
-/**
-  * This class enumerates all supported types of the Table API.
-  */
-object Types {
-
-  val STRING = BasicTypeInfo.STRING_TYPE_INFO
-  val BOOLEAN = BasicTypeInfo.BOOLEAN_TYPE_INFO
-
-  val BYTE = BasicTypeInfo.BYTE_TYPE_INFO
-  val SHORT = BasicTypeInfo.SHORT_TYPE_INFO
-  val INT = BasicTypeInfo.INT_TYPE_INFO
-  val LONG = BasicTypeInfo.LONG_TYPE_INFO
-  val FLOAT = BasicTypeInfo.FLOAT_TYPE_INFO
-  val DOUBLE = BasicTypeInfo.DOUBLE_TYPE_INFO
-  val DECIMAL = BasicTypeInfo.BIG_DEC_TYPE_INFO
-
-  val DATE = SqlTimeTypeInfo.DATE
-  val TIME = SqlTimeTypeInfo.TIME
-  val TIMESTAMP = SqlTimeTypeInfo.TIMESTAMP
-  val INTERVAL_MONTHS = TimeIntervalTypeInfo.INTERVAL_MONTHS
-  val INTERVAL_MILLIS = TimeIntervalTypeInfo.INTERVAL_MILLIS
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenException.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenException.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenException.scala
deleted file mode 100644
index 8b7559f..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenException.scala
+++ /dev/null
@@ -1,24 +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.api.table.codegen
-
-/**
-  * Exception for all errors occurring during code generation.
-  */
-class CodeGenException(msg: String) extends RuntimeException(msg)


[29/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TimeIntervalTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TimeIntervalTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TimeIntervalTypeInfo.scala
deleted file mode 100644
index b516745..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TimeIntervalTypeInfo.scala
+++ /dev/null
@@ -1,113 +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.api.table.typeutils
-
-import java.util.Objects
-
-import org.apache.flink.api.common.ExecutionConfig
-import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
-import org.apache.flink.api.common.typeutils.base.{IntComparator, IntSerializer, LongComparator, LongSerializer}
-import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
-import org.apache.flink.api.table.typeutils.TimeIntervalTypeInfo.instantiateComparator
-import org.apache.flink.util.Preconditions._
-
-/**
-  * TypeInformation for SQL INTERVAL types.
-  */
-@SerialVersionUID(-1816179424364825258L)
-class TimeIntervalTypeInfo[T](
-    val clazz: Class[T],
-    val serializer: TypeSerializer[T],
-    val comparatorClass: Class[_ <: TypeComparator[T]])
-  extends TypeInformation[T]
-  with AtomicType[T] {
-
-  checkNotNull(clazz)
-  checkNotNull(serializer)
-  checkNotNull(comparatorClass)
-
-  override def isBasicType: Boolean = false
-
-  override def isTupleType: Boolean = false
-
-  override def getArity: Int = 1
-
-  override def getTotalFields: Int = 1
-
-  override def getTypeClass: Class[T] = clazz
-
-  override def isKeyType: Boolean = true
-
-  override def createSerializer(config: ExecutionConfig): TypeSerializer[T] = serializer
-
-  override def createComparator(
-      sortOrderAscending: Boolean,
-      executionConfig: ExecutionConfig)
-    : TypeComparator[T] = instantiateComparator(comparatorClass, sortOrderAscending)
-
-  // ----------------------------------------------------------------------------------------------
-
-  override def hashCode: Int = Objects.hash(clazz, serializer, comparatorClass)
-
-  def canEqual(obj: Any): Boolean = obj.isInstanceOf[TimeIntervalTypeInfo[_]]
-
-  override def equals(obj: Any): Boolean = {
-    obj match {
-      case other: TimeIntervalTypeInfo[_] =>
-        other.canEqual(this) &&
-          (this.clazz eq other.clazz) &&
-          serializer == other.serializer &&
-          (this.comparatorClass eq other.comparatorClass)
-      case _ =>
-        false
-    }
-  }
-
-  override def toString: String = s"TimeIntervalTypeInfo(${clazz.getSimpleName})"
-}
-
-object TimeIntervalTypeInfo {
-
-  val INTERVAL_MONTHS = new TimeIntervalTypeInfo(
-    classOf[java.lang.Integer],
-    IntSerializer.INSTANCE,
-    classOf[IntComparator])
-
-  val INTERVAL_MILLIS = new TimeIntervalTypeInfo(
-    classOf[java.lang.Long],
-    LongSerializer.INSTANCE,
-    classOf[LongComparator])
-
-  // ----------------------------------------------------------------------------------------------
-
-  private def instantiateComparator[X](
-      comparatorClass: Class[_ <: TypeComparator[X]],
-      ascendingOrder: java.lang.Boolean)
-    : TypeComparator[X] = {
-    try {
-      val constructor = comparatorClass.getConstructor(java.lang.Boolean.TYPE)
-      constructor.newInstance(ascendingOrder)
-    } catch {
-      case e: Exception =>
-        throw new RuntimeException(
-          s"Could not initialize comparator ${comparatorClass.getName}", e)
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCheckUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCheckUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCheckUtils.scala
deleted file mode 100644
index e30e273..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCheckUtils.scala
+++ /dev/null
@@ -1,94 +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.api.table.typeutils
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{BIG_DEC_TYPE_INFO, BOOLEAN_TYPE_INFO, INT_TYPE_INFO, STRING_TYPE_INFO}
-import org.apache.flink.api.common.typeinfo._
-import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
-import org.apache.flink.api.table.validate._
-
-object TypeCheckUtils {
-
-  /**
-    * Checks if type information is an advanced type that can be converted to a
-    * SQL type but NOT vice versa.
-    */
-  def isAdvanced(dataType: TypeInformation[_]): Boolean = dataType match {
-    case _: BasicTypeInfo[_] => false
-    case _: SqlTimeTypeInfo[_] => false
-    case _: TimeIntervalTypeInfo[_] => false
-    case _ => true
-  }
-
-  /**
-    * Checks if type information is a simple type that can be converted to a
-    * SQL type and vice versa.
-    */
-  def isSimple(dataType: TypeInformation[_]): Boolean = !isAdvanced(dataType)
-
-  def isNumeric(dataType: TypeInformation[_]): Boolean = dataType match {
-    case _: NumericTypeInfo[_] => true
-    case BIG_DEC_TYPE_INFO => true
-    case _ => false
-  }
-
-  def isTemporal(dataType: TypeInformation[_]): Boolean =
-    isTimePoint(dataType) || isTimeInterval(dataType)
-
-  def isTimePoint(dataType: TypeInformation[_]): Boolean =
-    dataType.isInstanceOf[SqlTimeTypeInfo[_]]
-
-  def isTimeInterval(dataType: TypeInformation[_]): Boolean =
-    dataType.isInstanceOf[TimeIntervalTypeInfo[_]]
-
-  def isString(dataType: TypeInformation[_]): Boolean = dataType == STRING_TYPE_INFO
-
-  def isBoolean(dataType: TypeInformation[_]): Boolean = dataType == BOOLEAN_TYPE_INFO
-
-  def isDecimal(dataType: TypeInformation[_]): Boolean = dataType == BIG_DEC_TYPE_INFO
-
-  def isInteger(dataType: TypeInformation[_]): Boolean = dataType == INT_TYPE_INFO
-
-  def isArray(dataType: TypeInformation[_]): Boolean = dataType match {
-    case _: ObjectArrayTypeInfo[_, _] | _: PrimitiveArrayTypeInfo[_] => true
-    case _ => false
-  }
-
-  def isComparable(dataType: TypeInformation[_]): Boolean =
-    classOf[Comparable[_]].isAssignableFrom(dataType.getTypeClass) && !isArray(dataType)
-
-  def assertNumericExpr(
-      dataType: TypeInformation[_],
-      caller: String)
-    : ValidationResult = dataType match {
-    case _: NumericTypeInfo[_] =>
-      ValidationSuccess
-    case BIG_DEC_TYPE_INFO =>
-      ValidationSuccess
-    case _ =>
-      ValidationFailure(s"$caller requires numeric types, get $dataType here")
-  }
-
-  def assertOrderableExpr(dataType: TypeInformation[_], caller: String): ValidationResult = {
-    if (dataType.isSortKeyType) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(s"$caller requires orderable types, get $dataType here")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCoercion.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCoercion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCoercion.scala
deleted file mode 100644
index 23154a5..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeCoercion.scala
+++ /dev/null
@@ -1,121 +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.api.table.typeutils
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, NumericTypeInfo, TypeInformation}
-
-/**
-  * Utilities for type conversions.
-  */
-object TypeCoercion {
-
-  val numericWideningPrecedence: IndexedSeq[TypeInformation[_]] =
-    IndexedSeq(
-      BYTE_TYPE_INFO,
-      SHORT_TYPE_INFO,
-      INT_TYPE_INFO,
-      LONG_TYPE_INFO,
-      FLOAT_TYPE_INFO,
-      DOUBLE_TYPE_INFO)
-
-  def widerTypeOf(tp1: TypeInformation[_], tp2: TypeInformation[_]): Option[TypeInformation[_]] = {
-    (tp1, tp2) match {
-      case (ti1, ti2) if ti1 == ti2 => Some(ti1)
-
-      case (_, STRING_TYPE_INFO) => Some(STRING_TYPE_INFO)
-      case (STRING_TYPE_INFO, _) => Some(STRING_TYPE_INFO)
-
-      case (_, BIG_DEC_TYPE_INFO) => Some(BIG_DEC_TYPE_INFO)
-      case (BIG_DEC_TYPE_INFO, _) => Some(BIG_DEC_TYPE_INFO)
-
-      case (stti: SqlTimeTypeInfo[_], _: TimeIntervalTypeInfo[_]) => Some(stti)
-      case (_: TimeIntervalTypeInfo[_], stti: SqlTimeTypeInfo[_]) => Some(stti)
-
-      case tuple if tuple.productIterator.forall(numericWideningPrecedence.contains) =>
-        val higherIndex = numericWideningPrecedence.lastIndexWhere(t => t == tp1 || t == tp2)
-        Some(numericWideningPrecedence(higherIndex))
-
-      case _ => None
-    }
-  }
-
-  /**
-    * Test if we can do cast safely without lose of information.
-    */
-  def canSafelyCast(from: TypeInformation[_], to: TypeInformation[_]): Boolean = (from, to) match {
-    case (_, STRING_TYPE_INFO) => true
-
-    case (_: NumericTypeInfo[_], BIG_DEC_TYPE_INFO) => true
-
-    case tuple if tuple.productIterator.forall(numericWideningPrecedence.contains) =>
-      if (numericWideningPrecedence.indexOf(from) < numericWideningPrecedence.indexOf(to)) {
-        true
-      } else {
-        false
-      }
-
-    case _ => false
-  }
-
-  /**
-    * All the supported cast types in flink-table.
-    * Note: This may lose information during the cast.
-    */
-  def canCast(from: TypeInformation[_], to: TypeInformation[_]): Boolean = (from, to) match {
-    case (fromTp, toTp) if fromTp == toTp => true
-
-    case (_, STRING_TYPE_INFO) => true
-
-    case (_, CHAR_TYPE_INFO) => false // Character type not supported.
-
-    case (STRING_TYPE_INFO, _: NumericTypeInfo[_]) => true
-    case (STRING_TYPE_INFO, BOOLEAN_TYPE_INFO) => true
-    case (STRING_TYPE_INFO, BIG_DEC_TYPE_INFO) => true
-    case (STRING_TYPE_INFO, SqlTimeTypeInfo.DATE) => true
-    case (STRING_TYPE_INFO, SqlTimeTypeInfo.TIME) => true
-    case (STRING_TYPE_INFO, SqlTimeTypeInfo.TIMESTAMP) => true
-
-    case (BOOLEAN_TYPE_INFO, _: NumericTypeInfo[_]) => true
-    case (BOOLEAN_TYPE_INFO, BIG_DEC_TYPE_INFO) => true
-    case (_: NumericTypeInfo[_], BOOLEAN_TYPE_INFO) => true
-    case (BIG_DEC_TYPE_INFO, BOOLEAN_TYPE_INFO) => true
-
-    case (_: NumericTypeInfo[_], _: NumericTypeInfo[_]) => true
-    case (BIG_DEC_TYPE_INFO, _: NumericTypeInfo[_]) => true
-    case (_: NumericTypeInfo[_], BIG_DEC_TYPE_INFO) => true
-    case (INT_TYPE_INFO, SqlTimeTypeInfo.DATE) => true
-    case (INT_TYPE_INFO, SqlTimeTypeInfo.TIME) => true
-    case (LONG_TYPE_INFO, SqlTimeTypeInfo.TIMESTAMP) => true
-    case (INT_TYPE_INFO, TimeIntervalTypeInfo.INTERVAL_MONTHS) => true
-    case (LONG_TYPE_INFO, TimeIntervalTypeInfo.INTERVAL_MILLIS) => true
-
-    case (SqlTimeTypeInfo.DATE, SqlTimeTypeInfo.TIME) => false
-    case (SqlTimeTypeInfo.TIME, SqlTimeTypeInfo.DATE) => false
-    case (_: SqlTimeTypeInfo[_], _: SqlTimeTypeInfo[_]) => true
-    case (SqlTimeTypeInfo.DATE, INT_TYPE_INFO) => true
-    case (SqlTimeTypeInfo.TIME, INT_TYPE_INFO) => true
-    case (SqlTimeTypeInfo.TIMESTAMP, LONG_TYPE_INFO) => true
-
-    case (TimeIntervalTypeInfo.INTERVAL_MONTHS, INT_TYPE_INFO) => true
-    case (TimeIntervalTypeInfo.INTERVAL_MILLIS, LONG_TYPE_INFO) => true
-
-    case _ => false
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeConverter.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeConverter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeConverter.scala
deleted file mode 100644
index a81577c..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/TypeConverter.scala
+++ /dev/null
@@ -1,155 +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.api.table.typeutils
-
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.JoinRelType
-import org.apache.calcite.rel.core.JoinRelType._
-import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.java.operators.join.JoinType
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo}
-import org.apache.flink.api.table.{FlinkTypeFactory, TableException}
-import org.apache.flink.types.Row
-
-import scala.collection.JavaConversions._
-
-object TypeConverter {
-
-  val DEFAULT_ROW_TYPE = new RowTypeInfo().asInstanceOf[TypeInformation[Any]]
-
-  /**
-    * Determines the return type of Flink operators based on the logical fields, the expected
-    * physical type and configuration parameters.
-    *
-    * For example:
-    *   - No physical type expected, only 3 non-null fields and efficient type usage enabled
-    *       -> return Tuple3
-    *   - No physical type expected, efficient type usage enabled, but 3 nullable fields
-    *       -> return Row because Tuple does not support null values
-    *   - Physical type expected
-    *       -> check if physical type is compatible and return it
-    *
-    * @param logicalRowType logical row information
-    * @param expectedPhysicalType expected physical type
-    * @param nullable fields can be nullable
-    * @param useEfficientTypes use the most efficient types (e.g. Tuples and value types)
-    * @return suitable return type
-    */
-  def determineReturnType(
-      logicalRowType: RelDataType,
-      expectedPhysicalType: Option[TypeInformation[Any]],
-      nullable: Boolean,
-      useEfficientTypes: Boolean)
-    : TypeInformation[Any] = {
-    // convert to type information
-    val logicalFieldTypes = logicalRowType.getFieldList map { relDataType =>
-      FlinkTypeFactory.toTypeInfo(relDataType.getType)
-    }
-    // field names
-    val logicalFieldNames = logicalRowType.getFieldNames.toList
-
-    val returnType = expectedPhysicalType match {
-      // a certain physical type is expected (but not Row)
-      // check if expected physical type is compatible with logical field type
-      case Some(typeInfo) if typeInfo.getTypeClass != classOf[Row] =>
-        if (typeInfo.getArity != logicalFieldTypes.length) {
-          throw new TableException("Arity of result does not match expected type.")
-        }
-        typeInfo match {
-
-          // POJO type expected
-          case pt: PojoTypeInfo[_] =>
-            logicalFieldNames.zip(logicalFieldTypes) foreach {
-              case (fName, fType) =>
-                val pojoIdx = pt.getFieldIndex(fName)
-                if (pojoIdx < 0) {
-                  throw new TableException(s"POJO does not define field name: $fName")
-                }
-                val expectedTypeInfo = pt.getTypeAt(pojoIdx)
-                if (fType != expectedTypeInfo) {
-                  throw new TableException(s"Result field does not match expected type. " +
-                    s"Expected: $expectedTypeInfo; Actual: $fType")
-                }
-            }
-
-          // Tuple/Case class type expected
-          case ct: CompositeType[_] =>
-            logicalFieldTypes.zipWithIndex foreach {
-              case (fieldTypeInfo, i) =>
-                val expectedTypeInfo = ct.getTypeAt(i)
-                if (fieldTypeInfo != expectedTypeInfo) {
-                  throw new TableException(s"Result field does not match expected type. " +
-                    s"Expected: $expectedTypeInfo; Actual: $fieldTypeInfo")
-                }
-            }
-
-          // Atomic type expected
-          case at: AtomicType[_] =>
-            val fieldTypeInfo = logicalFieldTypes.head
-            if (fieldTypeInfo != at) {
-              throw new TableException(s"Result field does not match expected type. " +
-                s"Expected: $at; Actual: $fieldTypeInfo")
-            }
-
-          case _ =>
-            throw new TableException("Unsupported result type.")
-        }
-        typeInfo
-
-      // Row is expected, create the arity for it
-      case Some(typeInfo) if typeInfo.getTypeClass == classOf[Row] =>
-        new RowTypeInfo(logicalFieldTypes: _*)
-
-      // no physical type
-      // determine type based on logical fields and configuration parameters
-      case None =>
-        // no need for efficient types -> use Row
-        // we cannot use efficient types if row arity > tuple arity or nullable
-        if (!useEfficientTypes || logicalFieldTypes.length > Tuple.MAX_ARITY || nullable) {
-          new RowTypeInfo(logicalFieldTypes: _*)
-        }
-        // use efficient type tuple or atomic type
-        else {
-          if (logicalFieldTypes.length == 1) {
-            logicalFieldTypes.head
-          }
-          else {
-            new TupleTypeInfo[Tuple](logicalFieldTypes.toArray:_*)
-          }
-        }
-    }
-    returnType.asInstanceOf[TypeInformation[Any]]
-  }
-
-  def sqlJoinTypeToFlinkJoinType(sqlJoinType: JoinRelType): JoinType = sqlJoinType match {
-    case INNER => JoinType.INNER
-    case LEFT => JoinType.LEFT_OUTER
-    case RIGHT => JoinType.RIGHT_OUTER
-    case FULL => JoinType.FULL_OUTER
-  }
-
-  def flinkJoinTypeToRelType(joinType: JoinType) = joinType match {
-    case JoinType.INNER => JoinRelType.INNER
-    case JoinType.LEFT_OUTER => JoinRelType.LEFT
-    case JoinType.RIGHT_OUTER => JoinRelType.RIGHT
-    case JoinType.FULL_OUTER => JoinRelType.FULL
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala
deleted file mode 100644
index 8e409cc..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/FunctionCatalog.scala
+++ /dev/null
@@ -1,320 +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.api.table.validate
-
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-import org.apache.calcite.sql.util.{ChainedSqlOperatorTable, ListSqlOperatorTable, ReflectiveSqlOperatorTable}
-import org.apache.calcite.sql.{SqlFunction, SqlOperator, SqlOperatorTable}
-import org.apache.flink.api.table.ValidationException
-import org.apache.flink.api.table.expressions._
-import org.apache.flink.api.table.functions.{ScalarFunction, TableFunction}
-import org.apache.flink.api.table.functions.utils.{TableSqlFunction, UserDefinedFunctionUtils}
-
-import scala.collection.JavaConversions._
-import scala.collection.mutable
-import scala.util.{Failure, Success, Try}
-
-/**
-  * A catalog for looking up (user-defined) functions, used during validation phases
-  * of both Table API and SQL API.
-  */
-class FunctionCatalog {
-
-  private val functionBuilders = mutable.HashMap.empty[String, Class[_]]
-  private val sqlFunctions = mutable.ListBuffer[SqlFunction]()
-
-  def registerFunction(name: String, builder: Class[_]): Unit =
-    functionBuilders.put(name.toLowerCase, builder)
-
-  def registerSqlFunction(sqlFunction: SqlFunction): Unit = {
-    sqlFunctions --= sqlFunctions.filter(_.getName == sqlFunction.getName)
-    sqlFunctions += sqlFunction
-  }
-
-  /**
-    * Register multiple SQL functions at the same time. The functions have the same name.
-    */
-  def registerSqlFunctions(functions: Seq[SqlFunction]): Unit = {
-    if (functions.nonEmpty) {
-      val name = functions.head.getName
-      // check that all functions have the same name
-      if (functions.forall(_.getName == name)) {
-        sqlFunctions --= sqlFunctions.filter(_.getName == name)
-        sqlFunctions ++= functions
-      } else {
-        throw ValidationException("The SQL functions to be registered have different names.")
-      }
-    }
-  }
-
-  def getSqlOperatorTable: SqlOperatorTable =
-    ChainedSqlOperatorTable.of(
-      new BasicOperatorTable(),
-      new ListSqlOperatorTable(sqlFunctions)
-    )
-
-  /**
-    * Lookup and create an expression if we find a match.
-    */
-  def lookupFunction(name: String, children: Seq[Expression]): Expression = {
-    val funcClass = functionBuilders
-      .getOrElse(name.toLowerCase, throw ValidationException(s"Undefined function: $name"))
-
-    // Instantiate a function using the provided `children`
-    funcClass match {
-
-      // user-defined scalar function call
-      case sf if classOf[ScalarFunction].isAssignableFrom(sf) =>
-        Try(UserDefinedFunctionUtils.instantiate(sf.asInstanceOf[Class[ScalarFunction]])) match {
-          case Success(scalarFunction) => ScalarFunctionCall(scalarFunction, children)
-          case Failure(e) => throw ValidationException(e.getMessage)
-        }
-
-      // user-defined table function call
-      case tf if classOf[TableFunction[_]].isAssignableFrom(tf) =>
-        val tableSqlFunction = sqlFunctions
-          .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[TableSqlFunction])
-          .getOrElse(throw ValidationException(s"Undefined table function: $name"))
-          .asInstanceOf[TableSqlFunction]
-        val typeInfo = tableSqlFunction.getRowTypeInfo
-        val function = tableSqlFunction.getTableFunction
-        TableFunctionCall(name, function, children, typeInfo)
-
-      // general expression call
-      case expression if classOf[Expression].isAssignableFrom(expression) =>
-        // try to find a constructor accepts `Seq[Expression]`
-        Try(funcClass.getDeclaredConstructor(classOf[Seq[_]])) match {
-          case Success(seqCtor) =>
-            Try(seqCtor.newInstance(children).asInstanceOf[Expression]) match {
-              case Success(expr) => expr
-              case Failure(e) => throw new ValidationException(e.getMessage)
-            }
-          case Failure(e) =>
-            val childrenClass = Seq.fill(children.length)(classOf[Expression])
-            // try to find a constructor matching the exact number of children
-            Try(funcClass.getDeclaredConstructor(childrenClass: _*)) match {
-              case Success(ctor) =>
-                Try(ctor.newInstance(children: _*).asInstanceOf[Expression]) match {
-                  case Success(expr) => expr
-                  case Failure(exception) => throw ValidationException(exception.getMessage)
-                }
-              case Failure(exception) =>
-                throw ValidationException(s"Invalid number of arguments for function $funcClass")
-            }
-        }
-
-      case _ =>
-        throw ValidationException("Unsupported function.")
-    }
-  }
-
-  /**
-    * Drop a function and return if the function existed.
-    */
-  def dropFunction(name: String): Boolean =
-    functionBuilders.remove(name.toLowerCase).isDefined
-
-  /**
-    * Drop all registered functions.
-    */
-  def clear(): Unit = functionBuilders.clear()
-}
-
-object FunctionCatalog {
-
-  val builtInFunctions: Map[String, Class[_]] = Map(
-    // logic
-    "isNull" -> classOf[IsNull],
-    "isNotNull" -> classOf[IsNotNull],
-    "isTrue" -> classOf[IsTrue],
-    "isFalse" -> classOf[IsFalse],
-    "isNotTrue" -> classOf[IsNotTrue],
-    "isNotFalse" -> classOf[IsNotFalse],
-
-    // aggregate functions
-    "avg" -> classOf[Avg],
-    "count" -> classOf[Count],
-    "max" -> classOf[Max],
-    "min" -> classOf[Min],
-    "sum" -> classOf[Sum],
-
-    // string functions
-    "charLength" -> classOf[CharLength],
-    "initCap" -> classOf[InitCap],
-    "like" -> classOf[Like],
-    "lowerCase" -> classOf[Lower],
-    "similar" -> classOf[Similar],
-    "substring" -> classOf[Substring],
-    "trim" -> classOf[Trim],
-    "upperCase" -> classOf[Upper],
-    "position" -> classOf[Position],
-    "overlay" -> classOf[Overlay],
-
-    // math functions
-    "abs" -> classOf[Abs],
-    "ceil" -> classOf[Ceil],
-    "exp" -> classOf[Exp],
-    "floor" -> classOf[Floor],
-    "log10" -> classOf[Log10],
-    "ln" -> classOf[Ln],
-    "power" -> classOf[Power],
-    "mod" -> classOf[Mod],
-    "sqrt" -> classOf[Sqrt],
-
-    // temporal functions
-    "extract" -> classOf[Extract],
-    "currentDate" -> classOf[CurrentDate],
-    "currentTime" -> classOf[CurrentTime],
-    "currentTimestamp" -> classOf[CurrentTimestamp],
-    "localTime" -> classOf[LocalTime],
-    "localTimestamp" -> classOf[LocalTimestamp],
-    "quarter" -> classOf[Quarter],
-    "temporalOverlaps" -> classOf[TemporalOverlaps],
-
-    // array
-    "cardinality" -> classOf[ArrayCardinality],
-    "at" -> classOf[ArrayElementAt],
-    "element" -> classOf[ArrayElement]
-
-    // TODO implement function overloading here
-    // "floor" -> classOf[TemporalFloor]
-    // "ceil" -> classOf[TemporalCeil]
-  )
-
-  /**
-    * Create a new function catalog with built-in functions.
-    */
-  def withBuiltIns: FunctionCatalog = {
-    val catalog = new FunctionCatalog()
-    builtInFunctions.foreach { case (n, c) => catalog.registerFunction(n, c) }
-    catalog
-  }
-}
-
-class BasicOperatorTable extends ReflectiveSqlOperatorTable {
-
-  /**
-    * List of supported SQL operators / functions.
-    *
-    * This list should be kept in sync with [[SqlStdOperatorTable]].
-    */
-  private val builtInSqlOperators: Seq[SqlOperator] = Seq(
-    // SET OPERATORS
-    SqlStdOperatorTable.UNION,
-    SqlStdOperatorTable.UNION_ALL,
-    SqlStdOperatorTable.EXCEPT,
-    SqlStdOperatorTable.EXCEPT_ALL,
-    SqlStdOperatorTable.INTERSECT,
-    SqlStdOperatorTable.INTERSECT_ALL,
-    // BINARY OPERATORS
-    SqlStdOperatorTable.AND,
-    SqlStdOperatorTable.AS,
-    SqlStdOperatorTable.CONCAT,
-    SqlStdOperatorTable.DIVIDE,
-    SqlStdOperatorTable.DIVIDE_INTEGER,
-    SqlStdOperatorTable.DOT,
-    SqlStdOperatorTable.EQUALS,
-    SqlStdOperatorTable.GREATER_THAN,
-    SqlStdOperatorTable.IS_DISTINCT_FROM,
-    SqlStdOperatorTable.IS_NOT_DISTINCT_FROM,
-    SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
-    SqlStdOperatorTable.LESS_THAN,
-    SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
-    SqlStdOperatorTable.MINUS,
-    SqlStdOperatorTable.MULTIPLY,
-    SqlStdOperatorTable.NOT_EQUALS,
-    SqlStdOperatorTable.OR,
-    SqlStdOperatorTable.PLUS,
-    SqlStdOperatorTable.DATETIME_PLUS,
-    // POSTFIX OPERATORS
-    SqlStdOperatorTable.DESC,
-    SqlStdOperatorTable.NULLS_FIRST,
-    SqlStdOperatorTable.IS_NOT_NULL,
-    SqlStdOperatorTable.IS_NULL,
-    SqlStdOperatorTable.IS_NOT_TRUE,
-    SqlStdOperatorTable.IS_TRUE,
-    SqlStdOperatorTable.IS_NOT_FALSE,
-    SqlStdOperatorTable.IS_FALSE,
-    SqlStdOperatorTable.IS_NOT_UNKNOWN,
-    SqlStdOperatorTable.IS_UNKNOWN,
-    // PREFIX OPERATORS
-    SqlStdOperatorTable.NOT,
-    SqlStdOperatorTable.UNARY_MINUS,
-    SqlStdOperatorTable.UNARY_PLUS,
-    // AGGREGATE OPERATORS
-    SqlStdOperatorTable.SUM,
-    SqlStdOperatorTable.COUNT,
-    SqlStdOperatorTable.MIN,
-    SqlStdOperatorTable.MAX,
-    SqlStdOperatorTable.AVG,
-    // ARRAY OPERATORS
-    SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR,
-    SqlStdOperatorTable.ITEM,
-    SqlStdOperatorTable.CARDINALITY,
-    SqlStdOperatorTable.ELEMENT,
-    // SPECIAL OPERATORS
-    SqlStdOperatorTable.ROW,
-    SqlStdOperatorTable.OVERLAPS,
-    SqlStdOperatorTable.LITERAL_CHAIN,
-    SqlStdOperatorTable.BETWEEN,
-    SqlStdOperatorTable.SYMMETRIC_BETWEEN,
-    SqlStdOperatorTable.NOT_BETWEEN,
-    SqlStdOperatorTable.SYMMETRIC_NOT_BETWEEN,
-    SqlStdOperatorTable.NOT_LIKE,
-    SqlStdOperatorTable.LIKE,
-    SqlStdOperatorTable.NOT_SIMILAR_TO,
-    SqlStdOperatorTable.SIMILAR_TO,
-    SqlStdOperatorTable.CASE,
-    SqlStdOperatorTable.REINTERPRET,
-    SqlStdOperatorTable.EXTRACT_DATE,
-    // FUNCTIONS
-    SqlStdOperatorTable.SUBSTRING,
-    SqlStdOperatorTable.OVERLAY,
-    SqlStdOperatorTable.TRIM,
-    SqlStdOperatorTable.POSITION,
-    SqlStdOperatorTable.CHAR_LENGTH,
-    SqlStdOperatorTable.CHARACTER_LENGTH,
-    SqlStdOperatorTable.UPPER,
-    SqlStdOperatorTable.LOWER,
-    SqlStdOperatorTable.INITCAP,
-    SqlStdOperatorTable.POWER,
-    SqlStdOperatorTable.SQRT,
-    SqlStdOperatorTable.MOD,
-    SqlStdOperatorTable.LN,
-    SqlStdOperatorTable.LOG10,
-    SqlStdOperatorTable.ABS,
-    SqlStdOperatorTable.EXP,
-    SqlStdOperatorTable.NULLIF,
-    SqlStdOperatorTable.COALESCE,
-    SqlStdOperatorTable.FLOOR,
-    SqlStdOperatorTable.CEIL,
-    SqlStdOperatorTable.LOCALTIME,
-    SqlStdOperatorTable.LOCALTIMESTAMP,
-    SqlStdOperatorTable.CURRENT_TIME,
-    SqlStdOperatorTable.CURRENT_TIMESTAMP,
-    SqlStdOperatorTable.CURRENT_DATE,
-    SqlStdOperatorTable.CAST,
-    SqlStdOperatorTable.EXTRACT,
-    SqlStdOperatorTable.QUARTER,
-    SqlStdOperatorTable.SCALAR_QUERY,
-    SqlStdOperatorTable.EXISTS
-  )
-
-  builtInSqlOperators.foreach(register)
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/ValidationResult.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/ValidationResult.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/ValidationResult.scala
deleted file mode 100644
index 5cc7d03..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/validate/ValidationResult.scala
+++ /dev/null
@@ -1,53 +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.api.table.validate
-
-/**
-  * Represents the result of a validation.
-  */
-sealed trait ValidationResult {
-  def isFailure: Boolean = !isSuccess
-  def isSuccess: Boolean
-
-  /**
-    * Allows constructing a cascade of validation results.
-    * The first failure result will be returned.
-    */
-  def orElse(other: ValidationResult): ValidationResult = {
-    if (isSuccess) {
-      other
-    } else {
-      this
-    }
-  }
-}
-
-/**
-  * Represents the successful result of a validation.
-  */
-object ValidationSuccess extends ValidationResult {
-  val isSuccess: Boolean = true
-}
-
-/**
-  * Represents the failing result of a validation,
-  * with a error message to show the reason of failure.
-  */
-case class ValidationFailure(message: String) extends ValidationResult {
-  val isSuccess: Boolean = false
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/windows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/windows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/windows.scala
deleted file mode 100644
index 5637d7a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/windows.scala
+++ /dev/null
@@ -1,390 +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.api.table
-
-import org.apache.flink.api.table.expressions.{Expression, ExpressionParser}
-import org.apache.flink.api.table.plan.logical._
-
-/**
-  * A group-window specification.
-  *
-  * Group-windows group rows based on time or row-count intervals and is therefore essentially a
-  * special type of groupBy. Just like groupBy, group-windows allow to compute aggregates
-  * on groups of elements.
-  *
-  * Infinite streaming tables can only be grouped into time or row intervals. Hence window grouping
-  * is required to apply aggregations on streaming tables.
-  *
-  * For finite batch tables, group-windows provide shortcuts for time-based groupBy.
-  *
-  */
-trait GroupWindow {
-
-  /**
-    * Converts an API class to a logical window for planning.
-    */
-  private[flink] def toLogicalWindow: LogicalWindow
-}
-
-/**
-  * A group-window operating on event-time.
-  *
-  * @param timeField defines the time mode for streaming tables. For batch table it defines the
-  *                  time attribute on which is grouped.
-  */
-abstract class EventTimeWindow(val timeField: Expression) extends GroupWindow {
-
-  protected var name: Option[Expression] = None
-
-  /**
-    * Assigns an alias for this window that the following `select()` clause can refer to in order
-    * to access window properties such as window start or end time.
-    *
-    * @param alias alias for this window
-    * @return this window
-    */
-  def as(alias: Expression): EventTimeWindow = {
-    this.name = Some(alias)
-    this
-  }
-
-  /**
-    * Assigns an alias for this window that the following `select()` clause can refer to in order
-    * to access window properties such as window start or end time.
-    *
-    * @param alias alias for this window
-    * @return this window
-    */
-  def as(alias: String): EventTimeWindow = as(ExpressionParser.parseExpression(alias))
-}
-
-// ------------------------------------------------------------------------------------------------
-// Tumbling group-windows
-// ------------------------------------------------------------------------------------------------
-
-/**
-  * Tumbling group-window.
-  *
-  * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows are
-  * grouped by processing-time.
-  *
-  * @param size the size of the window either as time or row-count interval.
-  */
-class TumblingWindow(size: Expression) extends GroupWindow {
-
-  /**
-    * Tumbling group-window.
-    *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
-    *
-    * @param size the size of the window either as time or row-count interval.
-    */
-  def this(size: String) = this(ExpressionParser.parseExpression(size))
-
-  private var alias: Option[Expression] = None
-
-  /**
-    * Specifies the time attribute on which rows are grouped.
-    *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
-    *
-    * For batch tables, refer to a timestamp or long attribute.
-    *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return a tumbling group-window on event-time
-    */
-  def on(timeField: Expression): TumblingEventTimeWindow =
-    new TumblingEventTimeWindow(alias, timeField, size)
-
-  /**
-    * Specifies the time attribute on which rows are grouped.
-    *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
-    *
-    * For batch tables, refer to a timestamp or long attribute.
-    *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return a tumbling group-window on event-time
-    */
-  def on(timeField: String): TumblingEventTimeWindow =
-    on(ExpressionParser.parseExpression(timeField))
-
-  /**
-    * Assigns an alias for this window that the following `select()` clause can refer to in order
-    * to access window properties such as window start or end time.
-    *
-    * @param alias alias for this window
-    * @return this window
-    */
-  def as(alias: Expression): TumblingWindow = {
-    this.alias = Some(alias)
-    this
-  }
-
-  /**
-    * Assigns an alias for this window that the following `select()` clause can refer to in order
-    * to access window properties such as window start or end time.
-    *
-    * @param alias alias for this window
-    * @return this window
-    */
-  def as(alias: String): TumblingWindow = as(ExpressionParser.parseExpression(alias))
-
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    ProcessingTimeTumblingGroupWindow(alias, size)
-}
-
-/**
-  * Tumbling group-window on event-time.
-  */
-class TumblingEventTimeWindow(
-    alias: Option[Expression],
-    time: Expression,
-    size: Expression)
-  extends EventTimeWindow(time) {
-
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    EventTimeTumblingGroupWindow(name.orElse(alias), time, size)
-}
-
-// ------------------------------------------------------------------------------------------------
-// Sliding group windows
-// ------------------------------------------------------------------------------------------------
-
-/**
-  * Partially specified sliding window.
-  *
-  * @param size the size of the window either as time or row-count interval.
-  */
-class SlideWithSize(size: Expression) {
-
-  /**
-    * Partially specified sliding window.
-    *
-    * @param size the size of the window either as time or row-count interval.
-    */
-  def this(size: String) = this(ExpressionParser.parseExpression(size))
-
-  /**
-    * Specifies the window's slide as time or row-count interval.
-    *
-    * The slide determines the interval in which windows are started. Hence, sliding windows can
-    * overlap if the slide is smaller than the size of the window.
-    *
-    * For example, you could have windows of size 15 minutes that slide by 3 minutes. With this
-    * 15 minutes worth of elements are grouped every 3 minutes and each row contributes to 5
-    * windows.
-    *
-    * @param slide the slide of the window either as time or row-count interval.
-    * @return a sliding group-window
-    */
-  def every(slide: Expression): SlidingWindow = new SlidingWindow(size, slide)
-
-  /**
-    * Specifies the window's slide as time or row-count interval.
-    *
-    * The slide determines the interval in which windows are started. Hence, sliding windows can
-    * overlap if the slide is smaller than the size of the window.
-    *
-    * For example, you could have windows of size 15 minutes that slide by 3 minutes. With this
-    * 15 minutes worth of elements are grouped every 3 minutes and each row contributes to 5
-    * windows.
-    *
-    * @param slide the slide of the window either as time or row-count interval.
-    * @return a sliding group-window
-    */
-  def every(slide: String): SlidingWindow = every(ExpressionParser.parseExpression(slide))
-}
-
-/**
-  * Sliding group-window.
-  *
-  * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows are
-  * grouped by processing-time.
-  *
-  * @param size the size of the window either as time or row-count interval.
-  */
-class SlidingWindow(
-    size: Expression,
-    slide: Expression)
-  extends GroupWindow {
-
-  private var alias: Option[Expression] = None
-
-  /**
-    * Specifies the time attribute on which rows are grouped.
-    *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
-    *
-    * For batch tables, refer to a timestamp or long attribute.
-    *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return a sliding group-window on event-time
-    */
-  def on(timeField: Expression): SlidingEventTimeWindow =
-    new SlidingEventTimeWindow(alias, timeField, size, slide)
-
-  /**
-    * Specifies the time attribute on which rows are grouped.
-    *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
-    *
-    * For batch tables, refer to a timestamp or long attribute.
-    *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return a sliding group-window on event-time
-    */
-  def on(timeField: String): SlidingEventTimeWindow =
-    on(ExpressionParser.parseExpression(timeField))
-
-  /**
-    * Assigns an alias for this window that the following `select()` clause can refer to in order
-    * to access window properties such as window start or end time.
-    *
-    * @param alias alias for this window
-    * @return this window
-    */
-  def as(alias: Expression): SlidingWindow = {
-    this.alias = Some(alias)
-    this
-  }
-
-  /**
-    * Assigns an alias for this window that the following `select()` clause can refer to in order
-    * to access window properties such as window start or end time.
-    *
-    * @param alias alias for this window
-    * @return this window
-    */
-  def as(alias: String): SlidingWindow = as(ExpressionParser.parseExpression(alias))
-
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    ProcessingTimeSlidingGroupWindow(alias, size, slide)
-}
-
-/**
-  * Sliding group-window on event-time.
-  */
-class SlidingEventTimeWindow(
-    alias: Option[Expression],
-    timeField: Expression,
-    size: Expression,
-    slide: Expression)
-  extends EventTimeWindow(timeField) {
-
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    EventTimeSlidingGroupWindow(name.orElse(alias), timeField, size, slide)
-}
-
-// ------------------------------------------------------------------------------------------------
-// Session group windows
-// ------------------------------------------------------------------------------------------------
-
-/**
-  * Session group-window.
-  *
-  * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows are
-  * grouped by processing-time.
-  *
-  * @param gap the time interval of inactivity before a window is closed.
-  */
-class SessionWindow(gap: Expression) extends GroupWindow {
-
-  /**
-    * Session group-window.
-    *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
-    *
-    * @param gap the time interval of inactivity before a window is closed.
-    */
-  def this(gap: String) = this(ExpressionParser.parseExpression(gap))
-
-  private var alias: Option[Expression] = None
-
-  /**
-    * Specifies the time attribute on which rows are grouped.
-    *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
-    *
-    * For batch tables, refer to a timestamp or long attribute.
-    *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return a session group-window on event-time
-    */
-  def on(timeField: Expression): SessionEventTimeWindow =
-    new SessionEventTimeWindow(alias, timeField, gap)
-
-  /**
-    * Specifies the time attribute on which rows are grouped.
-    *
-    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
-    * are grouped by processing-time.
-    *
-    * For batch tables, refer to a timestamp or long attribute.
-    *
-    * @param timeField time mode for streaming tables and time attribute for batch tables
-    * @return a session group-window on event-time
-    */
-  def on(timeField: String): SessionEventTimeWindow =
-    on(ExpressionParser.parseExpression(timeField))
-
-  /**
-    * Assigns an alias for this window that the following `select()` clause can refer to in order
-    * to access window properties such as window start or end time.
-    *
-    * @param alias alias for this window
-    * @return this window
-    */
-  def as(alias: Expression): SessionWindow = {
-    this.alias = Some(alias)
-    this
-  }
-
-  /**
-    * Assigns an alias for this window that the following `select()` clause can refer to in order
-    * to access window properties such as window start or end time.
-    *
-    * @param alias alias for this window
-    * @return this window
-    */
-  def as(alias: String): SessionWindow = as(ExpressionParser.parseExpression(alias))
-
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    ProcessingTimeSessionGroupWindow(alias, gap)
-}
-
-/**
-  * Session group-window on event-time.
-  */
-class SessionEventTimeWindow(
-    alias: Option[Expression],
-    timeField: Expression,
-    gap: Expression)
-  extends EventTimeWindow(timeField) {
-
-  override private[flink] def toLogicalWindow: LogicalWindow =
-    EventTimeSessionGroupWindow(name.orElse(alias), timeField, gap)
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/StreamSQLExample.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/StreamSQLExample.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/StreamSQLExample.scala
deleted file mode 100644
index 63a5413..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/StreamSQLExample.scala
+++ /dev/null
@@ -1,76 +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.examples.scala
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
-
-/**
-  * Simple example for demonstrating the use of SQL on a Stream Table.
-  *
-  * This example shows how to:
-  *  - Convert DataStreams to Tables
-  *  - Register a Table under a name
-  *  - Run a StreamSQL query on the registered Table
-  *
-  */
-object StreamSQLExample {
-
-  // *************************************************************************
-  //     PROGRAM
-  // *************************************************************************
-
-  def main(args: Array[String]): Unit = {
-
-    // set up execution environment
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val orderA: DataStream[Order] = env.fromCollection(Seq(
-      Order(1L, "beer", 3),
-      Order(1L, "diaper", 4),
-      Order(3L, "rubber", 2)))
-
-    val orderB: DataStream[Order] = env.fromCollection(Seq(
-      Order(2L, "pen", 3),
-      Order(2L, "rubber", 3),
-      Order(4L, "beer", 1)))
-
-    // register the DataStreams under the name "OrderA" and "OrderB"
-    tEnv.registerDataStream("OrderA", orderA, 'user, 'product, 'amount)
-    tEnv.registerDataStream("OrderB", orderB, 'user, 'product, 'amount)
-
-    // union the two tables
-    val result = tEnv.sql(
-      "SELECT * FROM OrderA WHERE amount > 2 UNION ALL " +
-        "SELECT * FROM OrderB WHERE amount < 2")
-
-    result.toDataStream[Order].print()
-
-    env.execute()
-  }
-
-  // *************************************************************************
-  //     USER DATA TYPES
-  // *************************************************************************
-
-  case class Order(user: Long, product: String, amount: Int)
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/StreamTableExample.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/StreamTableExample.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/StreamTableExample.scala
deleted file mode 100644
index 2ce2684..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/StreamTableExample.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.examples.scala
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
-
-/**
-  * Simple example for demonstrating the use of Table API on a Stream Table.
-  *
-  * This example shows how to:
-  *  - Convert DataStreams to Tables
-  *  - Apply union, select, and filter operations
-  *
-  */
-object StreamTableExample {
-
-  // *************************************************************************
-  //     PROGRAM
-  // *************************************************************************
-
-  def main(args: Array[String]): Unit = {
-
-    // set up execution environment
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val orderA = env.fromCollection(Seq(
-      Order(1L, "beer", 3),
-      Order(1L, "diaper", 4),
-      Order(3L, "rubber", 2))).toTable(tEnv)
-
-    val orderB = env.fromCollection(Seq(
-      Order(2L, "pen", 3),
-      Order(2L, "rubber", 3),
-      Order(4L, "beer", 1))).toTable(tEnv)
-
-    // union the two tables
-    val result: DataStream[Order] = orderA.unionAll(orderB)
-      .select('user, 'product, 'amount)
-      .where('amount > 2)
-      .toDataStream[Order]
-
-    result.print()
-
-    env.execute()
-  }
-
-  // *************************************************************************
-  //     USER DATA TYPES
-  // *************************************************************************
-
-  case class Order(user: Long, product: String, amount: Int)
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/TPCHQuery3Table.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/TPCHQuery3Table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/TPCHQuery3Table.scala
deleted file mode 100644
index a950988..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/TPCHQuery3Table.scala
+++ /dev/null
@@ -1,180 +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.examples.scala
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.TableEnvironment
-
-/**
-  * This program implements a modified version of the TPC-H query 3. The
-  * example demonstrates how to assign names to fields by extending the Tuple class.
-  * The original query can be found at
-  * [http://www.tpc.org/tpch/spec/tpch2.16.0.pdf](http://www.tpc.org/tpch/spec/tpch2.16.0.pdf)
-  * (page 29).
-  *
-  * This program implements the following SQL equivalent:
-  *
-  * {{{
-  * SELECT
-  *      l_orderkey,
-  *      SUM(l_extendedprice*(1-l_discount)) AS revenue,
-  *      o_orderdate,
-  *      o_shippriority
-  * FROM customer,
-  *      orders,
-  *      lineitem
-  * WHERE
-  *      c_mktsegment = '[SEGMENT]'
-  *      AND c_custkey = o_custkey
-  *      AND l_orderkey = o_orderkey
-  *      AND o_orderdate < date '[DATE]'
-  *      AND l_shipdate > date '[DATE]'
-  * GROUP BY
-  *      l_orderkey,
-  *      o_orderdate,
-  *      o_shippriority
-  * ORDER BY
-  *      revenue desc,
-  *      o_orderdate;
-  * }}}
-  *
-  * Input files are plain text CSV files using the pipe character ('|') as field separator
-  * as generated by the TPC-H data generator which is available at
-  * [http://www.tpc.org/tpch/](a href="http://www.tpc.org/tpch/).
-  *
-  * Usage:
-  * {{{
-  * TPCHQuery3Expression <lineitem-csv path> <customer-csv path> <orders-csv path> <result path>
-  * }}}
-  *
-  * This example shows how to:
-  *  - Convert DataSets to Tables
-  *  - Use Table API expressions
-  *
-  */
-object TPCHQuery3Table {
-
-  // *************************************************************************
-  //     PROGRAM
-  // *************************************************************************
-
-  def main(args: Array[String]) {
-    if (!parseParameters(args)) {
-      return
-    }
-
-    // set filter date
-    val date = "1995-03-12".toDate
-
-    // get execution environment
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val lineitems = getLineitemDataSet(env)
-      .toTable(tEnv, 'id, 'extdPrice, 'discount, 'shipDate)
-      .filter('shipDate.toDate > date)
-
-    val customers = getCustomerDataSet(env)
-      .toTable(tEnv, 'id, 'mktSegment)
-      .filter('mktSegment === "AUTOMOBILE")
-
-    val orders = getOrdersDataSet(env)
-      .toTable(tEnv, 'orderId, 'custId, 'orderDate, 'shipPrio)
-      .filter('orderDate.toDate < date)
-
-    val items =
-      orders.join(customers)
-        .where('custId === 'id)
-        .select('orderId, 'orderDate, 'shipPrio)
-      .join(lineitems)
-        .where('orderId === 'id)
-        .select(
-          'orderId,
-          'extdPrice * (1.0f.toExpr - 'discount) as 'revenue,
-          'orderDate,
-          'shipPrio)
-
-    val result = items
-      .groupBy('orderId, 'orderDate, 'shipPrio)
-      .select('orderId, 'revenue.sum as 'revenue, 'orderDate, 'shipPrio)
-      .orderBy('revenue.desc, 'orderDate.asc)
-
-    // emit result
-    result.writeAsCsv(outputPath, "\n", "|")
-
-    // execute program
-    env.execute("Scala TPCH Query 3 (Table API Expression) Example")
-  }
-  
-  // *************************************************************************
-  //     USER DATA TYPES
-  // *************************************************************************
-  
-  case class Lineitem(id: Long, extdPrice: Double, discount: Double, shipDate: String)
-  case class Customer(id: Long, mktSegment: String)
-  case class Order(orderId: Long, custId: Long, orderDate: String, shipPrio: Long)
-
-  // *************************************************************************
-  //     UTIL METHODS
-  // *************************************************************************
-  
-  private var lineitemPath: String = _
-  private var customerPath: String = _
-  private var ordersPath: String = _
-  private var outputPath: String = _
-
-  private def parseParameters(args: Array[String]): Boolean = {
-    if (args.length == 4) {
-      lineitemPath = args(0)
-      customerPath = args(1)
-      ordersPath = args(2)
-      outputPath = args(3)
-      true
-    } else {
-      System.err.println("This program expects data from the TPC-H benchmark as input data.\n" +
-          " Due to legal restrictions, we can not ship generated data.\n" +
-          " You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" +
-          " Usage: TPCHQuery3 <lineitem-csv path> <customer-csv path> " +
-                             "<orders-csv path> <result path>")
-      false
-    }
-  }
-
-  private def getLineitemDataSet(env: ExecutionEnvironment): DataSet[Lineitem] = {
-    env.readCsvFile[Lineitem](
-        lineitemPath,
-        fieldDelimiter = "|",
-        includedFields = Array(0, 5, 6, 10) )
-  }
-
-  private def getCustomerDataSet(env: ExecutionEnvironment): DataSet[Customer] = {
-    env.readCsvFile[Customer](
-        customerPath,
-        fieldDelimiter = "|",
-        includedFields = Array(0, 6) )
-  }
-
-  private def getOrdersDataSet(env: ExecutionEnvironment): DataSet[Order] = {
-    env.readCsvFile[Order](
-        ordersPath,
-        fieldDelimiter = "|",
-        includedFields = Array(0, 1, 4, 7) )
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/WordCountSQL.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/WordCountSQL.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/WordCountSQL.scala
deleted file mode 100644
index 96a603e..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/WordCountSQL.scala
+++ /dev/null
@@ -1,62 +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.examples.scala
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.TableEnvironment
-
-/**
-  * Simple example that shows how the Batch SQL API is used in Scala.
-  *
-  * This example shows how to:
-  *  - Convert DataSets to Tables
-  *  - Register a Table under a name
-  *  - Run a SQL query on the registered Table
-  *
-  */
-object WordCountSQL {
-
-  // *************************************************************************
-  //     PROGRAM
-  // *************************************************************************
-
-  def main(args: Array[String]): Unit = {
-
-    // set up execution environment
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val input = env.fromElements(WC("hello", 1), WC("hello", 1), WC("ciao", 1))
-
-    // register the DataSet as table "WordCount"
-    tEnv.registerDataSet("WordCount", input, 'word, 'frequency)
-
-    // run a SQL query on the Table and retrieve the result as a new Table
-    val table = tEnv.sql("SELECT word, SUM(frequency) FROM WordCount GROUP BY word")
-
-    table.toDataSet[WC].print()
-  }
-
-  // *************************************************************************
-  //     USER DATA TYPES
-  // *************************************************************************
-
-  case class WC(word: String, frequency: Long)
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/WordCountTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/WordCountTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/WordCountTable.scala
deleted file mode 100644
index 587a716..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/examples/scala/WordCountTable.scala
+++ /dev/null
@@ -1,62 +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.examples.scala
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.TableEnvironment
-
-/**
-  * Simple example for demonstrating the use of the Table API for a Word Count in Scala.
-  *
-  * This example shows how to:
-  *  - Convert DataSets to Tables
-  *  - Apply group, aggregate, select, and filter operations
-  *
-  */
-object WordCountTable {
-
-  // *************************************************************************
-  //     PROGRAM
-  // *************************************************************************
-
-  def main(args: Array[String]): Unit = {
-
-    // set up execution environment
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val input = env.fromElements(WC("hello", 1), WC("hello", 1), WC("ciao", 1))
-    val expr = input.toTable(tEnv)
-    val result = expr
-      .groupBy('word)
-      .select('word, 'frequency.sum as 'frequency)
-      .filter('frequency === 2)
-      .toDataSet[WC]
-
-    result.print()
-  }
-
-  // *************************************************************************
-  //     USER DATA TYPES
-  // *************************************************************************
-
-  case class WC(word: String, frequency: Long)
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..59cad80
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
@@ -0,0 +1,312 @@
+/*
+ * 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.util.concurrent.atomic.AtomicInteger
+
+import org.apache.calcite.plan.RelOptPlanner.CannotPlanException
+import org.apache.calcite.plan.RelOptUtil
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.sql2rel.RelDecorrelator
+import org.apache.calcite.tools.{Programs, RuleSet}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.io.DiscardingOutputFormat
+import org.apache.flink.api.java.typeutils.TypeExtractor
+import org.apache.flink.api.java.{DataSet, ExecutionEnvironment}
+import org.apache.flink.table.calcite.FlinkPlannerImpl
+import org.apache.flink.table.explain.PlanJsonParser
+import org.apache.flink.table.expressions.Expression
+import org.apache.flink.table.plan.logical.{CatalogNode, LogicalRelNode}
+import org.apache.flink.table.plan.nodes.dataset.{DataSetConvention, DataSetRel}
+import org.apache.flink.table.plan.rules.FlinkRuleSets
+import org.apache.flink.table.plan.schema.{DataSetTable, TableSourceTable}
+import org.apache.flink.table.sinks.{BatchTableSink, TableSink}
+import org.apache.flink.table.sources.BatchTableSource
+import org.apache.flink.types.Row
+
+/**
+  * The abstract base class for batch TableEnvironments.
+  *
+  * A TableEnvironment can be used to:
+  * - convert a [[DataSet]] to a [[Table]]
+  * - register a [[DataSet]] in the [[TableEnvironment]]'s catalog
+  * - register a [[Table]] in the [[TableEnvironment]]'s catalog
+  * - scan a registered table to obtain a [[Table]]
+  * - specify a SQL query on registered tables to obtain a [[Table]]
+  * - convert a [[Table]] into a [[DataSet]]
+  * - explain the AST and execution plan of a [[Table]]
+  *
+  * @param execEnv The [[ExecutionEnvironment]] which is wrapped in this [[BatchTableEnvironment]].
+  * @param config The [[TableConfig]] of this [[BatchTableEnvironment]].
+  */
+abstract class BatchTableEnvironment(
+    private[flink] val execEnv: ExecutionEnvironment,
+    config: TableConfig)
+  extends TableEnvironment(config) {
+
+  // a counter for unique table names.
+  private val nameCntr: AtomicInteger = new AtomicInteger(0)
+
+  // the naming pattern for internally registered tables.
+  private val internalNamePattern = "^_DataSetTable_[0-9]+$".r
+
+  /**
+    * Checks if the chosen table name is valid.
+    *
+    * @param name The table name to check.
+    */
+  override protected def checkValidTableName(name: String): Unit = {
+    val m = internalNamePattern.findFirstIn(name)
+    m match {
+      case Some(_) =>
+        throw new TableException(s"Illegal Table name. " +
+          s"Please choose a name that does not contain the pattern $internalNamePattern")
+      case None =>
+    }
+  }
+
+  /** Returns a unique table name according to the internal naming pattern. */
+  protected def createUniqueTableName(): String = "_DataSetTable_" + nameCntr.getAndIncrement()
+
+  /**
+    * Scans a registered table and returns the resulting [[Table]].
+    *
+    * The table to scan must be registered in the [[TableEnvironment]]'s catalog.
+    *
+    * @param tableName The name of the table to scan.
+    * @throws ValidationException if no table is registered under the given name.
+    * @return The scanned table.
+    */
+  @throws[ValidationException]
+  def scan(tableName: String): Table = {
+    if (isRegistered(tableName)) {
+      new Table(this, CatalogNode(tableName, getRowType(tableName)))
+    } else {
+      throw new TableException(s"Table \'$tableName\' was not found in the registry.")
+    }
+  }
+
+  /**
+    * Registers an external [[BatchTableSource]] in this [[TableEnvironment]]'s catalog.
+    * Registered tables can be referenced in SQL queries.
+    *
+    * @param name The name under which the [[BatchTableSource]] is registered.
+    * @param tableSource The [[BatchTableSource]] to register.
+    */
+  def registerTableSource(name: String, tableSource: BatchTableSource[_]): Unit = {
+
+    checkValidTableName(name)
+    registerTableInternal(name, new TableSourceTable(tableSource))
+  }
+
+  /**
+    * Evaluates a SQL query on registered tables and retrieves the result as a [[Table]].
+    *
+    * All tables referenced by the query must be registered in the TableEnvironment.
+    *
+    * @param query The SQL query to evaluate.
+    * @return The result of the query as Table.
+    */
+  override def sql(query: String): Table = {
+
+    val planner = new FlinkPlannerImpl(getFrameworkConfig, getPlanner, getTypeFactory)
+    // parse the sql query
+    val parsed = planner.parse(query)
+    // validate the sql query
+    val validated = planner.validate(parsed)
+    // transform to a relational tree
+    val relational = planner.rel(validated)
+
+    new Table(this, LogicalRelNode(relational.rel))
+  }
+
+  /**
+    * Writes a [[Table]] to a [[TableSink]].
+    *
+    * Internally, the [[Table]] is translated into a [[DataSet]] and handed over to the
+    * [[TableSink]] to write it.
+    *
+    * @param table The [[Table]] to write.
+    * @param sink The [[TableSink]] to write the [[Table]] to.
+    * @tparam T The expected type of the [[DataSet]] which represents the [[Table]].
+    */
+  override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = {
+
+    sink match {
+      case batchSink: BatchTableSink[T] =>
+        val outputType = sink.getOutputType
+        // translate the Table into a DataSet and provide the type that the TableSink expects.
+        val result: DataSet[T] = translate(table)(outputType)
+        // Give the DataSet to the TableSink to emit it.
+        batchSink.emitDataSet(result)
+      case _ =>
+        throw new TableException("BatchTableSink required to emit batch Table")
+    }
+  }
+
+  /**
+    * Returns the AST of the specified Table API and SQL queries and the execution plan to compute
+    * the result of the given [[Table]].
+    *
+    * @param table The table for which the AST and execution plan will be returned.
+    * @param extended Flag to include detailed optimizer estimates.
+    */
+  private[flink] def explain(table: Table, extended: Boolean): String = {
+    val ast = table.getRelNode
+    val optimizedPlan = optimize(ast)
+    val dataSet = translate[Row](optimizedPlan) (TypeExtractor.createTypeInfo(classOf[Row]))
+    dataSet.output(new DiscardingOutputFormat[Row])
+    val env = dataSet.getExecutionEnvironment
+    val jasonSqlPlan = env.getExecutionPlan
+    val sqlPlan = PlanJsonParser.getSqlExecutionPlan(jasonSqlPlan, extended)
+
+    s"== Abstract Syntax Tree ==" +
+        System.lineSeparator +
+        s"${RelOptUtil.toString(ast)}" +
+        System.lineSeparator +
+        s"== Optimized Logical Plan ==" +
+        System.lineSeparator +
+        s"${RelOptUtil.toString(optimizedPlan)}" +
+        System.lineSeparator +
+        s"== Physical Execution Plan ==" +
+        System.lineSeparator +
+        s"$sqlPlan"
+  }
+
+  /**
+    * Returns the AST of the specified Table API and SQL queries and the execution plan to compute
+    * the result of the given [[Table]].
+    *
+    * @param table The table for which the AST and execution plan will be returned.
+    */
+  def explain(table: Table): String = explain(table: Table, extended = false)
+
+  /**
+    * Registers a [[DataSet]] as a table under a given name in the [[TableEnvironment]]'s catalog.
+    *
+    * @param name The name under which the table is registered in the catalog.
+    * @param dataSet The [[DataSet]] to register as table in the catalog.
+    * @tparam T the type of the [[DataSet]].
+    */
+  protected def registerDataSetInternal[T](name: String, dataSet: DataSet[T]): Unit = {
+
+    val (fieldNames, fieldIndexes) = getFieldInfo[T](dataSet.getType)
+    val dataSetTable = new DataSetTable[T](
+      dataSet,
+      fieldIndexes,
+      fieldNames
+    )
+    registerTableInternal(name, dataSetTable)
+  }
+
+  /**
+    * Registers a [[DataSet]] as a table under a given name with field names as specified by
+    * field expressions in the [[TableEnvironment]]'s catalog.
+    *
+    * @param name The name under which the table is registered in the catalog.
+    * @param dataSet The [[DataSet]] to register as table in the catalog.
+    * @param fields The field expressions to define the field names of the table.
+    * @tparam T The type of the [[DataSet]].
+    */
+  protected def registerDataSetInternal[T](
+      name: String, dataSet: DataSet[T], fields: Array[Expression]): Unit = {
+
+    val (fieldNames, fieldIndexes) = getFieldInfo[T](dataSet.getType, fields)
+    val dataSetTable = new DataSetTable[T](dataSet, fieldIndexes, fieldNames)
+    registerTableInternal(name, dataSetTable)
+  }
+
+  /**
+    * Returns the built-in rules that are defined by the environment.
+    */
+  protected def getBuiltInRuleSet: RuleSet = FlinkRuleSets.DATASET_OPT_RULES
+
+  /**
+    * Generates the optimized [[RelNode]] tree from the original relational node tree.
+    *
+    * @param relNode The original [[RelNode]] tree
+    * @return The optimized [[RelNode]] tree
+    */
+  private[flink] def optimize(relNode: RelNode): RelNode = {
+
+    // decorrelate
+    val decorPlan = RelDecorrelator.decorrelateQuery(relNode)
+
+    // optimize the logical Flink plan
+    val optProgram = Programs.ofRules(getRuleSet)
+    val flinkOutputProps = relNode.getTraitSet.replace(DataSetConvention.INSTANCE).simplify()
+
+    val dataSetPlan = try {
+      optProgram.run(getPlanner, decorPlan, flinkOutputProps)
+    } catch {
+      case e: CannotPlanException =>
+        throw new TableException(
+          s"Cannot generate a valid execution plan for the given query: \n\n" +
+            s"${RelOptUtil.toString(relNode)}\n" +
+            s"This exception indicates that the query uses an unsupported SQL feature.\n" +
+            s"Please check the documentation for the set of currently supported SQL features.")
+      case t: TableException =>
+        throw new TableException(
+          s"Cannot generate a valid execution plan for the given query: \n\n" +
+            s"${RelOptUtil.toString(relNode)}\n" +
+            s"${t.msg}\n" +
+            s"Please check the documentation for the set of currently supported SQL features.")
+      case a: AssertionError =>
+        throw a.getCause
+    }
+    dataSetPlan
+  }
+
+  /**
+    * Translates a [[Table]] into a [[DataSet]].
+    *
+    * The transformation involves optimizing the relational expression tree as defined by
+    * Table API calls and / or SQL queries and generating corresponding [[DataSet]] operators.
+    *
+    * @param table The root node of the relational expression tree.
+    * @param tpe   The [[TypeInformation]] of the resulting [[DataSet]].
+    * @tparam A The type of the resulting [[DataSet]].
+    * @return The [[DataSet]] that corresponds to the translated [[Table]].
+    */
+  protected def translate[A](table: Table)(implicit tpe: TypeInformation[A]): DataSet[A] = {
+    val dataSetPlan = optimize(table.getRelNode)
+    translate(dataSetPlan)
+  }
+
+  /**
+    * Translates a logical [[RelNode]] into a [[DataSet]].
+    *
+    * @param logicalPlan The root node of the relational expression tree.
+    * @param tpe         The [[TypeInformation]] of the resulting [[DataSet]].
+    * @tparam A The type of the resulting [[DataSet]].
+    * @return The [[DataSet]] that corresponds to the translated [[Table]].
+    */
+  protected def translate[A](logicalPlan: RelNode)(implicit tpe: TypeInformation[A]): DataSet[A] = {
+    validateType(tpe)
+
+    logicalPlan match {
+      case node: DataSetRel =>
+        node.translateToPlan(
+          this,
+          Some(tpe.asInstanceOf[TypeInformation[Any]])
+        ).asInstanceOf[DataSet[A]]
+      case _ => ???
+    }
+  }
+}


[34/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkRel.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkRel.scala
deleted file mode 100644
index 7932e11..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkRel.scala
+++ /dev/null
@@ -1,103 +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.api.table.plan.nodes
-
-import org.apache.calcite.rex._
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.TableConfig
-import org.apache.flink.api.table.codegen.CodeGenerator
-import org.apache.flink.api.table.runtime.MapRunner
-
-import scala.collection.JavaConversions._
-
-trait FlinkRel {
-
-  private[flink] def getExpressionString(
-    expr: RexNode,
-    inFields: List[String],
-    localExprsTable: Option[List[RexNode]]): String = {
-
-    expr match {
-      case i: RexInputRef =>
-        inFields.get(i.getIndex)
-
-      case l: RexLiteral =>
-        l.toString
-
-      case l: RexLocalRef if localExprsTable.isEmpty =>
-        throw new IllegalArgumentException("Encountered RexLocalRef without " +
-          "local expression table")
-
-      case l: RexLocalRef =>
-        val lExpr = localExprsTable.get(l.getIndex)
-        getExpressionString(lExpr, inFields, localExprsTable)
-
-      case c: RexCall =>
-        val op = c.getOperator.toString
-        val ops = c.getOperands.map(getExpressionString(_, inFields, localExprsTable))
-        s"$op(${ops.mkString(", ")})"
-
-      case fa: RexFieldAccess =>
-        val referenceExpr = getExpressionString(fa.getReferenceExpr, inFields, localExprsTable)
-        val field = fa.getField.getName
-        s"$referenceExpr.$field"
-
-      case _ =>
-        throw new IllegalArgumentException(s"Unknown expression type '${expr.getClass}': $expr")
-    }
-  }
-
-  private[flink] def getConversionMapper(
-      config: TableConfig,
-      nullableInput: Boolean,
-      inputType: TypeInformation[Any],
-      expectedType: TypeInformation[Any],
-      conversionOperatorName: String,
-      fieldNames: Seq[String],
-      inputPojoFieldMapping: Option[Array[Int]] = None)
-    : MapFunction[Any, Any] = {
-
-    val generator = new CodeGenerator(
-      config,
-      nullableInput,
-      inputType,
-      None,
-      inputPojoFieldMapping)
-    val conversion = generator.generateConverterResultExpression(expectedType, fieldNames)
-
-    val body =
-      s"""
-         |${conversion.code}
-         |return ${conversion.resultTerm};
-         |""".stripMargin
-
-    val genFunction = generator.generateFunction(
-      conversionOperatorName,
-      classOf[MapFunction[Any, Any]],
-      body,
-      expectedType)
-
-    new MapRunner[Any, Any](
-      genFunction.name,
-      genFunction.code,
-      genFunction.returnType)
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchScan.scala
deleted file mode 100644
index a6de237..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchScan.scala
+++ /dev/null
@@ -1,97 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan._
-import org.apache.calcite.rel.core.TableScan
-import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.java.typeutils.PojoTypeInfo
-import org.apache.flink.api.table.TableConfig
-import org.apache.flink.api.table.plan.schema.FlinkTable
-import org.apache.flink.api.table.typeutils.TypeConverter.determineReturnType
-
-import scala.collection.JavaConversions._
-import scala.collection.JavaConverters._
-
-abstract class BatchScan(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    table: RelOptTable)
-  extends TableScan(cluster, traitSet, table)
-  with DataSetRel {
-
-  override def toString: String = {
-    s"Source(from: (${getRowType.getFieldNames.asScala.toList.mkString(", ")}))"
-  }
-
-  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-
-    val rowCnt = metadata.getRowCount(this)
-    planner.getCostFactory.makeCost(rowCnt, rowCnt, 0)
-  }
-
-  protected def convertToExpectedType(
-      input: DataSet[Any],
-      flinkTable: FlinkTable[_],
-      expectedType: Option[TypeInformation[Any]],
-      config: TableConfig): DataSet[Any] = {
-
-    val inputType = input.getType
-
-    expectedType match {
-
-      // special case:
-      // if efficient type usage is enabled and no expected type is set
-      // we can simply forward the DataSet to the next operator.
-      // however, we cannot forward PojoTypes as their fields don't have an order
-      case None if config.getEfficientTypeUsage && !inputType.isInstanceOf[PojoTypeInfo[_]] =>
-        input
-
-      case _ =>
-        val determinedType = determineReturnType(
-          getRowType,
-          expectedType,
-          config.getNullCheck,
-          config.getEfficientTypeUsage)
-
-        // conversion
-        if (determinedType != inputType) {
-
-          val mapFunc = getConversionMapper(
-            config,
-            nullableInput = false,
-            inputType,
-            determinedType,
-            "DataSetSourceConversion",
-            getRowType.getFieldNames,
-            Some(flinkTable.fieldIndexes))
-
-          val opName = s"from: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
-
-          input.map(mapFunc).name(opName)
-        }
-        // no conversion necessary, forward
-        else {
-          input
-        }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchTableSourceScan.scala
deleted file mode 100644
index e368219..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/BatchTableSourceScan.scala
+++ /dev/null
@@ -1,71 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan._
-import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.calcite.rel.{RelNode, RelWriter}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.{BatchTableEnvironment, FlinkTypeFactory}
-import org.apache.flink.api.table.plan.schema.TableSourceTable
-import org.apache.flink.api.table.sources.BatchTableSource
-
-/** Flink RelNode to read data from an external source defined by a [[BatchTableSource]]. */
-class BatchTableSourceScan(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    table: RelOptTable,
-    val tableSource: BatchTableSource[_])
-  extends BatchScan(cluster, traitSet, table) {
-
-  override def deriveRowType() = {
-    val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-    flinkTypeFactory.buildRowDataType(tableSource.getFieldsNames, tableSource.getFieldTypes)
-  }
-
-  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-    val rowCnt = metadata.getRowCount(this)
-    planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * estimateRowSize(getRowType))
-  }
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new BatchTableSourceScan(
-      cluster,
-      traitSet,
-      getTable,
-      tableSource
-    )
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw)
-      .item("fields", tableSource.getFieldsNames.mkString(", "))
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
-
-    val config = tableEnv.getConfig
-    val inputDataSet = tableSource.getDataSet(tableEnv.execEnv).asInstanceOf[DataSet[Any]]
-
-    convertToExpectedType(inputDataSet, new TableSourceTable(tableSource), expectedType, config)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala
deleted file mode 100644
index 94513d9..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetAggregate.scala
+++ /dev/null
@@ -1,166 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.AggregateCall
-import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.plan.nodes.FlinkAggregate
-import org.apache.flink.api.table.runtime.aggregate.AggregateUtil
-import org.apache.flink.api.table.runtime.aggregate.AggregateUtil.CalcitePair
-import org.apache.flink.api.table.typeutils.TypeConverter
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.{BatchTableEnvironment, FlinkTypeFactory}
-import org.apache.flink.types.Row
-
-import scala.collection.JavaConverters._
-
-/**
-  * Flink RelNode which matches along with a LogicalAggregate.
-  */
-class DataSetAggregate(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    inputNode: RelNode,
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    rowRelDataType: RelDataType,
-    inputType: RelDataType,
-    grouping: Array[Int])
-  extends SingleRel(cluster, traitSet, inputNode)
-  with FlinkAggregate
-  with DataSetRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataSetAggregate(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      namedAggregates,
-      getRowType,
-      inputType,
-      grouping)
-  }
-
-  override def toString: String = {
-    s"Aggregate(${ if (!grouping.isEmpty) {
-      s"groupBy: (${groupingToString(inputType, grouping)}), "
-    } else {
-      ""
-    }}select: (${aggregationToString(inputType, grouping, getRowType, namedAggregates, Nil)}))"
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw)
-      .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty)
-      .item("select", aggregationToString(inputType, grouping, getRowType, namedAggregates, Nil))
-  }
-
-  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-
-    val child = this.getInput
-    val rowCnt = metadata.getRowCount(child)
-    val rowSize = this.estimateRowSize(child.getRowType)
-    val aggCnt = this.namedAggregates.size
-    planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize)
-  }
-
-  override def translateToPlan(
-    tableEnv: BatchTableEnvironment,
-    expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
-
-    val config = tableEnv.getConfig
-
-    val groupingKeys = grouping.indices.toArray
-
-    val mapFunction = AggregateUtil.createPrepareMapFunction(
-      namedAggregates,
-      grouping,
-      inputType)
-
-    val groupReduceFunction = AggregateUtil.createAggregateGroupReduceFunction(
-      namedAggregates,
-      inputType,
-      rowRelDataType,
-      grouping)
-
-    val inputDS = getInput.asInstanceOf[DataSetRel].translateToPlan(
-      tableEnv,
-      // tell the input operator that this operator currently only supports Rows as input
-      Some(TypeConverter.DEFAULT_ROW_TYPE))
-
-    // get the output types
-    val fieldTypes: Array[TypeInformation[_]] = getRowType.getFieldList.asScala
-    .map(field => FlinkTypeFactory.toTypeInfo(field.getType))
-    .toArray
-
-    val aggString = aggregationToString(inputType, grouping, getRowType, namedAggregates, Nil)
-    val prepareOpName = s"prepare select: ($aggString)"
-    val mappedInput = inputDS
-      .map(mapFunction)
-      .name(prepareOpName)
-
-    val rowTypeInfo = new RowTypeInfo(fieldTypes: _*)
-
-    val result = {
-      if (groupingKeys.length > 0) {
-        // grouped aggregation
-        val aggOpName = s"groupBy: (${groupingToString(inputType, grouping)}), " +
-          s"select: ($aggString)"
-
-        mappedInput.asInstanceOf[DataSet[Row]]
-          .groupBy(groupingKeys: _*)
-          .reduceGroup(groupReduceFunction)
-          .returns(rowTypeInfo)
-          .name(aggOpName)
-          .asInstanceOf[DataSet[Any]]
-      }
-      else {
-        // global aggregation
-        val aggOpName = s"select:($aggString)"
-        mappedInput.asInstanceOf[DataSet[Row]]
-          .reduceGroup(groupReduceFunction)
-          .returns(rowTypeInfo)
-          .name(aggOpName)
-          .asInstanceOf[DataSet[Any]]
-      }
-    }
-
-    // if the expected type is not a Row, inject a mapper to convert to the expected type
-    expectedType match {
-      case Some(typeInfo) if typeInfo.getTypeClass != classOf[Row] =>
-        val mapName = s"convert: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
-        result.map(getConversionMapper(
-          config = config,
-          nullableInput = false,
-          inputType = rowTypeInfo.asInstanceOf[TypeInformation[Any]],
-          expectedType = expectedType.get,
-          conversionOperatorName = "DataSetAggregateConversion",
-          fieldNames = getRowType.getFieldNames.asScala
-        ))
-        .name(mapName)
-      case _ => result
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCalc.scala
deleted file mode 100644
index c0881b7..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCalc.scala
+++ /dev/null
@@ -1,136 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan.{RelOptPlanner, RelOptCost, RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
-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.table.codegen.CodeGenerator
-import org.apache.flink.api.table.plan.nodes.FlinkCalc
-import org.apache.flink.api.table.typeutils.TypeConverter
-import TypeConverter._
-import org.apache.flink.api.table.BatchTableEnvironment
-import org.apache.calcite.rex._
-
-import scala.collection.JavaConverters._
-
-/**
-  * Flink RelNode which matches along with LogicalCalc.
-  *
-  */
-class DataSetCalc(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    input: RelNode,
-    rowRelDataType: RelDataType,
-    private[flink] val calcProgram: RexProgram, // for tests
-    ruleDescription: String)
-  extends SingleRel(cluster, traitSet, input)
-  with FlinkCalc
-  with DataSetRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataSetCalc(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      getRowType,
-      calcProgram,
-      ruleDescription)
-  }
-
-  override def toString: String = calcToString(calcProgram, getExpressionString)
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw)
-      .item("select", selectionToString(calcProgram, getExpressionString))
-      .itemIf("where",
-        conditionToString(calcProgram, getExpressionString),
-        calcProgram.getCondition != null)
-  }
-
-  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-
-    val child = this.getInput
-    val rowCnt = metadata.getRowCount(child)
-
-    // compute number of expressions that do not access a field or literal, i.e. computations,
-    //   conditions, etc. We only want to account for computations, not for simple projections.
-    val compCnt = calcProgram.getExprList.asScala.toList.count {
-      case i: RexInputRef => false
-      case l: RexLiteral => false
-      case _ => true
-    }
-
-    planner.getCostFactory.makeCost(rowCnt, rowCnt * compCnt, 0)
-  }
-
-  override def estimateRowCount(metadata: RelMetadataQuery): Double = {
-    val child = this.getInput
-    val rowCnt = metadata.getRowCount(child)
-
-    if (calcProgram.getCondition != null) {
-      // we reduce the result card to push filters down
-      (rowCnt * 0.75).min(1.0)
-    } else {
-      rowCnt
-    }
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
-
-    val config = tableEnv.getConfig
-
-    val inputDS = getInput.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-
-    val returnType = determineReturnType(
-      getRowType,
-      expectedType,
-      config.getNullCheck,
-      config.getEfficientTypeUsage)
-
-    val generator = new CodeGenerator(config, false, inputDS.getType)
-
-    val body = functionBody(
-      generator,
-      inputDS.getType,
-      getRowType,
-      calcProgram,
-      config,
-      expectedType)
-
-    val genFunction = generator.generateFunction(
-      ruleDescription,
-      classOf[FlatMapFunction[Any, Any]],
-      body,
-      returnType)
-
-    val mapFunc = calcMapFunction(genFunction)
-    inputDS.flatMap(mapFunc).name(calcOpName(calcProgram, getExpressionString))
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetConvention.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetConvention.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetConvention.scala
deleted file mode 100644
index 03d9a51..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetConvention.scala
+++ /dev/null
@@ -1,47 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan._
-
-class DataSetConvention extends Convention {
-
-  override def toString: String = getName
-
-  override def useAbstractConvertersForConversion(
-    fromTraits: RelTraitSet,
-    toTraits: RelTraitSet): Boolean = false
-
-  override def canConvertConvention(toConvention: Convention): Boolean = false
-
-  def getInterface: Class[_] = classOf[DataSetRel]
-
-  def getName: String = "DATASET"
-
-  def getTraitDef: RelTraitDef[_ <: RelTrait] = ConventionTraitDef.INSTANCE
-
-  def satisfies(`trait`: RelTrait): Boolean = this eq `trait`
-
-  def register(planner: RelOptPlanner): Unit = { }
-}
-
-object DataSetConvention {
-
-  val INSTANCE = new DataSetConvention
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCorrelate.scala
deleted file mode 100644
index 95eb15b..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetCorrelate.scala
+++ /dev/null
@@ -1,139 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.logical.LogicalTableFunctionScan
-import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
-import org.apache.calcite.rex.{RexNode, RexCall}
-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.table.BatchTableEnvironment
-import org.apache.flink.api.table.codegen.CodeGenerator
-import org.apache.flink.api.table.functions.utils.TableSqlFunction
-import org.apache.flink.api.table.plan.nodes.FlinkCorrelate
-import org.apache.flink.api.table.typeutils.TypeConverter._
-
-/**
-  * Flink RelNode which matches along with join a user defined table function.
-  */
-class DataSetCorrelate(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    inputNode: RelNode,
-    scan: LogicalTableFunctionScan,
-    condition: Option[RexNode],
-    relRowType: RelDataType,
-    joinRowType: RelDataType,
-    joinType: SemiJoinType,
-    ruleDescription: String)
-  extends SingleRel(cluster, traitSet, inputNode)
-  with FlinkCorrelate
-  with DataSetRel {
-
-  override def deriveRowType() = relRowType
-
-  override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-    val rowCnt = metadata.getRowCount(getInput) * 1.5
-    planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * 0.5)
-  }
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataSetCorrelate(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      scan,
-      condition,
-      relRowType,
-      joinRowType,
-      joinType,
-      ruleDescription)
-  }
-
-  override def toString: String = {
-    val rexCall = scan.getCall.asInstanceOf[RexCall]
-    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
-    correlateToString(rexCall, sqlFunction)
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    val rexCall = scan.getCall.asInstanceOf[RexCall]
-    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
-    super.explainTerms(pw)
-      .item("invocation", scan.getCall)
-      .item("function", sqlFunction.getTableFunction.getClass.getCanonicalName)
-      .item("rowType", relRowType)
-      .item("joinType", joinType)
-      .itemIf("condition", condition.orNull, condition.isDefined)
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]])
-    : DataSet[Any] = {
-
-    val config = tableEnv.getConfig
-    val returnType = determineReturnType(
-      getRowType,
-      expectedType,
-      config.getNullCheck,
-      config.getEfficientTypeUsage)
-
-    // we do not need to specify input type
-    val inputDS = inputNode.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-
-    val funcRel = scan.asInstanceOf[LogicalTableFunctionScan]
-    val rexCall = funcRel.getCall.asInstanceOf[RexCall]
-    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
-    val pojoFieldMapping = sqlFunction.getPojoFieldMapping
-    val udtfTypeInfo = sqlFunction.getRowTypeInfo.asInstanceOf[TypeInformation[Any]]
-
-    val generator = new CodeGenerator(
-      config,
-      false,
-      inputDS.getType,
-      Some(udtfTypeInfo),
-      None,
-      Some(pojoFieldMapping))
-
-    val body = functionBody(
-      generator,
-      udtfTypeInfo,
-      getRowType,
-      rexCall,
-      condition,
-      config,
-      joinType,
-      expectedType)
-
-    val genFunction = generator.generateFunction(
-      ruleDescription,
-      classOf[FlatMapFunction[Any, Any]],
-      body,
-      returnType)
-
-    val mapFunc = correlateMapFunction(genFunction)
-
-    inputDS.flatMap(mapFunc).name(correlateOpName(rexCall, sqlFunction, relRowType))
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetIntersect.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetIntersect.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetIntersect.scala
deleted file mode 100644
index d2203d0..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetIntersect.scala
+++ /dev/null
@@ -1,133 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.BatchTableEnvironment
-import org.apache.flink.api.table.runtime.IntersectCoGroupFunction
-import org.apache.flink.api.table.typeutils.TypeConverter._
-
-import scala.collection.JavaConversions._
-import scala.collection.JavaConverters._
-
-/**
-  * Flink RelNode which translates Intersect into CoGroup Operator.
-  *
-  */
-class DataSetIntersect(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    leftNode: RelNode,
-    rightNode: RelNode,
-    rowRelDataType: RelDataType,
-    all: Boolean)
-  extends BiRel(cluster, traitSet, leftNode, rightNode)
-    with DataSetRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataSetIntersect(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      inputs.get(1),
-      getRowType,
-      all
-    )
-  }
-
-  override def toString: String = {
-    s"Intersect(intersect: ($intersectSelectionToString))"
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw).item("intersect", intersectSelectionToString)
-  }
-
-  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-    val children = this.getInputs
-    children.foldLeft(planner.getCostFactory.makeCost(0, 0, 0)) { (cost, child) =>
-      val rowCnt = metadata.getRowCount(child)
-      val rowSize = this.estimateRowSize(child.getRowType)
-      cost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize))
-    }
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
-
-    val leftDataSet: DataSet[Any] = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-    val rightDataSet: DataSet[Any] = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-
-    val coGroupedDs = leftDataSet.coGroup(rightDataSet)
-
-    val coGroupOpName = s"intersect: ($intersectSelectionToString)"
-    val coGroupFunction = new IntersectCoGroupFunction[Any](all)
-
-    val intersectDs = coGroupedDs.where("*").equalTo("*")
-      .`with`(coGroupFunction).name(coGroupOpName)
-
-    val config = tableEnv.getConfig
-    val leftType = leftDataSet.getType
-
-    // here we only care about left type information, because we emit records from left DataSet
-    expectedType match {
-      case None if config.getEfficientTypeUsage =>
-        intersectDs
-
-      case _ =>
-        val determinedType = determineReturnType(
-          getRowType,
-          expectedType,
-          config.getNullCheck,
-          config.getEfficientTypeUsage)
-
-        // conversion
-        if (determinedType != leftType) {
-          val mapFunc = getConversionMapper(
-            config,
-            false,
-            leftType,
-            determinedType,
-            "DataSetIntersectConversion",
-            getRowType.getFieldNames)
-
-          val opName = s"convert: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
-
-          intersectDs.map(mapFunc).name(opName)
-        }
-        // no conversion necessary, forward
-        else {
-          intersectDs
-        }
-    }
-  }
-
-  private def intersectSelectionToString: String = {
-    getRowType.getFieldNames.asScala.toList.mkString(", ")
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala
deleted file mode 100644
index ccd84ca..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetJoin.scala
+++ /dev/null
@@ -1,229 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan._
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
-import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
-import org.apache.calcite.util.mapping.IntPair
-import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.codegen.CodeGenerator
-import org.apache.flink.api.table.runtime.FlatJoinRunner
-import org.apache.flink.api.table.typeutils.TypeConverter.determineReturnType
-import org.apache.flink.api.table.{BatchTableEnvironment, TableException}
-import org.apache.flink.api.common.functions.FlatJoinFunction
-import org.apache.calcite.rex.RexNode
-
-import scala.collection.JavaConversions._
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-/**
-  * Flink RelNode which matches along with JoinOperator and its related operations.
-  */
-class DataSetJoin(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    leftNode: RelNode,
-    rightNode: RelNode,
-    rowRelDataType: RelDataType,
-    joinCondition: RexNode,
-    joinRowType: RelDataType,
-    joinInfo: JoinInfo,
-    keyPairs: List[IntPair],
-    joinType: JoinRelType,
-    joinHint: JoinHint,
-    ruleDescription: String)
-  extends BiRel(cluster, traitSet, leftNode, rightNode)
-  with DataSetRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataSetJoin(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      inputs.get(1),
-      getRowType,
-      joinCondition,
-      joinRowType,
-      joinInfo,
-      keyPairs,
-      joinType,
-      joinHint,
-      ruleDescription)
-  }
-
-  override def toString: String = {
-    s"$joinTypeToString(where: ($joinConditionToString), join: ($joinSelectionToString))"
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw)
-      .item("where", joinConditionToString)
-      .item("join", joinSelectionToString)
-      .item("joinType", joinTypeToString)
-  }
-
-  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-
-    val leftRowCnt = metadata.getRowCount(getLeft)
-    val leftRowSize = estimateRowSize(getLeft.getRowType)
-
-    val rightRowCnt = metadata.getRowCount(getRight)
-    val rightRowSize = estimateRowSize(getRight.getRowType)
-
-    val ioCost = (leftRowCnt * leftRowSize) + (rightRowCnt * rightRowSize)
-    val cpuCost = leftRowCnt + rightRowCnt
-    val rowCnt = leftRowCnt + rightRowCnt
-
-    planner.getCostFactory.makeCost(rowCnt, cpuCost, ioCost)
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
-
-    val config = tableEnv.getConfig
-
-    val returnType = determineReturnType(
-      getRowType,
-      expectedType,
-      config.getNullCheck,
-      config.getEfficientTypeUsage)
-
-    // get the equality keys
-    val leftKeys = ArrayBuffer.empty[Int]
-    val rightKeys = ArrayBuffer.empty[Int]
-    if (keyPairs.isEmpty) {
-      // if no equality keys => not supported
-      throw TableException(
-        "Joins should have at least one equality condition.\n" +
-          s"\tLeft: ${left.toString},\n" +
-          s"\tRight: ${right.toString},\n" +
-          s"\tCondition: ($joinConditionToString)"
-      )
-    }
-    else {
-      // at least one equality expression
-      val leftFields = left.getRowType.getFieldList
-      val rightFields = right.getRowType.getFieldList
-
-      keyPairs.foreach(pair => {
-        val leftKeyType = leftFields.get(pair.source).getType.getSqlTypeName
-        val rightKeyType = rightFields.get(pair.target).getType.getSqlTypeName
-
-        // check if keys are compatible
-        if (leftKeyType == rightKeyType) {
-          // add key pair
-          leftKeys.add(pair.source)
-          rightKeys.add(pair.target)
-        } else {
-          throw TableException(
-            "Equality join predicate on incompatible types.\n" +
-              s"\tLeft: ${left.toString},\n" +
-              s"\tRight: ${right.toString},\n" +
-              s"\tCondition: ($joinConditionToString)"
-          )
-        }
-      })
-    }
-
-    val leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-    val rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-
-    val (joinOperator, nullCheck) = joinType match {
-      case JoinRelType.INNER => (leftDataSet.join(rightDataSet), false)
-      case JoinRelType.LEFT => (leftDataSet.leftOuterJoin(rightDataSet), true)
-      case JoinRelType.RIGHT => (leftDataSet.rightOuterJoin(rightDataSet), true)
-      case JoinRelType.FULL => (leftDataSet.fullOuterJoin(rightDataSet), true)
-    }
-
-    if (nullCheck && !config.getNullCheck) {
-      throw TableException("Null check in TableConfig must be enabled for outer joins.")
-    }
-
-    val generator = new CodeGenerator(
-      config,
-      nullCheck,
-      leftDataSet.getType,
-      Some(rightDataSet.getType))
-    val conversion = generator.generateConverterResultExpression(
-      returnType,
-      joinRowType.getFieldNames)
-
-    var body = ""
-
-    if (joinInfo.isEqui) {
-      // only equality condition
-      body = s"""
-           |${conversion.code}
-           |${generator.collectorTerm}.collect(${conversion.resultTerm});
-           |""".stripMargin
-    }
-    else {
-      val condition = generator.generateExpression(joinCondition)
-      body = s"""
-           |${condition.code}
-           |if (${condition.resultTerm}) {
-           |  ${conversion.code}
-           |  ${generator.collectorTerm}.collect(${conversion.resultTerm});
-           |}
-           |""".stripMargin
-    }
-    val genFunction = generator.generateFunction(
-      ruleDescription,
-      classOf[FlatJoinFunction[Any, Any, Any]],
-      body,
-      returnType)
-
-    val joinFun = new FlatJoinRunner[Any, Any, Any](
-      genFunction.name,
-      genFunction.code,
-      genFunction.returnType)
-
-    val joinOpName = s"where: ($joinConditionToString), join: ($joinSelectionToString)"
-
-    joinOperator.where(leftKeys.toArray: _*).equalTo(rightKeys.toArray: _*)
-      .`with`(joinFun).name(joinOpName).asInstanceOf[DataSet[Any]]
-  }
-
-  private def joinSelectionToString: String = {
-    getRowType.getFieldNames.asScala.toList.mkString(", ")
-  }
-
-  private def joinConditionToString: String = {
-
-    val inFields = joinRowType.getFieldNames.asScala.toList
-    getExpressionString(joinCondition, inFields, None)
-  }
-
-  private def joinTypeToString = joinType match {
-    case JoinRelType.INNER => "InnerJoin"
-    case JoinRelType.LEFT=> "LeftOuterJoin"
-    case JoinRelType.RIGHT => "RightOuterJoin"
-    case JoinRelType.FULL => "FullOuterJoin"
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetMinus.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetMinus.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetMinus.scala
deleted file mode 100644
index 6a5cbd1..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetMinus.scala
+++ /dev/null
@@ -1,145 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.BatchTableEnvironment
-import org.apache.flink.api.table.runtime.MinusCoGroupFunction
-import org.apache.flink.api.table.typeutils.TypeConverter._
-
-import scala.collection.JavaConversions._
-import scala.collection.JavaConverters._
-
-/**
-  * Flink RelNode which implements set minus operation.
-  *
-  */
-class DataSetMinus(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    leftNode: RelNode,
-    rightNode: RelNode,
-    rowRelDataType: RelDataType,
-    all: Boolean)
-  extends BiRel(cluster, traitSet, leftNode, rightNode)
-    with DataSetRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataSetMinus(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      inputs.get(1),
-      getRowType,
-      all
-    )
-  }
-
-  override def toString: String = {
-    s"Minus(minus: ($minusSelectionToString}))"
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw).item("minus", minusSelectionToString)
-  }
-
-  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-    val children = this.getInputs
-    children.foldLeft(planner.getCostFactory.makeCost(0, 0, 0)) { (cost, child) =>
-      val rowCnt = metadata.getRowCount(child)
-      val rowSize = this.estimateRowSize(child.getRowType)
-      cost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize))
-    }
-  }
-
-  override def estimateRowCount(mq: RelMetadataQuery): Double = {
-    // from org.apache.calcite.rel.metadata.RelMdUtil.getMinusRowCount
-    val children = this.getInputs
-    var rowCnt = mq.getRowCount(children.head)
-    getInputs.tail.foreach(rowCnt -= 0.5 * mq.getRowCount(_))
-    if (rowCnt < 0) {
-      rowCnt = 0.0
-    }
-    rowCnt
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
-
-    val leftDataSet: DataSet[Any] = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-    val rightDataSet: DataSet[Any] = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-
-    val coGroupedDs = leftDataSet.coGroup(rightDataSet)
-
-    val coGroupOpName = s"minus: ($minusSelectionToString)"
-    val coGroupFunction = new MinusCoGroupFunction[Any](all)
-
-    val minusDs = coGroupedDs.where("*").equalTo("*")
-      .`with`(coGroupFunction).name(coGroupOpName)
-
-    val config = tableEnv.getConfig
-    val leftType = leftDataSet.getType
-
-    // here we only care about left type information, because we emit records from left DataSet
-    expectedType match {
-      case None if config.getEfficientTypeUsage =>
-        minusDs
-
-      case _ =>
-        val determinedType = determineReturnType(
-          getRowType,
-          expectedType,
-          config.getNullCheck,
-          config.getEfficientTypeUsage)
-
-        // conversion
-        if (determinedType != leftType) {
-          val mapFunc = getConversionMapper(
-            config = config,
-            nullableInput = false,
-            inputType = leftType,
-            expectedType = determinedType,
-            conversionOperatorName = "DataSetMinusConversion",
-            fieldNames = getRowType.getFieldNames)
-
-          val opName = s"convert: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
-
-          minusDs.map(mapFunc).name(opName)
-        }
-        // no conversion necessary, forward
-        else {
-          minusDs
-        }
-    }
-  }
-
-  private def minusSelectionToString: String = {
-    getRowType.getFieldNames.asScala.toList.mkString(", ")
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala
deleted file mode 100644
index 82c75e1..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetRel.scala
+++ /dev/null
@@ -1,74 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.sql.`type`.SqlTypeName
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.codegen.CodeGenerator
-import org.apache.flink.api.table.plan.nodes.FlinkRel
-import org.apache.flink.api.table.runtime.MapRunner
-import org.apache.flink.api.table.{BatchTableEnvironment, TableConfig, TableException}
-
-import scala.collection.JavaConversions._
-
-trait DataSetRel extends RelNode with FlinkRel {
-
-  /**
-    * Translates the [[DataSetRel]] node into a [[DataSet]] operator.
-    *
-    * @param tableEnv [[org.apache.flink.api.table.BatchTableEnvironment]] of the translated Table.
-    * @param expectedType specifies the type the Flink operator should return. The type must
-    *                     have the same arity as the result. For instance, if the
-    *                     expected type is a RowTypeInfo this method will return a DataSet of
-    *                     type Row. If the expected type is Tuple2, the operator will return
-    *                     a Tuple2 if possible. Row otherwise.
-    * @return DataSet of type expectedType or RowTypeInfo
-    */
-  def translateToPlan(
-     tableEnv: BatchTableEnvironment,
-     expectedType: Option[TypeInformation[Any]] = None) : DataSet[Any]
-
-  private[flink] def estimateRowSize(rowType: RelDataType): Double = {
-
-    rowType.getFieldList.map(_.getType.getSqlTypeName).foldLeft(0) { (s, t) =>
-      t match {
-        case SqlTypeName.TINYINT => s + 1
-        case SqlTypeName.SMALLINT => s + 2
-        case SqlTypeName.INTEGER => s + 4
-        case SqlTypeName.BIGINT => s + 8
-        case SqlTypeName.BOOLEAN => s + 1
-        case SqlTypeName.FLOAT => s + 4
-        case SqlTypeName.DOUBLE => s + 8
-        case SqlTypeName.VARCHAR => s + 12
-        case SqlTypeName.CHAR => s + 1
-        case SqlTypeName.DECIMAL => s + 12
-        case typeName if SqlTypeName.YEAR_INTERVAL_TYPES.contains(typeName) => s + 8
-        case typeName if SqlTypeName.DAY_INTERVAL_TYPES.contains(typeName) => s + 4
-        case SqlTypeName.TIME | SqlTypeName.TIMESTAMP | SqlTypeName.DATE => s + 12
-        case _ => throw TableException(s"Unsupported data type encountered: $t")
-      }
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetScan.scala
deleted file mode 100644
index b783136..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetScan.scala
+++ /dev/null
@@ -1,64 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan._
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.BatchTableEnvironment
-import org.apache.flink.api.table.plan.schema.DataSetTable
-
-/**
-  * Flink RelNode which matches along with DataSource.
-  * It ensures that types without deterministic field order (e.g. POJOs) are not part of
-  * the plan translation.
-  */
-class DataSetScan(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    table: RelOptTable,
-    rowRelDataType: RelDataType)
-  extends BatchScan(cluster, traitSet, table) {
-
-  val dataSetTable: DataSetTable[Any] = getTable.unwrap(classOf[DataSetTable[Any]])
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataSetScan(
-      cluster,
-      traitSet,
-      getTable,
-      getRowType
-    )
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
-
-    val config = tableEnv.getConfig
-    val inputDataSet: DataSet[Any] = dataSetTable.dataSet
-
-    convertToExpectedType(inputDataSet, dataSetTable, expectedType, config)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSingleRowJoin.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSingleRowJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSingleRowJoin.scala
deleted file mode 100644
index 0306c00..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSingleRowJoin.scala
+++ /dev/null
@@ -1,193 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan._
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
-import org.apache.calcite.rex.RexNode
-import org.apache.flink.api.common.functions.{FlatJoinFunction, FlatMapFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.codegen.CodeGenerator
-import org.apache.flink.api.table.runtime.{MapJoinLeftRunner, MapJoinRightRunner}
-import org.apache.flink.api.table.typeutils.TypeConverter.determineReturnType
-import org.apache.flink.api.table.{BatchTableEnvironment, TableConfig}
-
-import scala.collection.JavaConversions._
-import scala.collection.JavaConverters._
-
-/**
-  * Flink RelNode that executes a Join where one of inputs is a single row.
-  */
-class DataSetSingleRowJoin(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    leftNode: RelNode,
-    rightNode: RelNode,
-    leftIsSingle: Boolean,
-    rowRelDataType: RelDataType,
-    joinCondition: RexNode,
-    joinRowType: RelDataType,
-    ruleDescription: String)
-  extends BiRel(cluster, traitSet, leftNode, rightNode)
-  with DataSetRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataSetSingleRowJoin(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      inputs.get(1),
-      leftIsSingle,
-      getRowType,
-      joinCondition,
-      joinRowType,
-      ruleDescription)
-  }
-
-  override def toString: String = {
-    s"$joinTypeToString(where: ($joinConditionToString), join: ($joinSelectionToString))"
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw)
-      .item("where", joinConditionToString)
-      .item("join", joinSelectionToString)
-      .item("joinType", joinTypeToString)
-  }
-
-  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-    val child = if (leftIsSingle) {
-      this.getRight
-    } else {
-      this.getLeft
-    }
-    val rowCnt = metadata.getRowCount(child)
-    val rowSize = this.estimateRowSize(child.getRowType)
-    planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize)
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
-
-    val leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-    val rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-    val broadcastSetName = "joinSet"
-    val mapSideJoin = generateMapFunction(
-      tableEnv.getConfig,
-      leftDataSet.getType,
-      rightDataSet.getType,
-      leftIsSingle,
-      joinCondition,
-      broadcastSetName,
-      expectedType)
-
-    val (multiRowDataSet, singleRowDataSet) =
-      if (leftIsSingle) {
-        (rightDataSet, leftDataSet)
-      } else {
-        (leftDataSet, rightDataSet)
-      }
-
-    multiRowDataSet
-      .flatMap(mapSideJoin)
-      .withBroadcastSet(singleRowDataSet, broadcastSetName)
-      .name(getMapOperatorName)
-      .asInstanceOf[DataSet[Any]]
-  }
-
-  private def generateMapFunction(
-      config: TableConfig,
-      inputType1: TypeInformation[Any],
-      inputType2: TypeInformation[Any],
-      firstIsSingle: Boolean,
-      joinCondition: RexNode,
-      broadcastInputSetName: String,
-      expectedType: Option[TypeInformation[Any]]): FlatMapFunction[Any, Any] = {
-
-    val codeGenerator = new CodeGenerator(
-      config,
-      false,
-      inputType1,
-      Some(inputType2))
-
-    val returnType = determineReturnType(
-      getRowType,
-      expectedType,
-      config.getNullCheck,
-      config.getEfficientTypeUsage)
-
-    val conversion = codeGenerator.generateConverterResultExpression(
-      returnType,
-      joinRowType.getFieldNames)
-
-    val condition = codeGenerator.generateExpression(joinCondition)
-
-    val joinMethodBody = s"""
-                  |${condition.code}
-                  |if (${condition.resultTerm}) {
-                  |  ${conversion.code}
-                  |  ${codeGenerator.collectorTerm}.collect(${conversion.resultTerm});
-                  |}
-                  |""".stripMargin
-
-    val genFunction = codeGenerator.generateFunction(
-      ruleDescription,
-      classOf[FlatJoinFunction[Any, Any, Any]],
-      joinMethodBody,
-      returnType)
-
-    if (firstIsSingle) {
-      new MapJoinRightRunner[Any, Any, Any](
-        genFunction.name,
-        genFunction.code,
-        genFunction.returnType,
-        broadcastInputSetName)
-    } else {
-      new MapJoinLeftRunner[Any, Any, Any](
-        genFunction.name,
-        genFunction.code,
-        genFunction.returnType,
-        broadcastInputSetName)
-    }
-  }
-
-  private def getMapOperatorName: String = {
-    s"where: ($joinConditionToString), join: ($joinSelectionToString)"
-  }
-
-  private def joinSelectionToString: String = {
-    getRowType.getFieldNames.asScala.toList.mkString(", ")
-  }
-
-  private def joinConditionToString: String = {
-    val inFields = joinRowType.getFieldNames.asScala.toList
-    getExpressionString(joinCondition, inFields, None)
-  }
-
-  private def joinTypeToString: String = {
-    "NestedLoopJoin"
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSort.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSort.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSort.scala
deleted file mode 100644
index 661aeef..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSort.scala
+++ /dev/null
@@ -1,194 +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.api.table.plan.nodes.dataset
-
-import java.util
-
-import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.RelFieldCollation.Direction
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.{RelCollation, RelNode, RelWriter, SingleRel}
-import org.apache.calcite.rex.{RexLiteral, RexNode}
-import org.apache.flink.api.common.operators.Order
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.{BatchTableEnvironment, TableException}
-import org.apache.flink.api.table.runtime.{CountPartitionFunction, LimitFilterFunction}
-import org.apache.flink.api.table.typeutils.TypeConverter._
-
-import scala.collection.JavaConverters._
-
-class DataSetSort(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    inp: RelNode,
-    collations: RelCollation,
-    rowRelDataType: RelDataType,
-    offset: RexNode,
-    fetch: RexNode)
-  extends SingleRel(cluster, traitSet, inp)
-  with DataSetRel {
-
-  private val limitStart: Long = if (offset != null) {
-    RexLiteral.intValue(offset)
-  } else {
-    0L
-  }
-
-  private val limitEnd: Long = if (fetch != null) {
-    RexLiteral.intValue(fetch) + limitStart
-  } else {
-    Long.MaxValue
-  }
-
-  override def deriveRowType(): RelDataType = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = {
-    new DataSetSort(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      collations,
-      getRowType,
-      offset,
-      fetch
-    )
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]] = None)
-    : DataSet[Any] = {
-
-    if (fieldCollations.isEmpty) {
-      throw TableException("Limiting the result without sorting is not allowed " +
-        "as it could lead to arbitrary results.")
-    }
-
-    val config = tableEnv.getConfig
-
-    val inputDs = inp.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-
-    val currentParallelism = inputDs.getExecutionEnvironment.getParallelism
-    var partitionedDs = if (currentParallelism == 1) {
-      inputDs
-    } else {
-      inputDs.partitionByRange(fieldCollations.map(_._1): _*)
-        .withOrders(fieldCollations.map(_._2): _*)
-    }
-
-    fieldCollations.foreach { fieldCollation =>
-      partitionedDs = partitionedDs.sortPartition(fieldCollation._1, fieldCollation._2)
-    }
-
-    val limitedDs = if (offset == null && fetch == null) {
-      partitionedDs
-    } else {
-      val countFunction = new CountPartitionFunction[Any]
-
-      val partitionCountName = s"prepare offset/fetch"
-
-      val partitionCount = partitionedDs
-        .mapPartition(countFunction)
-        .name(partitionCountName)
-
-      val broadcastName = "countPartition"
-
-      val limitFunction = new LimitFilterFunction[Any](
-        limitStart,
-        limitEnd,
-        broadcastName)
-
-      val limitName = s"offset: $offsetToString, fetch: $fetchToString"
-
-      partitionedDs
-        .filter(limitFunction)
-        .name(limitName)
-        .withBroadcastSet(partitionCount, broadcastName)
-    }
-
-    val inputType = partitionedDs.getType
-    expectedType match {
-
-      case None if config.getEfficientTypeUsage =>
-        limitedDs
-
-      case _ =>
-        val determinedType = determineReturnType(
-          getRowType,
-          expectedType,
-          config.getNullCheck,
-          config.getEfficientTypeUsage)
-
-        // conversion
-        if (determinedType != inputType) {
-
-          val mapFunc = getConversionMapper(
-            config = config,
-            nullableInput = false,
-            inputType = partitionedDs.getType,
-            expectedType = determinedType,
-            conversionOperatorName = "DataSetSortConversion",
-            fieldNames = getRowType.getFieldNames.asScala
-          )
-
-          val opName = s"convert: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
-
-          limitedDs.map(mapFunc).name(opName)
-        }
-        // no conversion necessary, forward
-        else {
-          limitedDs
-        }
-    }
-  }
-
-  private def directionToOrder(direction: Direction) = {
-    direction match {
-      case Direction.ASCENDING | Direction.STRICTLY_ASCENDING => Order.ASCENDING
-      case Direction.DESCENDING | Direction.STRICTLY_DESCENDING => Order.DESCENDING
-      case _ => throw new IllegalArgumentException("Unsupported direction.")
-    }
-
-  }
-
-  private val fieldCollations = collations.getFieldCollations.asScala
-    .map(c => (c.getFieldIndex, directionToOrder(c.getDirection)))
-
-  private val sortFieldsToString = fieldCollations
-    .map(col => s"${getRowType.getFieldNames.get(col._1)} ${col._2.getShortName}" ).mkString(", ")
-
-  private val offsetToString = s"$offset"
-
-  private val fetchToString = if (limitEnd == Long.MaxValue) {
-    "unlimited"
-  } else {
-    s"$limitEnd"
-  }
-
-  override def toString: String =
-    s"Sort(by: ($sortFieldsToString), offset: $offsetToString, fetch: $fetchToString)"
-
-  override def explainTerms(pw: RelWriter) : RelWriter = {
-    super.explainTerms(pw)
-      .item("orderBy", sortFieldsToString)
-      .item("offset", offsetToString)
-      .item("fetch", fetchToString)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetUnion.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetUnion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetUnion.scala
deleted file mode 100644
index 6e43fae..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetUnion.scala
+++ /dev/null
@@ -1,104 +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.api.table.plan.nodes.dataset
-
-import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.metadata.RelMetadataQuery
-import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.BatchTableEnvironment
-
-import scala.collection.JavaConversions._
-import scala.collection.JavaConverters._
-
-/**
-* Flink RelNode which matches along with UnionOperator.
-*
-*/
-class DataSetUnion(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    leftNode: RelNode,
-    rightNode: RelNode,
-    rowRelDataType: RelDataType)
-  extends BiRel(cluster, traitSet, leftNode, rightNode)
-  with DataSetRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataSetUnion(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      inputs.get(1),
-      rowRelDataType
-    )
-  }
-
-  override def toString: String = {
-    s"Union(union: ($unionSelectionToString))"
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw).item("union", unionSelectionToString)
-  }
-
-  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
-
-    val children = this.getInputs
-    val rowCnt = children.foldLeft(0D) { (rows, child) =>
-      rows + metadata.getRowCount(child)
-    }
-
-    planner.getCostFactory.makeCost(rowCnt, 0, 0)
-  }
-
-  override def estimateRowCount(mq: RelMetadataQuery): Double = {
-    // adopted from org.apache.calcite.rel.metadata.RelMdUtil.getUnionAllRowCount
-    getInputs.foldLeft(0.0)(_ + mq.getRowCount(_))
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
-
-    var leftDataSet: DataSet[Any] = null
-    var rightDataSet: DataSet[Any] = null
-
-    expectedType match {
-      case None =>
-        leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
-        rightDataSet =
-          right.asInstanceOf[DataSetRel].translateToPlan(tableEnv, Some(leftDataSet.getType))
-      case _ =>
-        leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv, expectedType)
-        rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv, expectedType)
-    }
-
-    leftDataSet.union(rightDataSet).asInstanceOf[DataSet[Any]]
-  }
-
-  private def unionSelectionToString: String = {
-    rowRelDataType.getFieldNames.asScala.toList.mkString(", ")
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetValues.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetValues.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetValues.scala
deleted file mode 100644
index 4f3a257..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetValues.scala
+++ /dev/null
@@ -1,111 +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.api.table.plan.nodes.dataset
-
-import com.google.common.collect.ImmutableList
-import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.Values
-import org.apache.calcite.rel.{RelNode, RelWriter}
-import org.apache.calcite.rex.RexLiteral
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.api.table.BatchTableEnvironment
-import org.apache.flink.api.table.codegen.CodeGenerator
-import org.apache.flink.api.table.runtime.io.ValuesInputFormat
-import org.apache.flink.api.table.typeutils.TypeConverter._
-
-import scala.collection.JavaConverters._
-
-/**
-  * DataSet RelNode for a LogicalValues.
-  *
-  */
-class DataSetValues(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    rowRelDataType: RelDataType,
-    tuples: ImmutableList[ImmutableList[RexLiteral]],
-    ruleDescription: String)
-  extends Values(cluster, rowRelDataType, tuples, traitSet)
-  with DataSetRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataSetValues(
-      cluster,
-      traitSet,
-      getRowType,
-      getTuples,
-      ruleDescription
-    )
-  }
-
-  override def toString: String = {
-    s"Values(values: (${getRowType.getFieldNames.asScala.toList.mkString(", ")}))"
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw).item("values", valuesFieldsToString)
-  }
-
-  override def translateToPlan(
-      tableEnv: BatchTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
-
-    val config = tableEnv.getConfig
-
-    val returnType = determineReturnType(
-      getRowType,
-      expectedType,
-      config.getNullCheck,
-      config.getEfficientTypeUsage)
-
-    val generator = new CodeGenerator(config)
-
-    // generate code for every record
-    val generatedRecords = getTuples.asScala.map { r =>
-      generator.generateResultExpression(
-        returnType,
-        getRowType.getFieldNames.asScala,
-        r.asScala)
-    }
-
-    // generate input format
-    val generatedFunction = generator.generateValuesInputFormat(
-      ruleDescription,
-      generatedRecords.map(_.code),
-      returnType)
-
-    val inputFormat = new ValuesInputFormat[Any](
-      generatedFunction.name,
-      generatedFunction.code,
-      generatedFunction.returnType)
-
-    tableEnv.execEnv.createInput(inputFormat, returnType).asInstanceOf[DataSet[Any]]
-  }
-
-  private def valuesFieldsToString: String = {
-    getRowType.getFieldNames.asScala.toList.mkString(", ")
-  }
-
-}
-
-


[35/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/ProjectionTranslator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/ProjectionTranslator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/ProjectionTranslator.scala
deleted file mode 100644
index a25c402..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/ProjectionTranslator.scala
+++ /dev/null
@@ -1,263 +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.api.table.plan
-
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.api.table.expressions._
-import org.apache.flink.api.table.plan.logical.{LogicalNode, Project}
-
-import scala.collection.mutable.ListBuffer
-
-object ProjectionTranslator {
-
-  /**
-    * Extracts and deduplicates all aggregation and window property expressions (zero, one, or more)
-    * from the given expressions.
-    *
-    * @param exprs    a list of expressions to extract
-    * @param tableEnv the TableEnvironment
-    * @return a Tuple2, the first field contains the extracted and deduplicated aggregations,
-    *         and the second field contains the extracted and deduplicated window properties.
-    */
-  def extractAggregationsAndProperties(
-      exprs: Seq[Expression],
-      tableEnv: TableEnvironment): (Map[Expression, String], Map[Expression, String]) = {
-    exprs.foldLeft((Map[Expression, String](), Map[Expression, String]())) {
-      (x, y) => identifyAggregationsAndProperties(y, tableEnv, x._1, x._2)
-    }
-  }
-
-  /** Identifies and deduplicates aggregation functions and window properties. */
-  private def identifyAggregationsAndProperties(
-      exp: Expression,
-      tableEnv: TableEnvironment,
-      aggNames: Map[Expression, String],
-      propNames: Map[Expression, String]) : (Map[Expression, String], Map[Expression, String]) = {
-
-    exp match {
-      case agg: Aggregation =>
-        if (aggNames contains agg) {
-          (aggNames, propNames)
-        } else {
-          (aggNames + (agg -> tableEnv.createUniqueAttributeName()), propNames)
-        }
-      case prop: WindowProperty =>
-        if (propNames contains prop) {
-          (aggNames, propNames)
-        } else {
-          (aggNames, propNames + (prop -> tableEnv.createUniqueAttributeName()))
-        }
-      case l: LeafExpression =>
-        (aggNames, propNames)
-      case u: UnaryExpression =>
-        identifyAggregationsAndProperties(u.child, tableEnv, aggNames, propNames)
-      case b: BinaryExpression =>
-        val l = identifyAggregationsAndProperties(b.left, tableEnv, aggNames, propNames)
-        identifyAggregationsAndProperties(b.right, tableEnv, l._1, l._2)
-
-      // Functions calls
-      case c @ Call(name, args) =>
-        args.foldLeft((aggNames, propNames)){
-          (x, y) => identifyAggregationsAndProperties(y, tableEnv, x._1, x._2)
-        }
-
-      case sfc @ ScalarFunctionCall(clazz, args) =>
-        args.foldLeft((aggNames, propNames)){
-          (x, y) => identifyAggregationsAndProperties(y, tableEnv, x._1, x._2)
-        }
-
-      // General expression
-      case e: Expression =>
-        e.productIterator.foldLeft((aggNames, propNames)){
-          (x, y) => y match {
-            case e: Expression => identifyAggregationsAndProperties(e, tableEnv, x._1, x._2)
-            case _ => (x._1, x._2)
-          }
-        }
-    }
-  }
-
-  /**
-    * Replaces expressions with deduplicated aggregations and properties.
-    *
-    * @param exprs     a list of expressions to replace
-    * @param tableEnv  the TableEnvironment
-    * @param aggNames  the deduplicated aggregations
-    * @param propNames the deduplicated properties
-    * @return a list of replaced expressions
-    */
-  def replaceAggregationsAndProperties(
-      exprs: Seq[Expression],
-      tableEnv: TableEnvironment,
-      aggNames: Map[Expression, String],
-      propNames: Map[Expression, String]): Seq[NamedExpression] = {
-    exprs.map(replaceAggregationsAndProperties(_, tableEnv, aggNames, propNames))
-        .map(UnresolvedAlias)
-  }
-
-  private def replaceAggregationsAndProperties(
-      exp: Expression,
-      tableEnv: TableEnvironment,
-      aggNames: Map[Expression, String],
-      propNames: Map[Expression, String]) : Expression = {
-
-    exp match {
-      case agg: Aggregation =>
-        val name = aggNames(agg)
-        Alias(UnresolvedFieldReference(name), tableEnv.createUniqueAttributeName())
-      case prop: WindowProperty =>
-        val name = propNames(prop)
-        Alias(UnresolvedFieldReference(name), tableEnv.createUniqueAttributeName())
-      case n @ Alias(agg: Aggregation, name, _) =>
-        val aName = aggNames(agg)
-        Alias(UnresolvedFieldReference(aName), name)
-      case n @ Alias(prop: WindowProperty, name, _) =>
-        val pName = propNames(prop)
-        Alias(UnresolvedFieldReference(pName), name)
-      case l: LeafExpression => l
-      case u: UnaryExpression =>
-        val c = replaceAggregationsAndProperties(u.child, tableEnv, aggNames, propNames)
-        u.makeCopy(Array(c))
-      case b: BinaryExpression =>
-        val l = replaceAggregationsAndProperties(b.left, tableEnv, aggNames, propNames)
-        val r = replaceAggregationsAndProperties(b.right, tableEnv, aggNames, propNames)
-        b.makeCopy(Array(l, r))
-
-      // Functions calls
-      case c @ Call(name, args) =>
-        val newArgs = args.map(replaceAggregationsAndProperties(_, tableEnv, aggNames, propNames))
-        c.makeCopy(Array(name, newArgs))
-
-      case sfc @ ScalarFunctionCall(clazz, args) =>
-        val newArgs: Seq[Expression] = args
-          .map(replaceAggregationsAndProperties(_, tableEnv, aggNames, propNames))
-        sfc.makeCopy(Array(clazz, newArgs))
-
-      // array constructor
-      case c @ ArrayConstructor(args) =>
-        val newArgs = c.elements
-          .map(replaceAggregationsAndProperties(_, tableEnv, aggNames, propNames))
-        c.makeCopy(Array(newArgs))
-
-      // General expression
-      case e: Expression =>
-        val newArgs = e.productIterator.map {
-          case arg: Expression =>
-            replaceAggregationsAndProperties(arg, tableEnv, aggNames, propNames)
-        }
-        e.makeCopy(newArgs.toArray)
-    }
-  }
-
-  /**
-    * Expands an UnresolvedFieldReference("*") to parent's full project list.
-    */
-  def expandProjectList(
-      exprs: Seq[Expression],
-      parent: LogicalNode,
-      tableEnv: TableEnvironment)
-    : Seq[Expression] = {
-
-    val projectList = new ListBuffer[Expression]
-
-    exprs.foreach {
-      case n: UnresolvedFieldReference if n.name == "*" =>
-        projectList ++= parent.output.map(a => UnresolvedFieldReference(a.name))
-
-      case Flattening(unresolved) =>
-        // simulate a simple project to resolve fields using current parent
-        val project = Project(Seq(UnresolvedAlias(unresolved)), parent).validate(tableEnv)
-        val resolvedExpr = project
-          .output
-          .headOption
-          .getOrElse(throw new RuntimeException("Could not find resolved composite."))
-        resolvedExpr.validateInput()
-        val newProjects = resolvedExpr.resultType match {
-          case ct: CompositeType[_] =>
-            (0 until ct.getArity).map { idx =>
-              projectList += GetCompositeField(unresolved, ct.getFieldNames()(idx))
-            }
-          case _ =>
-            projectList += unresolved
-        }
-
-      case e: Expression => projectList += e
-    }
-    projectList
-  }
-
-  /**
-    * Extract all field references from the given expressions.
-    *
-    * @param exprs a list of expressions to extract
-    * @return a list of field references extracted from the given expressions
-    */
-  def extractFieldReferences(exprs: Seq[Expression]): Seq[NamedExpression] = {
-    exprs.foldLeft(Set[NamedExpression]()) {
-      (fieldReferences, expr) => identifyFieldReferences(expr, fieldReferences)
-    }.toSeq
-  }
-
-  private def identifyFieldReferences(
-      expr: Expression,
-      fieldReferences: Set[NamedExpression]): Set[NamedExpression] = expr match {
-
-    case f: UnresolvedFieldReference =>
-      fieldReferences + UnresolvedAlias(f)
-
-    case b: BinaryExpression =>
-      val l = identifyFieldReferences(b.left, fieldReferences)
-      identifyFieldReferences(b.right, l)
-
-    // Functions calls
-    case c @ Call(name, args) =>
-      args.foldLeft(fieldReferences) {
-        (fieldReferences, expr) => identifyFieldReferences(expr, fieldReferences)
-      }
-    case sfc @ ScalarFunctionCall(clazz, args) =>
-      args.foldLeft(fieldReferences) {
-        (fieldReferences, expr) => identifyFieldReferences(expr, fieldReferences)
-      }
-
-    // array constructor
-    case c @ ArrayConstructor(args) =>
-      args.foldLeft(fieldReferences) {
-        (fieldReferences, expr) => identifyFieldReferences(expr, fieldReferences)
-      }
-
-    // ignore fields from window property
-    case w : WindowProperty =>
-      fieldReferences
-
-    // keep this case after all unwanted unary expressions
-    case u: UnaryExpression =>
-      identifyFieldReferences(u.child, fieldReferences)
-
-    // General expression
-    case e: Expression =>
-      e.productIterator.foldLeft(fieldReferences) {
-        (fieldReferences, expr) => expr match {
-          case e: Expression => identifyFieldReferences(e, fieldReferences)
-          case _ => fieldReferences
-        }
-      }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/cost/DataSetCost.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/cost/DataSetCost.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/cost/DataSetCost.scala
deleted file mode 100644
index 58537dd..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/cost/DataSetCost.scala
+++ /dev/null
@@ -1,150 +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.api.table.plan.cost
-
-import org.apache.calcite.plan.{RelOptUtil, RelOptCostFactory, RelOptCost}
-import org.apache.calcite.util.Util
-
-/**
-  * This class is based on Apache Calcite's `org.apache.calcite.plan.volcano.VolcanoCost` and has
-  * an adapted cost comparison method `isLe(other: RelOptCost)` that takes io and cpu into account.
-  */
-class DataSetCost(val rowCount: Double, val cpu: Double, val io: Double) extends RelOptCost {
-
-  def getCpu: Double = cpu
-
-  def isInfinite: Boolean = {
-    (this eq DataSetCost.Infinity) ||
-      (this.rowCount == Double.PositiveInfinity) ||
-      (this.cpu == Double.PositiveInfinity) ||
-      (this.io == Double.PositiveInfinity)
-  }
-
-  def getIo: Double = io
-
-  def isLe(other: RelOptCost): Boolean = {
-    val that: DataSetCost = other.asInstanceOf[DataSetCost]
-    (this eq that) ||
-      (this.io < that.io) ||
-      (this.io == that.io && this.cpu < that.cpu) ||
-      (this.io == that.io && this.cpu == that.cpu && this.rowCount < that.rowCount)
-  }
-
-  def isLt(other: RelOptCost): Boolean = {
-    isLe(other) && !(this == other)
-  }
-
-  def getRows: Double = rowCount
-
-  override def hashCode: Int = Util.hashCode(rowCount) + Util.hashCode(cpu) + Util.hashCode(io)
-
-  def equals(other: RelOptCost): Boolean = {
-    (this eq other) ||
-      other.isInstanceOf[DataSetCost] &&
-        (this.rowCount == other.asInstanceOf[DataSetCost].rowCount) &&
-        (this.cpu == other.asInstanceOf[DataSetCost].cpu) &&
-        (this.io == other.asInstanceOf[DataSetCost].io)
-  }
-
-  def isEqWithEpsilon(other: RelOptCost): Boolean = {
-    if (!other.isInstanceOf[DataSetCost]) {
-      return false
-    }
-    val that: DataSetCost = other.asInstanceOf[DataSetCost]
-    (this eq that) ||
-      ((Math.abs(this.rowCount - that.rowCount) < RelOptUtil.EPSILON) &&
-        (Math.abs(this.cpu - that.cpu) < RelOptUtil.EPSILON) &&
-        (Math.abs(this.io - that.io) < RelOptUtil.EPSILON))
-  }
-
-  def minus(other: RelOptCost): RelOptCost = {
-    if (this eq DataSetCost.Infinity) {
-      return this
-    }
-    val that: DataSetCost = other.asInstanceOf[DataSetCost]
-    new DataSetCost(this.rowCount - that.rowCount, this.cpu - that.cpu, this.io - that.io)
-  }
-
-  def multiplyBy(factor: Double): RelOptCost = {
-    if (this eq DataSetCost.Infinity) {
-      return this
-    }
-    new DataSetCost(rowCount * factor, cpu * factor, io * factor)
-  }
-
-  def divideBy(cost: RelOptCost): Double = {
-    val that: DataSetCost = cost.asInstanceOf[DataSetCost]
-    var d: Double = 1
-    var n: Double = 0
-    if ((this.rowCount != 0) && !this.rowCount.isInfinite &&
-      (that.rowCount != 0) && !that.rowCount.isInfinite)
-    {
-      d *= this.rowCount / that.rowCount
-      n += 1
-    }
-    if ((this.cpu != 0) && !this.cpu.isInfinite && (that.cpu != 0) && !that.cpu.isInfinite) {
-      d *= this.cpu / that.cpu
-      n += 1
-    }
-    if ((this.io != 0) && !this.io.isInfinite && (that.io != 0) && !that.io.isInfinite) {
-      d *= this.io / that.io
-      n += 1
-    }
-    if (n == 0) {
-      return 1.0
-    }
-    Math.pow(d, 1 / n)
-  }
-
-  def plus(other: RelOptCost): RelOptCost = {
-    val that: DataSetCost = other.asInstanceOf[DataSetCost]
-    if ((this eq DataSetCost.Infinity) || (that eq DataSetCost.Infinity)) {
-      return DataSetCost.Infinity
-    }
-    new DataSetCost(this.rowCount + that.rowCount, this.cpu + that.cpu, this.io + that.io)
-  }
-
-  override def toString: String = s"{$rowCount rows, $cpu cpu, $io io}"
-
-}
-
-object DataSetCost {
-
-  private[flink] val Infinity = new DataSetCost(
-    Double.PositiveInfinity,
-    Double.PositiveInfinity,
-    Double.PositiveInfinity)
-  {
-    override def toString: String = "{inf}"
-  }
-
-  private[flink] val Huge = new DataSetCost(Double.MaxValue, Double.MaxValue, Double.MaxValue) {
-    override def toString: String = "{huge}"
-  }
-
-  private[flink] val Zero = new DataSetCost(0.0, 0.0, 0.0) {
-    override def toString: String = "{0}"
-  }
-
-  private[flink] val Tiny = new DataSetCost(1.0, 1.0, 0.0) {
-    override def toString = "{tiny}"
-  }
-
-  val FACTORY: RelOptCostFactory = new DataSetCostFactory
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/cost/DataSetCostFactory.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/cost/DataSetCostFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/cost/DataSetCostFactory.scala
deleted file mode 100644
index 87d57d6..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/cost/DataSetCostFactory.scala
+++ /dev/null
@@ -1,47 +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.api.table.plan.cost
-
-import org.apache.calcite.plan.{RelOptCost, RelOptCostFactory}
-
-/**
-  * This class is based on Apache Calcite's `org.apache.calcite.plan.volcano.VolcanoCost#Factory`.
-  */
-class DataSetCostFactory extends RelOptCostFactory {
-
-  override def makeCost(dRows: Double, dCpu: Double, dIo: Double): RelOptCost = {
-    new DataSetCost(dRows, dCpu, dIo)
-  }
-
-  override def makeHugeCost: RelOptCost = {
-    DataSetCost.Huge
-  }
-
-  override def makeInfiniteCost: RelOptCost = {
-    DataSetCost.Infinity
-  }
-
-  override def makeTinyCost: RelOptCost = {
-    DataSetCost.Tiny
-  }
-
-  override def makeZeroCost: RelOptCost = {
-    DataSetCost.Zero
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalNode.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalNode.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalNode.scala
deleted file mode 100644
index 21290d4..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalNode.scala
+++ /dev/null
@@ -1,162 +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.api.table.plan.logical
-
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.table.{TableEnvironment, ValidationException}
-import org.apache.flink.api.table.expressions._
-import org.apache.flink.api.table.trees.TreeNode
-import org.apache.flink.api.table.typeutils.TypeCoercion
-import org.apache.flink.api.table.validate._
-
-/**
-  * LogicalNode is created and validated as we construct query plan using Table API.
-  *
-  * The main validation procedure is separated into two phases:
-  *
-  * Expressions' resolution and transformation ([[resolveExpressions]]):
-  *
-  * - translate [[UnresolvedFieldReference]] into [[ResolvedFieldReference]]
-  *     using child operator's output
-  * - translate [[Call]](UnresolvedFunction) into solid Expression
-  * - generate alias names for query output
-  * - ....
-  *
-  * LogicalNode validation ([[validate]]):
-  *
-  * - check no [[UnresolvedFieldReference]] exists any more
-  * - check if all expressions have children of needed type
-  * - check each logical operator have desired input
-  *
-  * Once we pass the validation phase, we can safely convert LogicalNode into Calcite's RelNode.
-  */
-abstract class LogicalNode extends TreeNode[LogicalNode] {
-  def output: Seq[Attribute]
-
-  def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = {
-    // resolve references and function calls
-    val exprResolved = expressionPostOrderTransform {
-      case u @ UnresolvedFieldReference(name) =>
-        resolveReference(tableEnv, name).getOrElse(u)
-      case c @ Call(name, children) if c.childrenValid =>
-        tableEnv.getFunctionCatalog.lookupFunction(name, children)
-    }
-
-    exprResolved.expressionPostOrderTransform {
-      case ips: InputTypeSpec if ips.childrenValid =>
-        var changed: Boolean = false
-        val newChildren = ips.expectedTypes.zip(ips.children).map { case (tpe, child) =>
-          val childType = child.resultType
-          if (childType != tpe && TypeCoercion.canSafelyCast(childType, tpe)) {
-            changed = true
-            Cast(child, tpe)
-          } else {
-            child
-          }
-        }.toArray[AnyRef]
-        if (changed) ips.makeCopy(newChildren) else ips
-    }
-  }
-
-  final def toRelNode(relBuilder: RelBuilder): RelNode = construct(relBuilder).build()
-
-  protected[logical] def construct(relBuilder: RelBuilder): RelBuilder
-
-  def validate(tableEnv: TableEnvironment): LogicalNode = {
-    val resolvedNode = resolveExpressions(tableEnv)
-    resolvedNode.expressionPostOrderTransform {
-      case a: Attribute if !a.valid =>
-        val from = children.flatMap(_.output).map(_.name).mkString(", ")
-        failValidation(s"Cannot resolve [${a.name}] given input [$from].")
-
-      case e: Expression if e.validateInput().isFailure =>
-        failValidation(s"Expression $e failed on input check: " +
-          s"${e.validateInput().asInstanceOf[ValidationFailure].message}")
-    }
-  }
-
-  /**
-    * Resolves the given strings to a [[NamedExpression]] using the input from all child
-    * nodes of this LogicalPlan.
-    */
-  def resolveReference(tableEnv: TableEnvironment, name: String): Option[NamedExpression] = {
-    val childrenOutput = children.flatMap(_.output)
-    val candidates = childrenOutput.filter(_.name.equalsIgnoreCase(name))
-    if (candidates.length > 1) {
-      failValidation(s"Reference $name is ambiguous.")
-    } else if (candidates.isEmpty) {
-      None
-    } else {
-      Some(candidates.head.withName(name))
-    }
-  }
-
-  /**
-    * Runs [[postOrderTransform]] with `rule` on all expressions present in this logical node.
-    *
-    * @param rule the rule to be applied to every expression in this logical node.
-    */
-  def expressionPostOrderTransform(rule: PartialFunction[Expression, Expression]): LogicalNode = {
-    var changed = false
-
-    def expressionPostOrderTransform(e: Expression): Expression = {
-      val newExpr = e.postOrderTransform(rule)
-      if (newExpr.fastEquals(e)) {
-        e
-      } else {
-        changed = true
-        newExpr
-      }
-    }
-
-    val newArgs = productIterator.map {
-      case e: Expression => expressionPostOrderTransform(e)
-      case Some(e: Expression) => Some(expressionPostOrderTransform(e))
-      case seq: Traversable[_] => seq.map {
-        case e: Expression => expressionPostOrderTransform(e)
-        case other => other
-      }
-      case r: Resolvable[_] => r.resolveExpressions(e => expressionPostOrderTransform(e))
-      case other: AnyRef => other
-    }.toArray
-
-    if (changed) makeCopy(newArgs) else this
-  }
-
-  protected def failValidation(msg: String): Nothing = {
-    throw new ValidationException(msg)
-  }
-}
-
-abstract class LeafNode extends LogicalNode {
-  override def children: Seq[LogicalNode] = Nil
-}
-
-abstract class UnaryNode extends LogicalNode {
-  def child: LogicalNode
-
-  override def children: Seq[LogicalNode] = child :: Nil
-}
-
-abstract class BinaryNode extends LogicalNode {
-  def left: LogicalNode
-  def right: LogicalNode
-
-  override def children: Seq[LogicalNode] = left :: right :: Nil
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalWindow.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalWindow.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalWindow.scala
deleted file mode 100644
index 19fd603..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/LogicalWindow.scala
+++ /dev/null
@@ -1,36 +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.api.table.plan.logical
-
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.api.table.expressions.{Expression, WindowReference}
-import org.apache.flink.api.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
-
-abstract class LogicalWindow(val alias: Option[Expression]) extends Resolvable[LogicalWindow] {
-
-  def resolveExpressions(resolver: (Expression) => Expression): LogicalWindow = this
-
-  def validate(tableEnv: TableEnvironment): ValidationResult = alias match {
-    case Some(WindowReference(_)) => ValidationSuccess
-    case Some(_) => ValidationFailure("Window reference for window expected.")
-    case None => ValidationSuccess
-  }
-
-  override def toString: String = getClass.getSimpleName
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/Resolvable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/Resolvable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/Resolvable.scala
deleted file mode 100644
index 7540d43..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/Resolvable.scala
+++ /dev/null
@@ -1,43 +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.api.table.plan.logical
-
-import org.apache.flink.api.table.expressions.Expression
-
-/**
-  * A class implementing this interface can resolve the expressions of its parameters and
-  * return a new instance with resolved parameters. This is necessary if expression are nested in
-  * a not supported structure. By default, the validation of a logical node can resolve common
-  * structures like `Expression`, `Option[Expression]`, `Traversable[Expression]`.
-  *
-  * See also [[LogicalNode.expressionPostOrderTransform(scala.PartialFunction)]].
-  *
-  * @tparam T class which expression parameters need to be resolved
-  */
-trait Resolvable[T <: AnyRef] {
-
-  /**
-    * An implementing class can resolve its expressions by applying the given resolver
-    * function on its parameters.
-    *
-    * @param resolver function that can resolve an expression
-    * @return class with resolved expression parameters
-    */
-  def resolveExpressions(resolver: (Expression) => Expression): T
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/groupWindows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/groupWindows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/groupWindows.scala
deleted file mode 100644
index aeb9676..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/groupWindows.scala
+++ /dev/null
@@ -1,258 +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.api.table.plan.logical
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.api.table.{BatchTableEnvironment, StreamTableEnvironment, TableEnvironment}
-import org.apache.flink.api.table.expressions._
-import org.apache.flink.api.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo, TypeCoercion}
-import org.apache.flink.api.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
-
-abstract class EventTimeGroupWindow(
-    name: Option[Expression],
-    time: Expression)
-  extends LogicalWindow(name) {
-
-  override def validate(tableEnv: TableEnvironment): ValidationResult = {
-    val valid = super.validate(tableEnv)
-    if (valid.isFailure) {
-        return valid
-    }
-
-    tableEnv match {
-      case _: StreamTableEnvironment =>
-        time match {
-          case RowtimeAttribute() =>
-            ValidationSuccess
-          case _ =>
-            ValidationFailure("Event-time window expects a 'rowtime' time field.")
-      }
-      case _: BatchTableEnvironment =>
-        if (!TypeCoercion.canCast(time.resultType, BasicTypeInfo.LONG_TYPE_INFO)) {
-          ValidationFailure(s"Event-time window expects a time field that can be safely cast " +
-            s"to Long, but is ${time.resultType}")
-        } else {
-          ValidationSuccess
-        }
-    }
-
-  }
-}
-
-abstract class ProcessingTimeGroupWindow(name: Option[Expression]) extends LogicalWindow(name)
-
-// ------------------------------------------------------------------------------------------------
-// Tumbling group windows
-// ------------------------------------------------------------------------------------------------
-
-object TumblingGroupWindow {
-  def validate(tableEnv: TableEnvironment, size: Expression): ValidationResult = size match {
-    case Literal(_, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-      ValidationSuccess
-    case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
-      ValidationSuccess
-    case _ =>
-      ValidationFailure("Tumbling window expects size literal of type Interval of Milliseconds " +
-        "or Interval of Rows.")
-  }
-}
-
-case class ProcessingTimeTumblingGroupWindow(
-    name: Option[Expression],
-    size: Expression)
-  extends ProcessingTimeGroupWindow(name) {
-
-  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    ProcessingTimeTumblingGroupWindow(
-      name.map(resolve),
-      resolve(size))
-
-  override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv).orElse(TumblingGroupWindow.validate(tableEnv, size))
-
-  override def toString: String = s"ProcessingTimeTumblingGroupWindow($name, $size)"
-}
-
-case class EventTimeTumblingGroupWindow(
-    name: Option[Expression],
-    timeField: Expression,
-    size: Expression)
-  extends EventTimeGroupWindow(
-    name,
-    timeField) {
-
-  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    EventTimeTumblingGroupWindow(
-      name.map(resolve),
-      resolve(timeField),
-      resolve(size))
-
-  override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv)
-      .orElse(TumblingGroupWindow.validate(tableEnv, size))
-      .orElse(size match {
-        case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
-          ValidationFailure(
-            "Event-time grouping windows on row intervals are currently not supported.")
-        case _ =>
-          ValidationSuccess
-      })
-
-  override def toString: String = s"EventTimeTumblingGroupWindow($name, $timeField, $size)"
-}
-
-// ------------------------------------------------------------------------------------------------
-// Sliding group windows
-// ------------------------------------------------------------------------------------------------
-
-object SlidingGroupWindow {
-  def validate(
-      tableEnv: TableEnvironment,
-      size: Expression,
-      slide: Expression)
-    : ValidationResult = {
-
-    val checkedSize = size match {
-      case Literal(_, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-        ValidationSuccess
-      case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
-        ValidationSuccess
-      case _ =>
-        ValidationFailure("Sliding window expects size literal of type Interval of " +
-          "Milliseconds or Interval of Rows.")
-    }
-
-    val checkedSlide = slide match {
-      case Literal(_, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-        ValidationSuccess
-      case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
-        ValidationSuccess
-      case _ =>
-        ValidationFailure("Sliding window expects slide literal of type Interval of " +
-          "Milliseconds or Interval of Rows.")
-    }
-
-    checkedSize
-      .orElse(checkedSlide)
-      .orElse {
-        if (size.resultType != slide.resultType) {
-          ValidationFailure("Sliding window expects same type of size and slide.")
-        } else {
-          ValidationSuccess
-        }
-      }
-  }
-}
-
-case class ProcessingTimeSlidingGroupWindow(
-    name: Option[Expression],
-    size: Expression,
-    slide: Expression)
-  extends ProcessingTimeGroupWindow(name) {
-
-  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    ProcessingTimeSlidingGroupWindow(
-      name.map(resolve),
-      resolve(size),
-      resolve(slide))
-
-  override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv).orElse(SlidingGroupWindow.validate(tableEnv, size, slide))
-
-  override def toString: String = s"ProcessingTimeSlidingGroupWindow($name, $size, $slide)"
-}
-
-case class EventTimeSlidingGroupWindow(
-    name: Option[Expression],
-    timeField: Expression,
-    size: Expression,
-    slide: Expression)
-  extends EventTimeGroupWindow(name, timeField) {
-
-  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    EventTimeSlidingGroupWindow(
-      name.map(resolve),
-      resolve(timeField),
-      resolve(size),
-      resolve(slide))
-
-  override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv)
-      .orElse(SlidingGroupWindow.validate(tableEnv, size, slide))
-      .orElse(size match {
-        case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
-          ValidationFailure(
-            "Event-time grouping windows on row intervals are currently not supported.")
-        case _ =>
-          ValidationSuccess
-      })
-
-  override def toString: String = s"EventTimeSlidingGroupWindow($name, $timeField, $size, $slide)"
-}
-
-// ------------------------------------------------------------------------------------------------
-// Session group windows
-// ------------------------------------------------------------------------------------------------
-
-object SessionGroupWindow {
-
-  def validate(tableEnv: TableEnvironment, gap: Expression): ValidationResult = gap match {
-    case Literal(timeInterval: Long, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-      ValidationSuccess
-    case _ =>
-      ValidationFailure(
-        "Session window expects gap literal of type Interval of Milliseconds.")
-  }
-}
-
-case class ProcessingTimeSessionGroupWindow(
-    name: Option[Expression],
-    gap: Expression)
-  extends ProcessingTimeGroupWindow(name) {
-
-  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    ProcessingTimeSessionGroupWindow(
-      name.map(resolve),
-      resolve(gap))
-
-  override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv).orElse(SessionGroupWindow.validate(tableEnv, gap))
-
-  override def toString: String = s"ProcessingTimeSessionGroupWindow($name, $gap)"
-}
-
-case class EventTimeSessionGroupWindow(
-    name: Option[Expression],
-    timeField: Expression,
-    gap: Expression)
-  extends EventTimeGroupWindow(
-    name,
-    timeField) {
-
-  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
-    EventTimeSessionGroupWindow(
-      name.map(resolve),
-      resolve(timeField),
-      resolve(gap))
-
-  override def validate(tableEnv: TableEnvironment): ValidationResult =
-    super.validate(tableEnv).orElse(SessionGroupWindow.validate(tableEnv, gap))
-
-  override def toString: String = s"EventTimeSessionGroupWindow($name, $timeField, $gap)"
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala
deleted file mode 100644
index 438698a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/operators.scala
+++ /dev/null
@@ -1,692 +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.api.table.plan.logical
-
-import java.lang.reflect.Method
-import java.util
-
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.CorrelationId
-import org.apache.calcite.rel.logical.{LogicalProject, LogicalTableFunctionScan}
-import org.apache.calcite.rex.{RexInputRef, RexNode}
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.operators.join.JoinType
-import org.apache.flink.api.table._
-import org.apache.flink.api.table.expressions._
-import org.apache.flink.api.table.functions.TableFunction
-import org.apache.flink.api.table.functions.utils.TableSqlFunction
-import org.apache.flink.api.table.functions.utils.UserDefinedFunctionUtils._
-import org.apache.flink.api.table.plan.schema.FlinkTableFunctionImpl
-import org.apache.flink.api.table.typeutils.TypeConverter
-import org.apache.flink.api.table.validate.{ValidationFailure, ValidationSuccess}
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-
-case class Project(projectList: Seq[NamedExpression], child: LogicalNode) extends UnaryNode {
-  override def output: Seq[Attribute] = projectList.map(_.toAttribute)
-
-  override def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = {
-    val afterResolve = super.resolveExpressions(tableEnv).asInstanceOf[Project]
-    val newProjectList =
-      afterResolve.projectList.zipWithIndex.map { case (e, i) =>
-        e match {
-          case u @ UnresolvedAlias(c) => c match {
-            case ne: NamedExpression => ne
-            case expr if !expr.valid => u
-            case c @ Cast(ne: NamedExpression, tp) => Alias(c, s"${ne.name}-$tp")
-            case gcf: GetCompositeField => Alias(gcf, gcf.aliasName().getOrElse(s"_c$i"))
-            case other => Alias(other, s"_c$i")
-          }
-          case _ =>
-            throw new RuntimeException("This should never be called and probably points to a bug.")
-        }
-    }
-    Project(newProjectList, child)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    val resolvedProject = super.validate(tableEnv).asInstanceOf[Project]
-    val names: mutable.Set[String] = mutable.Set()
-
-    def checkName(name: String): Unit = {
-      if (names.contains(name)) {
-        failValidation(s"Duplicate field name $name.")
-      } else if (tableEnv.isInstanceOf[StreamTableEnvironment] && name == "rowtime") {
-        failValidation("'rowtime' cannot be used as field name in a streaming environment.")
-      } else {
-        names.add(name)
-      }
-    }
-
-    resolvedProject.projectList.foreach {
-      case n: Alias =>
-        // explicit name
-        checkName(n.name)
-      case r: ResolvedFieldReference =>
-        // simple field forwarding
-        checkName(r.name)
-      case _ => // Do nothing
-    }
-    resolvedProject
-  }
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    val allAlias = projectList.forall(_.isInstanceOf[Alias])
-    child.construct(relBuilder)
-    if (allAlias) {
-      // Calcite's RelBuilder does not translate identity projects even if they rename fields.
-      //   Add a projection ourselves (will be automatically removed by translation rules).
-      val project = LogicalProject.create(relBuilder.peek(),
-        // avoid AS call
-        projectList.map(_.asInstanceOf[Alias].child.toRexNode(relBuilder)).asJava,
-        projectList.map(_.name).asJava)
-      relBuilder.build()  // pop previous relNode
-      relBuilder.push(project)
-    } else {
-      relBuilder.project(projectList.map(_.toRexNode(relBuilder)): _*)
-    }
-  }
-}
-
-case class AliasNode(aliasList: Seq[Expression], child: LogicalNode) extends UnaryNode {
-  override def output: Seq[Attribute] =
-    throw UnresolvedException("Invalid call to output on AliasNode")
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder =
-    throw UnresolvedException("Invalid call to toRelNode on AliasNode")
-
-  override def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = {
-    if (aliasList.length > child.output.length) {
-      failValidation("Aliasing more fields than we actually have")
-    } else if (!aliasList.forall(_.isInstanceOf[UnresolvedFieldReference])) {
-      failValidation("Alias only accept name expressions as arguments")
-    } else if (!aliasList.forall(_.asInstanceOf[UnresolvedFieldReference].name != "*")) {
-      failValidation("Alias can not accept '*' as name")
-    } else if (tableEnv.isInstanceOf[StreamTableEnvironment] && !aliasList.forall {
-          case UnresolvedFieldReference(name) => name != "rowtime"
-        }) {
-      failValidation("'rowtime' cannot be used as field name in a streaming environment.")
-    } else {
-      val names = aliasList.map(_.asInstanceOf[UnresolvedFieldReference].name)
-      val input = child.output
-      Project(
-        names.zip(input).map { case (name, attr) =>
-          Alias(attr, name)} ++ input.drop(names.length), child)
-    }
-  }
-}
-
-case class Distinct(child: LogicalNode) extends UnaryNode {
-  override def output: Seq[Attribute] = child.output
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    child.construct(relBuilder)
-    relBuilder.distinct()
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
-      failValidation(s"Distinct on stream tables is currently not supported.")
-    }
-    this
-  }
-}
-
-case class Sort(order: Seq[Ordering], child: LogicalNode) extends UnaryNode {
-  override def output: Seq[Attribute] = child.output
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    child.construct(relBuilder)
-    relBuilder.sort(order.map(_.toRexNode(relBuilder)).asJava)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
-      failValidation(s"Sort on stream tables is currently not supported.")
-    }
-    super.validate(tableEnv)
-  }
-}
-
-case class Limit(offset: Int, fetch: Int = -1, child: LogicalNode) extends UnaryNode {
-  override def output: Seq[Attribute] = child.output
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    child.construct(relBuilder)
-    relBuilder.limit(offset, fetch)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
-      failValidation(s"Limit on stream tables is currently not supported.")
-    }
-    if (!child.validate(tableEnv).isInstanceOf[Sort]) {
-      failValidation(s"Limit operator must be preceded by an OrderBy operator.")
-    }
-    if (offset < 0) {
-      failValidation(s"Offset should be greater than or equal to zero.")
-    }
-    super.validate(tableEnv)
-  }
-}
-
-case class Filter(condition: Expression, child: LogicalNode) extends UnaryNode {
-  override def output: Seq[Attribute] = child.output
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    child.construct(relBuilder)
-    relBuilder.filter(condition.toRexNode(relBuilder))
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    val resolvedFilter = super.validate(tableEnv).asInstanceOf[Filter]
-    if (resolvedFilter.condition.resultType != BOOLEAN_TYPE_INFO) {
-      failValidation(s"Filter operator requires a boolean expression as input," +
-        s" but ${resolvedFilter.condition} is of type ${resolvedFilter.condition.resultType}")
-    }
-    resolvedFilter
-  }
-}
-
-case class Aggregate(
-    groupingExpressions: Seq[Expression],
-    aggregateExpressions: Seq[NamedExpression],
-    child: LogicalNode) extends UnaryNode {
-
-  override def output: Seq[Attribute] = {
-    (groupingExpressions ++ aggregateExpressions) map {
-      case ne: NamedExpression => ne.toAttribute
-      case e => Alias(e, e.toString).toAttribute
-    }
-  }
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    child.construct(relBuilder)
-    relBuilder.aggregate(
-      relBuilder.groupKey(groupingExpressions.map(_.toRexNode(relBuilder)).asJava),
-      aggregateExpressions.map {
-        case Alias(agg: Aggregation, name, _) => agg.toAggCall(name)(relBuilder)
-        case _ => throw new RuntimeException("This should never happen.")
-      }.asJava)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
-      failValidation(s"Aggregate on stream tables is currently not supported.")
-    }
-
-    val resolvedAggregate = super.validate(tableEnv).asInstanceOf[Aggregate]
-    val groupingExprs = resolvedAggregate.groupingExpressions
-    val aggregateExprs = resolvedAggregate.aggregateExpressions
-    aggregateExprs.foreach(validateAggregateExpression)
-    groupingExprs.foreach(validateGroupingExpression)
-
-    def validateAggregateExpression(expr: Expression): Unit = expr match {
-      // check no nested aggregation exists.
-      case aggExpr: Aggregation =>
-        aggExpr.children.foreach { child =>
-          child.preOrderVisit {
-            case agg: Aggregation =>
-              failValidation(
-                "It's not allowed to use an aggregate function as " +
-                  "input of another aggregate function")
-            case _ => // OK
-          }
-        }
-      case a: Attribute if !groupingExprs.exists(_.checkEquals(a)) =>
-        failValidation(
-          s"expression '$a' is invalid because it is neither" +
-            " present in group by nor an aggregate function")
-      case e if groupingExprs.exists(_.checkEquals(e)) => // OK
-      case e => e.children.foreach(validateAggregateExpression)
-    }
-
-    def validateGroupingExpression(expr: Expression): Unit = {
-      if (!expr.resultType.isKeyType) {
-        failValidation(
-          s"expression $expr cannot be used as a grouping expression " +
-            "because it's not a valid key type which must be hashable and comparable")
-      }
-    }
-    resolvedAggregate
-  }
-}
-
-case class Minus(left: LogicalNode, right: LogicalNode, all: Boolean) extends BinaryNode {
-  override def output: Seq[Attribute] = left.output
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    left.construct(relBuilder)
-    right.construct(relBuilder)
-    relBuilder.minus(all)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
-      failValidation(s"Minus on stream tables is currently not supported.")
-    }
-
-    val resolvedMinus = super.validate(tableEnv).asInstanceOf[Minus]
-    if (left.output.length != right.output.length) {
-      failValidation(s"Minus two table of different column sizes:" +
-        s" ${left.output.size} and ${right.output.size}")
-    }
-    val sameSchema = left.output.zip(right.output).forall { case (l, r) =>
-      l.resultType == r.resultType
-    }
-    if (!sameSchema) {
-      failValidation(s"Minus two table of different schema:" +
-        s" [${left.output.map(a => (a.name, a.resultType)).mkString(", ")}] and" +
-        s" [${right.output.map(a => (a.name, a.resultType)).mkString(", ")}]")
-    }
-    resolvedMinus
-  }
-}
-
-case class Union(left: LogicalNode, right: LogicalNode, all: Boolean) extends BinaryNode {
-  override def output: Seq[Attribute] = left.output
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    left.construct(relBuilder)
-    right.construct(relBuilder)
-    relBuilder.union(all)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    if (tableEnv.isInstanceOf[StreamTableEnvironment] && !all) {
-      failValidation(s"Union on stream tables is currently not supported.")
-    }
-
-    val resolvedUnion = super.validate(tableEnv).asInstanceOf[Union]
-    if (left.output.length != right.output.length) {
-      failValidation(s"Union two tables of different column sizes:" +
-        s" ${left.output.size} and ${right.output.size}")
-    }
-    val sameSchema = left.output.zip(right.output).forall { case (l, r) =>
-      l.resultType == r.resultType
-    }
-    if (!sameSchema) {
-      failValidation(s"Union two tables of different schema:" +
-        s" [${left.output.map(a => (a.name, a.resultType)).mkString(", ")}] and" +
-        s" [${right.output.map(a => (a.name, a.resultType)).mkString(", ")}]")
-    }
-    resolvedUnion
-  }
-}
-
-case class Intersect(left: LogicalNode, right: LogicalNode, all: Boolean) extends BinaryNode {
-  override def output: Seq[Attribute] = left.output
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    left.construct(relBuilder)
-    right.construct(relBuilder)
-    relBuilder.intersect(all)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
-      failValidation(s"Intersect on stream tables is currently not supported.")
-    }
-
-    val resolvedIntersect = super.validate(tableEnv).asInstanceOf[Intersect]
-    if (left.output.length != right.output.length) {
-      failValidation(s"Intersect two tables of different column sizes:" +
-        s" ${left.output.size} and ${right.output.size}")
-    }
-    // allow different column names between tables
-    val sameSchema = left.output.zip(right.output).forall { case (l, r) =>
-      l.resultType == r.resultType
-    }
-    if (!sameSchema) {
-      failValidation(s"Intersect two tables of different schema:" +
-        s" [${left.output.map(a => (a.name, a.resultType)).mkString(", ")}] and" +
-        s" [${right.output.map(a => (a.name, a.resultType)).mkString(", ")}]")
-    }
-    resolvedIntersect
-  }
-}
-
-case class Join(
-    left: LogicalNode,
-    right: LogicalNode,
-    joinType: JoinType,
-    condition: Option[Expression],
-    correlated: Boolean) extends BinaryNode {
-
-  override def output: Seq[Attribute] = {
-    left.output ++ right.output
-  }
-
-  private case class JoinFieldReference(
-    name: String,
-    resultType: TypeInformation[_],
-    left: LogicalNode,
-    right: LogicalNode) extends Attribute {
-
-    val isFromLeftInput = left.output.map(_.name).contains(name)
-
-    val (indexInInput, indexInJoin) = if (isFromLeftInput) {
-      val indexInLeft = left.output.map(_.name).indexOf(name)
-      (indexInLeft, indexInLeft)
-    } else {
-      val indexInRight = right.output.map(_.name).indexOf(name)
-      (indexInRight, indexInRight + left.output.length)
-    }
-
-    override def toString = s"'$name"
-
-    override def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-      // look up type of field
-      val fieldType = relBuilder.field(2, if (isFromLeftInput) 0 else 1, name).getType
-      // create a new RexInputRef with index offset
-      new RexInputRef(indexInJoin, fieldType)
-    }
-
-    override def withName(newName: String): Attribute = {
-      if (newName == name) {
-        this
-      } else {
-        JoinFieldReference(newName, resultType, left, right)
-      }
-    }
-  }
-
-  override def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = {
-    val node = super.resolveExpressions(tableEnv).asInstanceOf[Join]
-    val partialFunction: PartialFunction[Expression, Expression] = {
-      case field: ResolvedFieldReference => JoinFieldReference(
-        field.name,
-        field.resultType,
-        left,
-        right)
-    }
-    val resolvedCondition = node.condition.map(_.postOrderTransform(partialFunction))
-    Join(node.left, node.right, node.joinType, resolvedCondition, correlated)
-  }
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    left.construct(relBuilder)
-    right.construct(relBuilder)
-
-    val corSet = mutable.Set[CorrelationId]()
-    if (correlated) {
-      corSet += relBuilder.peek().getCluster.createCorrel()
-    }
-
-    relBuilder.join(
-      TypeConverter.flinkJoinTypeToRelType(joinType),
-      condition.map(_.toRexNode(relBuilder)).getOrElse(relBuilder.literal(true)),
-      corSet.asJava)
-  }
-
-  private def ambiguousName: Set[String] =
-    left.output.map(_.name).toSet.intersect(right.output.map(_.name).toSet)
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    if (tableEnv.isInstanceOf[StreamTableEnvironment]
-      && !right.isInstanceOf[LogicalTableFunctionCall]) {
-      failValidation(s"Join on stream tables is currently not supported.")
-    }
-
-    val resolvedJoin = super.validate(tableEnv).asInstanceOf[Join]
-    if (!resolvedJoin.condition.forall(_.resultType == BOOLEAN_TYPE_INFO)) {
-      failValidation(s"Filter operator requires a boolean expression as input, " + 
-        s"but ${resolvedJoin.condition} is of type ${resolvedJoin.joinType}")
-    } else if (ambiguousName.nonEmpty) {
-      failValidation(s"join relations with ambiguous names: ${ambiguousName.mkString(", ")}")
-    }
-
-    resolvedJoin.condition.foreach(testJoinCondition)
-    resolvedJoin
-  }
-
-  private def testJoinCondition(expression: Expression): Unit = {
-
-    def checkIfJoinCondition(exp : BinaryComparison) = exp.children match {
-        case (x : JoinFieldReference) :: (y : JoinFieldReference) :: Nil
-          if x.isFromLeftInput != y.isFromLeftInput => Unit
-        case x => failValidation(
-          s"Invalid non-join predicate $exp. For non-join predicates use Table#where.")
-      }
-
-    var equiJoinFound = false
-    def validateConditions(exp: Expression, isAndBranch: Boolean): Unit = exp match {
-      case x: And => x.children.foreach(validateConditions(_, isAndBranch))
-      case x: Or => x.children.foreach(validateConditions(_, isAndBranch = false))
-      case x: EqualTo =>
-        if (isAndBranch) {
-          equiJoinFound = true
-        }
-        checkIfJoinCondition(x)
-      case x: BinaryComparison => checkIfJoinCondition(x)
-      case x => failValidation(
-        s"Unsupported condition type: ${x.getClass.getSimpleName}. Condition: $x")
-    }
-
-    validateConditions(expression, isAndBranch = true)
-    if (!equiJoinFound) {
-      failValidation(s"Invalid join condition: $expression. At least one equi-join required.")
-    }
-  }
-}
-
-case class CatalogNode(
-    tableName: String,
-    rowType: RelDataType) extends LeafNode {
-
-  val output: Seq[Attribute] = rowType.getFieldList.asScala.map { field =>
-    ResolvedFieldReference(field.getName, FlinkTypeFactory.toTypeInfo(field.getType))
-  }
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    relBuilder.scan(tableName)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = this
-}
-
-/**
-  * Wrapper for valid logical plans generated from SQL String.
-  */
-case class LogicalRelNode(
-    relNode: RelNode) extends LeafNode {
-
-  val output: Seq[Attribute] = relNode.getRowType.getFieldList.asScala.map { field =>
-    ResolvedFieldReference(field.getName, FlinkTypeFactory.toTypeInfo(field.getType))
-  }
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    relBuilder.push(relNode)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = this
-}
-
-case class WindowAggregate(
-    groupingExpressions: Seq[Expression],
-    window: LogicalWindow,
-    propertyExpressions: Seq[NamedExpression],
-    aggregateExpressions: Seq[NamedExpression],
-    child: LogicalNode)
-  extends UnaryNode {
-
-  override def output: Seq[Attribute] = {
-    (groupingExpressions ++ aggregateExpressions ++ propertyExpressions) map {
-      case ne: NamedExpression => ne.toAttribute
-      case e => Alias(e, e.toString).toAttribute
-    }
-  }
-
-  // resolve references of this operator's parameters
-  override def resolveReference(
-      tableEnv: TableEnvironment,
-      name: String)
-    : Option[NamedExpression] = tableEnv match {
-    // resolve reference to rowtime attribute in a streaming environment
-    case _: StreamTableEnvironment if name == "rowtime" =>
-      Some(RowtimeAttribute())
-    case _ =>
-      window.alias match {
-        // resolve reference to this window's alias
-        case Some(UnresolvedFieldReference(alias)) if name == alias =>
-          // check if reference can already be resolved by input fields
-          val found = super.resolveReference(tableEnv, name)
-          if (found.isDefined) {
-            failValidation(s"Reference $name is ambiguous.")
-          } else {
-            Some(WindowReference(name))
-          }
-        case _ =>
-          // resolve references as usual
-          super.resolveReference(tableEnv, name)
-      }
-  }
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    val flinkRelBuilder = relBuilder.asInstanceOf[FlinkRelBuilder]
-    child.construct(flinkRelBuilder)
-    flinkRelBuilder.aggregate(
-      window,
-      relBuilder.groupKey(groupingExpressions.map(_.toRexNode(relBuilder)).asJava),
-      propertyExpressions.map {
-        case Alias(prop: WindowProperty, name, _) => prop.toNamedWindowProperty(name)(relBuilder)
-        case _ => throw new RuntimeException("This should never happen.")
-      },
-      aggregateExpressions.map {
-        case Alias(agg: Aggregation, name, _) => agg.toAggCall(name)(relBuilder)
-        case _ => throw new RuntimeException("This should never happen.")
-      }.asJava)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    val resolvedWindowAggregate = super.validate(tableEnv).asInstanceOf[WindowAggregate]
-    val groupingExprs = resolvedWindowAggregate.groupingExpressions
-    val aggregateExprs = resolvedWindowAggregate.aggregateExpressions
-    aggregateExprs.foreach(validateAggregateExpression)
-    groupingExprs.foreach(validateGroupingExpression)
-
-    def validateAggregateExpression(expr: Expression): Unit = expr match {
-      // check no nested aggregation exists.
-      case aggExpr: Aggregation =>
-        aggExpr.children.foreach { child =>
-          child.preOrderVisit {
-            case agg: Aggregation =>
-              failValidation(
-                "It's not allowed to use an aggregate function as " +
-                  "input of another aggregate function")
-            case _ => // ok
-          }
-        }
-      case a: Attribute if !groupingExprs.exists(_.checkEquals(a)) =>
-        failValidation(
-          s"Expression '$a' is invalid because it is neither" +
-            " present in group by nor an aggregate function")
-      case e if groupingExprs.exists(_.checkEquals(e)) => // ok
-      case e => e.children.foreach(validateAggregateExpression)
-    }
-
-    def validateGroupingExpression(expr: Expression): Unit = {
-      if (!expr.resultType.isKeyType) {
-        failValidation(
-          s"Expression $expr cannot be used as a grouping expression " +
-            "because it's not a valid key type which must be hashable and comparable")
-      }
-    }
-
-    // validate window
-    resolvedWindowAggregate.window.validate(tableEnv) match {
-      case ValidationFailure(msg) =>
-        failValidation(s"$window is invalid: $msg")
-      case ValidationSuccess => // ok
-    }
-
-    resolvedWindowAggregate
-  }
-}
-
-/**
-  * LogicalNode for calling a user-defined table functions.
-  *
-  * @param functionName function name
-  * @param tableFunction table function to be called (might be overloaded)
-  * @param parameters actual parameters
-  * @param fieldNames output field names
-  * @param child child logical node
-  */
-case class LogicalTableFunctionCall(
-    functionName: String,
-    tableFunction: TableFunction[_],
-    parameters: Seq[Expression],
-    resultType: TypeInformation[_],
-    fieldNames: Array[String],
-    child: LogicalNode)
-  extends UnaryNode {
-
-  private val (_, fieldIndexes, fieldTypes) = getFieldInfo(resultType)
-  private var evalMethod: Method = _
-
-  override def output: Seq[Attribute] = fieldNames.zip(fieldTypes).map {
-    case (n, t) => ResolvedFieldReference(n, t)
-  }
-
-  override def validate(tableEnv: TableEnvironment): LogicalNode = {
-    val node = super.validate(tableEnv).asInstanceOf[LogicalTableFunctionCall]
-    // check if not Scala object
-    checkNotSingleton(tableFunction.getClass)
-    // check if class could be instantiated
-    checkForInstantiation(tableFunction.getClass)
-    // look for a signature that matches the input types
-    val signature = node.parameters.map(_.resultType)
-    val foundMethod = getEvalMethod(tableFunction, signature)
-    if (foundMethod.isEmpty) {
-      failValidation(
-        s"Given parameters of function '$functionName' do not match any signature. \n" +
-          s"Actual: ${signatureToString(signature)} \n" +
-          s"Expected: ${signaturesToString(tableFunction)}")
-    } else {
-      node.evalMethod = foundMethod.get
-    }
-    node
-  }
-
-  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
-    val fieldIndexes = getFieldInfo(resultType)._2
-    val function = new FlinkTableFunctionImpl(resultType, fieldIndexes, fieldNames, evalMethod)
-    val typeFactory = relBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-    val sqlFunction = TableSqlFunction(
-      tableFunction.toString,
-      tableFunction,
-      resultType,
-      typeFactory,
-      function)
-
-    val scan = LogicalTableFunctionScan.create(
-      relBuilder.peek().getCluster,
-      new util.ArrayList[RelNode](),
-      relBuilder.call(sqlFunction, parameters.map(_.toRexNode(relBuilder)).asJava),
-      function.getElementType(null),
-      function.getRowType(relBuilder.getTypeFactory, null),
-      null)
-
-    relBuilder.push(scan)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/rel/LogicalWindowAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/rel/LogicalWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/rel/LogicalWindowAggregate.scala
deleted file mode 100644
index 9615168..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/logical/rel/LogicalWindowAggregate.scala
+++ /dev/null
@@ -1,114 +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.api.table.plan.logical.rel
-
-import java.util
-
-import org.apache.calcite.plan.{Convention, RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.{Aggregate, AggregateCall}
-import org.apache.calcite.rel.{RelNode, RelShuttle}
-import org.apache.calcite.util.ImmutableBitSet
-import org.apache.flink.api.table.FlinkRelBuilder.NamedWindowProperty
-import org.apache.flink.api.table.FlinkTypeFactory
-import org.apache.flink.api.table.plan.logical.LogicalWindow
-
-class LogicalWindowAggregate(
-    window: LogicalWindow,
-    namedProperties: Seq[NamedWindowProperty],
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    child: RelNode,
-    indicator: Boolean,
-    groupSet: ImmutableBitSet,
-    groupSets: util.List[ImmutableBitSet],
-    aggCalls: util.List[AggregateCall])
-  extends Aggregate(
-    cluster,
-    traitSet,
-    child,
-    indicator,
-    groupSet,
-    groupSets,
-    aggCalls) {
-
-  def getWindow = window
-
-  def getNamedProperties = namedProperties
-
-  override def copy(
-      traitSet: RelTraitSet,
-      input: RelNode,
-      indicator: Boolean,
-      groupSet: ImmutableBitSet,
-      groupSets: util.List[ImmutableBitSet],
-      aggCalls: util.List[AggregateCall])
-    : Aggregate = {
-
-    new LogicalWindowAggregate(
-      window,
-      namedProperties,
-      cluster,
-      traitSet,
-      input,
-      indicator,
-      groupSet,
-      groupSets,
-      aggCalls)
-  }
-
-  override def accept(shuttle: RelShuttle): RelNode = shuttle.visit(this)
-
-  override def deriveRowType(): RelDataType = {
-    val aggregateRowType = super.deriveRowType()
-    val typeFactory = getCluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-    val builder = typeFactory.builder
-    builder.addAll(aggregateRowType.getFieldList)
-    namedProperties.foreach { namedProp =>
-      builder.add(
-        namedProp.name,
-        typeFactory.createTypeFromTypeInfo(namedProp.property.resultType)
-      )
-    }
-    builder.build()
-  }
-}
-
-object LogicalWindowAggregate {
-
-  def create(
-      window: LogicalWindow,
-      namedProperties: Seq[NamedWindowProperty],
-      aggregate: Aggregate)
-    : LogicalWindowAggregate = {
-
-    val cluster: RelOptCluster = aggregate.getCluster
-    val traitSet: RelTraitSet = cluster.traitSetOf(Convention.NONE)
-    new LogicalWindowAggregate(
-      window,
-      namedProperties,
-      cluster,
-      traitSet,
-      aggregate.getInput,
-      aggregate.indicator,
-      aggregate.getGroupSet,
-      aggregate.getGroupSets,
-      aggregate.getAggCallList)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkAggregate.scala
deleted file mode 100644
index 85129c4..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkAggregate.scala
+++ /dev/null
@@ -1,68 +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.api.table.plan.nodes
-
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.AggregateCall
-import org.apache.flink.api.table.FlinkRelBuilder.NamedWindowProperty
-import org.apache.flink.api.table.runtime.aggregate.AggregateUtil._
-
-import scala.collection.JavaConverters._
-
-trait FlinkAggregate {
-
-  private[flink] def groupingToString(inputType: RelDataType, grouping: Array[Int]): String = {
-
-    val inFields = inputType.getFieldNames.asScala
-    grouping.map( inFields(_) ).mkString(", ")
-  }
-
-  private[flink] def aggregationToString(
-      inputType: RelDataType,
-      grouping: Array[Int],
-      rowType: RelDataType,
-      namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-      namedProperties: Seq[NamedWindowProperty])
-    : String = {
-
-    val inFields = inputType.getFieldNames.asScala
-    val outFields = rowType.getFieldNames.asScala
-
-    val groupStrings = grouping.map( inFields(_) )
-
-    val aggs = namedAggregates.map(_.getKey)
-    val aggStrings = aggs.map( a => s"${a.getAggregation}(${
-      if (a.getArgList.size() > 0) {
-        inFields(a.getArgList.get(0))
-      } else {
-        "*"
-      }
-    })")
-
-    val propStrings = namedProperties.map(_.property.toString)
-
-    (groupStrings ++ aggStrings ++ propStrings).zip(outFields).map {
-      case (f, o) => if (f == o) {
-        f
-      } else {
-        s"$f AS $o"
-      }
-    }.mkString(", ")
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkCalc.scala
deleted file mode 100644
index d5f8010..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkCalc.scala
+++ /dev/null
@@ -1,172 +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.api.table.plan.nodes
-
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rex.{RexNode, RexProgram}
-import org.apache.flink.api.common.functions.{FlatMapFunction, RichFlatMapFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.TableConfig
-import org.apache.flink.api.table.codegen.{GeneratedFunction, CodeGenerator}
-import org.apache.flink.api.table.runtime.FlatMapRunner
-import org.apache.flink.api.table.typeutils.TypeConverter._
-
-import scala.collection.JavaConversions._
-import scala.collection.JavaConverters._
-
-trait FlinkCalc {
-
-  private[flink] def functionBody(
-    generator: CodeGenerator,
-    inputType: TypeInformation[Any],
-    rowType: RelDataType,
-    calcProgram: RexProgram,
-    config: TableConfig,
-    expectedType: Option[TypeInformation[Any]]): String = {
-
-    val returnType = determineReturnType(
-      rowType,
-      expectedType,
-      config.getNullCheck,
-      config.getEfficientTypeUsage)
-
-    val condition = calcProgram.getCondition
-    val expandedExpressions = calcProgram.getProjectList.map(
-      expr => calcProgram.expandLocalRef(expr))
-    val projection = generator.generateResultExpression(
-      returnType,
-      rowType.getFieldNames,
-      expandedExpressions)
-
-      // only projection
-      if (condition == null) {
-        s"""
-          |${projection.code}
-          |${generator.collectorTerm}.collect(${projection.resultTerm});
-          |""".stripMargin
-      }
-      else {
-        val filterCondition = generator.generateExpression(
-          calcProgram.expandLocalRef(calcProgram.getCondition))
-        // only filter
-        if (projection == null) {
-          // conversion
-          if (inputType != returnType) {
-            val conversion = generator.generateConverterResultExpression(
-              returnType,
-              rowType.getFieldNames)
-
-            s"""
-              |${filterCondition.code}
-              |if (${filterCondition.resultTerm}) {
-              |  ${conversion.code}
-              |  ${generator.collectorTerm}.collect(${conversion.resultTerm});
-              |}
-              |""".stripMargin
-          }
-          // no conversion
-          else {
-            s"""
-              |${filterCondition.code}
-              |if (${filterCondition.resultTerm}) {
-              |  ${generator.collectorTerm}.collect(${generator.input1Term});
-              |}
-              |""".stripMargin
-          }
-        }
-        // both filter and projection
-        else {
-          s"""
-            |${filterCondition.code}
-            |if (${filterCondition.resultTerm}) {
-            |  ${projection.code}
-            |  ${generator.collectorTerm}.collect(${projection.resultTerm});
-            |}
-            |""".stripMargin
-        }
-      }
-    }
-
-  private[flink] def calcMapFunction(
-      genFunction: GeneratedFunction[FlatMapFunction[Any, Any]]): RichFlatMapFunction[Any, Any] = {
-
-    new FlatMapRunner[Any, Any](
-      genFunction.name,
-      genFunction.code,
-      genFunction.returnType)
-  }
-
-  private[flink] def conditionToString(
-      calcProgram: RexProgram,
-      expression: (RexNode, List[String], Option[List[RexNode]]) => String): String = {
-
-    val cond = calcProgram.getCondition
-    val inFields = calcProgram.getInputRowType.getFieldNames.asScala.toList
-    val localExprs = calcProgram.getExprList.asScala.toList
-
-    if (cond != null) {
-      expression(cond, inFields, Some(localExprs))
-    } else {
-      ""
-    }
-  }
-
-  private[flink] def selectionToString(
-      calcProgram: RexProgram,
-      expression: (RexNode, List[String], Option[List[RexNode]]) => String): String = {
-
-    val proj = calcProgram.getProjectList.asScala.toList
-    val inFields = calcProgram.getInputRowType.getFieldNames.asScala.toList
-    val localExprs = calcProgram.getExprList.asScala.toList
-    val outFields = calcProgram.getOutputRowType.getFieldNames.asScala.toList
-
-    proj
-      .map(expression(_, inFields, Some(localExprs)))
-      .zip(outFields).map { case (e, o) => {
-      if (e != o) {
-        e + " AS " + o
-      } else {
-        e
-      }
-    }
-    }.mkString(", ")
-  }
-
-  private[flink] def calcOpName(
-      calcProgram: RexProgram,
-      expression: (RexNode, List[String], Option[List[RexNode]]) => String) = {
-
-    val conditionStr = conditionToString(calcProgram, expression)
-    val selectionStr = selectionToString(calcProgram, expression)
-
-    s"${if (calcProgram.getCondition != null) {
-      s"where: ($conditionStr), "
-    } else {
-      ""
-    }}select: ($selectionStr)"
-  }
-
-  private[flink] def calcToString(
-      calcProgram: RexProgram,
-      expression: (RexNode, List[String], Option[List[RexNode]]) => String) = {
-
-    val name = calcOpName(calcProgram, expression)
-    s"Calc($name)"
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkCorrelate.scala
deleted file mode 100644
index c058265..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/FlinkCorrelate.scala
+++ /dev/null
@@ -1,163 +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.api.table.plan.nodes
-
-import org.apache.calcite.rel.`type`.RelDataType
-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.table.codegen.{CodeGenerator, GeneratedExpression, GeneratedFunction}
-import org.apache.flink.api.table.codegen.CodeGenUtils.primitiveDefaultValue
-import org.apache.flink.api.table.codegen.GeneratedExpression.{ALWAYS_NULL, NO_CODE}
-import org.apache.flink.api.table.functions.utils.TableSqlFunction
-import org.apache.flink.api.table.runtime.FlatMapRunner
-import org.apache.flink.api.table.typeutils.TypeConverter._
-import org.apache.flink.api.table.{TableConfig, TableException}
-
-import scala.collection.JavaConverters._
-
-/**
-  * Join a user-defined table function
-  */
-trait FlinkCorrelate {
-
-  private[flink] def functionBody(
-      generator: CodeGenerator,
-      udtfTypeInfo: TypeInformation[Any],
-      rowType: RelDataType,
-      rexCall: RexCall,
-      condition: Option[RexNode],
-      config: TableConfig,
-      joinType: SemiJoinType,
-      expectedType: Option[TypeInformation[Any]]): String = {
-
-    val returnType = determineReturnType(
-      rowType,
-      expectedType,
-      config.getNullCheck,
-      config.getEfficientTypeUsage)
-
-    val (input1AccessExprs, input2AccessExprs) = generator.generateCorrelateAccessExprs
-
-    val call = generator.generateExpression(rexCall)
-    var body =
-      s"""
-         |${call.code}
-         |java.util.Iterator iter = ${call.resultTerm}.getRowsIterator();
-       """.stripMargin
-
-    if (joinType == SemiJoinType.INNER) {
-      // cross join
-      body +=
-        s"""
-           |if (!iter.hasNext()) {
-           |  return;
-           |}
-        """.stripMargin
-    } else if (joinType == SemiJoinType.LEFT) {
-      // left outer join
-
-      // in case of left outer join and the returned row of table function is empty,
-      // fill all fields of row with null
-      val input2NullExprs = input2AccessExprs.map { x =>
-        GeneratedExpression(
-          primitiveDefaultValue(x.resultType),
-          ALWAYS_NULL,
-          NO_CODE,
-          x.resultType)
-      }
-      val outerResultExpr = generator.generateResultExpression(
-        input1AccessExprs ++ input2NullExprs, returnType, rowType.getFieldNames.asScala)
-      body +=
-        s"""
-           |if (!iter.hasNext()) {
-           |  ${outerResultExpr.code}
-           |  ${generator.collectorTerm}.collect(${outerResultExpr.resultTerm});
-           |  return;
-           |}
-        """.stripMargin
-    } else {
-      throw TableException(s"Unsupported SemiJoinType: $joinType for correlate join.")
-    }
-
-    val crossResultExpr = generator.generateResultExpression(
-      input1AccessExprs ++ input2AccessExprs,
-      returnType,
-      rowType.getFieldNames.asScala)
-
-    val projection = if (condition.isEmpty) {
-      s"""
-         |${crossResultExpr.code}
-         |${generator.collectorTerm}.collect(${crossResultExpr.resultTerm});
-       """.stripMargin
-    } else {
-      val filterGenerator = new CodeGenerator(config, false, udtfTypeInfo)
-      filterGenerator.input1Term = filterGenerator.input2Term
-      val filterCondition = filterGenerator.generateExpression(condition.get)
-      s"""
-         |${filterGenerator.reuseInputUnboxingCode()}
-         |${filterCondition.code}
-         |if (${filterCondition.resultTerm}) {
-         |  ${crossResultExpr.code}
-         |  ${generator.collectorTerm}.collect(${crossResultExpr.resultTerm});
-         |}
-         |""".stripMargin
-    }
-
-    val outputTypeClass = udtfTypeInfo.getTypeClass.getCanonicalName
-    body +=
-      s"""
-         |while (iter.hasNext()) {
-         |  $outputTypeClass ${generator.input2Term} = ($outputTypeClass) iter.next();
-         |  $projection
-         |}
-       """.stripMargin
-    body
-  }
-
-  private[flink] def correlateMapFunction(
-      genFunction: GeneratedFunction[FlatMapFunction[Any, Any]])
-    : FlatMapRunner[Any, Any] = {
-
-    new FlatMapRunner[Any, Any](
-      genFunction.name,
-      genFunction.code,
-      genFunction.returnType)
-  }
-
-  private[flink] def selectToString(rowType: RelDataType): String = {
-    rowType.getFieldNames.asScala.mkString(",")
-  }
-
-  private[flink] def correlateOpName(
-      rexCall: RexCall,
-      sqlFunction: TableSqlFunction,
-      rowType: RelDataType)
-    : String = {
-
-    s"correlate: ${correlateToString(rexCall, sqlFunction)}, select: ${selectToString(rowType)}"
-  }
-
-  private[flink] def correlateToString(rexCall: RexCall, sqlFunction: TableSqlFunction): String = {
-    val udtfName = sqlFunction.getName
-    val operands = rexCall.getOperands.asScala.map(_.toString).mkString(",")
-    s"table($udtfName($operands))"
-  }
-
-}


[44/47] flink git commit: [FLINK-3848] [table] Add projection push down for StreamTableSource.

Posted by fh...@apache.org.
[FLINK-3848] [table] Add projection push down for StreamTableSource.

- Add plan tests for projection push down.
- Implement ProjectableTableSource in CsvTableSource.
- Refactored RowCsvInputFormat

This closes #2810.


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

Branch: refs/heads/master
Commit: ef575e87e8a77d56673d40d98d3d66c6511d57ab
Parents: d5c7bf6
Author: tonycox <to...@gmail.com>
Authored: Thu Dec 8 12:00:40 2016 +0400
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Dec 16 16:41:21 2016 +0100

----------------------------------------------------------------------
 .../flink/api/java/io/RowCsvInputFormat.java    | 113 +++++++++-----
 .../api/java/io/RowCsvInputFormatTest.java      | 153 +++++++++++-------
 .../flink/table/plan/nodes/FlinkRel.scala       |  26 +++-
 .../table/plan/nodes/dataset/DataSetRel.scala   |  31 +---
 .../plan/nodes/datastream/DataStreamCalc.scala  |   2 +-
 .../datastream/StreamTableSourceScan.scala      |  15 +-
 .../flink/table/plan/rules/FlinkRuleSets.scala  |   3 +-
 ...ushProjectIntoBatchTableSourceScanRule.scala |   4 +-
 ...shProjectIntoStreamTableSourceScanRule.scala |  85 ++++++++++
 .../flink/table/sources/CsvTableSource.scala    |  31 +++-
 .../batch/ProjectableTableSourceITCase.scala    |   2 +-
 .../batch/ProjectableTableSourceTest.scala      | 155 +++++++++++++++++++
 .../api/scala/batch/TableSourceITCase.scala     |  64 ++++----
 .../api/scala/stream/TableSourceITCase.scala    |  76 +++++----
 .../flink/table/utils/CommonTestData.scala      |  63 ++++++++
 15 files changed, 615 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java b/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java
index 34233f5..af2e9e4 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/io/RowCsvInputFormat.java
@@ -20,76 +20,108 @@ package org.apache.flink.api.java.io;
 
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.io.ParseException;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.types.Row;
 import org.apache.flink.types.parser.FieldParser;
 
+import java.util.Arrays;
+
 @PublicEvolving
-public class RowCsvInputFormat extends CsvInputFormat<Row> {
+public class RowCsvInputFormat extends CsvInputFormat<Row> implements ResultTypeQueryable<Row> {
 
 	private static final long serialVersionUID = 1L;
 
 	private int arity;
+	private TypeInformation[] fieldTypeInfos;
+	private int[] fieldPosMap;
 	private boolean emptyColumnAsNull;
 
-	public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, String lineDelimiter, String fieldDelimiter, boolean[] includedFieldsMask, boolean emptyColumnAsNull) {
+	public RowCsvInputFormat(Path filePath, TypeInformation[] fieldTypeInfos, String lineDelimiter, String fieldDelimiter, int[] selectedFields, boolean emptyColumnAsNull) {
+
 		super(filePath);
-		if (rowTypeInfo.getArity() == 0) {
-			throw new IllegalArgumentException("Row arity must be greater than 0.");
+		this.arity = fieldTypeInfos.length;
+		if (arity == 0) {
+			throw new IllegalArgumentException("At least one field must be specified");
 		}
-		this.arity = rowTypeInfo.getArity();
-
-		boolean[] fieldsMask;
-		if (includedFieldsMask != null) {
-			fieldsMask = includedFieldsMask;
-		} else {
-			fieldsMask = createDefaultMask(arity);
+		if (arity != selectedFields.length) {
+			throw new IllegalArgumentException("Number of field types and selected fields must be the same");
 		}
+
+		this.fieldTypeInfos = fieldTypeInfos;
+		this.fieldPosMap = toFieldPosMap(selectedFields);
 		this.emptyColumnAsNull = emptyColumnAsNull;
+
+		boolean[] fieldsMask = toFieldMask(selectedFields);
+
 		setDelimiter(lineDelimiter);
 		setFieldDelimiter(fieldDelimiter);
-		setFieldsGeneric(fieldsMask, extractTypeClasses(rowTypeInfo));
+		setFieldsGeneric(fieldsMask, extractTypeClasses(fieldTypeInfos));
 	}
 
-	public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, String lineDelimiter, String fieldDelimiter, int[] includedFieldsMask) {
-		this(filePath, rowTypeInfo, lineDelimiter, fieldDelimiter, toBoolMask(includedFieldsMask), false);
+	public RowCsvInputFormat(Path filePath, TypeInformation[] fieldTypes, String lineDelimiter, String fieldDelimiter, int[] selectedFields) {
+		this(filePath, fieldTypes, lineDelimiter, fieldDelimiter, selectedFields, false);
 	}
 
-	public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, String lineDelimiter, String fieldDelimiter) {
-		this(filePath, rowTypeInfo, lineDelimiter, fieldDelimiter, null, false);
+	public RowCsvInputFormat(Path filePath, TypeInformation[] fieldTypes, String lineDelimiter, String fieldDelimiter) {
+		this(filePath, fieldTypes, lineDelimiter, fieldDelimiter, sequentialScanOrder(fieldTypes.length));
 	}
 
-	public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, boolean[] includedFieldsMask) {
-		this(filePath, rowTypeInfo, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER, includedFieldsMask, false);
+	public RowCsvInputFormat(Path filePath, TypeInformation[] fieldTypes, int[] selectedFields) {
+		this(filePath, fieldTypes, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER, selectedFields);
 	}
 
-	public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, int[] includedFieldsMask) {
-		this(filePath, rowTypeInfo, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER, includedFieldsMask);
+	public RowCsvInputFormat(Path filePath, TypeInformation[] fieldTypes, boolean emptyColumnAsNull) {
+		this(filePath, fieldTypes, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER, sequentialScanOrder(fieldTypes.length), emptyColumnAsNull);
 	}
 
-	public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo, boolean emptyColumnAsNull) {
-		this(filePath, rowTypeInfo, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER, null, emptyColumnAsNull);
+	public RowCsvInputFormat(Path filePath, TypeInformation[] fieldTypes) {
+		this(filePath, fieldTypes, false);
 	}
 
-	public RowCsvInputFormat(Path filePath, RowTypeInfo rowTypeInfo) {
-		this(filePath, rowTypeInfo, DEFAULT_LINE_DELIMITER, DEFAULT_FIELD_DELIMITER, null);
+	private static Class<?>[] extractTypeClasses(TypeInformation[] fieldTypes) {
+		Class<?>[] classes = new Class<?>[fieldTypes.length];
+		for (int i = 0; i < fieldTypes.length; i++) {
+			classes[i] = fieldTypes[i].getTypeClass();
+		}
+		return classes;
 	}
 
-	private static Class<?>[] extractTypeClasses(RowTypeInfo rowTypeInfo) {
-		Class<?>[] classes = new Class<?>[rowTypeInfo.getArity()];
-		for (int i = 0; i < rowTypeInfo.getArity(); i++) {
-			classes[i] = rowTypeInfo.getTypeAt(i).getTypeClass();
+	private static int[] sequentialScanOrder(int arity) {
+		int[] sequentialOrder = new int[arity];
+		for (int i = 0; i < arity; i++) {
+			sequentialOrder[i] = i;
 		}
-		return classes;
+		return sequentialOrder;
 	}
 
-	private static boolean[] toBoolMask(int[] includedFieldsMask) {
-		if (includedFieldsMask == null) {
-			return null;
-		} else {
-			return toBooleanMask(includedFieldsMask);
+	private static boolean[] toFieldMask(int[] selectedFields) {
+		int maxField = 0;
+		for (int selectedField : selectedFields) {
+			maxField = Math.max(maxField, selectedField);
 		}
+		boolean[] mask = new boolean[maxField + 1];
+		Arrays.fill(mask, false);
+
+		for (int selectedField : selectedFields) {
+			mask[selectedField] = true;
+		}
+		return mask;
+	}
+
+	private static int[] toFieldPosMap(int[] selectedFields) {
+		int[] fieldIdxs = Arrays.copyOf(selectedFields, selectedFields.length);
+		Arrays.sort(fieldIdxs);
+
+		int[] fieldPosMap = new int[selectedFields.length];
+		for (int i = 0; i < selectedFields.length; i++) {
+			int pos = Arrays.binarySearch(fieldIdxs, selectedFields[i]);
+			fieldPosMap[pos] = i;
+		}
+
+		return fieldPosMap;
 	}
 
 	@Override
@@ -129,14 +161,14 @@ public class RowCsvInputFormat extends CsvInputFormat<Row> {
 
 			if (fieldIncluded[field]) {
 				// parse field
-				FieldParser<Object> parser = (FieldParser<Object>) this.getFieldParsers()[output];
+				FieldParser<Object> parser = (FieldParser<Object>) this.getFieldParsers()[fieldPosMap[output]];
 				int latestValidPos = startPos;
 				startPos = parser.resetErrorStateAndParse(
 					bytes,
 					startPos,
 					limit,
 					fieldDelimiter,
-					holders[output]);
+					holders[fieldPosMap[output]]);
 
 				if (!isLenient() && (parser.getErrorState() != FieldParser.ParseErrorState.NONE)) {
 					// the error state EMPTY_COLUMN is ignored
@@ -145,14 +177,14 @@ public class RowCsvInputFormat extends CsvInputFormat<Row> {
 							field, new String(bytes, offset, numBytes), parser.getClass().getSimpleName(), parser.getErrorState()));
 					}
 				}
-				holders[output] = parser.getLastResult();
+				holders[fieldPosMap[output]] = parser.getLastResult();
 
 				// check parse result:
 				// the result is null if it is invalid
 				// or empty with emptyColumnAsNull enabled
 				if (startPos < 0 ||
 					(emptyColumnAsNull && (parser.getErrorState().equals(FieldParser.ParseErrorState.EMPTY_COLUMN)))) {
-					holders[output] = null;
+					holders[fieldPosMap[output]] = null;
 					startPos = skipFields(bytes, latestValidPos, limit, fieldDelimiter);
 				}
 				output++;
@@ -171,4 +203,9 @@ public class RowCsvInputFormat extends CsvInputFormat<Row> {
 		}
 		return true;
 	}
+
+	@Override
+	public TypeInformation<Row> getProducedType() {
+		return new RowTypeInfo(this.fieldTypeInfos);
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java
index a68e81e..b819641 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/io/RowCsvInputFormatTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.api.java.io;
 import org.apache.flink.api.common.io.ParseException;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FileInputSplit;
@@ -68,9 +69,9 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO);
+		TypeInformation[] fieldTypes = new TypeInformation[]{BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.DOUBLE_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|");
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|");
 		format.setLenient(false);
 		Configuration parameters = new Configuration();
 		format.configure(new Configuration());
@@ -161,12 +162,12 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.STRING_TYPE_INFO,
 			BasicTypeInfo.INT_TYPE_INFO,
-			BasicTypeInfo.DOUBLE_TYPE_INFO);
+			BasicTypeInfo.DOUBLE_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|");
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|");
 		format.setCommentPrefix("#");
 		format.configure(new Configuration());
 		format.open(split);
@@ -200,12 +201,12 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.STRING_TYPE_INFO,
 			BasicTypeInfo.INT_TYPE_INFO,
-			BasicTypeInfo.DOUBLE_TYPE_INFO);
+			BasicTypeInfo.DOUBLE_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|");
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|");
 		format.setCommentPrefix("//");
 		format.configure(new Configuration());
 		format.open(split);
@@ -233,12 +234,12 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.STRING_TYPE_INFO,
 			BasicTypeInfo.STRING_TYPE_INFO,
-			BasicTypeInfo.STRING_TYPE_INFO);
+			BasicTypeInfo.STRING_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|");
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|");
 		format.configure(new Configuration());
 		format.open(split);
 
@@ -273,12 +274,12 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.STRING_TYPE_INFO,
 			BasicTypeInfo.STRING_TYPE_INFO,
-			BasicTypeInfo.STRING_TYPE_INFO);
+			BasicTypeInfo.STRING_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|");
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|");
 		format.configure(new Configuration());
 		format.enableQuotedStringParsing('@');
 		format.open(split);
@@ -314,12 +315,12 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.STRING_TYPE_INFO,
 			BasicTypeInfo.STRING_TYPE_INFO,
-			BasicTypeInfo.STRING_TYPE_INFO);
+			BasicTypeInfo.STRING_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|");
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|");
 		format.setFieldDelimiter("|-");
 		format.configure(new Configuration());
 		format.open(split);
@@ -355,14 +356,14 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.INT_TYPE_INFO,
 			BasicTypeInfo.INT_TYPE_INFO,
 			BasicTypeInfo.INT_TYPE_INFO,
 			BasicTypeInfo.INT_TYPE_INFO,
-			BasicTypeInfo.INT_TYPE_INFO);
+			BasicTypeInfo.INT_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, "\n", "|");
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, "\n", "|");
 
 		format.setFieldDelimiter("|");
 		format.configure(new Configuration());
@@ -405,7 +406,7 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.BOOLEAN_TYPE_INFO,
 			BasicTypeInfo.BYTE_TYPE_INFO,
 			BasicTypeInfo.DOUBLE_TYPE_INFO,
@@ -413,9 +414,9 @@ public class RowCsvInputFormatTest {
 			BasicTypeInfo.INT_TYPE_INFO,
 			BasicTypeInfo.LONG_TYPE_INFO,
 			BasicTypeInfo.SHORT_TYPE_INFO,
-			BasicTypeInfo.STRING_TYPE_INFO);
+			BasicTypeInfo.STRING_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo, true);
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes, true);
 		format.setFieldDelimiter(",");
 		format.configure(new Configuration());
 		format.open(split);
@@ -439,14 +440,14 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.DOUBLE_TYPE_INFO,
 			BasicTypeInfo.DOUBLE_TYPE_INFO,
 			BasicTypeInfo.DOUBLE_TYPE_INFO,
 			BasicTypeInfo.DOUBLE_TYPE_INFO,
-			BasicTypeInfo.DOUBLE_TYPE_INFO);
+			BasicTypeInfo.DOUBLE_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo);
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes);
 		format.setFieldDelimiter("|");
 		format.configure(new Configuration());
 		format.open(split);
@@ -480,11 +481,11 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.INT_TYPE_INFO,
-			BasicTypeInfo.INT_TYPE_INFO);
+			BasicTypeInfo.INT_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo);
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes);
 		format.setFieldDelimiter("|");
 		format.configure(new Configuration());
 		format.open(split);
@@ -513,15 +514,15 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.INT_TYPE_INFO,
 			BasicTypeInfo.INT_TYPE_INFO,
-			BasicTypeInfo.INT_TYPE_INFO);
+			BasicTypeInfo.INT_TYPE_INFO};
 
 		RowCsvInputFormat format = new RowCsvInputFormat(
 			PATH,
-			typeInfo,
-			new boolean[]{true, false, false, true, false, false, false, true});
+			fieldTypes,
+			new int[]{0,3,7});
 		format.setFieldDelimiter("|x|");
 		format.configure(new Configuration());
 		format.open(split);
@@ -552,14 +553,14 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.INT_TYPE_INFO,
 			BasicTypeInfo.INT_TYPE_INFO,
-			BasicTypeInfo.INT_TYPE_INFO);
+			BasicTypeInfo.INT_TYPE_INFO};
 
 		RowCsvInputFormat format = new RowCsvInputFormat(
 			PATH,
-			typeInfo,
+			fieldTypes,
 			new int[]{0, 3, 7});
 		format.setFieldDelimiter("|");
 		format.configure(new Configuration());
@@ -591,15 +592,15 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = RowCsvInputFormatTest.createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.INT_TYPE_INFO,
 			BasicTypeInfo.INT_TYPE_INFO,
-			BasicTypeInfo.INT_TYPE_INFO);
+			BasicTypeInfo.INT_TYPE_INFO};
 
 		RowCsvInputFormat format = new RowCsvInputFormat(
 			PATH,
-			typeInfo,
-			new boolean[]{true, false, false, true, false, false, false, true});
+			fieldTypes,
+			new int[]{0, 3, 7});
 		format.setFieldDelimiter("&&");
 		format.configure(new Configuration());
 		format.open(split);
@@ -660,14 +661,14 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.INT_TYPE_INFO,
 			BasicTypeInfo.STRING_TYPE_INFO,
 			BasicTypeInfo.STRING_TYPE_INFO,
 			BasicTypeInfo.STRING_TYPE_INFO,
-			BasicTypeInfo.DOUBLE_TYPE_INFO);
+			BasicTypeInfo.DOUBLE_TYPE_INFO};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo);
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes);
 		format.setSkipFirstLineAsHeader(true);
 		format.setFieldDelimiter(",");
 		format.configure(new Configuration());
@@ -749,14 +750,14 @@ public class RowCsvInputFormatTest {
 		writer.write(fileContent);
 		writer.close();
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.STRING_TYPE_INFO,
-			BasicTypeInfo.STRING_TYPE_INFO);
+			BasicTypeInfo.STRING_TYPE_INFO};
 
 		RowCsvInputFormat inputFormat = new RowCsvInputFormat(
 			new Path(tempFile.toURI().toString()),
-			typeInfo,
-			new boolean[]{true, false, true});
+			fieldTypes,
+			new int[]{0, 2});
 		inputFormat.enableQuotedStringParsing('"');
 		inputFormat.setFieldDelimiter("|");
 		inputFormat.setDelimiter('\n');
@@ -781,11 +782,11 @@ public class RowCsvInputFormatTest {
 		writer.write(fileContent);
 		writer.close();
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			BasicTypeInfo.STRING_TYPE_INFO,
-			BasicTypeInfo.STRING_TYPE_INFO);
+			BasicTypeInfo.STRING_TYPE_INFO};
 
-		RowCsvInputFormat inputFormat = new RowCsvInputFormat(new Path(tempFile.toURI().toString()), typeInfo);
+		RowCsvInputFormat inputFormat = new RowCsvInputFormat(new Path(tempFile.toURI().toString()), fieldTypes);
 		inputFormat.enableQuotedStringParsing('"');
 		inputFormat.setFieldDelimiter("|");
 		inputFormat.setDelimiter('\n');
@@ -806,13 +807,13 @@ public class RowCsvInputFormatTest {
 
 		FileInputSplit split = createTempFile(fileContent);
 
-		RowTypeInfo typeInfo = new RowTypeInfo(
+		TypeInformation[] fieldTypes = new TypeInformation[]{
 			SqlTimeTypeInfo.DATE,
 			SqlTimeTypeInfo.TIME,
 			SqlTimeTypeInfo.TIMESTAMP,
-			SqlTimeTypeInfo.TIMESTAMP);
+			SqlTimeTypeInfo.TIMESTAMP};
 
-		RowCsvInputFormat format = new RowCsvInputFormat(PATH, typeInfo);
+		RowCsvInputFormat format = new RowCsvInputFormat(PATH, fieldTypes);
 		format.setFieldDelimiter("|");
 		format.configure(new Configuration());
 		format.open(split);
@@ -838,6 +839,48 @@ public class RowCsvInputFormatTest {
 		assertTrue(format.reachedEnd());
 	}
 
+	@Test
+	public void testScanOrder() throws Exception {
+		String fileContent =
+			// first row
+			"111|222|333|444|555|666|777|888|999|000|\n" +
+			// second row
+			"000|999|888|777|666|555|444|333|222|111|";
+		FileInputSplit split = createTempFile(fileContent);
+
+		TypeInformation[] fieldTypes = new TypeInformation[]{
+			BasicTypeInfo.INT_TYPE_INFO,
+			BasicTypeInfo.INT_TYPE_INFO,
+			BasicTypeInfo.INT_TYPE_INFO};
+
+		int[] order = new int[]{7, 3, 0};
+		RowCsvInputFormat format = new RowCsvInputFormat(
+			PATH,
+			fieldTypes,
+			order);
+
+		format.setFieldDelimiter("|");
+		format.configure(new Configuration());
+		format.open(split);
+
+		Row result = new Row(3);
+
+		// check first row
+		result = format.nextRecord(result);
+		assertNotNull(result);
+		assertEquals(888, result.getField(0));
+		assertEquals(444, result.getField(1));
+		assertEquals(111, result.getField(2));
+
+		// check second row
+		result = format.nextRecord(result);
+		assertNotNull(result);
+		assertEquals(333, result.getField(0));
+		assertEquals(777, result.getField(1));
+		assertEquals(0, result.getField(2));
+
+	}
+
 	private static FileInputSplit createTempFile(String content) throws IOException {
 		File tempFile = File.createTempFile("test_contents", "tmp");
 		tempFile.deleteOnExit();
@@ -859,9 +902,9 @@ public class RowCsvInputFormatTest {
 		wrt.write(fileContent);
 		wrt.close();
 
-		RowTypeInfo typeInfo = new RowTypeInfo(BasicTypeInfo.STRING_TYPE_INFO);
+		TypeInformation[] fieldTypes = new TypeInformation[]{BasicTypeInfo.STRING_TYPE_INFO};
 
-		RowCsvInputFormat inputFormat = new RowCsvInputFormat(new Path(tempFile.toURI().toString()), typeInfo);
+		RowCsvInputFormat inputFormat = new RowCsvInputFormat(new Path(tempFile.toURI().toString()), fieldTypes);
 		inputFormat.configure(new Configuration());
 		inputFormat.setDelimiter(lineBreakerSetup);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkRel.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkRel.scala
index c8211a2..835f316 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkRel.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkRel.scala
@@ -18,10 +18,12 @@
 
 package org.apache.flink.table.plan.nodes
 
+import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rex._
+import org.apache.calcite.sql.`type`.SqlTypeName
 import org.apache.flink.api.common.functions.MapFunction
 import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.codegen.CodeGenerator
 import org.apache.flink.table.runtime.MapRunner
 
@@ -100,4 +102,26 @@ trait FlinkRel {
       genFunction.returnType)
 
   }
+
+  private[flink] def estimateRowSize(rowType: RelDataType): Double = {
+
+    rowType.getFieldList.map(_.getType.getSqlTypeName).foldLeft(0) { (s, t) =>
+      t match {
+        case SqlTypeName.TINYINT => s + 1
+        case SqlTypeName.SMALLINT => s + 2
+        case SqlTypeName.INTEGER => s + 4
+        case SqlTypeName.BIGINT => s + 8
+        case SqlTypeName.BOOLEAN => s + 1
+        case SqlTypeName.FLOAT => s + 4
+        case SqlTypeName.DOUBLE => s + 8
+        case SqlTypeName.VARCHAR => s + 12
+        case SqlTypeName.CHAR => s + 1
+        case SqlTypeName.DECIMAL => s + 12
+        case typeName if SqlTypeName.YEAR_INTERVAL_TYPES.contains(typeName) => s + 8
+        case typeName if SqlTypeName.DAY_INTERVAL_TYPES.contains(typeName) => s + 4
+        case SqlTypeName.TIME | SqlTypeName.TIMESTAMP | SqlTypeName.DATE => s + 12
+        case _ => throw TableException(s"Unsupported data type encountered: $t")
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetRel.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetRel.scala
index 210ae03..02138cf 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetRel.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetRel.scala
@@ -19,14 +19,10 @@
 package org.apache.flink.table.plan.nodes.dataset
 
 import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.table.api.BatchTableEnvironment
+import org.apache.flink.table.plan.nodes.FlinkRel
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.DataSet
-import org.apache.flink.table.api.{BatchTableEnvironment, TableException}
-import org.apache.flink.table.plan.nodes.FlinkRel
-
-import scala.collection.JavaConversions._
 
 trait DataSetRel extends RelNode with FlinkRel {
 
@@ -45,27 +41,4 @@ trait DataSetRel extends RelNode with FlinkRel {
      tableEnv: BatchTableEnvironment,
      expectedType: Option[TypeInformation[Any]] = None) : DataSet[Any]
 
-  private[flink] def estimateRowSize(rowType: RelDataType): Double = {
-
-    rowType.getFieldList.map(_.getType.getSqlTypeName).foldLeft(0) { (s, t) =>
-      t match {
-        case SqlTypeName.TINYINT => s + 1
-        case SqlTypeName.SMALLINT => s + 2
-        case SqlTypeName.INTEGER => s + 4
-        case SqlTypeName.BIGINT => s + 8
-        case SqlTypeName.BOOLEAN => s + 1
-        case SqlTypeName.FLOAT => s + 4
-        case SqlTypeName.DOUBLE => s + 8
-        case SqlTypeName.VARCHAR => s + 12
-        case SqlTypeName.CHAR => s + 1
-        case SqlTypeName.DECIMAL => s + 12
-        case typeName if SqlTypeName.YEAR_INTERVAL_TYPES.contains(typeName) => s + 8
-        case typeName if SqlTypeName.DAY_INTERVAL_TYPES.contains(typeName) => s + 4
-        case SqlTypeName.TIME | SqlTypeName.TIMESTAMP | SqlTypeName.DATE => s + 12
-        case _ => throw TableException(s"Unsupported data type encountered: $t")
-      }
-    }
-
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/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 774c17b..43f1fb6 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
@@ -39,7 +39,7 @@ class DataStreamCalc(
     traitSet: RelTraitSet,
     input: RelNode,
     rowRelDataType: RelDataType,
-    calcProgram: RexProgram,
+    private[flink] val calcProgram: RexProgram,
     ruleDescription: String)
   extends SingleRel(cluster, traitSet, input)
   with FlinkCalc

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/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 f86a54b..702b6eb 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
@@ -19,7 +19,8 @@
 package org.apache.flink.table.plan.nodes.datastream
 
 import org.apache.calcite.plan._
-import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.calcite.rel.{RelNode, RelWriter}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.plan.schema.TableSourceTable
@@ -32,7 +33,7 @@ class StreamTableSourceScan(
     cluster: RelOptCluster,
     traitSet: RelTraitSet,
     table: RelOptTable,
-    tableSource: StreamTableSource[_])
+    val tableSource: StreamTableSource[_])
   extends StreamScan(cluster, traitSet, table) {
 
   override def deriveRowType() = {
@@ -40,6 +41,11 @@ class StreamTableSourceScan(
     flinkTypeFactory.buildRowDataType(tableSource.getFieldsNames, tableSource.getFieldTypes)
   }
 
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+    val rowCnt = metadata.getRowCount(this)
+    planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * estimateRowSize(getRowType))
+  }
+
   override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
     new StreamTableSourceScan(
       cluster,
@@ -49,6 +55,11 @@ class StreamTableSourceScan(
     )
   }
 
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .item("fields", tableSource.getFieldsNames.mkString(", "))
+  }
+
   override def translateToPlan(
       tableEnv: StreamTableEnvironment,
       expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
index bcd12a4..8c8b304 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/FlinkRuleSets.scala
@@ -155,7 +155,8 @@ object FlinkRuleSets {
       DataStreamUnionRule.INSTANCE,
       DataStreamValuesRule.INSTANCE,
       DataStreamCorrelateRule.INSTANCE,
-      StreamTableSourceScanRule.INSTANCE
+      StreamTableSourceScanRule.INSTANCE,
+      PushProjectIntoStreamTableSourceScanRule.INSTANCE
   )
 
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala
index 5d91c62..7adec48 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala
@@ -46,7 +46,7 @@ class PushProjectIntoBatchTableSourceScanRule extends RelOptRule(
 
     val usedFields: Array[Int] = extractRefInputFields(calc.calcProgram)
 
-    // if no fields can be projected, there is no need to transform subtree
+    // if no fields can be projected, we keep the original plan.
     if (scan.tableSource.getNumberOfFields != usedFields.length) {
       val originTableSource = scan.tableSource.asInstanceOf[ProjectableTableSource[_]]
       val newTableSource = originTableSource.projectFields(usedFields)
@@ -62,8 +62,8 @@ class PushProjectIntoBatchTableSourceScanRule extends RelOptRule(
         usedFields,
         calc.getCluster.getRexBuilder)
 
-      // if project merely returns its input and doesn't exist filter, remove datasetCalc nodes
       if (newCalcProgram.isTrivial) {
+        // drop calc if the transformed program merely returns its input and doesn't exist filter
         call.transformTo(newScan)
       } else {
         val newCalc = new DataSetCalc(

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushProjectIntoStreamTableSourceScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushProjectIntoStreamTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushProjectIntoStreamTableSourceScanRule.scala
new file mode 100644
index 0000000..654fb8f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/PushProjectIntoStreamTableSourceScanRule.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.plan.rules.datastream
+
+import org.apache.calcite.plan.RelOptRule._
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.flink.table.plan.nodes.datastream.{DataStreamCalc, StreamTableSourceScan}
+import org.apache.flink.table.plan.rules.util.RexProgramProjectExtractor._
+import org.apache.flink.table.sources.{ProjectableTableSource, StreamTableSource}
+
+/**
+  * The rule is responsible for push project into a [[StreamTableSourceScan]]
+  */
+class PushProjectIntoStreamTableSourceScanRule extends RelOptRule(
+  operand(classOf[DataStreamCalc],
+    operand(classOf[StreamTableSourceScan], none())),
+  "PushProjectIntoStreamTableSourceScanRule") {
+
+  /** Rule must only match if [[StreamTableSource]] targets a [[ProjectableTableSource]] */
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val scan: StreamTableSourceScan = call.rel(1).asInstanceOf[StreamTableSourceScan]
+    scan.tableSource match {
+      case _: ProjectableTableSource[_] => true
+      case _ => false
+    }
+  }
+
+  override def onMatch(call: RelOptRuleCall): Unit = {
+    val calc = call.rel(0).asInstanceOf[DataStreamCalc]
+    val scan = call.rel(1).asInstanceOf[StreamTableSourceScan]
+
+    val usedFields = extractRefInputFields(calc.calcProgram)
+
+    // if no fields can be projected, we keep the original plan
+    if (scan.tableSource.getNumberOfFields != usedFields.length) {
+      val originTableSource = scan.tableSource.asInstanceOf[ProjectableTableSource[_]]
+      val newTableSource = originTableSource.projectFields(usedFields)
+      val newScan = new StreamTableSourceScan(
+        scan.getCluster,
+        scan.getTraitSet,
+        scan.getTable,
+        newTableSource.asInstanceOf[StreamTableSource[_]])
+
+      val newProgram = rewriteRexProgram(
+        calc.calcProgram,
+        newScan.getRowType,
+        usedFields,
+        calc.getCluster.getRexBuilder)
+
+      if (newProgram.isTrivial) {
+        // drop calc if the transformed program merely returns its input and doesn't exist filter
+        call.transformTo(newScan)
+      } else {
+        val newCalc = new DataStreamCalc(
+          calc.getCluster,
+          calc.getTraitSet,
+          newScan,
+          calc.getRowType,
+          newProgram,
+          description)
+        call.transformTo(newCalc)
+      }
+    }
+  }
+}
+
+object PushProjectIntoStreamTableSourceScanRule {
+  val INSTANCE: RelOptRule = new PushProjectIntoStreamTableSourceScanRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSource.scala
index 3f4e395..20e8bb9 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSource.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSource.scala
@@ -54,7 +54,8 @@ class CsvTableSource(
     ignoreComments: String = null,
     lenient: Boolean = false)
   extends BatchTableSource[Row]
-  with StreamTableSource[Row] {
+  with StreamTableSource[Row]
+  with ProjectableTableSource[Row] {
 
   /**
     * A [[BatchTableSource]] and [[StreamTableSource]] for simple CSV files with a
@@ -74,6 +75,8 @@ class CsvTableSource(
 
   private val returnType = new RowTypeInfo(fieldTypes: _*)
 
+  private var selectedFields: Array[Int] = fieldTypes.indices.toArray
+
   /**
     * Returns the data of the table as a [[DataSet]] of [[Row]].
     *
@@ -106,8 +109,32 @@ class CsvTableSource(
     streamExecEnv.createInput(createCsvInput(), returnType)
   }
 
+  /** Returns a copy of [[TableSource]] with ability to project fields */
+  override def projectFields(fields: Array[Int]): CsvTableSource = {
+
+    val newFieldNames: Array[String] = fields.map(fieldNames(_))
+    val newFieldTypes: Array[TypeInformation[_]] = fields.map(fieldTypes(_))
+
+    val source = new CsvTableSource(path,
+      newFieldNames,
+      newFieldTypes,
+      fieldDelim,
+      rowDelim,
+      quoteCharacter,
+      ignoreFirstLine,
+      ignoreComments,
+      lenient)
+    source.selectedFields = fields
+    source
+  }
+
   private def createCsvInput(): RowCsvInputFormat = {
-    val inputFormat = new RowCsvInputFormat(new Path(path), returnType, rowDelim, fieldDelim)
+    val inputFormat = new RowCsvInputFormat(
+      new Path(path),
+      fieldTypes,
+      rowDelim,
+      fieldDelim,
+      selectedFields)
 
     inputFormat.setSkipFirstLineAsHeader(ignoreFirstLine)
     inputFormat.setLenient(lenient)

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala
index ec4dc59..37407c8 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala
@@ -19,7 +19,6 @@
 package org.apache.flink.table.api.scala.batch
 
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.api.java.{DataSet => JavaSet, ExecutionEnvironment => JavaExecEnv}
 import org.apache.flink.api.scala.ExecutionEnvironment
 import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
@@ -27,6 +26,7 @@ import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableC
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.sources.{BatchTableSource, ProjectableTableSource}
 import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
 import org.apache.flink.test.util.TestBaseUtils
 import org.apache.flink.types.Row

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceTest.scala
new file mode 100644
index 0000000..b3097cf
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceTest.scala
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch
+
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.sources.CsvTableSource
+import org.apache.flink.table.utils.{CommonTestData, TableTestBase}
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+class ProjectableTableSourceTest extends TableTestBase {
+
+  private val projectedFields: Array[String] = Array("last", "id", "score")
+  private val noCalcFields: Array[String] = Array("id", "score", "first")
+
+  @Test
+  def testBatchProjectableSourceScanPlanTableApi(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = batchTestUtil()
+    val tEnv = util.tEnv
+
+    tEnv.registerTableSource(tableName, csvTable)
+
+    val result = tEnv
+      .scan(tableName)
+      .select('last.upperCase(), 'id.floor(), 'score * 2)
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      sourceBatchTableNode(tableName, projectedFields),
+      term("select", "UPPER(last) AS _c0", "FLOOR(id) AS _c1", "*(score, 2) AS _c2")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testBatchProjectableSourceScanPlanSQL(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = batchTestUtil()
+
+    util.tEnv.registerTableSource(tableName, csvTable)
+
+    val sqlQuery = s"SELECT last, floor(id), score * 2 FROM $tableName"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      sourceBatchTableNode(tableName, projectedFields),
+      term("select", "last", "FLOOR(id) AS EXPR$1", "*(score, 2) AS EXPR$2")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testBatchProjectableSourceScanNoIdentityCalc(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = batchTestUtil()
+    val tEnv = util.tEnv
+
+    tEnv.registerTableSource(tableName, csvTable)
+
+    val result = tEnv
+      .scan(tableName)
+      .select('id, 'score, 'first)
+
+    val expected = sourceBatchTableNode(tableName, noCalcFields)
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testStreamProjectableSourceScanPlanTableApi(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = streamTestUtil()
+    val tEnv = util.tEnv
+
+    tEnv.registerTableSource(tableName, csvTable)
+
+    val result = tEnv
+      .ingest(tableName)
+      .select('last, 'id.floor(), 'score * 2)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      sourceStreamTableNode(tableName, projectedFields),
+      term("select", "last", "FLOOR(id) AS _c1", "*(score, 2) AS _c2")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testStreamProjectableSourceScanPlanSQL(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = streamTestUtil()
+
+    util.tEnv.registerTableSource(tableName, csvTable)
+
+    val sqlQuery = s"SELECT last, floor(id), score * 2 FROM $tableName"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      sourceStreamTableNode(tableName, projectedFields),
+      term("select", "last", "FLOOR(id) AS EXPR$1", "*(score, 2) AS EXPR$2")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testStreamProjectableSourceScanNoIdentityCalc(): Unit = {
+    val (csvTable, tableName) = tableSource
+    val util = streamTestUtil()
+    val tEnv = util.tEnv
+
+    tEnv.registerTableSource(tableName, csvTable)
+
+    val result = tEnv
+      .ingest(tableName)
+      .select('id, 'score, 'first)
+
+    val expected = sourceStreamTableNode(tableName, noCalcFields)
+    util.verifyTable(result, expected)
+  }
+
+  def tableSource: (CsvTableSource, String) = {
+    val csvTable = CommonTestData.getCsvTableSource
+    val tableName = "csvTable"
+    (csvTable, tableName)
+  }
+
+  def sourceBatchTableNode(sourceName: String, fields: Array[String]): String = {
+    s"BatchTableSourceScan(table=[[$sourceName]], fields=[${fields.mkString(", ")}])"
+  }
+
+  def sourceStreamTableNode(sourceName: String, fields: Array[String] ): String = {
+    s"StreamTableSourceScan(table=[[$sourceName]], fields=[${fields.mkString(", ")}])"
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
index 0b2c8fc..e324aad 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
@@ -18,8 +18,6 @@
 
 package org.apache.flink.table.api.scala.batch
 
-import java.io.{File, FileOutputStream, OutputStreamWriter}
-
 import org.apache.flink.api.common.io.GenericInputFormat
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
 import org.apache.flink.api.java.{DataSet => JavaSet, ExecutionEnvironment => JavaExecEnv}
@@ -29,8 +27,9 @@ import org.apache.flink.types.Row
 import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
 import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.sources.{BatchTableSource, CsvTableSource}
+import org.apache.flink.table.sources.BatchTableSource
 import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.utils.CommonTestData
 import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
 import org.apache.flink.test.util.TestBaseUtils
 import org.junit.Test
@@ -83,43 +82,11 @@ class TableSourceITCase(
   @Test
   def testCsvTableSource(): Unit = {
 
-    val csvRecords = Seq(
-      "First#Id#Score#Last",
-      "Mike#1#12.3#Smith",
-      "Bob#2#45.6#Taylor",
-      "Sam#3#7.89#Miller",
-      "Peter#4#0.12#Smith",
-      "% Just a comment",
-      "Liz#5#34.5#Williams",
-      "Sally#6#6.78#Miller",
-      "Alice#7#90.1#Smith",
-      "Kelly#8#2.34#Williams"
-    )
-
-    val tempFile = File.createTempFile("csv-test", "tmp")
-    tempFile.deleteOnExit()
-    val tmpWriter = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8")
-    tmpWriter.write(csvRecords.mkString("$"))
-    tmpWriter.close()
+    val csvTable = CommonTestData.getCsvTableSource
 
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env, config)
 
-    val csvTable = new CsvTableSource(
-      tempFile.getAbsolutePath,
-      Array("first", "id", "score", "last"),
-      Array(
-        BasicTypeInfo.STRING_TYPE_INFO,
-        BasicTypeInfo.INT_TYPE_INFO,
-        BasicTypeInfo.DOUBLE_TYPE_INFO,
-        BasicTypeInfo.STRING_TYPE_INFO
-      ),
-      fieldDelim = "#",
-      rowDelim = "$",
-      ignoreFirstLine = true,
-      ignoreComments = "%"
-    )
-
     tEnv.registerTableSource("csvTable", csvTable)
     val results = tEnv.sql(
       "SELECT last, sum(score), max(id) FROM csvTable GROUP BY last").collect()
@@ -132,6 +99,31 @@ class TableSourceITCase(
     TestBaseUtils.compareResultAsText(results.asJava, expected)
   }
 
+  @Test
+  def testCsvTableSourceWithProjection(): Unit = {
+    val csvTable = CommonTestData.getCsvTableSource
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    tEnv.registerTableSource("csvTable", csvTable)
+
+    val results = tEnv
+      .scan("csvTable")
+      .select('last, 'id.floor(), 'score * 2)
+      .collect()
+
+    val expected = Seq(
+      "Smith,1,24.6",
+      "Taylor,2,91.2",
+      "Miller,3,15.78",
+      "Smith,4,0.24",
+      "Williams,5,69.0",
+      "Miller,6,13.56",
+      "Smith,7,180.2",
+      "Williams,8,4.68").mkString("\n")
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
 }
 
 class TestBatchTableSource extends BatchTableSource[Row] {

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
index ce910db..316f2f3 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
@@ -18,14 +18,12 @@
 
 package org.apache.flink.table.api.scala.stream
 
-import java.io.{File, FileOutputStream, OutputStreamWriter}
-
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala.stream.utils.StreamITCase
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.sources.{CsvTableSource, StreamTableSource}
+import org.apache.flink.table.sources.StreamTableSource
+import org.apache.flink.api.scala._
+import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment
 import org.apache.flink.streaming.api.functions.source.SourceFunction
@@ -33,6 +31,7 @@ import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceCont
 import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
 import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.utils.CommonTestData
 import org.apache.flink.types.Row
 import org.junit.Assert._
 import org.junit.Test
@@ -87,46 +86,14 @@ class TableSourceITCase extends StreamingMultipleProgramsTestBase {
   }
 
   @Test
-  def testCsvTableSource(): Unit = {
-
-    val csvRecords = Seq(
-      "First#Id#Score#Last",
-      "Mike#1#12.3#Smith",
-      "Bob#2#45.6#Taylor",
-      "Sam#3#7.89#Miller",
-      "Peter#4#0.12#Smith",
-      "% Just a comment",
-      "Liz#5#34.5#Williams",
-      "Sally#6#6.78#Miller",
-      "Alice#7#90.1#Smith",
-      "Kelly#8#2.34#Williams"
-    )
-
-    val tempFile = File.createTempFile("csv-test", "tmp")
-    tempFile.deleteOnExit()
-    val tmpWriter = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8")
-    tmpWriter.write(csvRecords.mkString("$"))
-    tmpWriter.close()
+  def testCsvTableSourceSQL(): Unit = {
+
+    val csvTable = CommonTestData.getCsvTableSource
 
     val env = StreamExecutionEnvironment.getExecutionEnvironment
     val tEnv = TableEnvironment.getTableEnvironment(env)
     StreamITCase.testResults = mutable.MutableList()
 
-    val csvTable = new CsvTableSource(
-      tempFile.getAbsolutePath,
-      Array("first", "id", "score", "last"),
-      Array(
-        BasicTypeInfo.STRING_TYPE_INFO,
-        BasicTypeInfo.INT_TYPE_INFO,
-        BasicTypeInfo.DOUBLE_TYPE_INFO,
-        BasicTypeInfo.STRING_TYPE_INFO
-      ),
-      fieldDelim = "#",
-      rowDelim = "$",
-      ignoreFirstLine = true,
-      ignoreComments = "%"
-    )
-
     tEnv.registerTableSource("csvTable", csvTable)
     tEnv.sql(
       "SELECT last, score, id FROM csvTable WHERE id < 4 ")
@@ -141,6 +108,35 @@ class TableSourceITCase extends StreamingMultipleProgramsTestBase {
       "Miller,7.89,3")
     assertEquals(expected.sorted, StreamITCase.testResults.sorted)
   }
+
+  @Test
+  def testCsvTableSourceTableAPI(): Unit = {
+
+    val csvTable = CommonTestData.getCsvTableSource
+    StreamITCase.testResults = mutable.MutableList()
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    tEnv.registerTableSource("csvTable", csvTable)
+    tEnv.ingest("csvTable")
+      .select('last, 'id.floor(), 'score * 2)
+      .toDataStream[Row]
+      .addSink(new StreamITCase.StringSink)
+
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "Smith,1,24.6",
+      "Taylor,2,91.2",
+      "Miller,3,15.78",
+      "Smith,4,0.24",
+      "Williams,5,69.0",
+      "Miller,6,13.56",
+      "Smith,7,180.2",
+      "Williams,8,4.68")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
 }
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ef575e87/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/CommonTestData.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/CommonTestData.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/CommonTestData.scala
new file mode 100644
index 0000000..349b369
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/CommonTestData.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.utils
+
+import java.io.{File, FileOutputStream, OutputStreamWriter}
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.table.sources.CsvTableSource
+
+object CommonTestData {
+
+  def getCsvTableSource: CsvTableSource = {
+    val csvRecords = Seq(
+      "First#Id#Score#Last",
+      "Mike#1#12.3#Smith",
+      "Bob#2#45.6#Taylor",
+      "Sam#3#7.89#Miller",
+      "Peter#4#0.12#Smith",
+      "% Just a comment",
+      "Liz#5#34.5#Williams",
+      "Sally#6#6.78#Miller",
+      "Alice#7#90.1#Smith",
+      "Kelly#8#2.34#Williams"
+    )
+
+    val tempFile = File.createTempFile("csv-test", "tmp")
+    tempFile.deleteOnExit()
+    val tmpWriter = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8")
+    tmpWriter.write(csvRecords.mkString("$"))
+    tmpWriter.close()
+
+    new CsvTableSource(
+      tempFile.getAbsolutePath,
+      Array("first", "id", "score", "last"),
+      Array(
+        BasicTypeInfo.STRING_TYPE_INFO,
+        BasicTypeInfo.INT_TYPE_INFO,
+        BasicTypeInfo.DOUBLE_TYPE_INFO,
+        BasicTypeInfo.STRING_TYPE_INFO
+      ),
+      fieldDelim = "#",
+      rowDelim = "$",
+      ignoreFirstLine = true,
+      ignoreComments = "%"
+    )
+  }
+}


[43/47] flink git commit: [FLINK-5008] [docs] Update IDE setup and quickstart documentation.

Posted by fh...@apache.org.
[FLINK-5008] [docs] Update IDE setup and quickstart documentation.

This closes #2764.


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

Branch: refs/heads/master
Commit: 67c4be648b1f51ceadae3a9e3dd41807802a89ef
Parents: 22af6cf
Author: Nico Kruber <ni...@data-artisans.com>
Authored: Thu Nov 3 16:19:49 2016 +0100
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Dec 16 16:41:21 2016 +0100

----------------------------------------------------------------------
 README.md                                       |  24 +---
 docs/internals/ide_setup.md                     | 129 +++++++------------
 docs/page/img/quickstart-setup/jobmanager-1.png | Bin 118647 -> 66956 bytes
 docs/page/img/quickstart-setup/jobmanager-2.png | Bin 136935 -> 77840 bytes
 docs/page/img/quickstart-setup/jobmanager-3.png | Bin 184257 -> 97567 bytes
 docs/page/img/quickstart-setup/setup.gif        | Bin 495692 -> 0 bytes
 docs/quickstart/java_api_quickstart.md          |  69 ++++++++--
 docs/quickstart/run_example_quickstart.md       |  34 ++---
 docs/quickstart/scala_api_quickstart.md         |  71 ++++++++--
 docs/quickstart/setup_quickstart.md             |  89 ++++++++-----
 .../examples/socket/SocketWindowWordCount.java  |   4 +-
 .../examples/socket/SocketWindowWordCount.scala |   2 +-
 12 files changed, 240 insertions(+), 182 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 8fd533b..025dfb9 100644
--- a/README.md
+++ b/README.md
@@ -85,7 +85,7 @@ To build unit tests with Java 8, use Java 8u51 or above to prevent failures in u
 
 ## Developing Flink
 
-The Flink committers use IntelliJ IDEA and Eclipse IDE to develop the Flink codebase.
+The Flink committers use IntelliJ IDEA to develop the Flink codebase.
 We recommend IntelliJ IDEA for developing projects that involve Scala code.
 
 Minimal requirements for an IDE are:
@@ -104,25 +104,11 @@ Check out our [Setting up IntelliJ](https://github.com/apache/flink/blob/master/
 
 ### Eclipse Scala IDE
 
-For Eclipse users, we recommend using Scala IDE 3.0.3, based on Eclipse Kepler. While this is a slightly older version,
-we found it to be the version that works most robustly for a complex project like Flink.
-
-Further details, and a guide to newer Scala IDE versions can be found in the
-[How to setup Eclipse](https://github.com/apache/flink/blob/master/docs/internals/ide_setup.md#eclipse) docs.
-
-**Note:** Before following this setup, make sure to run the build from the command line once
-(`mvn clean install -DskipTests`, see above)
-
-1. Download the Scala IDE (preferred) or install the plugin to Eclipse Kepler. See 
-   [How to setup Eclipse](https://github.com/apache/flink/blob/master/docs/internals/ide_setup.md#eclipse) for download links and instructions.
-2. Add the "macroparadise" compiler plugin to the Scala compiler.
-   Open "Window" -> "Preferences" -> "Scala" -> "Compiler" -> "Advanced" and put into the "Xplugin" field the path to
-   the *macroparadise* jar file (typically "/home/*-your-user-*/.m2/repository/org/scalamacros/paradise_2.10.4/2.0.1/paradise_2.10.4-2.0.1.jar").
-   Note: If you do not have the jar file, you probably did not run the command line build.
-3. Import the Flink Maven projects ("File" -> "Import" -> "Maven" -> "Existing Maven Projects") 
-4. During the import, Eclipse will ask to automatically install additional Maven build helper plugins.
-5. Close the "flink-java8" project. Since Eclipse Kepler does not support Java 8, you cannot develop this project.
+**NOTE:** From our experience, this setup does not work with Flink
+due to deficiencies of the old Eclipse version bundled with Scala IDE 3.0.3 or
+due to version incompatibilities with the bundled Scala version in Scala IDE 4.4.1.
 
+**We recommend to use IntelliJ instead (see above)**
 
 ## Support
 

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/docs/internals/ide_setup.md
----------------------------------------------------------------------
diff --git a/docs/internals/ide_setup.md b/docs/internals/ide_setup.md
index 1c514d0..f170588 100644
--- a/docs/internals/ide_setup.md
+++ b/docs/internals/ide_setup.md
@@ -25,105 +25,66 @@ under the License.
 * Replaced by the TOC
 {:toc}
 
+The sections below describe how to import the Flink project into an IDE
+for the development of Flink itself. For writing Flink programs, please
+refer to the [Java API]({{ site.baseurl }}/quickstart/java_api_quickstart.html)
+and the [Scala API]({{ site.baseurl }}/quickstart/scala_api_quickstart.html)
+quickstart guides.
+
+**NOTE:** Whenever something is not working in your IDE, try with the Maven
+command line first (`mvn clean package -DskipTests`) as it might be your IDE
+that has a bug or is not properly set up.
+
+## Preparation
+
+To get started, please first checkout the Flink sources from one of our
+[repositories](https://flink.apache.org/community.html#source-code),
+e.g.
+{% highlight bash %}
+git clone https://github.com/apache/flink.git
+{% endhighlight %}
+
 ## IntelliJ IDEA
 
 A brief guide on how to set up IntelliJ IDEA IDE for development of the Flink core.
 As Eclipse is known to have issues with mixed Scala and Java projects, more and more contributers are migrating to IntelliJ IDEA.
 
-The following documentation describes the steps to setup IntelliJ IDEA 14.0.3 (https://www.jetbrains.com/idea/download/) with the Flink sources.
-
-Prior to doing anything, make sure that the Flink project is built at least once from the terminal:
-`mvn clean package -DskipTests`
+The following documentation describes the steps to setup IntelliJ IDEA 2016.2.5
+([https://www.jetbrains.com/idea/download/](https://www.jetbrains.com/idea/download/))
+with the Flink sources.
 
 ### Installing the Scala plugin
-1. Go to IntelliJ plugins settings (File -> Settings -> Plugins) and click on "Install Jetbrains plugin...".
+
+The IntelliJ installation setup offers to install the Scala plugin.
+If it is not installed, follow these instructions before importing Flink
+to enable support for Scala projects and files:
+
+1. Go to IntelliJ plugins settings (File -> Settings -> Plugins) and
+   click on "Install Jetbrains plugin...".
 2. Select and install the "Scala" plugin.
 3. Restart IntelliJ
 
-### Installing the Scala compiler plugin
-1. Go to IntelliJ scala compiler settings (File -> Settings -> Build, Execution, Deployment -> Compiler -> Scala Compiler) and click on "Install Jetbrains plugin...".
-2. Click on the green plus icon on the right to add a compiler plugin
-3. Point to the paradise jar: ~/.m2/repository/org/scalamacros/paradise_2.10.4/2.0.1/paradise_2.10.4-2.0.1.jar If there is no such file, this means that you should build Flink from the terminal as explained above.
-
 ### Importing Flink
+
 1. Start IntelliJ IDEA and choose "Import Project"
 2. Select the root folder of the Flink repository
 3. Choose "Import project from external model" and select "Maven"
-4. Leave the default options and finish the import.
+4. Leave the default options and click on "Next" until you hit the SDK section.
+5. If there is no SDK, create a one with the "+" sign top left,
+   then click "JDK", select your JDK home directory and click "OK".
+   Otherwise simply select your SDK.
+6. Continue by clicking "Next" again and finish the import.
+7. Right-click on the imported Flink project -> Maven -> Generate Sources and Update Folders.
+   Note that this will install Flink libraries in your local Maven repository,
+   i.e. "/home/*-your-user-*/.m2/repository/org/apache/flink/".
+   Alternatively, `mvn clean package -DskipTests` also creates the necessary
+   files for the IDE to work with but without installing libraries.
+8. Build the Project (Build -> Make Project)
 
 ## Eclipse
 
-A brief guide how to set up Eclipse for development of the Flink core.
-Flink uses mixed Scala/Java projects, which pose a challenge to some IDEs.
-Below is the setup guide that works best from our personal experience.
-
-For Eclipse users, we currently recomment the Scala IDE 3.0.3, as the most robust solution.
-
-
-### Eclipse Scala IDE 3.0.3
-
-**NOTE:** While this version of the Scala IDE is not the newest, we have found it to be the most reliably working
-version for complex projects like Flink. One restriction is, though, that it works only with Java 7, not with Java 8.
-
-**Note:** Before following this setup, make sure to run the build from the command line once
-(`mvn clean package -DskipTests`)
-
-1. Download the Scala IDE (preferred) or install the plugin to Eclipse Kepler. See section below for download links
-   and instructions.
-2. Add the "macroparadise" compiler plugin to the Scala compiler.
-   Open "Window" -> "Preferences" -> "Scala" -> "Compiler" -> "Advanced" and put into the "Xplugin" field the path to
-   the *macroparadise* jar file (typically "/home/*-your-user-*/.m2/repository/org/scalamacros/paradise_2.10.4/2.0.1/paradise_2.10.4-2.0.1.jar").
-   Note: If you do not have the jar file, you probably did not ran the command line build.
-3. Import the Flink Maven projects ("File" -> "Import" -> "Maven" -> "Existing Maven Projects")
-4. During the import, Eclipse will ask to automatically install additional Maven build helper plugins.
-5. Close the "flink-java8" project. Since Eclipse Kepler does not support Java 8, you cannot develop this project.
-
-
-#### Download links for Scala IDE 3.0.3
-
-The Scala IDE 3.0.3 is a previous stable release, and download links are a bit hidden.
-
-The pre-packaged Scala IDE can be downloaded from the following links:
-
-* [Linux (64 bit)](http://downloads.typesafe.com/scalaide-pack/3.0.3.vfinal-210-20140327/scala-SDK-3.0.3-2.10-linux.gtk.x86_64.tar.gz)
-* [Linux (32 bit)](http://downloads.typesafe.com/scalaide-pack/3.0.3.vfinal-210-20140327/scala-SDK-3.0.3-2.10-linux.gtk.x86.tar.gz)
-* [MaxOS X Cocoa (64 bit)](http://downloads.typesafe.com/scalaide-pack/3.0.3.vfinal-210-20140327/scala-SDK-3.0.3-2.10-macosx.cocoa.x86_64.zip)
-* [MaxOS X Cocoa (32 bit)](http://downloads.typesafe.com/scalaide-pack/3.0.3.vfinal-210-20140327/scala-SDK-3.0.3-2.10-macosx.cocoa.x86.zip)
-* [Windows (64 bit)](http://downloads.typesafe.com/scalaide-pack/3.0.3.vfinal-210-20140327/scala-SDK-3.0.3-2.10-win32.win32.x86_64.zip)
-* [Windows (32 bit)](http://downloads.typesafe.com/scalaide-pack/3.0.3.vfinal-210-20140327/scala-SDK-3.0.3-2.10-win32.win32.x86.zip)
-
-Alternatively, you can download Eclipse Kepler from [https://eclipse.org/downloads/packages/release/Kepler/SR2](https://eclipse.org/downloads/packages/release/Kepler/SR2)
-and manually add the Scala and Maven plugins by plugin site at [http://scala-ide.org/download/prev-stable.html](http://scala-ide.org/download/prev-stable.html).
-
-* Either use the update site to install the plugin ("Help" -> "Install new Software")
-* Or download the [zip file](http://download.scala-ide.org/sdk/helium/e38/scala211/stable/update-site.zip), unpack it, and move the contents of the
-  "plugins" and "features" folders into the equally named folders of the Eclipse root directory
-
-**NOTE:** It might happen that some modules do not build in Eclipse correctly (even if the maven build succeeds).
-To fix this, right-click in the corresponding Eclipse project and choose "Properties" and than "Maven".
-Uncheck the box labeled "Resolve dependencies from Workspace projects", click "Apply" and then "OK". "
-
-
-### Eclipse Scala IDE 4.0.0
-
-**NOTE: From personal experience, the use of the Scala IDE 4.0.0 performs worse than previous versions for complex projects like Flink.**
-**Version 4.0.0 does not handle mixed Java/Scala projects as robustly and it frequently raises incorrect import and type errors.**
-
-*Note:* Before following this setup, make sure to run the build from the command line once
-(`mvn clean package -DskipTests`)
-
-1. Download the Scala IDE: [http://scala-ide.org/download/sdk.html](http://scala-ide.org/download/sdk.html)
-2. Import the Flink Maven projects (File -> Import -> Maven -> Existing Maven Projects)
-3. While importing the Flink project, the IDE may ask you to install an additional maven build helper plugin.
-4. After the import, you need to set the Scala version of your projects to Scala 2.10 (from the default 2.11).
-   To do that, select all projects that contain Scala code (marked by the small *S* on the project icon),
-   right click and select "Scala -> Set the Scala Installation" and pick "2.10.4".
-   Currently, the project to which that is relevant are "flink-runtime", "flink-scala", "flink-scala-examples",
-   "flink-streaming-example", "flink-streaming-scala", "flink-tests", "flink-test-utils", and "flink-yarn".
-5. Depending on your version of the Scala IDE, you may need to add the "macroparadise" compiler plugin to the
-   Scala compiler. Open "Window" -> "Preferences" -> "Scala" -> "Compiler" -> "Advanced" and put into the "Xplugin" field
-   the path to the *macroparadise* jar file (typically "/home/*-your-user-*/.m2/repository/org/scalamacros/paradise_2.10.4/2.0.1/paradise_2.10.4-2.0.1.jar")
-6. In order to compile the "flink-java-8" project, you may need to add a Java 8 execution environment.
-   See [this post](http://stackoverflow.com/questions/25391207/how-do-i-add-execution-environment-1-8-to-eclipse-luna)
-   for details.
+**NOTE:** From our experience, this setup does not work with Flink
+due to deficiencies of the old Eclipse version bundled with Scala IDE 3.0.3 or
+due to version incompatibilities with the bundled Scala version in Scala IDE 4.4.1.
 
+**We recommend to use IntelliJ instead (see [above](#intellij-idea))**

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/docs/page/img/quickstart-setup/jobmanager-1.png
----------------------------------------------------------------------
diff --git a/docs/page/img/quickstart-setup/jobmanager-1.png b/docs/page/img/quickstart-setup/jobmanager-1.png
index c6ecfb1..41243de 100644
Binary files a/docs/page/img/quickstart-setup/jobmanager-1.png and b/docs/page/img/quickstart-setup/jobmanager-1.png differ

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/docs/page/img/quickstart-setup/jobmanager-2.png
----------------------------------------------------------------------
diff --git a/docs/page/img/quickstart-setup/jobmanager-2.png b/docs/page/img/quickstart-setup/jobmanager-2.png
index 724f21e..09e7169 100644
Binary files a/docs/page/img/quickstart-setup/jobmanager-2.png and b/docs/page/img/quickstart-setup/jobmanager-2.png differ

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/docs/page/img/quickstart-setup/jobmanager-3.png
----------------------------------------------------------------------
diff --git a/docs/page/img/quickstart-setup/jobmanager-3.png b/docs/page/img/quickstart-setup/jobmanager-3.png
index 8572cf1..ac9e1ef 100644
Binary files a/docs/page/img/quickstart-setup/jobmanager-3.png and b/docs/page/img/quickstart-setup/jobmanager-3.png differ

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/docs/page/img/quickstart-setup/setup.gif
----------------------------------------------------------------------
diff --git a/docs/page/img/quickstart-setup/setup.gif b/docs/page/img/quickstart-setup/setup.gif
deleted file mode 100644
index bc9aad6..0000000
Binary files a/docs/page/img/quickstart-setup/setup.gif and /dev/null differ

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/docs/quickstart/java_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/java_api_quickstart.md b/docs/quickstart/java_api_quickstart.md
index c8f21d8..7384d91 100644
--- a/docs/quickstart/java_api_quickstart.md
+++ b/docs/quickstart/java_api_quickstart.md
@@ -46,38 +46,80 @@ Use one of the following commands to __create a project__:
     {% highlight bash %}
     $ mvn archetype:generate                               \
       -DarchetypeGroupId=org.apache.flink              \
-      -DarchetypeArtifactId=flink-quickstart-java      \
+      -DarchetypeArtifactId=flink-quickstart-java      \{% unless site.is_stable %}
+      -DarchetypeCatalog=https://repository.apache.org/content/repositories/snapshots/ \{% endunless %}
       -DarchetypeVersion={{site.version}}
     {% endhighlight %}
         This allows you to <strong>name your newly created project</strong>. It will interactively ask you for the groupId, artifactId, and package name.
     </div>
     <div class="tab-pane" id="quickstart-script">
     {% highlight bash %}
+{% if site.is_stable %}
     $ curl https://flink.apache.org/q/quickstart.sh | bash
+{% else %}
+    $ curl https://flink.apache.org/q/quickstart-SNAPSHOT.sh | bash
+{% endif %}
     {% endhighlight %}
     </div>
 </div>
 
 ## Inspect Project
 
-There will be a new directory in your working directory. If you've used the _curl_ approach, the directory is called `quickstart`. Otherwise, it has the name of your artifactId.
+There will be a new directory in your working directory. If you've used
+the _curl_ approach, the directory is called `quickstart`. Otherwise,
+it has the name of your `artifactId`:
+
+{% highlight bash %}
+$ tree quickstart/
+quickstart/
+\u251c\u2500\u2500 pom.xml
+\u2514\u2500\u2500 src
+    \u2514\u2500\u2500 main
+        \u251c\u2500\u2500 java
+        \u2502�� \u2514\u2500\u2500 org
+        \u2502��     \u2514\u2500\u2500 myorg
+        \u2502��         \u2514\u2500\u2500 quickstart
+        \u2502��             \u251c\u2500\u2500 BatchJob.java
+        \u2502��             \u251c\u2500\u2500 SocketTextStreamWordCount.java
+        \u2502��             \u251c\u2500\u2500 StreamingJob.java
+        \u2502��             \u2514\u2500\u2500 WordCount.java
+        \u2514\u2500\u2500 resources
+            \u2514\u2500\u2500 log4j.properties
+{% endhighlight %}
 
 The sample project is a __Maven project__, which contains four classes. _StreamingJob_ and _BatchJob_ are basic skeleton programs, _SocketTextStreamWordCount_ is a working streaming example and _WordCountJob_ is a working batch example. Please note that the _main_ method of all classes allow you to start Flink in a development/testing mode.
 
-We recommend you [import this project into your IDE]({{ site.baseurl }}/internals/ide_setup) to develop and test it. 
+We recommend you __import this project into your IDE__ to develop and
+test it. If you use Eclipse, the [m2e plugin](http://www.eclipse.org/m2e/)
+allows to [import Maven projects](http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import).
+Some Eclipse bundles include that plugin by default, others require you
+to install it manually. The IntelliJ IDE supports Maven projects out of
+the box.
 
-A note to Mac OS X users: The default JVM heapsize for Java is too small for Flink. You have to manually increase it. Choose "Run Configurations" -> Arguments and write into the "VM Arguments" box: "-Xmx800m" in Eclipse.
+
+*A note to Mac OS X users*: The default JVM heapsize for Java is too
+small for Flink. You have to manually increase it. In Eclipse, choose
+`Run Configurations -> Arguments` and write into the `VM Arguments`
+box: `-Xmx800m`.
 
 ## Build Project
 
-If you want to __build your project__, go to your project directory and issue the `mvn clean install -Pbuild-jar` command. You will __find a jar__ that runs on every Flink cluster in __target/your-artifact-id-{{ site.version }}.jar__. There is also a fat-jar,  __target/your-artifact-id-{{ site.version }}-flink-fat-jar.jar__. This
-also contains all dependencies that get added to the maven project.
+If you want to __build your project__, go to your project directory and
+issue the `mvn clean install -Pbuild-jar` command. You will
+__find a jar__ that runs on every Flink cluster with a compatible
+version, __target/original-your-artifact-id-your-version.jar__. There
+is also a fat-jar in __target/your-artifact-id-your-version.jar__ which,
+additionally, contains all dependencies that were added to the Maven
+project.
 
 ## Next Steps
 
 Write your application!
 
-The quickstart project contains a WordCount implementation, the "Hello World" of Big Data processing systems. The goal of WordCount is to determine the frequencies of words in a text, e.g., how often do the terms "the" or "house" occur in all Wikipedia texts.
+The quickstart project contains a `WordCount` implementation, the
+"Hello World" of Big Data processing systems. The goal of `WordCount`
+is to determine the frequencies of words in a text, e.g., how often do
+the terms "the" or "house" occur in all Wikipedia texts.
 
 __Sample Input__:
 
@@ -93,7 +135,10 @@ data 1
 is 1
 ~~~
 
-The following code shows the WordCount implementation from the Quickstart which processes some text lines with two operators (FlatMap and Reduce), and prints the resulting words and counts to std-out.
+The following code shows the `WordCount` implementation from the
+Quickstart which processes some text lines with two operators (a FlatMap
+and a Reduce operation via aggregating a sum), and prints the resulting
+words and counts to std-out.
 
 ~~~java
 public class WordCount {
@@ -116,9 +161,9 @@ public class WordCount {
         text.flatMap(new LineSplitter())
         // group by the tuple field "0" and sum up tuple field "1"
         .groupBy(0)
-        .aggregate(Aggregations.SUM, 1);
+        .sum(1);
 
-    // emit result
+    // execute and print result
     counts.print();
   }
 }
@@ -127,11 +172,11 @@ public class WordCount {
 The operations are defined by specialized classes, here the LineSplitter class.
 
 ~~~java
-public class LineSplitter implements FlatMapFunction<String, Tuple2<String, Integer>> {
+public static final class LineSplitter implements FlatMapFunction<String, Tuple2<String, Integer>> {
 
   @Override
   public void flatMap(String value, Collector<Tuple2<String, Integer>> out) {
-    // normalize and split the line into words
+    // normalize and split the line
     String[] tokens = value.toLowerCase().split("\\W+");
 
     // emit the pairs

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/docs/quickstart/run_example_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/run_example_quickstart.md b/docs/quickstart/run_example_quickstart.md
index 9ce4c84..d68a8a5 100644
--- a/docs/quickstart/run_example_quickstart.md
+++ b/docs/quickstart/run_example_quickstart.md
@@ -41,15 +41,16 @@ see [Java API Quickstart]({{ site.baseurl }}/quickstart/java_api_quickstart.html
 about this. For our purposes, the command to run is this:
 
 {% highlight bash %}
-$ mvn archetype:generate\
-    -DarchetypeGroupId=org.apache.flink\
-    -DarchetypeArtifactId=flink-quickstart-java\
-    -DarchetypeVersion={{ site.version }}\
-    -DgroupId=wiki-edits\
-    -DartifactId=wiki-edits\
-    -Dversion=0.1\
-    -Dpackage=wikiedits\
-    -DinteractiveMode=false\
+$ mvn archetype:generate \
+    -DarchetypeGroupId=org.apache.flink \
+    -DarchetypeArtifactId=flink-quickstart-java \{% unless site.is_stable %}
+    -DarchetypeCatalog=https://repository.apache.org/content/repositories/snapshots/ \{% endunless %}
+    -DarchetypeVersion={{ site.version }} \
+    -DgroupId=wiki-edits \
+    -DartifactId=wiki-edits \
+    -Dversion=0.1 \
+    -Dpackage=wikiedits \
+    -DinteractiveMode=false
 {% endhighlight %}
 
 You can edit the `groupId`, `artifactId` and `package` if you like. With the above parameters,
@@ -63,8 +64,9 @@ wiki-edits/
     \u2514\u2500\u2500 main
         \u251c\u2500\u2500 java
         \u2502�� \u2514\u2500\u2500 wikiedits
-        \u2502��     \u251c\u2500\u2500 Job.java
+        \u2502��     \u251c\u2500\u2500 BatchJob.java
         \u2502��     \u251c\u2500\u2500 SocketTextStreamWordCount.java
+        \u2502��     \u251c\u2500\u2500 StreamingJob.java
         \u2502��     \u2514\u2500\u2500 WordCount.java
         \u2514\u2500\u2500 resources
             \u2514\u2500\u2500 log4j.properties
@@ -79,7 +81,7 @@ $ rm wiki-edits/src/main/java/wikiedits/*.java
 {% endhighlight %}
 
 As a last step we need to add the Flink Wikipedia connector as a dependency so that we can
-use it in our program. Edit the `dependencies` section so that it looks like this:
+use it in our program. Edit the `dependencies` section of the `pom.xml` so that it looks like this:
 
 {% highlight xml %}
 <dependencies>
@@ -90,23 +92,23 @@ use it in our program. Edit the `dependencies` section so that it looks like thi
     </dependency>
     <dependency>
         <groupId>org.apache.flink</groupId>
-        <artifactId>flink-streaming-java_2.10</artifactId>
+        <artifactId>flink-streaming-java_2.11</artifactId>
         <version>${flink.version}</version>
     </dependency>
     <dependency>
         <groupId>org.apache.flink</groupId>
-        <artifactId>flink-clients_2.10</artifactId>
+        <artifactId>flink-clients_2.11</artifactId>
         <version>${flink.version}</version>
     </dependency>
     <dependency>
         <groupId>org.apache.flink</groupId>
-        <artifactId>flink-connector-wikiedits_2.10</artifactId>
+        <artifactId>flink-connector-wikiedits_2.11</artifactId>
         <version>${flink.version}</version>
     </dependency>
 </dependencies>
 {% endhighlight %}
 
-Notice the `flink-connector-wikiedits_2.10` dependency that was added. (This example and
+Notice the `flink-connector-wikiedits_2.11` dependency that was added. (This example and
 the Wikipedia connector were inspired by the *Hello Samza* example of Apache Samza.)
 
 ## Writing a Flink Program
@@ -295,7 +297,7 @@ use the Kafka sink. Add this to the `pom.xml` file in the dependencies section:
 {% highlight xml %}
 <dependency>
     <groupId>org.apache.flink</groupId>
-    <artifactId>flink-connector-kafka-0.8_2.10</artifactId>
+    <artifactId>flink-connector-kafka-0.8_2.11</artifactId>
     <version>${flink.version}</version>
 </dependency>
 {% endhighlight %}

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/docs/quickstart/scala_api_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/scala_api_quickstart.md b/docs/quickstart/scala_api_quickstart.md
index 388f0a7..8ee1e66 100644
--- a/docs/quickstart/scala_api_quickstart.md
+++ b/docs/quickstart/scala_api_quickstart.md
@@ -136,14 +136,19 @@ Use one of the following commands to __create a project__:
     {% highlight bash %}
     $ mvn archetype:generate                               \
       -DarchetypeGroupId=org.apache.flink              \
-      -DarchetypeArtifactId=flink-quickstart-scala     \
+      -DarchetypeArtifactId=flink-quickstart-scala     \{% unless site.is_stable %}
+      -DarchetypeCatalog=https://repository.apache.org/content/repositories/snapshots/ \{% endunless %}
       -DarchetypeVersion={{site.version}}
     {% endhighlight %}
     This allows you to <strong>name your newly created project</strong>. It will interactively ask you for the groupId, artifactId, and package name.
     </div>
     <div class="tab-pane" id="quickstart-script">
 {% highlight bash %}
-$ curl https://flink.apache.org/q/quickstart-scala.sh | bash
+{% if site.is_stable %}
+    $ curl https://flink.apache.org/q/quickstart-scala.sh | bash
+{% else %}
+    $ curl https://flink.apache.org/q/quickstart-scala-SNAPSHOT.sh | bash
+{% endif %}
 {% endhighlight %}
     </div>
 </div>
@@ -151,34 +156,63 @@ $ curl https://flink.apache.org/q/quickstart-scala.sh | bash
 
 ### Inspect Project
 
-There will be a new directory in your working directory. If you've used the _curl_ approach, the directory is called `quickstart`. Otherwise, it has the name of your artifactId.
+There will be a new directory in your working directory. If you've used
+the _curl_ approach, the directory is called `quickstart`. Otherwise,
+it has the name of your `artifactId`:
+
+{% highlight bash %}
+$ tree quickstart/
+quickstart/
+\u251c\u2500\u2500 pom.xml
+\u2514\u2500\u2500 src
+    \u2514\u2500\u2500 main
+        \u251c\u2500\u2500 resources
+        \u2502�� \u2514\u2500\u2500 log4j.properties
+        \u2514\u2500\u2500 scala
+            \u2514\u2500\u2500 org
+                \u2514\u2500\u2500 myorg
+                    \u2514\u2500\u2500 quickstart
+                        \u251c\u2500\u2500 BatchJob.scala
+                        \u251c\u2500\u2500 SocketTextStreamWordCount.scala
+                        \u251c\u2500\u2500 StreamingJob.scala
+                        \u2514\u2500\u2500 WordCount.scala
+{% endhighlight %}
 
 The sample project is a __Maven project__, which contains four classes. _StreamingJob_ and _BatchJob_ are basic skeleton programs, _SocketTextStreamWordCount_ is a working streaming example and _WordCountJob_ is a working batch example. Please note that the _main_ method of all classes allow you to start Flink in a development/testing mode.
 
 We recommend you __import this project into your IDE__. For Eclipse, you need the following plugins, which you can install from the provided Eclipse Update Sites:
 
 * _Eclipse 4.x_
-  * [Scala IDE](http://download.scala-ide.org/sdk/e38/scala210/stable/site)
+  * [Scala IDE](http://download.scala-ide.org/sdk/lithium/e44/scala211/stable/site)
   * [m2eclipse-scala](http://alchim31.free.fr/m2e-scala/update-site)
-  * [Build Helper Maven Plugin](https://repository.sonatype.org/content/repositories/forge-sites/m2e-extras/0.15.0/N/0.15.0.201206251206/)
-* _Eclipse 3.7_
-  * [Scala IDE](http://download.scala-ide.org/sdk/e37/scala210/stable/site)
+  * [Build Helper Maven Plugin](https://repo1.maven.org/maven2/.m2e/connectors/m2eclipse-buildhelper/0.15.0/N/0.15.0.201207090124/)
+* _Eclipse 3.8_
+  * [Scala IDE for Scala 2.11](http://download.scala-ide.org/sdk/helium/e38/scala211/stable/site) or [Scala IDE for Scala 2.10](http://download.scala-ide.org/sdk/helium/e38/scala210/stable/site)
   * [m2eclipse-scala](http://alchim31.free.fr/m2e-scala/update-site)
   * [Build Helper Maven Plugin](https://repository.sonatype.org/content/repositories/forge-sites/m2e-extras/0.14.0/N/0.14.0.201109282148/)
 
-The IntelliJ IDE also supports Maven and offers a plugin for Scala development.
+The IntelliJ IDE supports Maven out of the box and offers a plugin for
+Scala development.
 
 
 ### Build Project
 
-If you want to __build your project__, go to your project directory and issue the `mvn clean package -Pbuild-jar` command. You will __find a jar__ that runs on every Flink cluster in __target/your-artifact-id-{{ site.version }}.jar__. There is also a fat-jar,  __target/your-artifact-id-{{ site.version }}-flink-fat-jar.jar__. This
-also contains all dependencies that get added to the maven project.
+If you want to __build your project__, go to your project directory and
+issue the `mvn clean package -Pbuild-jar` command. You will
+__find a jar__ that runs on every Flink cluster with a compatible
+version, __target/original-your-artifact-id-your-version.jar__. There
+is also a fat-jar in  __target/your-artifact-id-your-version.jar__ which,
+additionally, contains all dependencies that were added to the Maven
+project.
 
 ## Next Steps
 
 Write your application!
 
-The quickstart project contains a WordCount implementation, the "Hello World" of Big Data processing systems. The goal of WordCount is to determine the frequencies of words in a text, e.g., how often do the terms "the" or "house" occur in all Wikipedia texts.
+The quickstart project contains a `WordCount` implementation, the
+"Hello World" of Big Data processing systems. The goal of `WordCount`
+is to determine the frequencies of words in a text, e.g., how often do
+the terms "the" or "house" occur in all Wikipedia texts.
 
 __Sample Input__:
 
@@ -194,7 +228,10 @@ data 1
 is 1
 ~~~
 
-The following code shows the WordCount implementation from the Quickstart which processes some text lines with two operators (FlatMap and Reduce), and prints the resulting words and counts to std-out.
+The following code shows the `WordCount` implementation from the
+Quickstart which processes some text lines with two operators (a FlatMap
+and a Reduce operation via aggregating a sum), and prints the resulting
+words and counts to std-out.
 
 ~~~scala
 object WordCountJob {
@@ -213,7 +250,7 @@ object WordCountJob {
       .groupBy(0)
       .sum(1)
 
-    // emit result
+    // emit result and print result
     counts.print()
   }
 }
@@ -221,4 +258,10 @@ object WordCountJob {
 
 {% gh_link flink-examples/flink-examples-batch/src/main/scala/org/apache/flink/examples/scala/wordcount/WordCount.scala "Check GitHub" %} for the full example code.
 
-For a complete overview over our API, have a look at the [DataStream API]({{ site.baseurl }}/dev/datastream_api.html) and [DataSet API]({{ site.baseurl }}/dev/batch/index.html) sections. If you have any trouble, ask on our [Mailing List](http://mail-archives.apache.org/mod_mbox/flink-dev/). We are happy to provide help.
+For a complete overview over our API, have a look at the
+[DataStream API]({{ site.baseurl }}/dev/datastream_api.html),
+[DataSet API]({{ site.baseurl }}/dev/batch/index.html), and
+[Scala API Extensions]({{ site.baseurl }}/dev/scala_api_extensions.html)
+sections. If you have any trouble, ask on our
+[Mailing List](http://mail-archives.apache.org/mod_mbox/flink-dev/).
+We are happy to provide help.

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/docs/quickstart/setup_quickstart.md
----------------------------------------------------------------------
diff --git a/docs/quickstart/setup_quickstart.md b/docs/quickstart/setup_quickstart.md
index 6aab686..0628176 100644
--- a/docs/quickstart/setup_quickstart.md
+++ b/docs/quickstart/setup_quickstart.md
@@ -48,21 +48,34 @@ Java HotSpot(TM) 64-Bit Server VM (build 25.111-b14, mixed mode)
 
 ### Download
 
-Download a binary from the [downloads page](http://flink.apache.org/downloads.html). You can pick
-any Hadoop/Scala combination you like. If you plan to just use the local file system, any Hadoop
-version will work fine.
-
-### Start a Local Flink Cluster
-
-1. Go to the download directory.
-2. Unpack the downloaded archive.
-3. Start Flink.
+{% if site.is_stable %}
+1. Download a binary from the [downloads page](http://flink.apache.org/downloads.html). You can pick
+   any Hadoop/Scala combination you like. If you plan to just use the local file system, any Hadoop
+   version will work fine.
+2. Go to the download directory.
+3. Unpack the downloaded archive.
 
 ~~~bash
 $ cd ~/Downloads        # Go to download directory
 $ tar xzf flink-*.tgz   # Unpack the downloaded archive
 $ cd flink-{{site.version}}
-$ bin/start-local.sh    # Start Flink
+~~~
+{% else %}
+Clone the source code from one of our
+[repositories](http://flink.apache.org/community.html#source-code), e.g.:
+
+~~~bash
+$ git clone https://github.com/apache/flink.git
+$ cd flink
+$ mvn clean package -DskipTests # this will take up to 10 minutes
+$ cd build-target               # this is where Flink is installed to
+~~~
+{% endif %}
+
+### Start a Local Flink Cluster
+
+~~~bash
+$ ./bin/start-local.sh  # Start Flink
 ~~~
 
 Check the __JobManager's web frontend__ at [http://localhost:8081](http://localhost:8081) and make sure everything is up and running. The web frontend should report a single available TaskManager instance.
@@ -89,7 +102,7 @@ You can find the complete source code for this SocketWindowWordCount example in
 object SocketWindowWordCount {
 
     def main(args: Array[String]) : Unit = {
-    
+
         // the port to connect to
         val port: Int = try {
             ParameterTool.fromArgs(args).getInt("port")
@@ -102,11 +115,11 @@ object SocketWindowWordCount {
 
         // get the execution environment
         val env: StreamExecutionEnvironment = StreamExecutionEnvironment.getExecutionEnvironment
-    
+
         // get input data by connecting to the socket
         val text = env.socketTextStream("localhost", port, '\n')
 
-        // parse the data, group it, window it, and aggregate the counts 
+        // parse the data, group it, window it, and aggregate the counts
         val windowCounts = text
             .flatMap { w => w.split("\\s") }
             .map { w => WordWithCount(w, 1) }
@@ -197,7 +210,10 @@ public class SocketWindowWordCount {
 
 ## Run the Example
 
-Now, we are going to run this Flink application. It will read text from a socket and once a second print the number of occurances of each distinct word during the previous 5 seconds.
+Now, we are going to run this Flink application. It will read text from
+a socket and once every 5 seconds print the number of occurrences of
+each distinct word during the previous 5 seconds, i.e. a tumbling
+window of processing time, as long as words are floating in.
 
 * First of all, we use **netcat** to start local server via
 
@@ -208,15 +224,21 @@ Now, we are going to run this Flink application. It will read text from a socket
 * Submit the Flink program:
 
   ~~~bash
-  $ bin/flink run examples/streaming/SocketWindowWordCount.jar --port 9000
-
-  03/08/2016 17:21:56 Job execution switched to status RUNNING.
-  03/08/2016 17:21:56 Source: Socket Stream -> Flat Map(1/1) switched to SCHEDULED
-  03/08/2016 17:21:56 Source: Socket Stream -> Flat Map(1/1) switched to DEPLOYING
-  03/08/2016 17:21:56 Keyed Aggregation -> Sink: Unnamed(1/1) switched to SCHEDULED
-  03/08/2016 17:21:56 Keyed Aggregation -> Sink: Unnamed(1/1) switched to DEPLOYING
-  03/08/2016 17:21:56 Source: Socket Stream -> Flat Map(1/1) switched to RUNNING
-  03/08/2016 17:21:56 Keyed Aggregation -> Sink: Unnamed(1/1) switched to RUNNING
+  $ ./bin/flink run examples/streaming/SocketWindowWordCount.jar --port 9000
+
+  Cluster configuration: Standalone cluster with JobManager at /127.0.0.1:6123
+  Using address 127.0.0.1:6123 to connect to JobManager.
+  JobManager web interface address http://127.0.0.1:8081
+  Starting execution of program
+  Submitting job with JobID: 574a10c8debda3dccd0c78a3bde55e1b. Waiting for job completion.
+  Connected to JobManager at Actor[akka.tcp://flink@127.0.0.1:6123/user/jobmanager#297388688]
+  11/04/2016 14:04:50     Job execution switched to status RUNNING.
+  11/04/2016 14:04:50     Source: Socket Stream -> Flat Map(1/1) switched to SCHEDULED
+  11/04/2016 14:04:50     Source: Socket Stream -> Flat Map(1/1) switched to DEPLOYING
+  11/04/2016 14:04:50     Fast TumblingProcessingTimeWindows(5000) of WindowedStream.main(SocketWindowWordCount.java:79) -> Sink: Unnamed(1/1) switched to SCHEDULED
+  11/04/2016 14:04:51     Fast TumblingProcessingTimeWindows(5000) of WindowedStream.main(SocketWindowWordCount.java:79) -> Sink: Unnamed(1/1) switched to DEPLOYING
+  11/04/2016 14:04:51     Fast TumblingProcessingTimeWindows(5000) of WindowedStream.main(SocketWindowWordCount.java:79) -> Sink: Unnamed(1/1) switched to RUNNING
+  11/04/2016 14:04:51     Source: Socket Stream -> Flat Map(1/1) switched to RUNNING
   ~~~
 
   The program connects to the socket and waits for input. You can check the web interface to verify that the job is running as expected:
@@ -230,7 +252,10 @@ Now, we are going to run this Flink application. It will read text from a socket
     </div>
   </div>
 
-* Counts are printed to `stdout`. Monitor the JobManager's output file and write some text in `nc`:
+* Words are counted in time windows of 5 seconds (processing time, tumbling
+  windows) and are printed to `stdout`. Monitor the JobManager's output file
+  and write some text in `nc` (input is sent to Flink line by line after
+  hitting <RETURN>):
 
   ~~~bash
   $ nc -l 9000
@@ -239,26 +264,22 @@ Now, we are going to run this Flink application. It will read text from a socket
   bye
   ~~~
 
-  The `.out` file will print the counts immediately:
+  The `.out` file will print the counts at the end of each time window as long
+  as words are floating in, e.g.:
 
   ~~~bash
   $ tail -f log/flink-*-jobmanager-*.out
-  (lorem,1)
-  (ipsum,1)
-  (ipsum,2)
-  (ipsum,3)
-  (ipsum,4)
-  (bye,1)
+  lorem : 1
+  bye : 1
+  ipsum : 4
   ~~~~
 
   To **stop** Flink when you're done type:
 
   ~~~bash
-  $ bin/stop-local.sh
+  $ ./bin/stop-local.sh
   ~~~
 
-  <a href="{{ site.baseurl }}/page/img/quickstart-setup/setup.gif" ><img class="img-responsive" src="{{ site.baseurl }}/page/img/quickstart-setup/setup.gif" alt="Quickstart: Setup"/></a>
-
 ## Next Steps
 
 Check out some more [examples]({{ site.baseurl }}/examples) to get a better feel for Flink's programming APIs. When you are done with that, go ahead and read the [streaming guide]({{ site.baseurl }}/dev/datastream_api.html).

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java
index fe1b6e7..dd2e061 100644
--- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java
+++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/socket/SocketWindowWordCount.java
@@ -74,7 +74,7 @@ public class SocketWindowWordCount {
 				})
 
 				.keyBy("word")
-				.timeWindow(Time.seconds(5), Time.seconds(1))
+				.timeWindow(Time.seconds(5))
 
 				.reduce(new ReduceFunction<WordWithCount>() {
 					@Override
@@ -111,4 +111,4 @@ public class SocketWindowWordCount {
 			return word + " : " + count;
 		}
 	}
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/67c4be64/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketWindowWordCount.scala
----------------------------------------------------------------------
diff --git a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketWindowWordCount.scala b/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketWindowWordCount.scala
index 3b432ec..1761b84 100644
--- a/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketWindowWordCount.scala
+++ b/flink-examples/flink-examples-streaming/src/main/scala/org/apache/flink/streaming/scala/examples/socket/SocketWindowWordCount.scala
@@ -62,7 +62,7 @@ object SocketWindowWordCount {
           .flatMap { w => w.split("\\s") }
           .map { w => WordWithCount(w, 1) }
           .keyBy("word")
-          .timeWindow(Time.seconds(5), Time.seconds(1))
+          .timeWindow(Time.seconds(5))
           .sum("count")
 
     // print the results with a single thread, rather than in parallel


[16/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/MinAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/MinAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/MinAggregate.scala
new file mode 100644
index 0000000..88cb058
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/MinAggregate.scala
@@ -0,0 +1,171 @@
+/*
+ * 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.math.BigDecimal
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.types.Row
+
+abstract class MinAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] {
+
+  protected var minIndex: Int = _
+
+  /**
+   * Initiate the intermediate aggregate value in Row.
+   *
+   * @param intermediate The intermediate aggregate row to initiate.
+   */
+  override def initiate(intermediate: Row): Unit = {
+    intermediate.setField(minIndex, null)
+  }
+
+  /**
+   * Accessed in MapFunction, prepare the input of partial aggregate.
+   *
+   * @param value
+   * @param partial
+   */
+  override def prepare(value: Any, partial: Row): Unit = {
+    if (value == null) {
+      initiate(partial)
+    } else {
+      partial.setField(minIndex, value)
+    }
+  }
+
+  /**
+   * Accessed in CombineFunction and GroupReduceFunction, merge partial
+   * aggregate result into aggregate buffer.
+   *
+   * @param partial
+   * @param buffer
+   */
+  override def merge(partial: Row, buffer: Row): Unit = {
+    val partialValue = partial.getField(minIndex).asInstanceOf[T]
+    if (partialValue != null) {
+      val bufferValue = buffer.getField(minIndex).asInstanceOf[T]
+      if (bufferValue != null) {
+        val min : T = if (ord.compare(partialValue, bufferValue) < 0) partialValue else bufferValue
+        buffer.setField(minIndex, min)
+      } else {
+        buffer.setField(minIndex, partialValue)
+      }
+    }
+  }
+
+  /**
+   * Return the final aggregated result based on aggregate buffer.
+   *
+   * @param buffer
+   * @return
+   */
+  override def evaluate(buffer: Row): T = {
+    buffer.getField(minIndex).asInstanceOf[T]
+  }
+
+  override def supportPartial: Boolean = true
+
+  override def setAggOffsetInRow(aggOffset: Int): Unit = {
+    minIndex = aggOffset
+  }
+}
+
+class ByteMinAggregate extends MinAggregate[Byte] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.BYTE_TYPE_INFO)
+
+}
+
+class ShortMinAggregate extends MinAggregate[Short] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.SHORT_TYPE_INFO)
+
+}
+
+class IntMinAggregate extends MinAggregate[Int] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.INT_TYPE_INFO)
+
+}
+
+class LongMinAggregate extends MinAggregate[Long] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO)
+
+}
+
+class FloatMinAggregate extends MinAggregate[Float] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.FLOAT_TYPE_INFO)
+
+}
+
+class DoubleMinAggregate extends MinAggregate[Double] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.DOUBLE_TYPE_INFO)
+
+}
+
+class BooleanMinAggregate extends MinAggregate[Boolean] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.BOOLEAN_TYPE_INFO)
+
+}
+
+class DecimalMinAggregate extends Aggregate[BigDecimal] {
+
+  protected var minIndex: Int = _
+
+  override def intermediateDataType = Array(BasicTypeInfo.BIG_DEC_TYPE_INFO)
+
+  override def initiate(intermediate: Row): Unit = {
+    intermediate.setField(minIndex, null)
+  }
+
+  override def prepare(value: Any, partial: Row): Unit = {
+    if (value == null) {
+      initiate(partial)
+    } else {
+      partial.setField(minIndex, value)
+    }
+  }
+
+  override def merge(partial: Row, buffer: Row): Unit = {
+    val partialValue = partial.getField(minIndex).asInstanceOf[BigDecimal]
+    if (partialValue != null) {
+      val bufferValue = buffer.getField(minIndex).asInstanceOf[BigDecimal]
+      if (bufferValue != null) {
+        val min = if (partialValue.compareTo(bufferValue) < 0) partialValue else bufferValue
+        buffer.setField(minIndex, min)
+      } else {
+        buffer.setField(minIndex, partialValue)
+      }
+    }
+  }
+
+  override def evaluate(buffer: Row): BigDecimal = {
+    buffer.getField(minIndex).asInstanceOf[BigDecimal]
+  }
+
+  override def supportPartial: Boolean = true
+
+  override def setAggOffsetInRow(aggOffset: Int): Unit = {
+    minIndex = aggOffset
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SumAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SumAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SumAggregate.scala
new file mode 100644
index 0000000..cd88112
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/SumAggregate.scala
@@ -0,0 +1,131 @@
+/*
+ * 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.math.BigDecimal
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.types.Row
+
+abstract class SumAggregate[T: Numeric]
+  extends Aggregate[T] {
+
+  private val numeric = implicitly[Numeric[T]]
+  protected var sumIndex: Int = _
+
+  override def initiate(partial: Row): Unit = {
+    partial.setField(sumIndex, null)
+  }
+
+  override def merge(partial1: Row, buffer: Row): Unit = {
+    val partialValue = partial1.getField(sumIndex).asInstanceOf[T]
+    if (partialValue != null) {
+      val bufferValue = buffer.getField(sumIndex).asInstanceOf[T]
+      if (bufferValue != null) {
+        buffer.setField(sumIndex, numeric.plus(partialValue, bufferValue))
+      } else {
+        buffer.setField(sumIndex, partialValue)
+      }
+    }
+  }
+
+  override def evaluate(buffer: Row): T = {
+    buffer.getField(sumIndex).asInstanceOf[T]
+  }
+
+  override def prepare(value: Any, partial: Row): Unit = {
+    if (value == null) {
+      initiate(partial)
+    } else {
+      val input = value.asInstanceOf[T]
+      partial.setField(sumIndex, input)
+    }
+  }
+
+  override def supportPartial: Boolean = true
+
+  override def setAggOffsetInRow(aggOffset: Int): Unit = {
+    sumIndex = aggOffset
+  }
+}
+
+class ByteSumAggregate extends SumAggregate[Byte] {
+  override def intermediateDataType = Array(BasicTypeInfo.BYTE_TYPE_INFO)
+}
+
+class ShortSumAggregate extends SumAggregate[Short] {
+  override def intermediateDataType = Array(BasicTypeInfo.SHORT_TYPE_INFO)
+}
+
+class IntSumAggregate extends SumAggregate[Int] {
+  override def intermediateDataType = Array(BasicTypeInfo.INT_TYPE_INFO)
+}
+
+class LongSumAggregate extends SumAggregate[Long] {
+  override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO)
+}
+
+class FloatSumAggregate extends SumAggregate[Float] {
+  override def intermediateDataType = Array(BasicTypeInfo.FLOAT_TYPE_INFO)
+}
+
+class DoubleSumAggregate extends SumAggregate[Double] {
+  override def intermediateDataType = Array(BasicTypeInfo.DOUBLE_TYPE_INFO)
+}
+
+class DecimalSumAggregate extends Aggregate[BigDecimal] {
+
+  protected var sumIndex: Int = _
+
+  override def intermediateDataType = Array(BasicTypeInfo.BIG_DEC_TYPE_INFO)
+
+  override def initiate(partial: Row): Unit = {
+    partial.setField(sumIndex, null)
+  }
+
+  override def merge(partial1: Row, buffer: Row): Unit = {
+    val partialValue = partial1.getField(sumIndex).asInstanceOf[BigDecimal]
+    if (partialValue != null) {
+      val bufferValue = buffer.getField(sumIndex).asInstanceOf[BigDecimal]
+      if (bufferValue != null) {
+        buffer.setField(sumIndex, partialValue.add(bufferValue))
+      } else {
+        buffer.setField(sumIndex, partialValue)
+      }
+    }
+  }
+
+  override def evaluate(buffer: Row): BigDecimal = {
+    buffer.getField(sumIndex).asInstanceOf[BigDecimal]
+  }
+
+  override def prepare(value: Any, partial: Row): Unit = {
+    if (value == null) {
+      initiate(partial)
+    } else {
+      val input = value.asInstanceOf[BigDecimal]
+      partial.setField(sumIndex, input)
+    }
+  }
+
+  override def supportPartial: Boolean = true
+
+  override def setAggOffsetInRow(aggOffset: Int): Unit = {
+    sumIndex = aggOffset
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala
new file mode 100644
index 0000000..d2ac454
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/TimeWindowPropertyCollector.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.aggregate
+
+import org.apache.calcite.runtime.SqlFunctions
+import org.apache.flink.types.Row
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow
+import org.apache.flink.util.Collector
+
+/**
+  * Adds TimeWindow properties to specified fields of a row before it emits the row to a wrapped
+  * collector.
+  */
+class TimeWindowPropertyCollector(windowStartOffset: Option[Int], windowEndOffset: Option[Int])
+    extends Collector[Row] {
+
+  var wrappedCollector: Collector[Row] = _
+  var timeWindow: TimeWindow = _
+
+  override def collect(record: Row): Unit = {
+
+    val lastFieldPos = record.getArity - 1
+
+    if (windowStartOffset.isDefined) {
+      record.setField(
+        lastFieldPos + windowStartOffset.get,
+        SqlFunctions.internalToTimestamp(timeWindow.getStart))
+    }
+    if (windowEndOffset.isDefined) {
+      record.setField(
+        lastFieldPos + windowEndOffset.get,
+        SqlFunctions.internalToTimestamp(timeWindow.getEnd))
+    }
+    wrappedCollector.collect(record)
+  }
+
+  override def close(): Unit = wrappedCollector.close()
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
new file mode 100644
index 0000000..1a339e6
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/io/ValuesInputFormat.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.io
+
+import org.apache.flink.api.common.io.{GenericInputFormat, NonParallelInput}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.core.io.GenericInputSplit
+import org.slf4j.LoggerFactory
+
+class ValuesInputFormat[OUT](
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[OUT])
+  extends GenericInputFormat[OUT]
+  with NonParallelInput
+  with ResultTypeQueryable[OUT]
+  with Compiler[GenericInputFormat[OUT]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  private var format: GenericInputFormat[OUT] = _
+
+  override def open(split: GenericInputSplit): Unit = {
+    LOG.debug(s"Compiling GenericInputFormat: $name \n\n Code:\n$code")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+    LOG.debug("Instantiating GenericInputFormat.")
+    format = clazz.newInstance()
+  }
+
+  override def reachedEnd(): Boolean = format.reachedEnd()
+
+  override def nextRecord(reuse: OUT): OUT = format.nextRecord(reuse)
+
+  override def getProducedType: TypeInformation[OUT] = returnType
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/BatchTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/BatchTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/BatchTableSink.scala
new file mode 100644
index 0000000..edfe113
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/BatchTableSink.scala
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.sinks
+
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.api.Table
+
+/** Defines an external [[TableSink]] to emit a batch [[Table]].
+  *
+  * @tparam T Type of [[DataSet]] that this [[TableSink]] expects and supports.
+  */
+trait BatchTableSink[T] extends TableSink[T] {
+
+  /** Emits the DataSet. */
+  def emitDataSet(dataSet: DataSet[T]): Unit
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
new file mode 100644
index 0000000..9cf76dd
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.sinks
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.types.Row
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.datastream.DataStream
+
+/**
+  * A simple [[TableSink]] to emit data as CSV files.
+  *
+  * @param path The output path to write the Table to.
+  * @param fieldDelim The field delimiter, ',' by default.
+  */
+class CsvTableSink(
+    path: String,
+    fieldDelim: String = ",")
+  extends TableSinkBase[Row] with BatchTableSink[Row] with StreamTableSink[Row] {
+
+  override def emitDataSet(dataSet: DataSet[Row]): Unit = {
+    dataSet
+      .map(new CsvFormatter(fieldDelim))
+      .writeAsText(path)
+  }
+
+  override def emitDataStream(dataStream: DataStream[Row]): Unit = {
+    dataStream
+      .map(new CsvFormatter(fieldDelim))
+      .writeAsText(path)
+  }
+
+  override protected def copy: TableSinkBase[Row] = {
+    new CsvTableSink(path, fieldDelim)
+  }
+
+  override def getOutputType: TypeInformation[Row] = {
+    new RowTypeInfo(getFieldTypes: _*)
+  }
+}
+
+/**
+  * Formats a [[Row]] into a [[String]] with fields separated by the field delimiter.
+  *
+  * @param fieldDelim The field delimiter.
+  */
+class CsvFormatter(fieldDelim: String) extends MapFunction[Row, String] {
+  override def map(row: Row): String = {
+
+    val builder = new StringBuilder
+
+    // write first value
+    val v = row.getField(0)
+    if (v != null) {
+      builder.append(v.toString)
+    }
+
+    // write following values
+    for (i <- 1 until row.getArity) {
+      builder.append(fieldDelim)
+      val v = row.getField(i)
+      if (v != null) {
+        builder.append(v.toString)
+      }
+    }
+    builder.mkString
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala
new file mode 100644
index 0000000..360252e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/StreamTableSink.scala
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.sinks
+
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.Table
+
+/** Defines an external [[TableSink]] to emit a batch [[Table]].
+  *
+  * @tparam T Type of [[DataStream]] that this [[TableSink]] expects and supports.
+  */
+trait StreamTableSink[T] extends TableSink[T] {
+
+  /** Emits the DataStream. */
+  def emitDataStream(dataStream: DataStream[T]): Unit
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/TableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/TableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/TableSink.scala
new file mode 100644
index 0000000..8304867
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/TableSink.scala
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.sinks
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.Table
+
+/** A [[TableSink]] specifies how to emit a [[Table]] to an external
+  * system or location.
+  *
+  * The interface is generic such that it can support different storage locations and formats.
+  *
+  * @tparam T The return type of the [[TableSink]].
+  */
+trait TableSink[T] {
+
+  /**
+    * Return the type expected by this [[TableSink]].
+    *
+    * This type should depend on the types returned by [[getFieldNames]].
+    *
+    * @return The type expected by this [[TableSink]].
+    */
+  def getOutputType: TypeInformation[T]
+
+  /** Returns the names of the table fields. */
+  def getFieldNames: Array[String]
+
+  /** Returns the types of the table fields. */
+  def getFieldTypes: Array[TypeInformation[_]]
+
+  /**
+    * Return a copy of this [[TableSink]] configured with the field names and types of the
+    * [[Table]] to emit.
+    *
+    * @param fieldNames The field names of the table to emit.
+    * @param fieldTypes The field types of the table to emit.
+    * @return A copy of this [[TableSink]] configured with the field names and types of the
+    *         [[Table]] to emit.
+    */
+  def configure(fieldNames: Array[String],
+                fieldTypes: Array[TypeInformation[_]]): TableSink[T]
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/TableSinkBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/TableSinkBase.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/TableSinkBase.scala
new file mode 100644
index 0000000..45866ca
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/TableSinkBase.scala
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.sinks
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.Table
+
+trait TableSinkBase[T] extends TableSink[T] {
+
+  private var fieldNames: Option[Array[String]] = None
+  private var fieldTypes: Option[Array[TypeInformation[_]]] = None
+
+  /** Return a deep copy of the [[TableSink]]. */
+  protected def copy: TableSinkBase[T]
+
+  /**
+    * Return the field names of the [[Table]] to emit. */
+  def getFieldNames: Array[String] = {
+    fieldNames match {
+      case Some(n) => n
+      case None => throw new IllegalStateException(
+        "TableSink must be configured to retrieve field names.")
+    }
+  }
+
+  /** Return the field types of the [[Table]] to emit. */
+  def getFieldTypes: Array[TypeInformation[_]] = {
+    fieldTypes match {
+      case Some(t) => t
+      case None => throw new IllegalStateException(
+        "TableSink must be configured to retrieve field types.")
+    }
+  }
+
+  /**
+    * Return a copy of this [[TableSink]] configured with the field names and types of the
+    * [[Table]] to emit.
+    *
+    * @param fieldNames The field names of the table to emit.
+    * @param fieldTypes The field types of the table to emit.
+    * @return A copy of this [[TableSink]] configured with the field names and types of the
+    *         [[Table]] to emit.
+    */
+  final def configure(fieldNames: Array[String],
+                      fieldTypes: Array[TypeInformation[_]]): TableSink[T] = {
+
+    val configuredSink = this.copy
+    configuredSink.fieldNames = Some(fieldNames)
+    configuredSink.fieldTypes = Some(fieldTypes)
+
+    configuredSink
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/BatchTableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/BatchTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/BatchTableSource.scala
new file mode 100644
index 0000000..0478dc9
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/BatchTableSource.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.sources
+
+import org.apache.flink.api.java.{ExecutionEnvironment, DataSet}
+
+/** Defines an external batch table and provides access to its data.
+  *
+  * @tparam T Type of the [[DataSet]] created by this [[TableSource]].
+  */
+trait BatchTableSource[T] extends TableSource[T] {
+
+  /**
+    * Returns the data of the table as a [[DataSet]].
+    *
+    * NOTE: This method is for internal use only for defining a [[TableSource]].
+    *       Do not use it in Table API programs.
+    */
+  def getDataSet(execEnv: ExecutionEnvironment): DataSet[T]
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSource.scala
new file mode 100644
index 0000000..3f4e395
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/CsvTableSource.scala
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.sources
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.io.CsvInputFormat
+import org.apache.flink.api.java.{DataSet, ExecutionEnvironment}
+import org.apache.flink.types.Row
+import org.apache.flink.api.java.io.RowCsvInputFormat
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.core.fs.Path
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+import org.apache.flink.table.api.TableException
+
+/**
+  * A [[BatchTableSource]] and [[StreamTableSource]] for simple CSV files with a
+  * (logically) unlimited number of fields.
+  *
+  * @param path The path to the CSV file.
+  * @param fieldNames The names of the table fields.
+  * @param fieldTypes The types of the table fields.
+  * @param fieldDelim The field delimiter, "," by default.
+  * @param rowDelim The row delimiter, "\n" by default.
+  * @param quoteCharacter An optional quote character for String values, null by default.
+  * @param ignoreFirstLine Flag to ignore the first line, false by default.
+  * @param ignoreComments An optional prefix to indicate comments, null by default.
+  * @param lenient Flag to skip records with parse error instead to fail, false by default.
+  */
+class CsvTableSource(
+    path: String,
+    fieldNames: Array[String],
+    fieldTypes: Array[TypeInformation[_]],
+    fieldDelim: String = CsvInputFormat.DEFAULT_FIELD_DELIMITER,
+    rowDelim: String = CsvInputFormat.DEFAULT_LINE_DELIMITER,
+    quoteCharacter: Character = null,
+    ignoreFirstLine: Boolean = false,
+    ignoreComments: String = null,
+    lenient: Boolean = false)
+  extends BatchTableSource[Row]
+  with StreamTableSource[Row] {
+
+  /**
+    * A [[BatchTableSource]] and [[StreamTableSource]] for simple CSV files with a
+    * (logically) unlimited number of fields.
+    *
+    * @param path The path to the CSV file.
+    * @param fieldNames The names of the table fields.
+    * @param fieldTypes The types of the table fields.
+    */
+  def this(path: String, fieldNames: Array[String], fieldTypes: Array[TypeInformation[_]]) =
+    this(path, fieldNames, fieldTypes, CsvInputFormat.DEFAULT_FIELD_DELIMITER,
+      CsvInputFormat.DEFAULT_LINE_DELIMITER, null, false, null, false)
+
+  if (fieldNames.length != fieldTypes.length) {
+    throw TableException("Number of field names and field types must be equal.")
+  }
+
+  private val returnType = new RowTypeInfo(fieldTypes: _*)
+
+  /**
+    * Returns the data of the table as a [[DataSet]] of [[Row]].
+    *
+    * NOTE: This method is for internal use only for defining a [[TableSource]].
+    *       Do not use it in Table API programs.
+    */
+  override def getDataSet(execEnv: ExecutionEnvironment): DataSet[Row] = {
+    execEnv.createInput(createCsvInput(), returnType)
+  }
+
+  /** Returns the types of the table fields. */
+  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
+
+  /** Returns the names of the table fields. */
+  override def getFieldsNames: Array[String] = fieldNames
+
+  /** Returns the number of fields of the table. */
+  override def getNumberOfFields: Int = fieldNames.length
+
+  /** Returns the [[RowTypeInfo]] for the return type of the [[CsvTableSource]]. */
+  override def getReturnType: RowTypeInfo = returnType
+
+  /**
+    * Returns the data of the table as a [[DataStream]] of [[Row]].
+    *
+    * NOTE: This method is for internal use only for defining a [[TableSource]].
+    *       Do not use it in Table API programs.
+    */
+  override def getDataStream(streamExecEnv: StreamExecutionEnvironment): DataStream[Row] = {
+    streamExecEnv.createInput(createCsvInput(), returnType)
+  }
+
+  private def createCsvInput(): RowCsvInputFormat = {
+    val inputFormat = new RowCsvInputFormat(new Path(path), returnType, rowDelim, fieldDelim)
+
+    inputFormat.setSkipFirstLineAsHeader(ignoreFirstLine)
+    inputFormat.setLenient(lenient)
+    if (quoteCharacter != null) {
+      inputFormat.enableQuotedStringParsing(quoteCharacter)
+    }
+    if (ignoreComments != null) {
+      inputFormat.setCommentPrefix(ignoreComments)
+    }
+
+    inputFormat
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/ProjectableTableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/ProjectableTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/ProjectableTableSource.scala
new file mode 100644
index 0000000..429cccb
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/ProjectableTableSource.scala
@@ -0,0 +1,38 @@
+/*
+ * 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
+
+/**
+  * Adds support for projection push-down to a [[TableSource]].
+  * A [[TableSource]] extending this interface is able to project the fields of the return table.
+  *
+  * @tparam T The return type of the [[ProjectableTableSource]].
+  */
+trait ProjectableTableSource[T] {
+
+  /**
+    * Creates a copy of the [[ProjectableTableSource]] that projects its output on the specified
+    * fields.
+    *
+    * @param fields The indexes of the fields to return.
+    * @return A copy of the [[ProjectableTableSource]] that projects its output.
+    */
+  def projectFields(fields: Array[Int]): ProjectableTableSource[T]
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/StreamTableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/StreamTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/StreamTableSource.scala
new file mode 100644
index 0000000..7a2737c
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/StreamTableSource.scala
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.sources
+
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+
+/** Defines an external stream table and provides access to its data.
+  *
+  * @tparam T Type of the [[DataStream]] created by this [[TableSource]].
+  */
+trait StreamTableSource[T] extends TableSource[T] {
+
+  /**
+    * Returns the data of the table as a [[DataStream]].
+    *
+    * NOTE: This method is for internal use only for defining a [[TableSource]].
+    *       Do not use it in Table API programs.
+    */
+  def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[T]
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSource.scala
new file mode 100644
index 0000000..9d4ba68
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sources/TableSource.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.sources
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+
+/** Defines an external table by providing schema information, i.e., field names and types.
+  *
+  * @tparam T The return type of the [[TableSource]].
+  */
+trait TableSource[T] {
+
+  /** Returns the number of fields of the table. */
+  def getNumberOfFields: Int
+
+  /** Returns the names of the table fields. */
+  def getFieldsNames: Array[String]
+
+  /** Returns the types of the table fields. */
+  def getFieldTypes: Array[TypeInformation[_]]
+
+  /** Returns the [[TypeInformation]] for the return type of the [[TableSource]]. */
+  def getReturnType: TypeInformation[T]
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/InternalTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/InternalTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/InternalTypeInfo.scala
new file mode 100644
index 0000000..b4af152
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/InternalTypeInfo.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.typeutils
+
+import java.util.Objects
+
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
+import org.apache.flink.util.Preconditions._
+
+/**
+  * TypeInformation for internal types of the Table API that are for translation purposes only
+  * and should not be contained in final plan.
+  */
+@SerialVersionUID(-13064574364925255L)
+abstract class InternalTypeInfo[T](val clazz: Class[T])
+  extends TypeInformation[T]
+  with AtomicType[T] {
+
+  checkNotNull(clazz)
+
+  override def isBasicType: Boolean =
+    throw new UnsupportedOperationException("This type is for internal use only.")
+
+  override def isTupleType: Boolean =
+    throw new UnsupportedOperationException("This type is for internal use only.")
+
+  override def getArity: Int =
+    throw new UnsupportedOperationException("This type is for internal use only.")
+
+  override def getTotalFields: Int =
+    throw new UnsupportedOperationException("This type is for internal use only.")
+
+  override def getTypeClass: Class[T] = clazz
+
+  override def isKeyType: Boolean =
+    throw new UnsupportedOperationException("This type is for internal use only.")
+
+  override def createSerializer(config: ExecutionConfig): TypeSerializer[T] =
+    throw new UnsupportedOperationException("This type is for internal use only.")
+
+  override def createComparator(
+      sortOrderAscending: Boolean,
+      executionConfig: ExecutionConfig)
+    : TypeComparator[T] =
+    throw new UnsupportedOperationException("This type is for internal use only.")
+
+  // ----------------------------------------------------------------------------------------------
+
+  override def hashCode: Int = Objects.hash(clazz)
+
+  def canEqual(obj: Any): Boolean
+
+  override def equals(obj: Any): Boolean = {
+    obj match {
+      case other: InternalTypeInfo[_] =>
+        other.canEqual(this) && (this.clazz eq other.clazz)
+      case _ =>
+        false
+    }
+  }
+
+  override def toString: String = s"InternalTypeInfo"
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/RowIntervalTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/RowIntervalTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/RowIntervalTypeInfo.scala
new file mode 100644
index 0000000..bbc20aa
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/RowIntervalTypeInfo.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.typeutils
+
+/**
+  * TypeInformation for row intervals.
+  */
+@SerialVersionUID(-1306179424364925258L)
+class RowIntervalTypeInfo extends InternalTypeInfo[Long](classOf[Long]) {
+
+  def canEqual(obj: Any): Boolean = obj.isInstanceOf[RowIntervalTypeInfo]
+
+  override def toString: String = s"RowIntervalTypeInfo"
+}
+
+object RowIntervalTypeInfo {
+
+  val INTERVAL_ROWS = new RowIntervalTypeInfo()
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIntervalTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIntervalTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIntervalTypeInfo.scala
new file mode 100644
index 0000000..9d76050
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TimeIntervalTypeInfo.scala
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.typeutils
+
+import java.util.Objects
+
+import org.apache.flink.api.common.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.common.typeutils.base.{IntComparator, IntSerializer, LongComparator, LongSerializer}
+import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
+import org.apache.flink.table.typeutils.TimeIntervalTypeInfo.instantiateComparator
+import org.apache.flink.util.Preconditions._
+
+/**
+  * TypeInformation for SQL INTERVAL types.
+  */
+@SerialVersionUID(-1816179424364825258L)
+class TimeIntervalTypeInfo[T](
+    val clazz: Class[T],
+    val serializer: TypeSerializer[T],
+    val comparatorClass: Class[_ <: TypeComparator[T]])
+  extends TypeInformation[T]
+  with AtomicType[T] {
+
+  checkNotNull(clazz)
+  checkNotNull(serializer)
+  checkNotNull(comparatorClass)
+
+  override def isBasicType: Boolean = false
+
+  override def isTupleType: Boolean = false
+
+  override def getArity: Int = 1
+
+  override def getTotalFields: Int = 1
+
+  override def getTypeClass: Class[T] = clazz
+
+  override def isKeyType: Boolean = true
+
+  override def createSerializer(config: ExecutionConfig): TypeSerializer[T] = serializer
+
+  override def createComparator(
+      sortOrderAscending: Boolean,
+      executionConfig: ExecutionConfig)
+    : TypeComparator[T] = instantiateComparator(comparatorClass, sortOrderAscending)
+
+  // ----------------------------------------------------------------------------------------------
+
+  override def hashCode: Int = Objects.hash(clazz, serializer, comparatorClass)
+
+  def canEqual(obj: Any): Boolean = obj.isInstanceOf[TimeIntervalTypeInfo[_]]
+
+  override def equals(obj: Any): Boolean = {
+    obj match {
+      case other: TimeIntervalTypeInfo[_] =>
+        other.canEqual(this) &&
+          (this.clazz eq other.clazz) &&
+          serializer == other.serializer &&
+          (this.comparatorClass eq other.comparatorClass)
+      case _ =>
+        false
+    }
+  }
+
+  override def toString: String = s"TimeIntervalTypeInfo(${clazz.getSimpleName})"
+}
+
+object TimeIntervalTypeInfo {
+
+  val INTERVAL_MONTHS = new TimeIntervalTypeInfo(
+    classOf[java.lang.Integer],
+    IntSerializer.INSTANCE,
+    classOf[IntComparator])
+
+  val INTERVAL_MILLIS = new TimeIntervalTypeInfo(
+    classOf[java.lang.Long],
+    LongSerializer.INSTANCE,
+    classOf[LongComparator])
+
+  // ----------------------------------------------------------------------------------------------
+
+  private def instantiateComparator[X](
+      comparatorClass: Class[_ <: TypeComparator[X]],
+      ascendingOrder: java.lang.Boolean)
+    : TypeComparator[X] = {
+    try {
+      val constructor = comparatorClass.getConstructor(java.lang.Boolean.TYPE)
+      constructor.newInstance(ascendingOrder)
+    } catch {
+      case e: Exception =>
+        throw new RuntimeException(
+          s"Could not initialize comparator ${comparatorClass.getName}", e)
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCheckUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCheckUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCheckUtils.scala
new file mode 100644
index 0000000..40f0cf2
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCheckUtils.scala
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.typeutils
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{BIG_DEC_TYPE_INFO, BOOLEAN_TYPE_INFO, INT_TYPE_INFO, STRING_TYPE_INFO}
+import org.apache.flink.api.common.typeinfo._
+import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
+import org.apache.flink.table.validate._
+
+object TypeCheckUtils {
+
+  /**
+    * Checks if type information is an advanced type that can be converted to a
+    * SQL type but NOT vice versa.
+    */
+  def isAdvanced(dataType: TypeInformation[_]): Boolean = dataType match {
+    case _: BasicTypeInfo[_] => false
+    case _: SqlTimeTypeInfo[_] => false
+    case _: TimeIntervalTypeInfo[_] => false
+    case _ => true
+  }
+
+  /**
+    * Checks if type information is a simple type that can be converted to a
+    * SQL type and vice versa.
+    */
+  def isSimple(dataType: TypeInformation[_]): Boolean = !isAdvanced(dataType)
+
+  def isNumeric(dataType: TypeInformation[_]): Boolean = dataType match {
+    case _: NumericTypeInfo[_] => true
+    case BIG_DEC_TYPE_INFO => true
+    case _ => false
+  }
+
+  def isTemporal(dataType: TypeInformation[_]): Boolean =
+    isTimePoint(dataType) || isTimeInterval(dataType)
+
+  def isTimePoint(dataType: TypeInformation[_]): Boolean =
+    dataType.isInstanceOf[SqlTimeTypeInfo[_]]
+
+  def isTimeInterval(dataType: TypeInformation[_]): Boolean =
+    dataType.isInstanceOf[TimeIntervalTypeInfo[_]]
+
+  def isString(dataType: TypeInformation[_]): Boolean = dataType == STRING_TYPE_INFO
+
+  def isBoolean(dataType: TypeInformation[_]): Boolean = dataType == BOOLEAN_TYPE_INFO
+
+  def isDecimal(dataType: TypeInformation[_]): Boolean = dataType == BIG_DEC_TYPE_INFO
+
+  def isInteger(dataType: TypeInformation[_]): Boolean = dataType == INT_TYPE_INFO
+
+  def isArray(dataType: TypeInformation[_]): Boolean = dataType match {
+    case _: ObjectArrayTypeInfo[_, _] | _: PrimitiveArrayTypeInfo[_] => true
+    case _ => false
+  }
+
+  def isComparable(dataType: TypeInformation[_]): Boolean =
+    classOf[Comparable[_]].isAssignableFrom(dataType.getTypeClass) && !isArray(dataType)
+
+  def assertNumericExpr(
+      dataType: TypeInformation[_],
+      caller: String)
+    : ValidationResult = dataType match {
+    case _: NumericTypeInfo[_] =>
+      ValidationSuccess
+    case BIG_DEC_TYPE_INFO =>
+      ValidationSuccess
+    case _ =>
+      ValidationFailure(s"$caller requires numeric types, get $dataType here")
+  }
+
+  def assertOrderableExpr(dataType: TypeInformation[_], caller: String): ValidationResult = {
+    if (dataType.isSortKeyType) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(s"$caller requires orderable types, get $dataType here")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCoercion.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCoercion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCoercion.scala
new file mode 100644
index 0000000..99e94b3
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeCoercion.scala
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.typeutils
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, NumericTypeInfo, TypeInformation}
+
+/**
+  * Utilities for type conversions.
+  */
+object TypeCoercion {
+
+  val numericWideningPrecedence: IndexedSeq[TypeInformation[_]] =
+    IndexedSeq(
+      BYTE_TYPE_INFO,
+      SHORT_TYPE_INFO,
+      INT_TYPE_INFO,
+      LONG_TYPE_INFO,
+      FLOAT_TYPE_INFO,
+      DOUBLE_TYPE_INFO)
+
+  def widerTypeOf(tp1: TypeInformation[_], tp2: TypeInformation[_]): Option[TypeInformation[_]] = {
+    (tp1, tp2) match {
+      case (ti1, ti2) if ti1 == ti2 => Some(ti1)
+
+      case (_, STRING_TYPE_INFO) => Some(STRING_TYPE_INFO)
+      case (STRING_TYPE_INFO, _) => Some(STRING_TYPE_INFO)
+
+      case (_, BIG_DEC_TYPE_INFO) => Some(BIG_DEC_TYPE_INFO)
+      case (BIG_DEC_TYPE_INFO, _) => Some(BIG_DEC_TYPE_INFO)
+
+      case (stti: SqlTimeTypeInfo[_], _: TimeIntervalTypeInfo[_]) => Some(stti)
+      case (_: TimeIntervalTypeInfo[_], stti: SqlTimeTypeInfo[_]) => Some(stti)
+
+      case tuple if tuple.productIterator.forall(numericWideningPrecedence.contains) =>
+        val higherIndex = numericWideningPrecedence.lastIndexWhere(t => t == tp1 || t == tp2)
+        Some(numericWideningPrecedence(higherIndex))
+
+      case _ => None
+    }
+  }
+
+  /**
+    * Test if we can do cast safely without lose of information.
+    */
+  def canSafelyCast(from: TypeInformation[_], to: TypeInformation[_]): Boolean = (from, to) match {
+    case (_, STRING_TYPE_INFO) => true
+
+    case (_: NumericTypeInfo[_], BIG_DEC_TYPE_INFO) => true
+
+    case tuple if tuple.productIterator.forall(numericWideningPrecedence.contains) =>
+      if (numericWideningPrecedence.indexOf(from) < numericWideningPrecedence.indexOf(to)) {
+        true
+      } else {
+        false
+      }
+
+    case _ => false
+  }
+
+  /**
+    * All the supported cast types in flink-table.
+    * Note: This may lose information during the cast.
+    */
+  def canCast(from: TypeInformation[_], to: TypeInformation[_]): Boolean = (from, to) match {
+    case (fromTp, toTp) if fromTp == toTp => true
+
+    case (_, STRING_TYPE_INFO) => true
+
+    case (_, CHAR_TYPE_INFO) => false // Character type not supported.
+
+    case (STRING_TYPE_INFO, _: NumericTypeInfo[_]) => true
+    case (STRING_TYPE_INFO, BOOLEAN_TYPE_INFO) => true
+    case (STRING_TYPE_INFO, BIG_DEC_TYPE_INFO) => true
+    case (STRING_TYPE_INFO, SqlTimeTypeInfo.DATE) => true
+    case (STRING_TYPE_INFO, SqlTimeTypeInfo.TIME) => true
+    case (STRING_TYPE_INFO, SqlTimeTypeInfo.TIMESTAMP) => true
+
+    case (BOOLEAN_TYPE_INFO, _: NumericTypeInfo[_]) => true
+    case (BOOLEAN_TYPE_INFO, BIG_DEC_TYPE_INFO) => true
+    case (_: NumericTypeInfo[_], BOOLEAN_TYPE_INFO) => true
+    case (BIG_DEC_TYPE_INFO, BOOLEAN_TYPE_INFO) => true
+
+    case (_: NumericTypeInfo[_], _: NumericTypeInfo[_]) => true
+    case (BIG_DEC_TYPE_INFO, _: NumericTypeInfo[_]) => true
+    case (_: NumericTypeInfo[_], BIG_DEC_TYPE_INFO) => true
+    case (INT_TYPE_INFO, SqlTimeTypeInfo.DATE) => true
+    case (INT_TYPE_INFO, SqlTimeTypeInfo.TIME) => true
+    case (LONG_TYPE_INFO, SqlTimeTypeInfo.TIMESTAMP) => true
+    case (INT_TYPE_INFO, TimeIntervalTypeInfo.INTERVAL_MONTHS) => true
+    case (LONG_TYPE_INFO, TimeIntervalTypeInfo.INTERVAL_MILLIS) => true
+
+    case (SqlTimeTypeInfo.DATE, SqlTimeTypeInfo.TIME) => false
+    case (SqlTimeTypeInfo.TIME, SqlTimeTypeInfo.DATE) => false
+    case (_: SqlTimeTypeInfo[_], _: SqlTimeTypeInfo[_]) => true
+    case (SqlTimeTypeInfo.DATE, INT_TYPE_INFO) => true
+    case (SqlTimeTypeInfo.TIME, INT_TYPE_INFO) => true
+    case (SqlTimeTypeInfo.TIMESTAMP, LONG_TYPE_INFO) => true
+
+    case (TimeIntervalTypeInfo.INTERVAL_MONTHS, INT_TYPE_INFO) => true
+    case (TimeIntervalTypeInfo.INTERVAL_MILLIS, LONG_TYPE_INFO) => true
+
+    case _ => false
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeConverter.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeConverter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeConverter.scala
new file mode 100644
index 0000000..a2a120b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/typeutils/TypeConverter.scala
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.typeutils
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.JoinRelType
+import org.apache.calcite.rel.core.JoinRelType._
+import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.api.java.operators.join.JoinType
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.types.Row
+import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo}
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.calcite.FlinkTypeFactory
+
+import scala.collection.JavaConversions._
+
+object TypeConverter {
+
+  val DEFAULT_ROW_TYPE = new RowTypeInfo().asInstanceOf[TypeInformation[Any]]
+
+  /**
+    * Determines the return type of Flink operators based on the logical fields, the expected
+    * physical type and configuration parameters.
+    *
+    * For example:
+    *   - No physical type expected, only 3 non-null fields and efficient type usage enabled
+    *       -> return Tuple3
+    *   - No physical type expected, efficient type usage enabled, but 3 nullable fields
+    *       -> return Row because Tuple does not support null values
+    *   - Physical type expected
+    *       -> check if physical type is compatible and return it
+    *
+    * @param logicalRowType logical row information
+    * @param expectedPhysicalType expected physical type
+    * @param nullable fields can be nullable
+    * @param useEfficientTypes use the most efficient types (e.g. Tuples and value types)
+    * @return suitable return type
+    */
+  def determineReturnType(
+      logicalRowType: RelDataType,
+      expectedPhysicalType: Option[TypeInformation[Any]],
+      nullable: Boolean,
+      useEfficientTypes: Boolean)
+    : TypeInformation[Any] = {
+    // convert to type information
+    val logicalFieldTypes = logicalRowType.getFieldList map { relDataType =>
+      FlinkTypeFactory.toTypeInfo(relDataType.getType)
+    }
+    // field names
+    val logicalFieldNames = logicalRowType.getFieldNames.toList
+
+    val returnType = expectedPhysicalType match {
+      // a certain physical type is expected (but not Row)
+      // check if expected physical type is compatible with logical field type
+      case Some(typeInfo) if typeInfo.getTypeClass != classOf[Row] =>
+        if (typeInfo.getArity != logicalFieldTypes.length) {
+          throw new TableException("Arity of result does not match expected type.")
+        }
+        typeInfo match {
+
+          // POJO type expected
+          case pt: PojoTypeInfo[_] =>
+            logicalFieldNames.zip(logicalFieldTypes) foreach {
+              case (fName, fType) =>
+                val pojoIdx = pt.getFieldIndex(fName)
+                if (pojoIdx < 0) {
+                  throw new TableException(s"POJO does not define field name: $fName")
+                }
+                val expectedTypeInfo = pt.getTypeAt(pojoIdx)
+                if (fType != expectedTypeInfo) {
+                  throw new TableException(s"Result field does not match expected type. " +
+                    s"Expected: $expectedTypeInfo; Actual: $fType")
+                }
+            }
+
+          // Tuple/Case class type expected
+          case ct: CompositeType[_] =>
+            logicalFieldTypes.zipWithIndex foreach {
+              case (fieldTypeInfo, i) =>
+                val expectedTypeInfo = ct.getTypeAt(i)
+                if (fieldTypeInfo != expectedTypeInfo) {
+                  throw new TableException(s"Result field does not match expected type. " +
+                    s"Expected: $expectedTypeInfo; Actual: $fieldTypeInfo")
+                }
+            }
+
+          // Atomic type expected
+          case at: AtomicType[_] =>
+            val fieldTypeInfo = logicalFieldTypes.head
+            if (fieldTypeInfo != at) {
+              throw new TableException(s"Result field does not match expected type. " +
+                s"Expected: $at; Actual: $fieldTypeInfo")
+            }
+
+          case _ =>
+            throw new TableException("Unsupported result type.")
+        }
+        typeInfo
+
+      // Row is expected, create the arity for it
+      case Some(typeInfo) if typeInfo.getTypeClass == classOf[Row] =>
+        new RowTypeInfo(logicalFieldTypes: _*)
+
+      // no physical type
+      // determine type based on logical fields and configuration parameters
+      case None =>
+        // no need for efficient types -> use Row
+        // we cannot use efficient types if row arity > tuple arity or nullable
+        if (!useEfficientTypes || logicalFieldTypes.length > Tuple.MAX_ARITY || nullable) {
+          new RowTypeInfo(logicalFieldTypes: _*)
+        }
+        // use efficient type tuple or atomic type
+        else {
+          if (logicalFieldTypes.length == 1) {
+            logicalFieldTypes.head
+          }
+          else {
+            new TupleTypeInfo[Tuple](logicalFieldTypes.toArray:_*)
+          }
+        }
+    }
+    returnType.asInstanceOf[TypeInformation[Any]]
+  }
+
+  def sqlJoinTypeToFlinkJoinType(sqlJoinType: JoinRelType): JoinType = sqlJoinType match {
+    case INNER => JoinType.INNER
+    case LEFT => JoinType.LEFT_OUTER
+    case RIGHT => JoinType.RIGHT_OUTER
+    case FULL => JoinType.FULL_OUTER
+  }
+
+  def flinkJoinTypeToRelType(joinType: JoinType) = joinType match {
+    case JoinType.INNER => JoinRelType.INNER
+    case JoinType.LEFT_OUTER => JoinRelType.LEFT
+    case JoinType.RIGHT_OUTER => JoinRelType.RIGHT
+    case JoinType.FULL_OUTER => JoinRelType.FULL
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
new file mode 100644
index 0000000..f92b3a1
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
@@ -0,0 +1,320 @@
+/*
+ * 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.validate
+
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.sql.util.{ChainedSqlOperatorTable, ListSqlOperatorTable, ReflectiveSqlOperatorTable}
+import org.apache.calcite.sql.{SqlFunction, SqlOperator, SqlOperatorTable}
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.functions.{ScalarFunction, TableFunction}
+import org.apache.flink.table.functions.utils.{TableSqlFunction, UserDefinedFunctionUtils}
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable
+import scala.util.{Failure, Success, Try}
+
+/**
+  * A catalog for looking up (user-defined) functions, used during validation phases
+  * of both Table API and SQL API.
+  */
+class FunctionCatalog {
+
+  private val functionBuilders = mutable.HashMap.empty[String, Class[_]]
+  private val sqlFunctions = mutable.ListBuffer[SqlFunction]()
+
+  def registerFunction(name: String, builder: Class[_]): Unit =
+    functionBuilders.put(name.toLowerCase, builder)
+
+  def registerSqlFunction(sqlFunction: SqlFunction): Unit = {
+    sqlFunctions --= sqlFunctions.filter(_.getName == sqlFunction.getName)
+    sqlFunctions += sqlFunction
+  }
+
+  /**
+    * Register multiple SQL functions at the same time. The functions have the same name.
+    */
+  def registerSqlFunctions(functions: Seq[SqlFunction]): Unit = {
+    if (functions.nonEmpty) {
+      val name = functions.head.getName
+      // check that all functions have the same name
+      if (functions.forall(_.getName == name)) {
+        sqlFunctions --= sqlFunctions.filter(_.getName == name)
+        sqlFunctions ++= functions
+      } else {
+        throw ValidationException("The SQL functions to be registered have different names.")
+      }
+    }
+  }
+
+  def getSqlOperatorTable: SqlOperatorTable =
+    ChainedSqlOperatorTable.of(
+      new BasicOperatorTable(),
+      new ListSqlOperatorTable(sqlFunctions)
+    )
+
+  /**
+    * Lookup and create an expression if we find a match.
+    */
+  def lookupFunction(name: String, children: Seq[Expression]): Expression = {
+    val funcClass = functionBuilders
+      .getOrElse(name.toLowerCase, throw ValidationException(s"Undefined function: $name"))
+
+    // Instantiate a function using the provided `children`
+    funcClass match {
+
+      // user-defined scalar function call
+      case sf if classOf[ScalarFunction].isAssignableFrom(sf) =>
+        Try(UserDefinedFunctionUtils.instantiate(sf.asInstanceOf[Class[ScalarFunction]])) match {
+          case Success(scalarFunction) => ScalarFunctionCall(scalarFunction, children)
+          case Failure(e) => throw ValidationException(e.getMessage)
+        }
+
+      // user-defined table function call
+      case tf if classOf[TableFunction[_]].isAssignableFrom(tf) =>
+        val tableSqlFunction = sqlFunctions
+          .find(f => f.getName.equalsIgnoreCase(name) && f.isInstanceOf[TableSqlFunction])
+          .getOrElse(throw ValidationException(s"Undefined table function: $name"))
+          .asInstanceOf[TableSqlFunction]
+        val typeInfo = tableSqlFunction.getRowTypeInfo
+        val function = tableSqlFunction.getTableFunction
+        TableFunctionCall(name, function, children, typeInfo)
+
+      // general expression call
+      case expression if classOf[Expression].isAssignableFrom(expression) =>
+        // try to find a constructor accepts `Seq[Expression]`
+        Try(funcClass.getDeclaredConstructor(classOf[Seq[_]])) match {
+          case Success(seqCtor) =>
+            Try(seqCtor.newInstance(children).asInstanceOf[Expression]) match {
+              case Success(expr) => expr
+              case Failure(e) => throw new ValidationException(e.getMessage)
+            }
+          case Failure(e) =>
+            val childrenClass = Seq.fill(children.length)(classOf[Expression])
+            // try to find a constructor matching the exact number of children
+            Try(funcClass.getDeclaredConstructor(childrenClass: _*)) match {
+              case Success(ctor) =>
+                Try(ctor.newInstance(children: _*).asInstanceOf[Expression]) match {
+                  case Success(expr) => expr
+                  case Failure(exception) => throw ValidationException(exception.getMessage)
+                }
+              case Failure(exception) =>
+                throw ValidationException(s"Invalid number of arguments for function $funcClass")
+            }
+        }
+
+      case _ =>
+        throw ValidationException("Unsupported function.")
+    }
+  }
+
+  /**
+    * Drop a function and return if the function existed.
+    */
+  def dropFunction(name: String): Boolean =
+    functionBuilders.remove(name.toLowerCase).isDefined
+
+  /**
+    * Drop all registered functions.
+    */
+  def clear(): Unit = functionBuilders.clear()
+}
+
+object FunctionCatalog {
+
+  val builtInFunctions: Map[String, Class[_]] = Map(
+    // logic
+    "isNull" -> classOf[IsNull],
+    "isNotNull" -> classOf[IsNotNull],
+    "isTrue" -> classOf[IsTrue],
+    "isFalse" -> classOf[IsFalse],
+    "isNotTrue" -> classOf[IsNotTrue],
+    "isNotFalse" -> classOf[IsNotFalse],
+
+    // aggregate functions
+    "avg" -> classOf[Avg],
+    "count" -> classOf[Count],
+    "max" -> classOf[Max],
+    "min" -> classOf[Min],
+    "sum" -> classOf[Sum],
+
+    // string functions
+    "charLength" -> classOf[CharLength],
+    "initCap" -> classOf[InitCap],
+    "like" -> classOf[Like],
+    "lowerCase" -> classOf[Lower],
+    "similar" -> classOf[Similar],
+    "substring" -> classOf[Substring],
+    "trim" -> classOf[Trim],
+    "upperCase" -> classOf[Upper],
+    "position" -> classOf[Position],
+    "overlay" -> classOf[Overlay],
+
+    // math functions
+    "abs" -> classOf[Abs],
+    "ceil" -> classOf[Ceil],
+    "exp" -> classOf[Exp],
+    "floor" -> classOf[Floor],
+    "log10" -> classOf[Log10],
+    "ln" -> classOf[Ln],
+    "power" -> classOf[Power],
+    "mod" -> classOf[Mod],
+    "sqrt" -> classOf[Sqrt],
+
+    // temporal functions
+    "extract" -> classOf[Extract],
+    "currentDate" -> classOf[CurrentDate],
+    "currentTime" -> classOf[CurrentTime],
+    "currentTimestamp" -> classOf[CurrentTimestamp],
+    "localTime" -> classOf[LocalTime],
+    "localTimestamp" -> classOf[LocalTimestamp],
+    "quarter" -> classOf[Quarter],
+    "temporalOverlaps" -> classOf[TemporalOverlaps],
+
+    // array
+    "cardinality" -> classOf[ArrayCardinality],
+    "at" -> classOf[ArrayElementAt],
+    "element" -> classOf[ArrayElement]
+
+    // TODO implement function overloading here
+    // "floor" -> classOf[TemporalFloor]
+    // "ceil" -> classOf[TemporalCeil]
+  )
+
+  /**
+    * Create a new function catalog with built-in functions.
+    */
+  def withBuiltIns: FunctionCatalog = {
+    val catalog = new FunctionCatalog()
+    builtInFunctions.foreach { case (n, c) => catalog.registerFunction(n, c) }
+    catalog
+  }
+}
+
+class BasicOperatorTable extends ReflectiveSqlOperatorTable {
+
+  /**
+    * List of supported SQL operators / functions.
+    *
+    * This list should be kept in sync with [[SqlStdOperatorTable]].
+    */
+  private val builtInSqlOperators: Seq[SqlOperator] = Seq(
+    // SET OPERATORS
+    SqlStdOperatorTable.UNION,
+    SqlStdOperatorTable.UNION_ALL,
+    SqlStdOperatorTable.EXCEPT,
+    SqlStdOperatorTable.EXCEPT_ALL,
+    SqlStdOperatorTable.INTERSECT,
+    SqlStdOperatorTable.INTERSECT_ALL,
+    // BINARY OPERATORS
+    SqlStdOperatorTable.AND,
+    SqlStdOperatorTable.AS,
+    SqlStdOperatorTable.CONCAT,
+    SqlStdOperatorTable.DIVIDE,
+    SqlStdOperatorTable.DIVIDE_INTEGER,
+    SqlStdOperatorTable.DOT,
+    SqlStdOperatorTable.EQUALS,
+    SqlStdOperatorTable.GREATER_THAN,
+    SqlStdOperatorTable.IS_DISTINCT_FROM,
+    SqlStdOperatorTable.IS_NOT_DISTINCT_FROM,
+    SqlStdOperatorTable.GREATER_THAN_OR_EQUAL,
+    SqlStdOperatorTable.LESS_THAN,
+    SqlStdOperatorTable.LESS_THAN_OR_EQUAL,
+    SqlStdOperatorTable.MINUS,
+    SqlStdOperatorTable.MULTIPLY,
+    SqlStdOperatorTable.NOT_EQUALS,
+    SqlStdOperatorTable.OR,
+    SqlStdOperatorTable.PLUS,
+    SqlStdOperatorTable.DATETIME_PLUS,
+    // POSTFIX OPERATORS
+    SqlStdOperatorTable.DESC,
+    SqlStdOperatorTable.NULLS_FIRST,
+    SqlStdOperatorTable.IS_NOT_NULL,
+    SqlStdOperatorTable.IS_NULL,
+    SqlStdOperatorTable.IS_NOT_TRUE,
+    SqlStdOperatorTable.IS_TRUE,
+    SqlStdOperatorTable.IS_NOT_FALSE,
+    SqlStdOperatorTable.IS_FALSE,
+    SqlStdOperatorTable.IS_NOT_UNKNOWN,
+    SqlStdOperatorTable.IS_UNKNOWN,
+    // PREFIX OPERATORS
+    SqlStdOperatorTable.NOT,
+    SqlStdOperatorTable.UNARY_MINUS,
+    SqlStdOperatorTable.UNARY_PLUS,
+    // AGGREGATE OPERATORS
+    SqlStdOperatorTable.SUM,
+    SqlStdOperatorTable.COUNT,
+    SqlStdOperatorTable.MIN,
+    SqlStdOperatorTable.MAX,
+    SqlStdOperatorTable.AVG,
+    // ARRAY OPERATORS
+    SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR,
+    SqlStdOperatorTable.ITEM,
+    SqlStdOperatorTable.CARDINALITY,
+    SqlStdOperatorTable.ELEMENT,
+    // SPECIAL OPERATORS
+    SqlStdOperatorTable.ROW,
+    SqlStdOperatorTable.OVERLAPS,
+    SqlStdOperatorTable.LITERAL_CHAIN,
+    SqlStdOperatorTable.BETWEEN,
+    SqlStdOperatorTable.SYMMETRIC_BETWEEN,
+    SqlStdOperatorTable.NOT_BETWEEN,
+    SqlStdOperatorTable.SYMMETRIC_NOT_BETWEEN,
+    SqlStdOperatorTable.NOT_LIKE,
+    SqlStdOperatorTable.LIKE,
+    SqlStdOperatorTable.NOT_SIMILAR_TO,
+    SqlStdOperatorTable.SIMILAR_TO,
+    SqlStdOperatorTable.CASE,
+    SqlStdOperatorTable.REINTERPRET,
+    SqlStdOperatorTable.EXTRACT_DATE,
+    // FUNCTIONS
+    SqlStdOperatorTable.SUBSTRING,
+    SqlStdOperatorTable.OVERLAY,
+    SqlStdOperatorTable.TRIM,
+    SqlStdOperatorTable.POSITION,
+    SqlStdOperatorTable.CHAR_LENGTH,
+    SqlStdOperatorTable.CHARACTER_LENGTH,
+    SqlStdOperatorTable.UPPER,
+    SqlStdOperatorTable.LOWER,
+    SqlStdOperatorTable.INITCAP,
+    SqlStdOperatorTable.POWER,
+    SqlStdOperatorTable.SQRT,
+    SqlStdOperatorTable.MOD,
+    SqlStdOperatorTable.LN,
+    SqlStdOperatorTable.LOG10,
+    SqlStdOperatorTable.ABS,
+    SqlStdOperatorTable.EXP,
+    SqlStdOperatorTable.NULLIF,
+    SqlStdOperatorTable.COALESCE,
+    SqlStdOperatorTable.FLOOR,
+    SqlStdOperatorTable.CEIL,
+    SqlStdOperatorTable.LOCALTIME,
+    SqlStdOperatorTable.LOCALTIMESTAMP,
+    SqlStdOperatorTable.CURRENT_TIME,
+    SqlStdOperatorTable.CURRENT_TIMESTAMP,
+    SqlStdOperatorTable.CURRENT_DATE,
+    SqlStdOperatorTable.CAST,
+    SqlStdOperatorTable.EXTRACT,
+    SqlStdOperatorTable.QUARTER,
+    SqlStdOperatorTable.SCALAR_QUERY,
+    SqlStdOperatorTable.EXISTS
+  )
+
+  builtInSqlOperators.foreach(register)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/ValidationResult.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/ValidationResult.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/ValidationResult.scala
new file mode 100644
index 0000000..64a568b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/ValidationResult.scala
@@ -0,0 +1,53 @@
+/*
+ * 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.validate
+
+/**
+  * Represents the result of a validation.
+  */
+sealed trait ValidationResult {
+  def isFailure: Boolean = !isSuccess
+  def isSuccess: Boolean
+
+  /**
+    * Allows constructing a cascade of validation results.
+    * The first failure result will be returned.
+    */
+  def orElse(other: ValidationResult): ValidationResult = {
+    if (isSuccess) {
+      other
+    } else {
+      this
+    }
+  }
+}
+
+/**
+  * Represents the successful result of a validation.
+  */
+object ValidationSuccess extends ValidationResult {
+  val isSuccess: Boolean = true
+}
+
+/**
+  * Represents the failing result of a validation,
+  * with a error message to show the reason of failure.
+  */
+case class ValidationFailure(message: String) extends ValidationResult {
+  val isSuccess: Boolean = false
+}


[09/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/ExpressionReductionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/ExpressionReductionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/ExpressionReductionTest.scala
deleted file mode 100644
index b8156a2..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/ExpressionReductionTest.scala
+++ /dev/null
@@ -1,424 +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.api.table
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.utils.TableTestBase
-import org.apache.flink.api.table.utils.TableTestUtil._
-import org.junit.Test
-
-class ExpressionReductionTest extends TableTestBase {
-
-  @Test
-  def testReduceCalcExpressionForBatchSQL(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT " +
-      "(3+4)+a, " +
-      "b+(1+2), " +
-      "CASE 11 WHEN 1 THEN 'a' ELSE 'b' END, " +
-      "TRIM(BOTH ' STRING '),  " +
-      "'test' || 'string', " +
-      "NULLIF(1, 1), " +
-      "TIMESTAMP '1990-10-14 23:00:00.123' + INTERVAL '10 00:00:01' DAY TO SECOND, " +
-      "EXTRACT(DAY FROM INTERVAL '19 12:10:10.123' DAY TO SECOND(3)),  " +
-      "1 IS NULL, " +
-      "'TEST' LIKE '%EST', " +
-      "FLOOR(2.5), " +
-      "'TEST' IN ('west', 'TEST', 'rest'), " +
-      "CAST(TRUE AS VARCHAR) || 'X'" +
-      "FROM MyTable WHERE a>(1+7)"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select",
-        "+(7, a) AS EXPR$0",
-        "+(b, 3) AS EXPR$1",
-        "'b' AS EXPR$2",
-        "'STRING' AS EXPR$3",
-        "'teststring' AS EXPR$4",
-        "null AS EXPR$5",
-        "1990-10-24 23:00:01 AS EXPR$6",
-        "19 AS EXPR$7",
-        "false AS EXPR$8",
-        "true AS EXPR$9",
-        "2 AS EXPR$10",
-        "true AS EXPR$11",
-        "'trueX' AS EXPR$12"
-      ),
-      term("where", ">(a, 8)")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testReduceProjectExpressionForBatchSQL(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT " +
-      "(3+4)+a, " +
-      "b+(1+2), " +
-      "CASE 11 WHEN 1 THEN 'a' ELSE 'b' END, " +
-      "TRIM(BOTH ' STRING '),  " +
-      "'test' || 'string', " +
-      "NULLIF(1, 1), " +
-      "TIMESTAMP '1990-10-14 23:00:00.123' + INTERVAL '10 00:00:01' DAY TO SECOND, " +
-      "EXTRACT(DAY FROM INTERVAL '19 12:10:10.123' DAY TO SECOND(3)),  " +
-      "1 IS NULL, " +
-      "'TEST' LIKE '%EST', " +
-      "FLOOR(2.5), " +
-      "'TEST' IN ('west', 'TEST', 'rest'), " +
-      "CAST(TRUE AS VARCHAR) || 'X'" +
-      "FROM MyTable"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select",
-        "+(7, a) AS EXPR$0",
-        "+(b, 3) AS EXPR$1",
-        "'b' AS EXPR$2",
-        "'STRING' AS EXPR$3",
-        "'teststring' AS EXPR$4",
-        "null AS EXPR$5",
-        "1990-10-24 23:00:01 AS EXPR$6",
-        "19 AS EXPR$7",
-        "false AS EXPR$8",
-        "true AS EXPR$9",
-        "2 AS EXPR$10",
-        "true AS EXPR$11",
-        "'trueX' AS EXPR$12"
-      )
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testReduceFilterExpressionForBatchSQL(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT " +
-      "*" +
-      "FROM MyTable WHERE a>(1+7)"
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select", "a", "b", "c"),
-      term("where", ">(a, 8)")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testReduceCalcExpressionForBatchTableAPI(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val result = table
-      .where('a > (1 + 7))
-      .select((3 + 4).toExpr + 6,
-              (11 === 1) ? ("a", "b"),
-              " STRING ".trim,
-              "test" + "string",
-              "1990-10-14 23:00:00.123".toTimestamp + 10.days + 1.second,
-              1.isNull,
-              "TEST".like("%EST"),
-              2.5.toExpr.floor(),
-              true.cast(Types.STRING) + "X")
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select",
-        "13 AS _c0",
-        "'b' AS _c1",
-        "'STRING' AS _c2",
-        "'teststring' AS _c3",
-        "1990-10-24 23:00:01 AS _c4",
-        "false AS _c5",
-        "true AS _c6",
-        "2E0 AS _c7",
-        "'trueX' AS _c8"
-      ),
-      term("where", ">(a, 8)")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testReduceProjectExpressionForBatchTableAPI(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val result = table
-      .select((3 + 4).toExpr + 6,
-              (11 === 1) ? ("a", "b"),
-              " STRING ".trim,
-              "test" + "string",
-              "1990-10-14 23:00:00.123".toTimestamp + 10.days + 1.second,
-              1.isNull,
-              "TEST".like("%EST"),
-              2.5.toExpr.floor(),
-              true.cast(Types.STRING) + "X")
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select",
-        "13 AS _c0",
-        "'b' AS _c1",
-        "'STRING' AS _c2",
-        "'teststring' AS _c3",
-        "1990-10-24 23:00:01 AS _c4",
-        "false AS _c5",
-        "true AS _c6",
-        "2E0 AS _c7",
-        "'trueX' AS _c8"
-      )
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testReduceFilterExpressionForBatchTableAPI(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val result = table
-      .where('a > (1 + 7))
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select", "a", "b", "c"),
-      term("where", ">(a, 8)")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testReduceCalcExpressionForStreamSQL(): Unit = {
-    val util = streamTestUtil()
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT " +
-      "(3+4)+a, " +
-      "b+(1+2), " +
-      "CASE 11 WHEN 1 THEN 'a' ELSE 'b' END, " +
-      "TRIM(BOTH ' STRING '),  " +
-      "'test' || 'string', " +
-      "NULLIF(1, 1), " +
-      "TIMESTAMP '1990-10-14 23:00:00.123' + INTERVAL '10 00:00:01' DAY TO SECOND, " +
-      "EXTRACT(DAY FROM INTERVAL '19 12:10:10.123' DAY TO SECOND(3)),  " +
-      "1 IS NULL, " +
-      "'TEST' LIKE '%EST', " +
-      "FLOOR(2.5), " +
-      "'TEST' IN ('west', 'TEST', 'rest'), " +
-      "CAST(TRUE AS VARCHAR) || 'X'" +
-      "FROM MyTable WHERE a>(1+7)"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      streamTableNode(0),
-      term("select",
-        "+(7, a) AS EXPR$0",
-        "+(b, 3) AS EXPR$1",
-        "'b' AS EXPR$2",
-        "'STRING' AS EXPR$3",
-        "'teststring' AS EXPR$4",
-        "null AS EXPR$5",
-        "1990-10-24 23:00:01 AS EXPR$6",
-        "19 AS EXPR$7",
-        "false AS EXPR$8",
-        "true AS EXPR$9",
-        "2 AS EXPR$10",
-        "true AS EXPR$11",
-        "'trueX' AS EXPR$12"
-      ),
-      term("where", ">(a, 8)")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testReduceProjectExpressionForStreamSQL(): Unit = {
-    val util = streamTestUtil()
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT " +
-      "(3+4)+a, " +
-      "b+(1+2), " +
-      "CASE 11 WHEN 1 THEN 'a' ELSE 'b' END, " +
-      "TRIM(BOTH ' STRING '),  " +
-      "'test' || 'string', " +
-      "NULLIF(1, 1), " +
-      "TIMESTAMP '1990-10-14 23:00:00.123' + INTERVAL '10 00:00:01' DAY TO SECOND, " +
-      "EXTRACT(DAY FROM INTERVAL '19 12:10:10.123' DAY TO SECOND(3)),  " +
-      "1 IS NULL, " +
-      "'TEST' LIKE '%EST', " +
-      "FLOOR(2.5), " +
-      "'TEST' IN ('west', 'TEST', 'rest'), " +
-      "CAST(TRUE AS VARCHAR) || 'X'" +
-      "FROM MyTable"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      streamTableNode(0),
-      term("select",
-        "+(7, a) AS EXPR$0",
-        "+(b, 3) AS EXPR$1",
-        "'b' AS EXPR$2",
-        "'STRING' AS EXPR$3",
-        "'teststring' AS EXPR$4",
-        "null AS EXPR$5",
-        "1990-10-24 23:00:01 AS EXPR$6",
-        "19 AS EXPR$7",
-        "false AS EXPR$8",
-        "true AS EXPR$9",
-        "2 AS EXPR$10",
-        "true AS EXPR$11",
-        "'trueX' AS EXPR$12"
-      )
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testReduceFilterExpressionForStreamSQL(): Unit = {
-    val util = streamTestUtil()
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT " +
-      "*" +
-      "FROM MyTable WHERE a>(1+7)"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      streamTableNode(0),
-      term("select", "a", "b", "c"),
-      term("where", ">(a, 8)")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testReduceCalcExpressionForStreamTableAPI(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val result = table
-      .where('a > (1 + 7))
-      .select((3 + 4).toExpr + 6,
-              (11 === 1) ? ("a", "b"),
-              " STRING ".trim,
-              "test" + "string",
-              "1990-10-14 23:00:00.123".toTimestamp + 10.days + 1.second,
-              1.isNull,
-              "TEST".like("%EST"),
-              2.5.toExpr.floor(),
-              true.cast(Types.STRING) + "X")
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      streamTableNode(0),
-      term("select",
-        "13 AS _c0",
-        "'b' AS _c1",
-        "'STRING' AS _c2",
-        "'teststring' AS _c3",
-        "1990-10-24 23:00:01 AS _c4",
-        "false AS _c5",
-        "true AS _c6",
-        "2E0 AS _c7",
-        "'trueX' AS _c8"
-      ),
-      term("where", ">(a, 8)")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testReduceProjectExpressionForStreamTableAPI(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val result =  table
-      .select((3 + 4).toExpr + 6,
-              (11 === 1) ? ("a", "b"),
-              " STRING ".trim,
-              "test" + "string",
-              "1990-10-14 23:00:00.123".toTimestamp + 10.days + 1.second,
-              1.isNull,
-              "TEST".like("%EST"),
-              2.5.toExpr.floor(),
-              true.cast(Types.STRING) + "X")
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      streamTableNode(0),
-      term("select",
-        "13 AS _c0",
-        "'b' AS _c1",
-        "'STRING' AS _c2",
-        "'teststring' AS _c3",
-        "1990-10-24 23:00:01 AS _c4",
-        "false AS _c5",
-        "true AS _c6",
-        "2E0 AS _c7",
-        "'trueX' AS _c8"
-      )
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testReduceFilterExpressionForStreamTableAPI(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-
-    val result = table
-      .where('a > (1 + 7))
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      streamTableNode(0),
-      term("select", "a", "b", "c"),
-      term("where", ">(a, 8)")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/TableEnvironmentTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/TableEnvironmentTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/TableEnvironmentTest.scala
deleted file mode 100644
index db86ef3..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/TableEnvironmentTest.scala
+++ /dev/null
@@ -1,292 +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.api.table
-
-import org.apache.calcite.tools.RuleSet
-import org.apache.flink.api.scala._
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor}
-import org.apache.flink.api.table.expressions.{Alias, UnresolvedFieldReference}
-import org.apache.flink.api.table.sinks.TableSink
-import org.junit.Test
-import org.junit.Assert.assertEquals
-
-class TableEnvironmentTest {
-
-  val tEnv = new MockTableEnvironment
-
-  val tupleType = new TupleTypeInfo(
-    INT_TYPE_INFO,
-    STRING_TYPE_INFO,
-    DOUBLE_TYPE_INFO)
-
-  val caseClassType = implicitly[TypeInformation[CClass]]
-
-  val pojoType = TypeExtractor.createTypeInfo(classOf[PojoClass])
-
-  val atomicType = INT_TYPE_INFO
-
-  @Test
-  def testGetFieldInfoTuple(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(tupleType)
-
-    fieldInfo._1.zip(Array("f0", "f1", "f2")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test
-  def testGetFieldInfoCClass(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(caseClassType)
-
-    fieldInfo._1.zip(Array("cf1", "cf2", "cf3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test
-  def testGetFieldInfoPojo(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(pojoType)
-
-    fieldInfo._1.zip(Array("pf1", "pf2", "pf3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test(expected = classOf[TableException])
-  def testGetFieldInfoAtomic(): Unit = {
-    tEnv.getFieldInfo(atomicType)
-  }
-
-  @Test
-  def testGetFieldInfoTupleNames(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      tupleType,
-      Array(
-        new UnresolvedFieldReference("name1"),
-        new UnresolvedFieldReference("name2"),
-        new UnresolvedFieldReference("name3")
-    ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test
-  def testGetFieldInfoCClassNames(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      caseClassType,
-      Array(
-        new UnresolvedFieldReference("name1"),
-        new UnresolvedFieldReference("name2"),
-        new UnresolvedFieldReference("name3")
-    ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test(expected = classOf[TableException])
-  def testGetFieldInfoPojoNames1(): Unit = {
-    tEnv.getFieldInfo(
-      pojoType,
-      Array(
-        new UnresolvedFieldReference("name1"),
-        new UnresolvedFieldReference("name2"),
-        new UnresolvedFieldReference("name3")
-      ))
-  }
-
-  @Test
-  def testGetFieldInfoPojoNames2(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      pojoType,
-      Array(
-        new UnresolvedFieldReference("pf3"),
-        new UnresolvedFieldReference("pf1"),
-        new UnresolvedFieldReference("pf2")
-      ))
-
-    fieldInfo._1.zip(Array("pf3", "pf1", "pf2")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test
-  def testGetFieldInfoAtomicName1(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      atomicType,
-      Array(new UnresolvedFieldReference("name"))
-    )
-
-    fieldInfo._1.zip(Array("name")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test(expected = classOf[TableException])
-  def testGetFieldInfoAtomicName2(): Unit = {
-    tEnv.getFieldInfo(
-      atomicType,
-      Array(
-        new UnresolvedFieldReference("name1"),
-        new UnresolvedFieldReference("name2")
-      ))
-  }
-
-  @Test
-  def testGetFieldInfoTupleAlias1(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      tupleType,
-      Array(
-        new Alias(UnresolvedFieldReference("f0"), "name1"),
-        new Alias(UnresolvedFieldReference("f1"), "name2"),
-        new Alias(UnresolvedFieldReference("f2"), "name3")
-      ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test
-  def testGetFieldInfoTupleAlias2(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      tupleType,
-      Array(
-        new Alias(UnresolvedFieldReference("f2"), "name1"),
-        new Alias(UnresolvedFieldReference("f0"), "name2"),
-        new Alias(UnresolvedFieldReference("f1"), "name3")
-      ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test(expected = classOf[TableException])
-  def testGetFieldInfoTupleAlias3(): Unit = {
-    tEnv.getFieldInfo(
-      tupleType,
-      Array(
-        new Alias(UnresolvedFieldReference("xxx"), "name1"),
-        new Alias(UnresolvedFieldReference("yyy"), "name2"),
-        new Alias(UnresolvedFieldReference("zzz"), "name3")
-      ))
-  }
-
-  @Test
-  def testGetFieldInfoCClassAlias1(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      caseClassType,
-      Array(
-        new Alias(new UnresolvedFieldReference("cf1"), "name1"),
-        new Alias(new UnresolvedFieldReference("cf2"), "name2"),
-        new Alias(new UnresolvedFieldReference("cf3"), "name3")
-      ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test
-  def testGetFieldInfoCClassAlias2(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      caseClassType,
-      Array(
-        new Alias(new UnresolvedFieldReference("cf3"), "name1"),
-        new Alias(new UnresolvedFieldReference("cf1"), "name2"),
-        new Alias(new UnresolvedFieldReference("cf2"), "name3")
-      ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test(expected = classOf[TableException])
-  def testGetFieldInfoCClassAlias3(): Unit = {
-    tEnv.getFieldInfo(
-      caseClassType,
-      Array(
-        new Alias(new UnresolvedFieldReference("xxx"), "name1"),
-        new Alias(new UnresolvedFieldReference("yyy"), "name2"),
-        new Alias(new UnresolvedFieldReference("zzz"), "name3")
-      ))
-  }
-
-  @Test
-  def testGetFieldInfoPojoAlias1(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      pojoType,
-      Array(
-        new Alias(new UnresolvedFieldReference("pf1"), "name1"),
-        new Alias(new UnresolvedFieldReference("pf2"), "name2"),
-        new Alias(new UnresolvedFieldReference("pf3"), "name3")
-      ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test
-  def testGetFieldInfoPojoAlias2(): Unit = {
-    val fieldInfo = tEnv.getFieldInfo(
-      pojoType,
-      Array(
-        new Alias(new UnresolvedFieldReference("pf3"), "name1"),
-        new Alias(new UnresolvedFieldReference("pf1"), "name2"),
-        new Alias(new UnresolvedFieldReference("pf2"), "name3")
-      ))
-
-    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
-    fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
-  }
-
-  @Test(expected = classOf[TableException])
-  def testGetFieldInfoPojoAlias3(): Unit = {
-    tEnv.getFieldInfo(
-      pojoType,
-      Array(
-        new Alias(new UnresolvedFieldReference("xxx"), "name1"),
-        new Alias(new UnresolvedFieldReference("yyy"), "name2"),
-        new Alias(new UnresolvedFieldReference("zzz"), "name3")
-      ))
-  }
-
-  @Test(expected = classOf[TableException])
-  def testGetFieldInfoAtomicAlias(): Unit = {
-    tEnv.getFieldInfo(
-      atomicType,
-      Array(
-        new Alias(new UnresolvedFieldReference("name1"), "name2")
-      ))
-  }
-
-}
-
-class MockTableEnvironment extends TableEnvironment(new TableConfig) {
-
-  override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = ???
-
-  override protected def checkValidTableName(name: String): Unit = ???
-
-  override protected def getBuiltInRuleSet: RuleSet = ???
-
-  override def sql(query: String): Table = ???
-}
-
-case class CClass(cf1: Int, cf2: String, cf3: Double)
-
-class PojoClass(var pf1: Int, var pf2: String, var pf3: Double) {
-  def this() = this(0, "", 0.0)
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ArrayTypeTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ArrayTypeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ArrayTypeTest.scala
deleted file mode 100644
index 2ba76ad..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ArrayTypeTest.scala
+++ /dev/null
@@ -1,360 +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.api.table.expressions
-
-import java.sql.Date
-
-import org.apache.flink.api.common.typeinfo.{PrimitiveArrayTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.expressions.utils.ExpressionTestBase
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.{Types, ValidationException}
-import org.apache.flink.types.Row
-import org.junit.Test
-
-class ArrayTypeTest extends ExpressionTestBase {
-
-  @Test(expected = classOf[ValidationException])
-  def testObviousInvalidIndexTableApi(): Unit = {
-    testTableApi('f2.at(0), "FAIL", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testEmptyArraySql(): Unit = {
-    testSqlApi("ARRAY[]", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testEmptyArrayTableApi(): Unit = {
-    testTableApi("FAIL", "array()", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testNullArraySql(): Unit = {
-    testSqlApi("ARRAY[NULL]", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testDifferentTypesArraySql(): Unit = {
-    testSqlApi("ARRAY[1, TRUE]", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testDifferentTypesArrayTableApi(): Unit = {
-    testTableApi("FAIL", "array(1, true)", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testUnsupportedComparison(): Unit = {
-    testAllApis(
-      'f2 <= 'f5.at(1),
-      "f2 <= f5.at(1)",
-      "f2 <= f5[1]",
-      "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testElementNonArray(): Unit = {
-    testTableApi(
-      'f0.element(),
-      "FAIL",
-      "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testElementNonArraySql(): Unit = {
-    testSqlApi(
-      "ELEMENT(f0)",
-      "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testCardinalityOnNonArray(): Unit = {
-    testTableApi('f0.cardinality(), "FAIL", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testCardinalityOnNonArraySql(): Unit = {
-    testSqlApi("CARDINALITY(f0)", "FAIL")
-  }
-
-  @Test
-  def testArrayLiterals(): Unit = {
-    // primitive literals
-    testAllApis(array(1, 2, 3), "array(1, 2, 3)", "ARRAY[1, 2, 3]", "[1, 2, 3]")
-
-    testAllApis(
-      array(true, true, true),
-      "array(true, true, true)",
-      "ARRAY[TRUE, TRUE, TRUE]",
-      "[true, true, true]")
-
-    // object literals
-    testTableApi(array(BigDecimal(1), BigDecimal(1)), "array(1p, 1p)", "[1, 1]")
-
-    testAllApis(
-      array(array(array(1), array(1))),
-      "array(array(array(1), array(1)))",
-      "ARRAY[ARRAY[ARRAY[1], ARRAY[1]]]",
-      "[[[1], [1]]]")
-
-    testAllApis(
-      array(1 + 1, 3 * 3),
-      "array(1 + 1, 3 * 3)",
-      "ARRAY[1 + 1, 3 * 3]",
-      "[2, 9]")
-
-    testAllApis(
-      array(Null(Types.INT), 1),
-      "array(Null(INT), 1)",
-      "ARRAY[NULLIF(1,1), 1]",
-      "[null, 1]")
-
-    testAllApis(
-      array(array(Null(Types.INT), 1)),
-      "array(array(Null(INT), 1))",
-      "ARRAY[ARRAY[NULLIF(1,1), 1]]",
-      "[[null, 1]]")
-
-    // implicit conversion
-    testTableApi(
-      Array(1, 2, 3),
-      "array(1, 2, 3)",
-      "[1, 2, 3]")
-
-    testTableApi(
-      Array[Integer](1, 2, 3),
-      "array(1, 2, 3)",
-      "[1, 2, 3]")
-
-    testAllApis(
-      Array(Date.valueOf("1985-04-11")),
-      "array('1985-04-11'.toDate)",
-      "ARRAY[DATE '1985-04-11']",
-      "[1985-04-11]")
-
-    testAllApis(
-      Array(BigDecimal(2.0002), BigDecimal(2.0003)),
-      "Array(2.0002p, 2.0003p)",
-      "ARRAY[CAST(2.0002 AS DECIMAL), CAST(2.0003 AS DECIMAL)]",
-      "[2.0002, 2.0003]")
-
-    testAllApis(
-      Array(Array(x = true)),
-      "Array(Array(true))",
-      "ARRAY[ARRAY[TRUE]]",
-      "[[true]]")
-
-    testAllApis(
-      Array(Array(1, 2, 3), Array(3, 2, 1)),
-      "Array(Array(1, 2, 3), Array(3, 2, 1))",
-      "ARRAY[ARRAY[1, 2, 3], ARRAY[3, 2, 1]]",
-      "[[1, 2, 3], [3, 2, 1]]")
-  }
-
-  @Test
-  def testArrayField(): Unit = {
-    testAllApis(
-      array('f0, 'f1),
-      "array(f0, f1)",
-      "ARRAY[f0, f1]",
-      "[null, 42]")
-
-    testAllApis(
-      array('f0, 'f1),
-      "array(f0, f1)",
-      "ARRAY[f0, f1]",
-      "[null, 42]")
-
-    testAllApis(
-      'f2,
-      "f2",
-      "f2",
-      "[1, 2, 3]")
-
-    testAllApis(
-      'f3,
-      "f3",
-      "f3",
-      "[1984-03-12, 1984-02-10]")
-
-    testAllApis(
-      'f5,
-      "f5",
-      "f5",
-      "[[1, 2, 3], null]")
-
-    testAllApis(
-      'f6,
-      "f6",
-      "f6",
-      "[1, null, null, 4]")
-
-    testAllApis(
-      'f2,
-      "f2",
-      "f2",
-      "[1, 2, 3]")
-
-    testAllApis(
-      'f2.at(1),
-      "f2.at(1)",
-      "f2[1]",
-      "1")
-
-    testAllApis(
-      'f3.at(1),
-      "f3.at(1)",
-      "f3[1]",
-      "1984-03-12")
-
-    testAllApis(
-      'f3.at(2),
-      "f3.at(2)",
-      "f3[2]",
-      "1984-02-10")
-
-    testAllApis(
-      'f5.at(1).at(2),
-      "f5.at(1).at(2)",
-      "f5[1][2]",
-      "2")
-
-    testAllApis(
-      'f5.at(2).at(2),
-      "f5.at(2).at(2)",
-      "f5[2][2]",
-      "null")
-
-    testAllApis(
-      'f4.at(2).at(2),
-      "f4.at(2).at(2)",
-      "f4[2][2]",
-      "null")
-  }
-
-  @Test
-  def testArrayOperations(): Unit = {
-    // cardinality
-    testAllApis(
-      'f2.cardinality(),
-      "f2.cardinality()",
-      "CARDINALITY(f2)",
-      "3")
-
-    testAllApis(
-      'f4.cardinality(),
-      "f4.cardinality()",
-      "CARDINALITY(f4)",
-      "null")
-
-    // element
-    testAllApis(
-      'f9.element(),
-      "f9.element()",
-      "ELEMENT(f9)",
-      "1")
-
-    testAllApis(
-      'f8.element(),
-      "f8.element()",
-      "ELEMENT(f8)",
-      "4.0")
-
-    testAllApis(
-      'f10.element(),
-      "f10.element()",
-      "ELEMENT(f10)",
-      "null")
-
-    testAllApis(
-      'f4.element(),
-      "f4.element()",
-      "ELEMENT(f4)",
-      "null")
-
-    // comparison
-    testAllApis(
-      'f2 === 'f5.at(1),
-      "f2 === f5.at(1)",
-      "f2 = f5[1]",
-      "true")
-
-    testAllApis(
-      'f6 === array(1, 2, 3),
-      "f6 === array(1, 2, 3)",
-      "f6 = ARRAY[1, 2, 3]",
-      "false")
-
-    testAllApis(
-      'f2 !== 'f5.at(1),
-      "f2 !== f5.at(1)",
-      "f2 <> f5[1]",
-      "false")
-
-    testAllApis(
-      'f2 === 'f7,
-      "f2 === f7",
-      "f2 = f7",
-      "false")
-
-    testAllApis(
-      'f2 !== 'f7,
-      "f2 !== f7",
-      "f2 <> f7",
-      "true")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  case class MyCaseClass(string: String, int: Int)
-
-  override def testData: Any = {
-    val testData = new Row(11)
-    testData.setField(0, null)
-    testData.setField(1, 42)
-    testData.setField(2, Array(1, 2, 3))
-    testData.setField(3, Array(Date.valueOf("1984-03-12"), Date.valueOf("1984-02-10")))
-    testData.setField(4, null)
-    testData.setField(5, Array(Array(1, 2, 3), null))
-    testData.setField(6, Array[Integer](1, null, null, 4))
-    testData.setField(7, Array(1, 2, 3, 4))
-    testData.setField(8, Array(4.0))
-    testData.setField(9, Array[Integer](1))
-    testData.setField(10, Array[Integer]())
-    testData
-  }
-
-  override def typeInfo: TypeInformation[Any] = {
-    new RowTypeInfo(
-      Types.INT,
-      Types.INT,
-      PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO,
-      ObjectArrayTypeInfo.getInfoFor(Types.DATE),
-      ObjectArrayTypeInfo.getInfoFor(ObjectArrayTypeInfo.getInfoFor(Types.INT)),
-      ObjectArrayTypeInfo.getInfoFor(PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO),
-      ObjectArrayTypeInfo.getInfoFor(Types.INT),
-      PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO,
-      PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO,
-      ObjectArrayTypeInfo.getInfoFor(Types.INT),
-      ObjectArrayTypeInfo.getInfoFor(Types.INT)
-    ).asInstanceOf[TypeInformation[Any]]
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/CompositeAccessTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/CompositeAccessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/CompositeAccessTest.scala
deleted file mode 100644
index 879f68d..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/CompositeAccessTest.scala
+++ /dev/null
@@ -1,193 +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.api.table.expressions
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor}
-import org.apache.flink.api.scala.createTypeInformation
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.{Types, ValidationException}
-import org.apache.flink.types.Row
-import org.apache.flink.api.table.expressions.CompositeAccessTest.{MyCaseClass, MyCaseClass2, MyPojo}
-import org.apache.flink.api.table.expressions.utils.ExpressionTestBase
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.junit.Test
-
-
-class CompositeAccessTest extends ExpressionTestBase {
-
-  @Test
-  def testGetField(): Unit = {
-
-    // single field by string key
-    testAllApis(
-      'f0.get("intField"),
-      "f0.get('intField')",
-      "testTable.f0.intField",
-      "42")
-
-    testSqlApi("testTable.f0.stringField", "Bob")
-
-    testSqlApi("testTable.f0.booleanField", "true")
-
-    // single field by int key
-    testTableApi(
-      'f0.get(0),
-      "f0.get(0)",
-      "42")
-
-    // nested single field
-    testAllApis(
-      'f1.get("objectField").get("intField"),
-      "f1.get('objectField').get('intField')",
-      "testTable.f1.objectField.intField",
-      "25")
-
-    testSqlApi("testTable.f1.objectField.stringField", "Timo")
-
-    testSqlApi("testTable.f1.objectField.booleanField", "false")
-
-    testAllApis(
-      'f2.get(0),
-      "f2.get(0)",
-      "testTable.f2._1",
-      "a")
-
-    testSqlApi("testTable.f3.f1", "b")
-
-    testSqlApi("testTable.f4.myString", "Hello")
-
-    testSqlApi("testTable.f5", "13")
-
-    testAllApis(
-      'f7.get("_1"),
-      "get(f7, '_1')",
-      "testTable.f7._1",
-      "true")
-
-    // composite field return type
-    testSqlApi("testTable.f6", "MyCaseClass2(null)")
-
-    testAllApis(
-      'f1.get("objectField"),
-      "f1.get('objectField')",
-      "testTable.f1.objectField",
-      "MyCaseClass(25,Timo,false)")
-
-    testAllApis(
-      'f0,
-      "f0",
-      "testTable.f0",
-      "MyCaseClass(42,Bob,true)")
-
-    // flattening (test base only returns first column)
-    testAllApis(
-      'f1.get("objectField").flatten(),
-      "f1.get('objectField').flatten()",
-      "testTable.f1.objectField.*",
-      "25")
-
-    testAllApis(
-      'f0.flatten(),
-      "flatten(f0)",
-      "testTable.f0.*",
-      "42")
-
-    testTableApi(12.flatten(), "12.flatten()", "12")
-
-    testTableApi('f5.flatten(), "f5.flatten()", "13")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testWrongSqlField(): Unit = {
-    testSqlApi("testTable.f5.test", "13")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testWrongIntKeyField(): Unit = {
-    testTableApi('f0.get(555), "'fail'", "fail")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testWrongIntKeyField2(): Unit = {
-    testTableApi("fail", "f0.get(555)", "fail")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testWrongStringKeyField(): Unit = {
-    testTableApi('f0.get("fghj"), "'fail'", "fail")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testWrongStringKeyField2(): Unit = {
-    testTableApi("fail", "f0.get('fghj')", "fail")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  def testData = {
-    val testData = new Row(8)
-    testData.setField(0, MyCaseClass(42, "Bob", booleanField = true))
-    testData.setField(1, MyCaseClass2(MyCaseClass(25, "Timo", booleanField = false)))
-    testData.setField(2, ("a", "b"))
-    testData.setField(3, new org.apache.flink.api.java.tuple.Tuple2[String, String]("a", "b"))
-    testData.setField(4, new MyPojo())
-    testData.setField(5, 13)
-    testData.setField(6, MyCaseClass2(null))
-    testData.setField(7, Tuple1(true))
-    testData
-  }
-
-  def typeInfo = {
-    new RowTypeInfo(
-      createTypeInformation[MyCaseClass],
-      createTypeInformation[MyCaseClass2],
-      createTypeInformation[(String, String)],
-      new TupleTypeInfo(Types.STRING, Types.STRING),
-      TypeExtractor.createTypeInfo(classOf[MyPojo]),
-      Types.INT,
-      createTypeInformation[MyCaseClass2],
-      createTypeInformation[Tuple1[Boolean]]
-      ).asInstanceOf[TypeInformation[Any]]
-  }
-
-}
-
-object CompositeAccessTest {
-  case class MyCaseClass(intField: Int, stringField: String, booleanField: Boolean)
-
-  case class MyCaseClass2(objectField: MyCaseClass)
-
-  class MyPojo {
-    private var myInt: Int = 0
-    private var myString: String = "Hello"
-
-    def getMyInt = myInt
-
-    def setMyInt(value: Int) = {
-      myInt = value
-    }
-
-    def getMyString = myString
-
-    def setMyString(value: String) = {
-      myString = myString
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/DecimalTypeTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/DecimalTypeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/DecimalTypeTest.scala
deleted file mode 100644
index a986365..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/DecimalTypeTest.scala
+++ /dev/null
@@ -1,311 +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.api.table.expressions
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.Types
-import org.apache.flink.types.Row
-import org.apache.flink.api.table.expressions.utils.ExpressionTestBase
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.junit.Test
-
-class DecimalTypeTest extends ExpressionTestBase {
-
-  @Test
-  def testDecimalLiterals(): Unit = {
-    // implicit double
-    testAllApis(
-      11.2,
-      "11.2",
-      "11.2",
-      "11.2")
-
-    // implicit double
-    testAllApis(
-      0.7623533651719233,
-      "0.7623533651719233",
-      "0.7623533651719233",
-      "0.7623533651719233")
-
-    // explicit decimal (with precision of 19)
-    testAllApis(
-      BigDecimal("1234567891234567891"),
-      "1234567891234567891p",
-      "1234567891234567891",
-      "1234567891234567891")
-
-    // explicit decimal (high precision, not SQL compliant)
-    testTableApi(
-      BigDecimal("123456789123456789123456789"),
-      "123456789123456789123456789p",
-      "123456789123456789123456789")
-
-    // explicit decimal (high precision, not SQL compliant)
-    testTableApi(
-      BigDecimal("12.3456789123456789123456789"),
-      "12.3456789123456789123456789p",
-      "12.3456789123456789123456789")
-  }
-
-  @Test
-  def testDecimalBorders(): Unit = {
-    testAllApis(
-      Double.MaxValue,
-      Double.MaxValue.toString,
-      Double.MaxValue.toString,
-      Double.MaxValue.toString)
-
-    testAllApis(
-      Double.MinValue,
-      Double.MinValue.toString,
-      Double.MinValue.toString,
-      Double.MinValue.toString)
-
-    testAllApis(
-      Double.MinValue.cast(Types.FLOAT),
-      s"${Double.MinValue}.cast(FLOAT)",
-      s"CAST(${Double.MinValue} AS FLOAT)",
-      Float.NegativeInfinity.toString)
-
-    testAllApis(
-      Byte.MinValue.cast(Types.BYTE),
-      s"(${Byte.MinValue}).cast(BYTE)",
-      s"CAST(${Byte.MinValue} AS TINYINT)",
-      Byte.MinValue.toString)
-
-    testAllApis(
-      Byte.MinValue.cast(Types.BYTE) - 1.cast(Types.BYTE),
-      s"(${Byte.MinValue}).cast(BYTE) - (1).cast(BYTE)",
-      s"CAST(${Byte.MinValue} AS TINYINT) - CAST(1 AS TINYINT)",
-      Byte.MaxValue.toString)
-
-    testAllApis(
-      Short.MinValue.cast(Types.SHORT),
-      s"(${Short.MinValue}).cast(SHORT)",
-      s"CAST(${Short.MinValue} AS SMALLINT)",
-      Short.MinValue.toString)
-
-    testAllApis(
-      Int.MinValue.cast(Types.INT) - 1,
-      s"(${Int.MinValue}).cast(INT) - 1",
-      s"CAST(${Int.MinValue} AS INT) - 1",
-      Int.MaxValue.toString)
-
-    testAllApis(
-      Long.MinValue.cast(Types.LONG),
-      s"(${Long.MinValue}L).cast(LONG)",
-      s"CAST(${Long.MinValue} AS BIGINT)",
-      Long.MinValue.toString)
-  }
-
-  @Test
-  def testDecimalCasting(): Unit = {
-    // from String
-    testTableApi(
-      "123456789123456789123456789".cast(Types.DECIMAL),
-      "'123456789123456789123456789'.cast(DECIMAL)",
-      "123456789123456789123456789")
-
-    // from double
-    testAllApis(
-      'f3.cast(Types.DECIMAL),
-      "f3.cast(DECIMAL)",
-      "CAST(f3 AS DECIMAL)",
-      "4.2")
-
-    // to double
-    testAllApis(
-      'f0.cast(Types.DOUBLE),
-      "f0.cast(DOUBLE)",
-      "CAST(f0 AS DOUBLE)",
-      "1.2345678912345679E8")
-
-    // to int
-    testAllApis(
-      'f4.cast(Types.INT),
-      "f4.cast(INT)",
-      "CAST(f4 AS INT)",
-      "123456789")
-
-    // to long
-    testAllApis(
-      'f4.cast(Types.LONG),
-      "f4.cast(LONG)",
-      "CAST(f4 AS BIGINT)",
-      "123456789")
-
-    // to boolean (not SQL compliant)
-    testTableApi(
-      'f1.cast(Types.BOOLEAN),
-      "f1.cast(BOOL)",
-      "true")
-
-    testTableApi(
-      'f5.cast(Types.BOOLEAN),
-      "f5.cast(BOOL)",
-      "false")
-
-    testTableApi(
-      BigDecimal("123456789.123456789123456789").cast(Types.DOUBLE),
-      "(123456789.123456789123456789p).cast(DOUBLE)",
-      "1.2345678912345679E8")
-  }
-
-  @Test
-  def testDecimalArithmetic(): Unit = {
-    // implicit cast to decimal
-    testAllApis(
-      'f1 + 12,
-      "f1 + 12",
-      "f1 + 12",
-      "123456789123456789123456801")
-
-    // implicit cast to decimal
-    testAllApis(
-      Literal(12) + 'f1,
-      "12 + f1",
-      "12 + f1",
-      "123456789123456789123456801")
-
-    // implicit cast to decimal
-    testAllApis(
-      'f1 + 12.3,
-      "f1 + 12.3",
-      "f1 + 12.3",
-      "123456789123456789123456801.3")
-
-    // implicit cast to decimal
-    testAllApis(
-      Literal(12.3) + 'f1,
-      "12.3 + f1",
-      "12.3 + f1",
-      "123456789123456789123456801.3")
-
-    testAllApis(
-      'f1 + 'f1,
-      "f1 + f1",
-      "f1 + f1",
-      "246913578246913578246913578")
-
-    testAllApis(
-      'f1 - 'f1,
-      "f1 - f1",
-      "f1 - f1",
-      "0")
-
-    testAllApis(
-      'f1 * 'f1,
-      "f1 * f1",
-      "f1 * f1",
-      "15241578780673678546105778281054720515622620750190521")
-
-    testAllApis(
-      'f1 / 'f1,
-      "f1 / f1",
-      "f1 / f1",
-      "1")
-
-    testAllApis(
-      'f1 % 'f1,
-      "f1 % f1",
-      "MOD(f1, f1)",
-      "0")
-
-    testAllApis(
-      -'f0,
-      "-f0",
-      "-f0",
-      "-123456789.123456789123456789")
-  }
-
-  @Test
-  def testDecimalComparison(): Unit = {
-    testAllApis(
-      'f1 < 12,
-      "f1 < 12",
-      "f1 < 12",
-      "false")
-
-    testAllApis(
-      'f1 > 12,
-      "f1 > 12",
-      "f1 > 12",
-      "true")
-
-    testAllApis(
-      'f1 === 12,
-      "f1 === 12",
-      "f1 = 12",
-      "false")
-
-    testAllApis(
-      'f5 === 0,
-      "f5 === 0",
-      "f5 = 0",
-      "true")
-
-    testAllApis(
-      'f1 === BigDecimal("123456789123456789123456789"),
-      "f1 === 123456789123456789123456789p",
-      "f1 = CAST('123456789123456789123456789' AS DECIMAL)",
-      "true")
-
-    testAllApis(
-      'f1 !== BigDecimal("123456789123456789123456789"),
-      "f1 !== 123456789123456789123456789p",
-      "f1 <> CAST('123456789123456789123456789' AS DECIMAL)",
-      "false")
-
-    testAllApis(
-      'f4 < 'f0,
-      "f4 < f0",
-      "f4 < f0",
-      "true")
-
-    // TODO add all tests if FLINK-4070 is fixed
-    testSqlApi(
-      "12 < f1",
-      "true")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  def testData = {
-    val testData = new Row(6)
-    testData.setField(0, BigDecimal("123456789.123456789123456789").bigDecimal)
-    testData.setField(1, BigDecimal("123456789123456789123456789").bigDecimal)
-    testData.setField(2, 42)
-    testData.setField(3, 4.2)
-    testData.setField(4, BigDecimal("123456789").bigDecimal)
-    testData.setField(5, BigDecimal("0.000").bigDecimal)
-    testData
-  }
-
-  def typeInfo = {
-    new RowTypeInfo(
-      Types.DECIMAL,
-      Types.DECIMAL,
-      Types.INT,
-      Types.DOUBLE,
-      Types.DECIMAL,
-      Types.DECIMAL).asInstanceOf[TypeInformation[Any]]
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala
deleted file mode 100644
index 0b39d4d..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/NonDeterministicTests.scala
+++ /dev/null
@@ -1,89 +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.api.table.expressions
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala.table._
-import org.apache.flink.types.Row
-import org.apache.flink.api.table.expressions.utils.ExpressionTestBase
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.junit.{Ignore, Test}
-
-/**
-  * Tests that can only be checked manually as they are non-deterministic.
-  */
-class NonDeterministicTests extends ExpressionTestBase {
-
-  @Ignore
-  @Test
-  def testCurrentDate(): Unit = {
-    testAllApis(
-      currentDate(),
-      "currentDate()",
-      "CURRENT_DATE",
-      "PLEASE CHECK MANUALLY")
-  }
-
-  @Ignore
-  @Test
-  def testCurrentTime(): Unit = {
-    testAllApis(
-      currentTime(),
-      "currentTime()",
-      "CURRENT_TIME",
-      "PLEASE CHECK MANUALLY")
-  }
-
-  @Ignore
-  @Test
-  def testCurrentTimestamp(): Unit = {
-    testAllApis(
-      currentTimestamp(),
-      "currentTimestamp()",
-      "CURRENT_TIMESTAMP",
-      "PLEASE CHECK MANUALLY")
-  }
-
-  @Ignore
-  @Test
-  def testLocalTimestamp(): Unit = {
-    testAllApis(
-      localTimestamp(),
-      "localTimestamp()",
-      "LOCALTIMESTAMP",
-      "PLEASE CHECK MANUALLY")
-  }
-
-  @Ignore
-  @Test
-  def testLocalTime(): Unit = {
-    testAllApis(
-      localTime(),
-      "localTime()",
-      "LOCALTIME",
-      "PLEASE CHECK MANUALLY")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  override def testData: Any = new Row(0)
-
-  override def typeInfo: TypeInformation[Any] =
-    new RowTypeInfo().asInstanceOf[TypeInformation[Any]]
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala
deleted file mode 100644
index 3ef02a9..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarFunctionsTest.scala
+++ /dev/null
@@ -1,1166 +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.api.table.expressions
-
-import java.sql.{Date, Time, Timestamp}
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.expressions.utils.ExpressionTestBase
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.{Types, ValidationException}
-import org.apache.flink.types.Row
-import org.junit.Test
-
-class ScalarFunctionsTest extends ExpressionTestBase {
-
-  // ----------------------------------------------------------------------------------------------
-  // String functions
-  // ----------------------------------------------------------------------------------------------
-
-  @Test
-  def testOverlay(): Unit = {
-    testAllApis(
-      "xxxxxtest".overlay("xxxx", 6),
-      "'xxxxxtest'.overlay('xxxx', 6)",
-      "OVERLAY('xxxxxtest' PLACING 'xxxx' FROM 6)",
-      "xxxxxxxxx")
-
-    testAllApis(
-      "xxxxxtest".overlay("xxxx", 6, 2),
-      "'xxxxxtest'.overlay('xxxx', 6, 2)",
-      "OVERLAY('xxxxxtest' PLACING 'xxxx' FROM 6 FOR 2)",
-      "xxxxxxxxxst")
-  }
-
-  @Test
-  def testPosition(): Unit = {
-    testAllApis(
-      "test".position("xxxtest"),
-      "'test'.position('xxxtest')",
-      "POSITION('test' IN 'xxxtest')",
-      "4")
-
-    testAllApis(
-      "testx".position("xxxtest"),
-      "'testx'.position('xxxtest')",
-      "POSITION('testx' IN 'xxxtest')",
-      "0")
-  }
-
-  @Test
-  def testSubstring(): Unit = {
-    testAllApis(
-      'f0.substring(2),
-      "f0.substring(2)",
-      "SUBSTRING(f0, 2)",
-      "his is a test String.")
-
-    testAllApis(
-      'f0.substring(2, 5),
-      "f0.substring(2, 5)",
-      "SUBSTRING(f0, 2, 5)",
-      "his i")
-
-    testAllApis(
-      'f0.substring(1, 'f7),
-      "f0.substring(1, f7)",
-      "SUBSTRING(f0, 1, f7)",
-      "Thi")
-
-    testAllApis(
-      'f0.substring(1.cast(Types.BYTE), 'f7),
-      "f0.substring(1.cast(BYTE), f7)",
-      "SUBSTRING(f0, CAST(1 AS TINYINT), f7)",
-      "Thi")
-
-    testSqlApi(
-      "SUBSTRING(f0 FROM 2 FOR 1)",
-      "h")
-
-    testSqlApi(
-      "SUBSTRING(f0 FROM 2)",
-      "his is a test String.")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidSubstring1(): Unit = {
-    // Must fail. Parameter of substring must be an Integer not a Double.
-    testTableApi("test".substring(2.0.toExpr), "FAIL", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidSubstring2(): Unit = {
-    // Must fail. Parameter of substring must be an Integer not a String.
-    testTableApi("test".substring("test".toExpr), "FAIL", "FAIL")
-  }
-
-  @Test
-  def testTrim(): Unit = {
-    testAllApis(
-      'f8.trim(),
-      "f8.trim()",
-      "TRIM(f8)",
-      "This is a test String.")
-
-    testAllApis(
-      'f8.trim(removeLeading = true, removeTrailing = true, " "),
-      "trim(f8)",
-      "TRIM(f8)",
-      "This is a test String.")
-
-    testAllApis(
-      'f8.trim(removeLeading = false, removeTrailing = true, " "),
-      "f8.trim(TRAILING, ' ')",
-      "TRIM(TRAILING FROM f8)",
-      " This is a test String.")
-
-    testAllApis(
-      'f0.trim(removeLeading = true, removeTrailing = true, "."),
-      "trim(BOTH, '.', f0)",
-      "TRIM(BOTH '.' FROM f0)",
-      "This is a test String")
-  }
-
-  @Test
-  def testCharLength(): Unit = {
-    testAllApis(
-      'f0.charLength(),
-      "f0.charLength()",
-      "CHAR_LENGTH(f0)",
-      "22")
-
-    testAllApis(
-      'f0.charLength(),
-      "charLength(f0)",
-      "CHARACTER_LENGTH(f0)",
-      "22")
-  }
-
-  @Test
-  def testUpperCase(): Unit = {
-    testAllApis(
-      'f0.upperCase(),
-      "f0.upperCase()",
-      "UPPER(f0)",
-      "THIS IS A TEST STRING.")
-  }
-
-  @Test
-  def testLowerCase(): Unit = {
-    testAllApis(
-      'f0.lowerCase(),
-      "f0.lowerCase()",
-      "LOWER(f0)",
-      "this is a test string.")
-  }
-
-  @Test
-  def testInitCap(): Unit = {
-    testAllApis(
-      'f0.initCap(),
-      "f0.initCap()",
-      "INITCAP(f0)",
-      "This Is A Test String.")
-  }
-
-  @Test
-  def testConcat(): Unit = {
-    testAllApis(
-      'f0 + 'f0,
-      "f0 + f0",
-      "f0||f0",
-      "This is a test String.This is a test String.")
-  }
-
-  @Test
-  def testLike(): Unit = {
-    testAllApis(
-      'f0.like("Th_s%"),
-      "f0.like('Th_s%')",
-      "f0 LIKE 'Th_s%'",
-      "true")
-
-    testAllApis(
-      'f0.like("%is a%"),
-      "f0.like('%is a%')",
-      "f0 LIKE '%is a%'",
-      "true")
-  }
-
-  @Test
-  def testNotLike(): Unit = {
-    testAllApis(
-      !'f0.like("Th_s%"),
-      "!f0.like('Th_s%')",
-      "f0 NOT LIKE 'Th_s%'",
-      "false")
-
-    testAllApis(
-      !'f0.like("%is a%"),
-      "!f0.like('%is a%')",
-      "f0 NOT LIKE '%is a%'",
-      "false")
-  }
-
-  @Test
-  def testLikeWithEscape(): Unit = {
-    testSqlApi(
-      "f23 LIKE '&%Th_s%' ESCAPE '&'",
-      "true")
-
-    testSqlApi(
-      "f23 LIKE '&%%is a%' ESCAPE '&'",
-      "true")
-
-    testSqlApi(
-      "f0 LIKE 'Th_s%' ESCAPE '&'",
-      "true")
-
-    testSqlApi(
-      "f0 LIKE '%is a%' ESCAPE '&'",
-      "true")
-  }
-
-  @Test
-  def testNotLikeWithEscape(): Unit = {
-    testSqlApi(
-      "f23 NOT LIKE '&%Th_s%' ESCAPE '&'",
-      "false")
-
-    testSqlApi(
-      "f23 NOT LIKE '&%%is a%' ESCAPE '&'",
-      "false")
-
-    testSqlApi(
-      "f0 NOT LIKE 'Th_s%' ESCAPE '&'",
-      "false")
-
-    testSqlApi(
-      "f0 NOT LIKE '%is a%' ESCAPE '&'",
-      "false")
-  }
-
-  @Test
-  def testSimilar(): Unit = {
-    testAllApis(
-      'f0.similar("_*"),
-      "f0.similar('_*')",
-      "f0 SIMILAR TO '_*'",
-      "true")
-
-    testAllApis(
-      'f0.similar("This (is)? a (test)+ Strin_*"),
-      "f0.similar('This (is)? a (test)+ Strin_*')",
-      "f0 SIMILAR TO 'This (is)? a (test)+ Strin_*'",
-      "true")
-  }
-
-  @Test
-  def testNotSimilar(): Unit = {
-    testAllApis(
-      !'f0.similar("_*"),
-      "!f0.similar('_*')",
-      "f0 NOT SIMILAR TO '_*'",
-      "false")
-
-    testAllApis(
-      !'f0.similar("This (is)? a (test)+ Strin_*"),
-      "!f0.similar('This (is)? a (test)+ Strin_*')",
-      "f0 NOT SIMILAR TO 'This (is)? a (test)+ Strin_*'",
-      "false")
-  }
-
-  @Test
-  def testSimilarWithEscape(): Unit = {
-    testSqlApi(
-      "f24 SIMILAR TO '&*&__*' ESCAPE '&'",
-      "true")
-
-    testSqlApi(
-      "f0 SIMILAR TO '_*' ESCAPE '&'",
-      "true")
-
-    testSqlApi(
-      "f24 SIMILAR TO '&*&_This (is)? a (test)+ Strin_*' ESCAPE '&'",
-      "true")
-
-    testSqlApi(
-      "f0 SIMILAR TO 'This (is)? a (test)+ Strin_*' ESCAPE '&'",
-      "true")
-  }
-
-  @Test
-  def testNotSimilarWithEscape(): Unit = {
-    testSqlApi(
-      "f24 NOT SIMILAR TO '&*&__*' ESCAPE '&'",
-      "false")
-
-    testSqlApi(
-      "f0 NOT SIMILAR TO '_*' ESCAPE '&'",
-      "false")
-
-    testSqlApi(
-      "f24 NOT SIMILAR TO '&*&_This (is)? a (test)+ Strin_*' ESCAPE '&'",
-      "false")
-
-    testSqlApi(
-      "f0 NOT SIMILAR TO 'This (is)? a (test)+ Strin_*' ESCAPE '&'",
-      "false")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-  // Math functions
-  // ----------------------------------------------------------------------------------------------
-
-  @Test
-  def testMod(): Unit = {
-    testAllApis(
-      'f4.mod('f7),
-      "f4.mod(f7)",
-      "MOD(f4, f7)",
-      "2")
-
-    testAllApis(
-      'f4.mod(3),
-      "mod(f4, 3)",
-      "MOD(f4, 3)",
-      "2")
-
-    testAllApis(
-      'f4 % 3,
-      "mod(44, 3)",
-      "MOD(44, 3)",
-      "2")
-
-  }
-
-  @Test
-  def testExp(): Unit = {
-    testAllApis(
-      'f2.exp(),
-      "f2.exp()",
-      "EXP(f2)",
-      math.exp(42.toByte).toString)
-
-    testAllApis(
-      'f3.exp(),
-      "f3.exp()",
-      "EXP(f3)",
-      math.exp(43.toShort).toString)
-
-    testAllApis(
-      'f4.exp(),
-      "f4.exp()",
-      "EXP(f4)",
-      math.exp(44.toLong).toString)
-
-    testAllApis(
-      'f5.exp(),
-      "f5.exp()",
-      "EXP(f5)",
-      math.exp(4.5.toFloat).toString)
-
-    testAllApis(
-      'f6.exp(),
-      "f6.exp()",
-      "EXP(f6)",
-      math.exp(4.6).toString)
-
-    testAllApis(
-      'f7.exp(),
-      "exp(3)",
-      "EXP(3)",
-      math.exp(3).toString)
-
-    testAllApis(
-      'f7.exp(),
-      "exp(3)",
-      "EXP(3)",
-      math.exp(3).toString)
-  }
-
-  @Test
-  def testLog10(): Unit = {
-    testAllApis(
-      'f2.log10(),
-      "f2.log10()",
-      "LOG10(f2)",
-      math.log10(42.toByte).toString)
-
-    testAllApis(
-      'f3.log10(),
-      "f3.log10()",
-      "LOG10(f3)",
-      math.log10(43.toShort).toString)
-
-    testAllApis(
-      'f4.log10(),
-      "f4.log10()",
-      "LOG10(f4)",
-      math.log10(44.toLong).toString)
-
-    testAllApis(
-      'f5.log10(),
-      "f5.log10()",
-      "LOG10(f5)",
-      math.log10(4.5.toFloat).toString)
-
-    testAllApis(
-      'f6.log10(),
-      "f6.log10()",
-      "LOG10(f6)",
-      math.log10(4.6).toString)
-  }
-
-  @Test
-  def testPower(): Unit = {
-    // f7: int , f4: long, f6: double
-    testAllApis(
-      'f2.power('f7),
-      "f2.power(f7)",
-      "POWER(f2, f7)",
-      math.pow(42.toByte, 3).toString)
-
-    testAllApis(
-      'f3.power('f6),
-      "f3.power(f6)",
-      "POWER(f3, f6)",
-      math.pow(43.toShort, 4.6D).toString)
-
-    testAllApis(
-      'f4.power('f5),
-      "f4.power(f5)",
-      "POWER(f4, f5)",
-      math.pow(44.toLong, 4.5.toFloat).toString)
-
-    testAllApis(
-      'f4.power('f5),
-      "f4.power(f5)",
-      "POWER(f4, f5)",
-      math.pow(44.toLong, 4.5.toFloat).toString)
-
-    // f5: float
-    testAllApis('f5.power('f5),
-      "f5.power(f5)",
-      "power(f5, f5)",
-      math.pow(4.5F, 4.5F).toString)
-
-    testAllApis('f5.power('f6),
-      "f5.power(f6)",
-      "power(f5, f6)",
-      math.pow(4.5F, 4.6D).toString)
-
-    testAllApis('f5.power('f7),
-      "f5.power(f7)",
-      "power(f5, f7)",
-      math.pow(4.5F, 3).toString)
-
-    testAllApis('f5.power('f4),
-      "f5.power(f4)",
-      "power(f5, f4)",
-      math.pow(4.5F, 44L).toString)
-
-    // f22: bigDecimal
-    // TODO delete casting in SQL when CALCITE-1467 is fixed
-    testAllApis(
-      'f22.cast(Types.DOUBLE).power('f5),
-      "f22.cast(DOUBLE).power(f5)",
-      "power(CAST(f22 AS DOUBLE), f5)",
-      math.pow(2, 4.5F).toString)
-
-    testAllApis(
-      'f22.cast(Types.DOUBLE).power('f6),
-      "f22.cast(DOUBLE).power(f6)",
-      "power(CAST(f22 AS DOUBLE), f6)",
-      math.pow(2, 4.6D).toString)
-
-    testAllApis(
-      'f22.cast(Types.DOUBLE).power('f7),
-      "f22.cast(DOUBLE).power(f7)",
-      "power(CAST(f22 AS DOUBLE), f7)",
-      math.pow(2, 3).toString)
-
-    testAllApis(
-      'f22.cast(Types.DOUBLE).power('f4),
-      "f22.cast(DOUBLE).power(f4)",
-      "power(CAST(f22 AS DOUBLE), f4)",
-      math.pow(2, 44L).toString)
-
-    testAllApis(
-      'f6.power('f22.cast(Types.DOUBLE)),
-      "f6.power(f22.cast(DOUBLE))",
-      "power(f6, f22)",
-      math.pow(4.6D, 2).toString)
-  }
-
-  @Test
-  def testSqrt(): Unit = {
-    testAllApis(
-      'f6.sqrt(),
-      "f6.sqrt",
-      "SQRT(f6)",
-      math.sqrt(4.6D).toString)
-
-    testAllApis(
-      'f7.sqrt(),
-      "f7.sqrt",
-      "SQRT(f7)",
-      math.sqrt(3).toString)
-
-    testAllApis(
-      'f4.sqrt(),
-      "f4.sqrt",
-      "SQRT(f4)",
-      math.sqrt(44L).toString)
-
-    testAllApis(
-      'f22.cast(Types.DOUBLE).sqrt(),
-      "f22.cast(DOUBLE).sqrt",
-      "SQRT(CAST(f22 AS DOUBLE))",
-      math.sqrt(2.0).toString)
-
-    testAllApis(
-      'f5.sqrt(),
-      "f5.sqrt",
-      "SQRT(f5)",
-      math.pow(4.5F, 0.5).toString)
-
-    testAllApis(
-      25.sqrt(),
-      "25.sqrt()",
-      "SQRT(25)",
-      "5.0")
-
-    testAllApis(
-      2.2.sqrt(),
-      "2.2.sqrt()",
-      "POWER(CAST(2.2 AS DOUBLE), CAST(0.5 AS DOUBLE))", // TODO fix FLINK-4621
-      math.sqrt(2.2).toString)
-  }
-
-  @Test
-  def testLn(): Unit = {
-    testAllApis(
-      'f2.ln(),
-      "f2.ln()",
-      "LN(f2)",
-      math.log(42.toByte).toString)
-
-    testAllApis(
-      'f3.ln(),
-      "f3.ln()",
-      "LN(f3)",
-      math.log(43.toShort).toString)
-
-    testAllApis(
-      'f4.ln(),
-      "f4.ln()",
-      "LN(f4)",
-      math.log(44.toLong).toString)
-
-    testAllApis(
-      'f5.ln(),
-      "f5.ln()",
-      "LN(f5)",
-      math.log(4.5.toFloat).toString)
-
-    testAllApis(
-      'f6.ln(),
-      "f6.ln()",
-      "LN(f6)",
-      math.log(4.6).toString)
-  }
-
-  @Test
-  def testAbs(): Unit = {
-    testAllApis(
-      'f2.abs(),
-      "f2.abs()",
-      "ABS(f2)",
-      "42")
-
-    testAllApis(
-      'f3.abs(),
-      "f3.abs()",
-      "ABS(f3)",
-      "43")
-
-    testAllApis(
-      'f4.abs(),
-      "f4.abs()",
-      "ABS(f4)",
-      "44")
-
-    testAllApis(
-      'f5.abs(),
-      "f5.abs()",
-      "ABS(f5)",
-      "4.5")
-
-    testAllApis(
-      'f6.abs(),
-      "f6.abs()",
-      "ABS(f6)",
-      "4.6")
-
-    testAllApis(
-      'f9.abs(),
-      "f9.abs()",
-      "ABS(f9)",
-      "42")
-
-    testAllApis(
-      'f10.abs(),
-      "f10.abs()",
-      "ABS(f10)",
-      "43")
-
-    testAllApis(
-      'f11.abs(),
-      "f11.abs()",
-      "ABS(f11)",
-      "44")
-
-    testAllApis(
-      'f12.abs(),
-      "f12.abs()",
-      "ABS(f12)",
-      "4.5")
-
-    testAllApis(
-      'f13.abs(),
-      "f13.abs()",
-      "ABS(f13)",
-      "4.6")
-
-    testAllApis(
-      'f15.abs(),
-      "f15.abs()",
-      "ABS(f15)",
-      "1231.1231231321321321111")
-  }
-
-  @Test
-  def testArithmeticFloorCeil(): Unit = {
-    testAllApis(
-      'f5.floor(),
-      "f5.floor()",
-      "FLOOR(f5)",
-      "4.0")
-
-    testAllApis(
-     'f5.ceil(),
-      "f5.ceil()",
-      "CEIL(f5)",
-      "5.0")
-
-    testAllApis(
-      'f3.floor(),
-      "f3.floor()",
-      "FLOOR(f3)",
-      "43")
-
-    testAllApis(
-      'f3.ceil(),
-      "f3.ceil()",
-      "CEIL(f3)",
-      "43")
-
-    testAllApis(
-      'f15.floor(),
-      "f15.floor()",
-      "FLOOR(f15)",
-      "-1232")
-
-    testAllApis(
-      'f15.ceil(),
-      "f15.ceil()",
-      "CEIL(f15)",
-      "-1231")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-  // Temporal functions
-  // ----------------------------------------------------------------------------------------------
-
-  @Test
-  def testExtract(): Unit = {
-    testAllApis(
-      'f16.extract(TimeIntervalUnit.YEAR),
-      "f16.extract(YEAR)",
-      "EXTRACT(YEAR FROM f16)",
-      "1996")
-
-    testAllApis(
-      'f16.extract(TimeIntervalUnit.MONTH),
-      "extract(f16, MONTH)",
-      "EXTRACT(MONTH FROM f16)",
-      "11")
-
-    testAllApis(
-      'f16.extract(TimeIntervalUnit.DAY),
-      "f16.extract(DAY)",
-      "EXTRACT(DAY FROM f16)",
-      "10")
-
-    testAllApis(
-      'f18.extract(TimeIntervalUnit.YEAR),
-      "f18.extract(YEAR)",
-      "EXTRACT(YEAR FROM f18)",
-      "1996")
-
-    testAllApis(
-      'f18.extract(TimeIntervalUnit.MONTH),
-      "f18.extract(MONTH)",
-      "EXTRACT(MONTH FROM f18)",
-      "11")
-
-    testAllApis(
-      'f18.extract(TimeIntervalUnit.DAY),
-      "f18.extract(DAY)",
-      "EXTRACT(DAY FROM f18)",
-      "10")
-
-    testAllApis(
-      'f18.extract(TimeIntervalUnit.HOUR),
-      "f18.extract(HOUR)",
-      "EXTRACT(HOUR FROM f18)",
-      "6")
-
-    testAllApis(
-      'f17.extract(TimeIntervalUnit.HOUR),
-      "f17.extract(HOUR)",
-      "EXTRACT(HOUR FROM f17)",
-      "6")
-
-    testAllApis(
-      'f18.extract(TimeIntervalUnit.MINUTE),
-      "f18.extract(MINUTE)",
-      "EXTRACT(MINUTE FROM f18)",
-      "55")
-
-    testAllApis(
-      'f17.extract(TimeIntervalUnit.MINUTE),
-      "f17.extract(MINUTE)",
-      "EXTRACT(MINUTE FROM f17)",
-      "55")
-
-    testAllApis(
-      'f18.extract(TimeIntervalUnit.SECOND),
-      "f18.extract(SECOND)",
-      "EXTRACT(SECOND FROM f18)",
-      "44")
-
-    testAllApis(
-      'f17.extract(TimeIntervalUnit.SECOND),
-      "f17.extract(SECOND)",
-      "EXTRACT(SECOND FROM f17)",
-      "44")
-
-    testAllApis(
-      'f19.extract(TimeIntervalUnit.DAY),
-      "f19.extract(DAY)",
-      "EXTRACT(DAY FROM f19)",
-      "16979")
-
-    testAllApis(
-      'f19.extract(TimeIntervalUnit.HOUR),
-      "f19.extract(HOUR)",
-      "EXTRACT(HOUR FROM f19)",
-      "7")
-
-    testAllApis(
-      'f19.extract(TimeIntervalUnit.MINUTE),
-      "f19.extract(MINUTE)",
-      "EXTRACT(MINUTE FROM f19)",
-      "23")
-
-    testAllApis(
-      'f19.extract(TimeIntervalUnit.SECOND),
-      "f19.extract(SECOND)",
-      "EXTRACT(SECOND FROM f19)",
-      "33")
-
-    testAllApis(
-      'f20.extract(TimeIntervalUnit.MONTH),
-      "f20.extract(MONTH)",
-      "EXTRACT(MONTH FROM f20)",
-      "1")
-
-    testAllApis(
-      'f20.extract(TimeIntervalUnit.YEAR),
-      "f20.extract(YEAR)",
-      "EXTRACT(YEAR FROM f20)",
-      "2")
-  }
-
-  @Test
-  def testTemporalFloor(): Unit = {
-    testAllApis(
-      'f18.floor(TimeIntervalUnit.YEAR),
-      "f18.floor(YEAR)",
-      "FLOOR(f18 TO YEAR)",
-      "1996-01-01 00:00:00.0")
-
-    testAllApis(
-      'f18.floor(TimeIntervalUnit.MONTH),
-      "f18.floor(MONTH)",
-      "FLOOR(f18 TO MONTH)",
-      "1996-11-01 00:00:00.0")
-
-    testAllApis(
-      'f18.floor(TimeIntervalUnit.DAY),
-      "f18.floor(DAY)",
-      "FLOOR(f18 TO DAY)",
-      "1996-11-10 00:00:00.0")
-
-    testAllApis(
-      'f18.floor(TimeIntervalUnit.MINUTE),
-      "f18.floor(MINUTE)",
-      "FLOOR(f18 TO MINUTE)",
-      "1996-11-10 06:55:00.0")
-
-    testAllApis(
-      'f18.floor(TimeIntervalUnit.SECOND),
-      "f18.floor(SECOND)",
-      "FLOOR(f18 TO SECOND)",
-      "1996-11-10 06:55:44.0")
-
-    testAllApis(
-      'f17.floor(TimeIntervalUnit.HOUR),
-      "f17.floor(HOUR)",
-      "FLOOR(f17 TO HOUR)",
-      "06:00:00")
-
-    testAllApis(
-      'f17.floor(TimeIntervalUnit.MINUTE),
-      "f17.floor(MINUTE)",
-      "FLOOR(f17 TO MINUTE)",
-      "06:55:00")
-
-    testAllApis(
-      'f17.floor(TimeIntervalUnit.SECOND),
-      "f17.floor(SECOND)",
-      "FLOOR(f17 TO SECOND)",
-      "06:55:44")
-
-    testAllApis(
-      'f16.floor(TimeIntervalUnit.YEAR),
-      "f16.floor(YEAR)",
-      "FLOOR(f16 TO YEAR)",
-      "1996-01-01")
-
-    testAllApis(
-      'f16.floor(TimeIntervalUnit.MONTH),
-      "f16.floor(MONTH)",
-      "FLOOR(f16 TO MONTH)",
-      "1996-11-01")
-
-    testAllApis(
-      'f18.ceil(TimeIntervalUnit.YEAR),
-      "f18.ceil(YEAR)",
-      "CEIL(f18 TO YEAR)",
-      "1997-01-01 00:00:00.0")
-
-    testAllApis(
-      'f18.ceil(TimeIntervalUnit.MONTH),
-      "f18.ceil(MONTH)",
-      "CEIL(f18 TO MONTH)",
-      "1996-12-01 00:00:00.0")
-
-    testAllApis(
-      'f18.ceil(TimeIntervalUnit.DAY),
-      "f18.ceil(DAY)",
-      "CEIL(f18 TO DAY)",
-      "1996-11-11 00:00:00.0")
-
-    testAllApis(
-      'f18.ceil(TimeIntervalUnit.MINUTE),
-      "f18.ceil(MINUTE)",
-      "CEIL(f18 TO MINUTE)",
-      "1996-11-10 06:56:00.0")
-
-    testAllApis(
-      'f18.ceil(TimeIntervalUnit.SECOND),
-      "f18.ceil(SECOND)",
-      "CEIL(f18 TO SECOND)",
-      "1996-11-10 06:55:45.0")
-
-    testAllApis(
-      'f17.ceil(TimeIntervalUnit.HOUR),
-      "f17.ceil(HOUR)",
-      "CEIL(f17 TO HOUR)",
-      "07:00:00")
-
-    testAllApis(
-      'f17.ceil(TimeIntervalUnit.MINUTE),
-      "f17.ceil(MINUTE)",
-      "CEIL(f17 TO MINUTE)",
-      "06:56:00")
-
-    testAllApis(
-      'f17.ceil(TimeIntervalUnit.SECOND),
-      "f17.ceil(SECOND)",
-      "CEIL(f17 TO SECOND)",
-      "06:55:44")
-
-    testAllApis(
-      'f16.ceil(TimeIntervalUnit.YEAR),
-      "f16.ceil(YEAR)",
-      "CEIL(f16 TO YEAR)",
-      "1996-01-01")
-
-    testAllApis(
-      'f16.ceil(TimeIntervalUnit.MONTH),
-      "f16.ceil(MONTH)",
-      "CEIL(f16 TO MONTH)",
-      "1996-11-01")
-  }
-
-  @Test
-  def testCurrentTimePoint(): Unit = {
-
-    // current time points are non-deterministic
-    // we just test the format of the output
-    // manual test can be found in NonDeterministicTests
-
-    testAllApis(
-      currentDate().cast(Types.STRING).charLength() >= 5,
-      "currentDate().cast(STRING).charLength() >= 5",
-      "CHAR_LENGTH(CAST(CURRENT_DATE AS VARCHAR)) >= 5",
-      "true")
-
-    testAllApis(
-      currentTime().cast(Types.STRING).charLength() >= 5,
-      "currentTime().cast(STRING).charLength() >= 5",
-      "CHAR_LENGTH(CAST(CURRENT_TIME AS VARCHAR)) >= 5",
-      "true")
-
-    testAllApis(
-      currentTimestamp().cast(Types.STRING).charLength() >= 12,
-      "currentTimestamp().cast(STRING).charLength() >= 12",
-      "CHAR_LENGTH(CAST(CURRENT_TIMESTAMP AS VARCHAR)) >= 12",
-      "true")
-
-    testAllApis(
-      localTimestamp().cast(Types.STRING).charLength() >= 12,
-      "localTimestamp().cast(STRING).charLength() >= 12",
-      "CHAR_LENGTH(CAST(LOCALTIMESTAMP AS VARCHAR)) >= 12",
-      "true")
-
-    testAllApis(
-      localTime().cast(Types.STRING).charLength() >= 5,
-      "localTime().cast(STRING).charLength() >= 5",
-      "CHAR_LENGTH(CAST(LOCALTIME AS VARCHAR)) >= 5",
-      "true")
-
-    // comparisons are deterministic
-    testAllApis(
-      localTimestamp() === localTimestamp(),
-      "localTimestamp() === localTimestamp()",
-      "LOCALTIMESTAMP = LOCALTIMESTAMP",
-      "true")
-  }
-
-  @Test
-  def testOverlaps(): Unit = {
-    testAllApis(
-      temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hours),
-      "temporalOverlaps('2:55:00'.toTime, 1.hour, '3:30:00'.toTime, 2.hours)",
-      "(TIME '2:55:00', INTERVAL '1' HOUR) OVERLAPS (TIME '3:30:00', INTERVAL '2' HOUR)",
-      "true")
-
-    testAllApis(
-      temporalOverlaps("9:00:00".toTime, "9:30:00".toTime, "9:29:00".toTime, "9:31:00".toTime),
-      "temporalOverlaps('9:00:00'.toTime, '9:30:00'.toTime, '9:29:00'.toTime, '9:31:00'.toTime)",
-      "(TIME '9:00:00', TIME '9:30:00') OVERLAPS (TIME '9:29:00', TIME '9:31:00')",
-      "true")
-
-    testAllApis(
-      temporalOverlaps("9:00:00".toTime, "10:00:00".toTime, "10:15:00".toTime, 3.hours),
-      "temporalOverlaps('9:00:00'.toTime, '10:00:00'.toTime, '10:15:00'.toTime, 3.hours)",
-      "(TIME '9:00:00', TIME '10:00:00') OVERLAPS (TIME '10:15:00', INTERVAL '3' HOUR)",
-      "false")
-
-    testAllApis(
-      temporalOverlaps("2011-03-10".toDate, 10.days, "2011-03-19".toDate, 10.days),
-      "temporalOverlaps('2011-03-10'.toDate, 10.days, '2011-03-19'.toDate, 10.days)",
-      "(DATE '2011-03-10', INTERVAL '10' DAY) OVERLAPS (DATE '2011-03-19', INTERVAL '10' DAY)",
-      "true")
-
-    testAllApis(
-      temporalOverlaps("2011-03-10 05:02:02".toTimestamp, 0.milli,
-        "2011-03-10 05:02:02".toTimestamp, "2011-03-10 05:02:01".toTimestamp),
-      "temporalOverlaps('2011-03-10 05:02:02'.toTimestamp, 0.milli, " +
-        "'2011-03-10 05:02:02'.toTimestamp, '2011-03-10 05:02:01'.toTimestamp)",
-      "(TIMESTAMP '2011-03-10 05:02:02', INTERVAL '0' SECOND) OVERLAPS " +
-        "(TIMESTAMP '2011-03-10 05:02:02', TIMESTAMP '2011-03-10 05:02:01')",
-      "false")
-
-    // TODO enable once CALCITE-1435 is fixed
-    // comparison of timestamps based on milliseconds is buggy
-    //testAllApis(
-    //  temporalOverlaps("2011-03-10 02:02:02.001".toTimestamp, 0.milli,
-    //    "2011-03-10 02:02:02.002".toTimestamp, "2011-03-10 02:02:02.002".toTimestamp),
-    //  "temporalOverlaps('2011-03-10 02:02:02.001'.toTimestamp, 0.milli, " +
-    //    "'2011-03-10 02:02:02.002'.toTimestamp, '2011-03-10 02:02:02.002'.toTimestamp)",
-    //  "(TIMESTAMP '2011-03-10 02:02:02.001', INTERVAL '0' SECOND) OVERLAPS " +
-    //    "(TIMESTAMP '2011-03-10 02:02:02.002', TIMESTAMP '2011-03-10 02:02:02.002')",
-    //  "false")
-  }
-
-  @Test
-  def testQuarter(): Unit = {
-    testAllApis(
-      "1997-01-27".toDate.quarter(),
-      "'1997-01-27'.toDate.quarter()",
-      "QUARTER(DATE '1997-01-27')",
-      "1")
-
-    testAllApis(
-      "1997-04-27".toDate.quarter(),
-      "'1997-04-27'.toDate.quarter()",
-      "QUARTER(DATE '1997-04-27')",
-      "2")
-
-    testAllApis(
-      "1997-12-31".toDate.quarter(),
-      "'1997-12-31'.toDate.quarter()",
-      "QUARTER(DATE '1997-12-31')",
-      "4")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-  // Other functions
-  // ----------------------------------------------------------------------------------------------
-
-  @Test
-  def testIsTrueIsFalse(): Unit = {
-    testAllApis(
-      'f1.isTrue,
-      "f1.isTrue",
-      "f1 IS TRUE",
-      "true")
-
-    testAllApis(
-      'f21.isTrue,
-      "f21.isTrue",
-      "f21 IS TRUE",
-      "false")
-
-    testAllApis(
-      false.isFalse,
-      "false.isFalse",
-      "FALSE IS FALSE",
-      "true")
-
-    testAllApis(
-      'f21.isFalse,
-      "f21.isFalse",
-      "f21 IS FALSE",
-      "false")
-
-    testAllApis(
-      'f1.isNotTrue,
-      "f1.isNotTrue",
-      "f1 IS NOT TRUE",
-      "false")
-
-    testAllApis(
-      'f21.isNotTrue,
-      "f21.isNotTrue",
-      "f21 IS NOT TRUE",
-      "true")
-
-    testAllApis(
-      false.isNotFalse,
-      "false.isNotFalse",
-      "FALSE IS NOT FALSE",
-      "false")
-
-    testAllApis(
-      'f21.isNotFalse,
-      "f21.isNotFalse",
-      "f21 IS NOT FALSE",
-      "true")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  def testData = {
-    val testData = new Row(25)
-    testData.setField(0, "This is a test String.")
-    testData.setField(1, true)
-    testData.setField(2, 42.toByte)
-    testData.setField(3, 43.toShort)
-    testData.setField(4, 44.toLong)
-    testData.setField(5, 4.5.toFloat)
-    testData.setField(6, 4.6)
-    testData.setField(7, 3)
-    testData.setField(8, " This is a test String. ")
-    testData.setField(9, -42.toByte)
-    testData.setField(10, -43.toShort)
-    testData.setField(11, -44.toLong)
-    testData.setField(12, -4.5.toFloat)
-    testData.setField(13, -4.6)
-    testData.setField(14, -3)
-    testData.setField(15, BigDecimal("-1231.1231231321321321111").bigDecimal)
-    testData.setField(16, Date.valueOf("1996-11-10"))
-    testData.setField(17, Time.valueOf("06:55:44"))
-    testData.setField(18, Timestamp.valueOf("1996-11-10 06:55:44.333"))
-    testData.setField(19, 1467012213000L) // +16979 07:23:33.000
-    testData.setField(20, 25) // +2-01
-    testData.setField(21, null)
-    testData.setField(22, BigDecimal("2").bigDecimal)
-    testData.setField(23, "%This is a test String.")
-    testData.setField(24, "*_This is a test String.")
-    testData
-  }
-
-  def typeInfo = {
-    new RowTypeInfo(
-      Types.STRING,
-      Types.BOOLEAN,
-      Types.BYTE,
-      Types.SHORT,
-      Types.LONG,
-      Types.FLOAT,
-      Types.DOUBLE,
-      Types.INT,
-      Types.STRING,
-      Types.BYTE,
-      Types.SHORT,
-      Types.LONG,
-      Types.FLOAT,
-      Types.DOUBLE,
-      Types.INT,
-      Types.DECIMAL,
-      Types.DATE,
-      Types.TIME,
-      Types.TIMESTAMP,
-      Types.INTERVAL_MILLIS,
-      Types.INTERVAL_MONTHS,
-      Types.BOOLEAN,
-      Types.DECIMAL,
-      Types.STRING,
-      Types.STRING).asInstanceOf[TypeInformation[Any]]
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarOperatorsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarOperatorsTest.scala
deleted file mode 100644
index 86f884f..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/ScalarOperatorsTest.scala
+++ /dev/null
@@ -1,220 +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.api.table.expressions
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.expressions.utils.ExpressionTestBase
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.{Types, ValidationException}
-import org.apache.flink.types.Row
-import org.junit.Test
-
-class ScalarOperatorsTest extends ExpressionTestBase {
-
-  @Test
-  def testCasting(): Unit = {
-    // test casting
-    // * -> String
-    testTableApi('f2.cast(Types.STRING), "f2.cast(STRING)", "1")
-    testTableApi('f5.cast(Types.STRING), "f5.cast(STRING)", "1.0")
-    testTableApi('f3.cast(Types.STRING), "f3.cast(STRING)", "1")
-    testTableApi('f6.cast(Types.STRING), "f6.cast(STRING)", "true")
-    // NUMERIC TYPE -> Boolean
-    testTableApi('f2.cast(Types.BOOLEAN), "f2.cast(BOOLEAN)", "true")
-    testTableApi('f7.cast(Types.BOOLEAN), "f7.cast(BOOLEAN)", "false")
-    testTableApi('f3.cast(Types.BOOLEAN), "f3.cast(BOOLEAN)", "true")
-    // NUMERIC TYPE -> NUMERIC TYPE
-    testTableApi('f2.cast(Types.DOUBLE), "f2.cast(DOUBLE)", "1.0")
-    testTableApi('f7.cast(Types.INT), "f7.cast(INT)", "0")
-    testTableApi('f3.cast(Types.SHORT), "f3.cast(SHORT)", "1")
-    // Boolean -> NUMERIC TYPE
-    testTableApi('f6.cast(Types.DOUBLE), "f6.cast(DOUBLE)", "1.0")
-    // identity casting
-    testTableApi('f2.cast(Types.INT), "f2.cast(INT)", "1")
-    testTableApi('f7.cast(Types.DOUBLE), "f7.cast(DOUBLE)", "0.0")
-    testTableApi('f3.cast(Types.LONG), "f3.cast(LONG)", "1")
-    testTableApi('f6.cast(Types.BOOLEAN), "f6.cast(BOOLEAN)", "true")
-    // String -> BASIC TYPE (not String, Date, Void, Character)
-    testTableApi('f2.cast(Types.BYTE), "f2.cast(BYTE)", "1")
-    testTableApi('f2.cast(Types.SHORT), "f2.cast(SHORT)", "1")
-    testTableApi('f2.cast(Types.INT), "f2.cast(INT)", "1")
-    testTableApi('f2.cast(Types.LONG), "f2.cast(LONG)", "1")
-    testTableApi('f3.cast(Types.DOUBLE), "f3.cast(DOUBLE)", "1.0")
-    testTableApi('f3.cast(Types.FLOAT), "f3.cast(FLOAT)", "1.0")
-    testTableApi('f5.cast(Types.BOOLEAN), "f5.cast(BOOLEAN)", "true")
-
-    // numeric auto cast in arithmetic
-    testTableApi('f0 + 1, "f0 + 1", "2")
-    testTableApi('f1 + 1, "f1 + 1", "2")
-    testTableApi('f2 + 1L, "f2 + 1L", "2")
-    testTableApi('f3 + 1.0f, "f3 + 1.0f", "2.0")
-    testTableApi('f3 + 1.0d, "f3 + 1.0d", "2.0")
-    testTableApi('f5 + 1, "f5 + 1", "2.0")
-    testTableApi('f3 + 1.0d, "f3 + 1.0d", "2.0")
-    testTableApi('f4 + 'f0, "f4 + f0", "2.0")
-
-    // numeric auto cast in comparison
-    testTableApi(
-      'f0 > 0 && 'f1 > 0 && 'f2 > 0L && 'f4 > 0.0f && 'f5 > 0.0d  && 'f3 > 0,
-      "f0 > 0 && f1 > 0 && f2 > 0L && f4 > 0.0f && f5 > 0.0d  && f3 > 0",
-      "true")
-  }
-
-  @Test
-  def testArithmetic(): Unit = {
-    // math arthmetic
-    testTableApi('f8 - 5, "f8 - 5", "0")
-    testTableApi('f8 + 5, "f8 + 5", "10")
-    testTableApi('f8 / 2, "f8 / 2", "2")
-    testTableApi('f8 * 2, "f8 * 2", "10")
-    testTableApi('f8 % 2, "f8 % 2", "1")
-    testTableApi(-'f8, "-f8", "-5")
-    testTableApi(3.toExpr + 'f8, "3 + f8", "8")
-
-    // boolean arithmetic
-    testTableApi('f6 && true, "f6 && true", "true")
-    testTableApi('f6 && false, "f6 && false", "false")
-    testTableApi('f6 || false, "f6 || false", "true")
-    testTableApi(!'f6, "!f6", "false")
-
-    // comparison
-    testTableApi('f8 > 'f2, "f8 > f2", "true")
-    testTableApi('f8 >= 'f8, "f8 >= f8", "true")
-    testTableApi('f8 < 'f2, "f8 < f2", "false")
-    testTableApi('f8.isNull, "f8.isNull", "false")
-    testTableApi('f8.isNotNull, "f8.isNotNull", "true")
-    testTableApi(12.toExpr <= 'f8, "12 <= f8", "false")
-
-    // string arithmetic
-    testTableApi(42.toExpr + 'f10 + 'f9, "42 + f10 + f9", "42String10")
-    testTableApi('f10 + 'f9, "f10 + f9", "String10")
-  }
-
-  @Test
-  def testOtherExpressions(): Unit = {
-    // null
-    testAllApis(Null(Types.INT), "Null(INT)", "CAST(NULL AS INT)", "null")
-    testAllApis(
-      Null(Types.STRING) === "",
-      "Null(STRING) === ''",
-      "CAST(NULL AS VARCHAR) = ''",
-      "null")
-
-    // if
-    testTableApi(('f6 && true).?("true", "false"), "(f6 && true).?('true', 'false')", "true")
-    testTableApi(false.?("true", "false"), "false.?('true', 'false')", "false")
-    testTableApi(
-      true.?(true.?(true.?(10, 4), 4), 4),
-      "true.?(true.?(true.?(10, 4), 4), 4)",
-      "10")
-    testTableApi(true, "?((f6 && true), 'true', 'false')", "true")
-    testSqlApi("CASE 11 WHEN 1 THEN 'a' ELSE 'b' END", "b")
-    testSqlApi("CASE 2 WHEN 1 THEN 'a' ELSE 'b' END", "b")
-    testSqlApi(
-      "CASE 1 WHEN 1, 2 THEN '1 or 2' WHEN 2 THEN 'not possible' WHEN 3, 2 THEN '3' " +
-      "ELSE 'none of the above' END",
-      "1 or 2           ")
-    testSqlApi("CASE WHEN 'a'='a' THEN 1 END", "1")
-    testSqlApi("CASE 2 WHEN 1 THEN 'a' WHEN 2 THEN 'bcd' END", "bcd")
-    testSqlApi("CASE f2 WHEN 1 THEN 11 WHEN 2 THEN 4 ELSE NULL END", "11")
-    testSqlApi("CASE f7 WHEN 1 THEN 11 WHEN 2 THEN 4 ELSE NULL END", "null")
-    testSqlApi("CASE 42 WHEN 1 THEN 'a' WHEN 2 THEN 'bcd' END", "null")
-    testSqlApi("CASE 1 WHEN 1 THEN true WHEN 2 THEN false ELSE NULL END", "true")
-
-    // case insensitive as
-    testTableApi(5 as 'test, "5 As test", "5")
-
-    // complex expressions
-    testTableApi('f0.isNull.isNull, "f0.isNull().isNull", "false")
-    testTableApi(
-      'f8.abs() + 'f8.abs().abs().abs().abs(),
-      "f8.abs() + f8.abs().abs().abs().abs()",
-      "10")
-    testTableApi(
-      'f8.cast(Types.STRING) + 'f8.cast(Types.STRING),
-      "f8.cast(STRING) + f8.cast(STRING)",
-      "55")
-    testTableApi('f8.isNull.cast(Types.INT), "CAST(ISNULL(f8), INT)", "0")
-    testTableApi(
-      'f8.cast(Types.INT).abs().isNull === false,
-      "ISNULL(CAST(f8, INT).abs()) === false",
-      "true")
-    testTableApi(
-      (((true === true) || false).cast(Types.STRING) + "X ").trim(),
-      "((((true) === true) || false).cast(STRING) + 'X ').trim",
-      "trueX")
-    testTableApi(12.isNull, "12.isNull", "false")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testIfInvalidTypesScala(): Unit = {
-    testTableApi(('f6 && true).?(5, "false"), "FAIL", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testIfInvalidTypesJava(): Unit = {
-    testTableApi("FAIL", "(f8 && true).?(5, 'false')", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidStringComparison1(): Unit = {
-    testTableApi("w" === 4, "FAIL", "FAIL")
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidStringComparison2(): Unit = {
-    testTableApi("w" > 4.toExpr, "FAIL", "FAIL")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  def testData = {
-    val testData = new Row(11)
-    testData.setField(0, 1: Byte)
-    testData.setField(1, 1: Short)
-    testData.setField(2, 1)
-    testData.setField(3, 1L)
-    testData.setField(4, 1.0f)
-    testData.setField(5, 1.0d)
-    testData.setField(6, true)
-    testData.setField(7, 0.0d)
-    testData.setField(8, 5)
-    testData.setField(9, 10)
-    testData.setField(10, "String")
-    testData
-  }
-
-  def typeInfo = {
-    new RowTypeInfo(
-      Types.BYTE,
-      Types.SHORT,
-      Types.INT,
-      Types.LONG,
-      Types.FLOAT,
-      Types.DOUBLE,
-      Types.BOOLEAN,
-      Types.DOUBLE,
-      Types.INT,
-      Types.INT,
-      Types.STRING
-      ).asInstanceOf[TypeInformation[Any]]
-  }
-
-}


[24/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/MethodCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/MethodCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/MethodCallGen.scala
new file mode 100644
index 0000000..2e0d340
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/MethodCallGen.scala
@@ -0,0 +1,44 @@
+/*
+ * 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.codegen.calls
+
+import java.lang.reflect.Method
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.codegen.CodeGenUtils.qualifyMethod
+import org.apache.flink.table.codegen.calls.CallGenerator.generateCallIfArgsNotNull
+import org.apache.flink.table.codegen.{CodeGenerator, GeneratedExpression}
+
+/**
+  * Generates a function call by using a [[java.lang.reflect.Method]].
+  */
+class MethodCallGen(returnType: TypeInformation[_], method: Method) extends CallGenerator {
+
+  override def generate(
+      codeGenerator: CodeGenerator,
+      operands: Seq[GeneratedExpression])
+    : GeneratedExpression = {
+    generateCallIfArgsNotNull(codeGenerator.nullCheck, returnType, operands) {
+      (terms) =>
+        s"""
+          |${qualifyMethod(method)}(${terms.mkString(", ")})
+          |""".stripMargin
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/MultiTypeMethodCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/MultiTypeMethodCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/MultiTypeMethodCallGen.scala
new file mode 100644
index 0000000..e5958a0
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/MultiTypeMethodCallGen.scala
@@ -0,0 +1,44 @@
+/*
+ * 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.codegen.calls
+
+import java.lang.reflect.Method
+
+import org.apache.flink.table.codegen.calls.CallGenerator._
+import org.apache.flink.table.codegen.{CodeGenerator, GeneratedExpression}
+
+/**
+  * Generates a function call that calls a method which returns the same type that it
+  * takes as first argument.
+  */
+class MultiTypeMethodCallGen(method: Method) extends CallGenerator {
+
+  override def generate(
+      codeGenerator: CodeGenerator,
+      operands: Seq[GeneratedExpression])
+    : GeneratedExpression = {
+    generateCallIfArgsNotNull(codeGenerator.nullCheck, operands.head.resultType, operands) {
+      (operandResultTerms) =>
+        s"""
+          |${method.getDeclaringClass.getCanonicalName}.
+          |  ${method.getName}(${operandResultTerms.mkString(", ")})
+         """.stripMargin
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/NotCallGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/NotCallGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/NotCallGenerator.scala
new file mode 100644
index 0000000..52397c9
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/NotCallGenerator.scala
@@ -0,0 +1,37 @@
+/*
+ * 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.codegen.calls
+
+import org.apache.flink.table.codegen.calls.ScalarOperators.generateNot
+import org.apache.flink.table.codegen.{GeneratedExpression, CodeGenerator}
+
+/**
+  * Inverts the boolean value of a CallGenerator result.
+  */
+class NotCallGenerator(callGenerator: CallGenerator) extends CallGenerator {
+
+  override def generate(
+      codeGenerator: CodeGenerator,
+      operands: Seq[GeneratedExpression])
+    : GeneratedExpression = {
+    val expr = callGenerator.generate(codeGenerator, operands)
+    generateNot(codeGenerator.nullCheck, expr)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarFunctionCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarFunctionCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarFunctionCallGen.scala
new file mode 100644
index 0000000..ac840df
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarFunctionCallGen.scala
@@ -0,0 +1,95 @@
+/*
+ * 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.codegen.calls
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.codegen.CodeGenUtils._
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, GeneratedExpression}
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
+
+/**
+  * Generates a call to user-defined [[ScalarFunction]].
+  *
+  * @param scalarFunction user-defined [[ScalarFunction]] that might be overloaded
+  * @param signature actual signature with which the function is called
+  * @param returnType actual return type required by the surrounding
+  */
+class ScalarFunctionCallGen(
+    scalarFunction: ScalarFunction,
+    signature: Seq[TypeInformation[_]],
+    returnType: TypeInformation[_])
+  extends CallGenerator {
+
+  override def generate(
+      codeGenerator: CodeGenerator,
+      operands: Seq[GeneratedExpression])
+    : GeneratedExpression = {
+    // determine function signature and result class
+    val matchingSignature = getSignature(scalarFunction, signature)
+      .getOrElse(throw new CodeGenException("No matching signature found."))
+    val resultClass = getResultTypeClass(scalarFunction, matchingSignature)
+
+    // convert parameters for function (output boxing)
+    val parameters = matchingSignature
+        .zip(operands)
+        .map { case (paramClass, operandExpr) =>
+          if (paramClass.isPrimitive) {
+            operandExpr
+          } else {
+            val boxedTypeTerm = boxedTypeTermForTypeInfo(operandExpr.resultType)
+            val boxedExpr = codeGenerator.generateOutputFieldBoxing(operandExpr)
+            val exprOrNull: String = if (codeGenerator.nullCheck) {
+              s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}"
+            } else {
+              boxedExpr.resultTerm
+            }
+            boxedExpr.copy(resultTerm = exprOrNull)
+          }
+        }
+
+    // generate function call
+    val functionReference = codeGenerator.addReusableFunction(scalarFunction)
+    val resultTypeTerm = if (resultClass.isPrimitive) {
+      primitiveTypeTermForTypeInfo(returnType)
+    } else {
+      boxedTypeTermForTypeInfo(returnType)
+    }
+    val resultTerm = newName("result")
+    val functionCallCode =
+      s"""
+        |${parameters.map(_.code).mkString("\n")}
+        |$resultTypeTerm $resultTerm = $functionReference.eval(
+        |  ${parameters.map(_.resultTerm).mkString(", ")});
+        |""".stripMargin
+
+    // convert result of function to internal representation (input unboxing)
+    val resultUnboxing = if (resultClass.isPrimitive) {
+      codeGenerator.generateNonNullLiteral(returnType, resultTerm)
+    } else {
+      codeGenerator.generateInputFieldUnboxing(returnType, resultTerm)
+    }
+    resultUnboxing.copy(code =
+      s"""
+        |$functionCallCode
+        |${resultUnboxing.code}
+        |""".stripMargin
+    )
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala
new file mode 100644
index 0000000..3f7c91f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/ScalarOperators.scala
@@ -0,0 +1,1025 @@
+/*
+ * 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.codegen.calls
+
+import org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY
+import org.apache.calcite.avatica.util.{DateTimeUtils, TimeUnitRange}
+import org.apache.calcite.util.BuiltInMethod
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{NumericTypeInfo, PrimitiveArrayTypeInfo, SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
+import org.apache.flink.table.codegen.CodeGenUtils._
+import org.apache.flink.table.codegen.{CodeGenerator, CodeGenException, GeneratedExpression}
+import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
+import org.apache.flink.table.typeutils.TypeCheckUtils._
+
+object ScalarOperators {
+
+  def generateStringConcatOperator(
+      nullCheck: Boolean,
+      left: GeneratedExpression,
+      right: GeneratedExpression)
+    : GeneratedExpression = {
+    generateOperatorIfNotNull(nullCheck, STRING_TYPE_INFO, left, right) {
+      (leftTerm, rightTerm) => s"$leftTerm + $rightTerm"
+    }
+  }
+
+  def generateArithmeticOperator(
+      operator: String,
+      nullCheck: Boolean,
+      resultType: TypeInformation[_],
+      left: GeneratedExpression,
+      right: GeneratedExpression)
+    : GeneratedExpression = {
+    val leftCasting = numericCasting(left.resultType, resultType)
+    val rightCasting = numericCasting(right.resultType, resultType)
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
+
+    generateOperatorIfNotNull(nullCheck, resultType, left, right) {
+      (leftTerm, rightTerm) =>
+        if (isDecimal(resultType)) {
+          s"${leftCasting(leftTerm)}.${arithOpToDecMethod(operator)}(${rightCasting(rightTerm)})"
+        } else {
+          s"($resultTypeTerm) (${leftCasting(leftTerm)} $operator ${rightCasting(rightTerm)})"
+        }
+    }
+  }
+
+  def generateUnaryArithmeticOperator(
+      operator: String,
+      nullCheck: Boolean,
+      resultType: TypeInformation[_],
+      operand: GeneratedExpression)
+    : GeneratedExpression = {
+    generateUnaryOperatorIfNotNull(nullCheck, resultType, operand) {
+      (operandTerm) =>
+        if (isDecimal(operand.resultType) && operator == "-") {
+          s"$operandTerm.negate()"
+        } else if (isDecimal(operand.resultType) && operator == "+") {
+          s"$operandTerm"
+        } else {
+          s"$operator($operandTerm)"
+        }
+    }
+  }
+
+  def generateEquals(
+      nullCheck: Boolean,
+      left: GeneratedExpression,
+      right: GeneratedExpression)
+    : GeneratedExpression = {
+    // numeric types
+    if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
+      generateComparison("==", nullCheck, left, right)
+    }
+    // temporal types
+    else if (isTemporal(left.resultType) && left.resultType == right.resultType) {
+      generateComparison("==", nullCheck, left, right)
+    }
+    // array types
+    else if (isArray(left.resultType) && left.resultType == right.resultType) {
+      generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
+        (leftTerm, rightTerm) => s"java.util.Arrays.equals($leftTerm, $rightTerm)"
+      }
+    }
+    // comparable types of same type
+    else if (isComparable(left.resultType) && left.resultType == right.resultType) {
+      generateComparison("==", nullCheck, left, right)
+    }
+    // non comparable types
+    else {
+      generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
+        if (isReference(left)) {
+          (leftTerm, rightTerm) => s"$leftTerm.equals($rightTerm)"
+        }
+        else if (isReference(right)) {
+          (leftTerm, rightTerm) => s"$rightTerm.equals($leftTerm)"
+        }
+        else {
+          throw new CodeGenException(s"Incomparable types: ${left.resultType} and " +
+            s"${right.resultType}")
+        }
+      }
+    }
+  }
+
+  def generateNotEquals(
+      nullCheck: Boolean,
+      left: GeneratedExpression,
+      right: GeneratedExpression)
+    : GeneratedExpression = {
+    // numeric types
+    if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
+      generateComparison("!=", nullCheck, left, right)
+    }
+    // temporal types
+    else if (isTemporal(left.resultType) && left.resultType == right.resultType) {
+      generateComparison("!=", nullCheck, left, right)
+    }
+    // array types
+    else if (isArray(left.resultType) && left.resultType == right.resultType) {
+      generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
+        (leftTerm, rightTerm) => s"!java.util.Arrays.equals($leftTerm, $rightTerm)"
+      }
+    }
+    // comparable types
+    else if (isComparable(left.resultType) && left.resultType == right.resultType) {
+      generateComparison("!=", nullCheck, left, right)
+    }
+    // non-comparable types
+    else {
+      generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
+        if (isReference(left)) {
+          (leftTerm, rightTerm) => s"!($leftTerm.equals($rightTerm))"
+        }
+        else if (isReference(right)) {
+          (leftTerm, rightTerm) => s"!($rightTerm.equals($leftTerm))"
+        }
+        else {
+          throw new CodeGenException(s"Incomparable types: ${left.resultType} and " +
+            s"${right.resultType}")
+        }
+      }
+    }
+  }
+
+  /**
+    * Generates comparison code for numeric types and comparable types of same type.
+    */
+  def generateComparison(
+      operator: String,
+      nullCheck: Boolean,
+      left: GeneratedExpression,
+      right: GeneratedExpression)
+    : GeneratedExpression = {
+    generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
+      // left is decimal or both sides are decimal
+      if (isDecimal(left.resultType) && isNumeric(right.resultType)) {
+        (leftTerm, rightTerm) => {
+          val operandCasting = numericCasting(right.resultType, left.resultType)
+          s"$leftTerm.compareTo(${operandCasting(rightTerm)}) $operator 0"
+        }
+      }
+      // right is decimal
+      else if (isNumeric(left.resultType) && isDecimal(right.resultType)) {
+        (leftTerm, rightTerm) => {
+          val operandCasting = numericCasting(left.resultType, right.resultType)
+          s"${operandCasting(leftTerm)}.compareTo($rightTerm) $operator 0"
+        }
+      }
+      // both sides are numeric
+      else if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
+        (leftTerm, rightTerm) => s"$leftTerm $operator $rightTerm"
+      }
+      // both sides are temporal of same type
+      else if (isTemporal(left.resultType) && left.resultType == right.resultType) {
+        (leftTerm, rightTerm) => s"$leftTerm $operator $rightTerm"
+      }
+      // both sides are boolean
+      else if (isBoolean(left.resultType) && left.resultType == right.resultType) {
+        operator match {
+          case "==" | "!=" => (leftTerm, rightTerm) => s"$leftTerm $operator $rightTerm"
+          case _ => throw new CodeGenException(s"Unsupported boolean comparison '$operator'.")
+        }
+      }
+      // both sides are same comparable type
+      else if (isComparable(left.resultType) && left.resultType == right.resultType) {
+        (leftTerm, rightTerm) => s"$leftTerm.compareTo($rightTerm) $operator 0"
+      }
+      else {
+        throw new CodeGenException(s"Incomparable types: ${left.resultType} and " +
+            s"${right.resultType}")
+      }
+    }
+  }
+
+  def generateIsNull(
+      nullCheck: Boolean,
+      operand: GeneratedExpression)
+    : GeneratedExpression = {
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+    val operatorCode = if (nullCheck) {
+      s"""
+        |${operand.code}
+        |boolean $resultTerm = ${operand.nullTerm};
+        |boolean $nullTerm = false;
+        |""".stripMargin
+    }
+    else if (!nullCheck && isReference(operand)) {
+      s"""
+        |${operand.code}
+        |boolean $resultTerm = ${operand.resultTerm} == null;
+        |boolean $nullTerm = false;
+        |""".stripMargin
+    }
+    else {
+      s"""
+        |${operand.code}
+        |boolean $resultTerm = false;
+        |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, operatorCode, BOOLEAN_TYPE_INFO)
+  }
+
+  def generateIsNotNull(
+      nullCheck: Boolean,
+      operand: GeneratedExpression)
+    : GeneratedExpression = {
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+    val operatorCode = if (nullCheck) {
+      s"""
+        |${operand.code}
+        |boolean $resultTerm = !${operand.nullTerm};
+        |boolean $nullTerm = false;
+        |""".stripMargin
+    }
+    else if (!nullCheck && isReference(operand)) {
+      s"""
+        |${operand.code}
+        |boolean $resultTerm = ${operand.resultTerm} != null;
+        |boolean $nullTerm = false;
+        |""".stripMargin
+    }
+    else {
+      s"""
+        |${operand.code}
+        |boolean $resultTerm = true;
+        |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, operatorCode, BOOLEAN_TYPE_INFO)
+  }
+
+  def generateAnd(
+      nullCheck: Boolean,
+      left: GeneratedExpression,
+      right: GeneratedExpression)
+    : GeneratedExpression = {
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+
+    val operatorCode = if (nullCheck) {
+      // Three-valued logic:
+      // no Unknown -> Two-valued logic
+      // True && Unknown -> Unknown
+      // False && Unknown -> False
+      // Unknown && True -> Unknown
+      // Unknown && False -> False
+      // Unknown && Unknown -> Unknown
+      s"""
+        |${left.code}
+        |${right.code}
+        |boolean $resultTerm;
+        |boolean $nullTerm;
+        |if (!${left.nullTerm} && !${right.nullTerm}) {
+        |  $resultTerm = ${left.resultTerm} && ${right.resultTerm};
+        |  $nullTerm = false;
+        |}
+        |else if (!${left.nullTerm} && ${left.resultTerm} && ${right.nullTerm}) {
+        |  $resultTerm = false;
+        |  $nullTerm = true;
+        |}
+        |else if (!${left.nullTerm} && !${left.resultTerm} && ${right.nullTerm}) {
+        |  $resultTerm = false;
+        |  $nullTerm = false;
+        |}
+        |else if (${left.nullTerm} && !${right.nullTerm} && ${right.resultTerm}) {
+        |  $resultTerm = false;
+        |  $nullTerm = true;
+        |}
+        |else if (${left.nullTerm} && !${right.nullTerm} && !${right.resultTerm}) {
+        |  $resultTerm = false;
+        |  $nullTerm = false;
+        |}
+        |else {
+        |  $resultTerm = false;
+        |  $nullTerm = true;
+        |}
+        |""".stripMargin
+    }
+    else {
+      s"""
+        |${left.code}
+        |${right.code}
+        |boolean $resultTerm = ${left.resultTerm} && ${right.resultTerm};
+        |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, operatorCode, BOOLEAN_TYPE_INFO)
+  }
+
+  def generateOr(
+      nullCheck: Boolean,
+      left: GeneratedExpression,
+      right: GeneratedExpression)
+    : GeneratedExpression = {
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+
+    val operatorCode = if (nullCheck) {
+      // Three-valued logic:
+      // no Unknown -> Two-valued logic
+      // True && Unknown -> True
+      // False && Unknown -> Unknown
+      // Unknown && True -> True
+      // Unknown && False -> Unknown
+      // Unknown && Unknown -> Unknown
+      s"""
+        |${left.code}
+        |${right.code}
+        |boolean $resultTerm;
+        |boolean $nullTerm;
+        |if (!${left.nullTerm} && !${right.nullTerm}) {
+        |  $resultTerm = ${left.resultTerm} || ${right.resultTerm};
+        |  $nullTerm = false;
+        |}
+        |else if (!${left.nullTerm} && ${left.resultTerm} && ${right.nullTerm}) {
+        |  $resultTerm = true;
+        |  $nullTerm = false;
+        |}
+        |else if (!${left.nullTerm} && !${left.resultTerm} && ${right.nullTerm}) {
+        |  $resultTerm = false;
+        |  $nullTerm = true;
+        |}
+        |else if (${left.nullTerm} && !${right.nullTerm} && ${right.resultTerm}) {
+        |  $resultTerm = true;
+        |  $nullTerm = false;
+        |}
+        |else if (${left.nullTerm} && !${right.nullTerm} && !${right.resultTerm}) {
+        |  $resultTerm = false;
+        |  $nullTerm = true;
+        |}
+        |else {
+        |  $resultTerm = false;
+        |  $nullTerm = true;
+        |}
+        |""".stripMargin
+    }
+    else {
+      s"""
+        |${left.code}
+        |${right.code}
+        |boolean $resultTerm = ${left.resultTerm} || ${right.resultTerm};
+        |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, operatorCode, BOOLEAN_TYPE_INFO)
+  }
+
+  def generateNot(
+      nullCheck: Boolean,
+      operand: GeneratedExpression)
+    : GeneratedExpression = {
+    // Three-valued logic:
+    // no Unknown -> Two-valued logic
+    // Unknown -> Unknown
+    generateUnaryOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, operand) {
+      (operandTerm) => s"!($operandTerm)"
+    }
+  }
+
+  def generateIsTrue(operand: GeneratedExpression): GeneratedExpression = {
+    GeneratedExpression(
+      operand.resultTerm, // unknown is always false by default
+      GeneratedExpression.NEVER_NULL,
+      operand.code,
+      BOOLEAN_TYPE_INFO)
+  }
+
+  def generateIsNotTrue(operand: GeneratedExpression): GeneratedExpression = {
+    GeneratedExpression(
+      s"(!${operand.resultTerm})", // unknown is always false by default
+      GeneratedExpression.NEVER_NULL,
+      operand.code,
+      BOOLEAN_TYPE_INFO)
+  }
+
+  def generateIsFalse(operand: GeneratedExpression): GeneratedExpression = {
+    GeneratedExpression(
+      s"(!${operand.resultTerm} && !${operand.nullTerm})",
+      GeneratedExpression.NEVER_NULL,
+      operand.code,
+      BOOLEAN_TYPE_INFO)
+  }
+
+  def generateIsNotFalse(operand: GeneratedExpression): GeneratedExpression = {
+    GeneratedExpression(
+      s"(${operand.resultTerm} || ${operand.nullTerm})",
+      GeneratedExpression.NEVER_NULL,
+      operand.code,
+      BOOLEAN_TYPE_INFO)
+  }
+
+  def generateCast(
+      nullCheck: Boolean,
+      operand: GeneratedExpression,
+      targetType: TypeInformation[_])
+    : GeneratedExpression = (operand.resultType, targetType) match {
+    // identity casting
+    case (fromTp, toTp) if fromTp == toTp =>
+      operand
+
+    // Date/Time/Timestamp -> String
+    case (dtt: SqlTimeTypeInfo[_], STRING_TYPE_INFO) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"${internalToTimePointCode(dtt, operandTerm)}.toString()"
+      }
+
+    // Interval Months -> String
+    case (TimeIntervalTypeInfo.INTERVAL_MONTHS, STRING_TYPE_INFO) =>
+      val method = qualifyMethod(BuiltInMethod.INTERVAL_YEAR_MONTH_TO_STRING.method)
+      val timeUnitRange = qualifyEnum(TimeUnitRange.YEAR_TO_MONTH)
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"$method($operandTerm, $timeUnitRange)"
+      }
+
+    // Interval Millis -> String
+    case (TimeIntervalTypeInfo.INTERVAL_MILLIS, STRING_TYPE_INFO) =>
+      val method = qualifyMethod(BuiltInMethod.INTERVAL_DAY_TIME_TO_STRING.method)
+      val timeUnitRange = qualifyEnum(TimeUnitRange.DAY_TO_SECOND)
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"$method($operandTerm, $timeUnitRange, 3)" // milli second precision
+      }
+
+    // Object array -> String
+    case (_:ObjectArrayTypeInfo[_, _], STRING_TYPE_INFO) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"java.util.Arrays.deepToString($operandTerm)"
+      }
+
+    // Primitive array -> String
+    case (_:PrimitiveArrayTypeInfo[_], STRING_TYPE_INFO) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"java.util.Arrays.toString($operandTerm)"
+      }
+
+    // * (not Date/Time/Timestamp) -> String
+    case (_, STRING_TYPE_INFO) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s""" "" + $operandTerm"""
+      }
+
+    // * -> Character
+    case (_, CHAR_TYPE_INFO) =>
+      throw new CodeGenException("Character type not supported.")
+
+    // String -> NUMERIC TYPE (not Character), Boolean
+    case (STRING_TYPE_INFO, _: NumericTypeInfo[_])
+        | (STRING_TYPE_INFO, BOOLEAN_TYPE_INFO) =>
+      val wrapperClass = targetType.getTypeClass.getCanonicalName
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"$wrapperClass.valueOf($operandTerm)"
+      }
+
+    // String -> BigDecimal
+    case (STRING_TYPE_INFO, BIG_DEC_TYPE_INFO) =>
+      val wrapperClass = targetType.getTypeClass.getCanonicalName
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"new $wrapperClass($operandTerm)"
+      }
+
+    // String -> Date
+    case (STRING_TYPE_INFO, SqlTimeTypeInfo.DATE) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"${qualifyMethod(BuiltInMethod.STRING_TO_DATE.method)}($operandTerm)"
+      }
+
+    // String -> Time
+    case (STRING_TYPE_INFO, SqlTimeTypeInfo.TIME) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"${qualifyMethod(BuiltInMethod.STRING_TO_TIME.method)}($operandTerm)"
+      }
+
+    // String -> Timestamp
+    case (STRING_TYPE_INFO, SqlTimeTypeInfo.TIMESTAMP) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"${qualifyMethod(BuiltInMethod.STRING_TO_TIMESTAMP.method)}" +
+          s"($operandTerm)"
+      }
+
+    // Boolean -> NUMERIC TYPE
+    case (BOOLEAN_TYPE_INFO, nti: NumericTypeInfo[_]) =>
+      val targetTypeTerm = primitiveTypeTermForTypeInfo(nti)
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"($targetTypeTerm) ($operandTerm ? 1 : 0)"
+      }
+
+    // Boolean -> BigDecimal
+    case (BOOLEAN_TYPE_INFO, BIG_DEC_TYPE_INFO) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"$operandTerm ? java.math.BigDecimal.ONE : java.math.BigDecimal.ZERO"
+      }
+
+    // NUMERIC TYPE -> Boolean
+    case (_: NumericTypeInfo[_], BOOLEAN_TYPE_INFO) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"$operandTerm != 0"
+      }
+
+    // BigDecimal -> Boolean
+    case (BIG_DEC_TYPE_INFO, BOOLEAN_TYPE_INFO) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"$operandTerm.compareTo(java.math.BigDecimal.ZERO) != 0"
+      }
+
+    // NUMERIC TYPE, BigDecimal -> NUMERIC TYPE, BigDecimal
+    case (_: NumericTypeInfo[_], _: NumericTypeInfo[_])
+        | (BIG_DEC_TYPE_INFO, _: NumericTypeInfo[_])
+        | (_: NumericTypeInfo[_], BIG_DEC_TYPE_INFO) =>
+      val operandCasting = numericCasting(operand.resultType, targetType)
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"${operandCasting(operandTerm)}"
+      }
+
+    // Date -> Timestamp
+    case (SqlTimeTypeInfo.DATE, SqlTimeTypeInfo.TIMESTAMP) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) =>
+          s"$operandTerm * ${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY"
+      }
+
+    // Timestamp -> Date
+    case (SqlTimeTypeInfo.TIMESTAMP, SqlTimeTypeInfo.DATE) =>
+      val targetTypeTerm = primitiveTypeTermForTypeInfo(targetType)
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) =>
+          s"($targetTypeTerm) ($operandTerm / " +
+            s"${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY)"
+      }
+
+    // Time -> Timestamp
+    case (SqlTimeTypeInfo.TIME, SqlTimeTypeInfo.TIMESTAMP) =>
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) => s"$operandTerm"
+      }
+
+    // Timestamp -> Time
+    case (SqlTimeTypeInfo.TIMESTAMP, SqlTimeTypeInfo.TIME) =>
+      val targetTypeTerm = primitiveTypeTermForTypeInfo(targetType)
+      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
+        (operandTerm) =>
+          s"($targetTypeTerm) ($operandTerm % " +
+            s"${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY)"
+      }
+
+    // internal temporal casting
+    // Date -> Integer
+    // Time -> Integer
+    // Timestamp -> Long
+    // Integer -> Date
+    // Integer -> Time
+    // Long -> Timestamp
+    // Integer -> Interval Months
+    // Long -> Interval Millis
+    // Interval Months -> Integer
+    // Interval Millis -> Long
+    case (SqlTimeTypeInfo.DATE, INT_TYPE_INFO) |
+         (SqlTimeTypeInfo.TIME, INT_TYPE_INFO) |
+         (SqlTimeTypeInfo.TIMESTAMP, LONG_TYPE_INFO) |
+         (INT_TYPE_INFO, SqlTimeTypeInfo.DATE) |
+         (INT_TYPE_INFO, SqlTimeTypeInfo.TIME) |
+         (LONG_TYPE_INFO, SqlTimeTypeInfo.TIMESTAMP) |
+         (INT_TYPE_INFO, TimeIntervalTypeInfo.INTERVAL_MONTHS) |
+         (LONG_TYPE_INFO, TimeIntervalTypeInfo.INTERVAL_MILLIS) |
+         (TimeIntervalTypeInfo.INTERVAL_MONTHS, INT_TYPE_INFO) |
+         (TimeIntervalTypeInfo.INTERVAL_MILLIS, LONG_TYPE_INFO) =>
+      internalExprCasting(operand, targetType)
+
+    // internal reinterpretation of temporal types
+    // Date, Time, Interval Months -> Long
+    case  (SqlTimeTypeInfo.DATE, LONG_TYPE_INFO)
+        | (SqlTimeTypeInfo.TIME, LONG_TYPE_INFO)
+        | (TimeIntervalTypeInfo.INTERVAL_MONTHS, LONG_TYPE_INFO) =>
+      internalExprCasting(operand, targetType)
+
+    case (from, to) =>
+      throw new CodeGenException(s"Unsupported cast from '$from' to '$to'.")
+  }
+
+  def generateIfElse(
+      nullCheck: Boolean,
+      operands: Seq[GeneratedExpression],
+      resultType: TypeInformation[_],
+      i: Int = 0)
+    : GeneratedExpression = {
+    // else part
+    if (i == operands.size - 1) {
+      generateCast(nullCheck, operands(i), resultType)
+    }
+    else {
+      // check that the condition is boolean
+      // we do not check for null instead we use the default value
+      // thus null is false
+      requireBoolean(operands(i))
+      val condition = operands(i)
+      val trueAction = generateCast(nullCheck, operands(i + 1), resultType)
+      val falseAction = generateIfElse(nullCheck, operands, resultType, i + 2)
+
+      val resultTerm = newName("result")
+      val nullTerm = newName("isNull")
+      val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
+
+      val operatorCode = if (nullCheck) {
+        s"""
+          |${condition.code}
+          |$resultTypeTerm $resultTerm;
+          |boolean $nullTerm;
+          |if (${condition.resultTerm}) {
+          |  ${trueAction.code}
+          |  $resultTerm = ${trueAction.resultTerm};
+          |  $nullTerm = ${trueAction.nullTerm};
+          |}
+          |else {
+          |  ${falseAction.code}
+          |  $resultTerm = ${falseAction.resultTerm};
+          |  $nullTerm = ${falseAction.nullTerm};
+          |}
+          |""".stripMargin
+      }
+      else {
+        s"""
+          |${condition.code}
+          |$resultTypeTerm $resultTerm;
+          |if (${condition.resultTerm}) {
+          |  ${trueAction.code}
+          |  $resultTerm = ${trueAction.resultTerm};
+          |}
+          |else {
+          |  ${falseAction.code}
+          |  $resultTerm = ${falseAction.resultTerm};
+          |}
+          |""".stripMargin
+      }
+
+      GeneratedExpression(resultTerm, nullTerm, operatorCode, resultType)
+    }
+  }
+
+  def generateTemporalPlusMinus(
+      plus: Boolean,
+      nullCheck: Boolean,
+      left: GeneratedExpression,
+      right: GeneratedExpression)
+    : GeneratedExpression = {
+
+    val op = if (plus) "+" else "-"
+
+    (left.resultType, right.resultType) match {
+      case (l: TimeIntervalTypeInfo[_], r: TimeIntervalTypeInfo[_]) if l == r =>
+        generateArithmeticOperator(op, nullCheck, l, left, right)
+
+      case (SqlTimeTypeInfo.DATE, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
+        generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.DATE, left, right) {
+            (l, r) => s"$l $op ((int) ($r / ${MILLIS_PER_DAY}L))"
+        }
+
+      case (SqlTimeTypeInfo.DATE, TimeIntervalTypeInfo.INTERVAL_MONTHS) =>
+        generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.DATE, left, right) {
+            (l, r) => s"${qualifyMethod(BuiltInMethod.ADD_MONTHS.method)}($l, $op($r))"
+        }
+
+      case (SqlTimeTypeInfo.TIME, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
+        generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.TIME, left, right) {
+            (l, r) => s"$l $op ((int) ($r))"
+        }
+
+      case (SqlTimeTypeInfo.TIMESTAMP, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
+        generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.TIMESTAMP, left, right) {
+          (l, r) => s"$l $op $r"
+        }
+
+      case (SqlTimeTypeInfo.TIMESTAMP, TimeIntervalTypeInfo.INTERVAL_MONTHS) =>
+        generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.TIMESTAMP, left, right) {
+          (l, r) => s"${qualifyMethod(BuiltInMethod.ADD_MONTHS.method)}($l, $op($r))"
+        }
+
+      case _ =>
+        throw new CodeGenException("Unsupported temporal arithmetic.")
+    }
+  }
+
+  def generateUnaryIntervalPlusMinus(
+      plus: Boolean,
+      nullCheck: Boolean,
+      operand: GeneratedExpression)
+    : GeneratedExpression = {
+    val operator = if (plus) "+" else "-"
+    generateUnaryArithmeticOperator(operator, nullCheck, operand.resultType, operand)
+  }
+
+  def generateArray(
+      codeGenerator: CodeGenerator,
+      resultType: TypeInformation[_],
+      elements: Seq[GeneratedExpression])
+    : GeneratedExpression = {
+    val arrayTerm = codeGenerator.addReusableArray(resultType.getTypeClass, elements.size)
+
+    val boxedElements: Seq[GeneratedExpression] = resultType match {
+
+      case oati: ObjectArrayTypeInfo[_, _] =>
+        // we box the elements to also represent null values
+        val boxedTypeTerm = boxedTypeTermForTypeInfo(oati.getComponentInfo)
+
+        elements.map { e =>
+          val boxedExpr = codeGenerator.generateOutputFieldBoxing(e)
+          val exprOrNull: String = if (codeGenerator.nullCheck) {
+            s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}"
+          } else {
+            boxedExpr.resultTerm
+          }
+          boxedExpr.copy(resultTerm = exprOrNull)
+        }
+
+      // no boxing necessary
+      case _: PrimitiveArrayTypeInfo[_] => elements
+    }
+
+    val code = boxedElements
+      .zipWithIndex
+      .map { case (element, idx) =>
+        s"""
+          |${element.code}
+          |$arrayTerm[$idx] = ${element.resultTerm};
+          |""".stripMargin
+      }
+      .mkString("\n")
+
+    GeneratedExpression(arrayTerm, GeneratedExpression.NEVER_NULL, code, resultType)
+  }
+
+  def generateArrayElementAt(
+      codeGenerator: CodeGenerator,
+      array: GeneratedExpression,
+      index: GeneratedExpression)
+    : GeneratedExpression = {
+
+    val resultTerm = newName("result")
+
+    array.resultType match {
+
+      // unbox object array types
+      case oati: ObjectArrayTypeInfo[_, _] =>
+        // get boxed array element
+        val resultTypeTerm = boxedTypeTermForTypeInfo(oati.getComponentInfo)
+
+        val arrayAccessCode = if (codeGenerator.nullCheck) {
+          s"""
+            |${array.code}
+            |${index.code}
+            |$resultTypeTerm $resultTerm = (${array.nullTerm} || ${index.nullTerm}) ?
+            |  null : ${array.resultTerm}[${index.resultTerm} - 1];
+            |""".stripMargin
+        } else {
+          s"""
+            |${array.code}
+            |${index.code}
+            |$resultTypeTerm $resultTerm = ${array.resultTerm}[${index.resultTerm} - 1];
+            |""".stripMargin
+        }
+
+        // generate unbox code
+        val unboxing = codeGenerator.generateInputFieldUnboxing(oati.getComponentInfo, resultTerm)
+
+        unboxing.copy(code =
+          s"""
+            |$arrayAccessCode
+            |${unboxing.code}
+            |""".stripMargin
+        )
+
+      // no unboxing necessary
+      case pati: PrimitiveArrayTypeInfo[_] =>
+        generateOperatorIfNotNull(codeGenerator.nullCheck, pati.getComponentType, array, index) {
+          (leftTerm, rightTerm) => s"$leftTerm[$rightTerm - 1]"
+        }
+    }
+  }
+
+  def generateArrayElement(
+      codeGenerator: CodeGenerator,
+      array: GeneratedExpression)
+    : GeneratedExpression = {
+
+    val nullTerm = newName("isNull")
+    val resultTerm = newName("result")
+    val resultType = array.resultType match {
+      case oati: ObjectArrayTypeInfo[_, _] => oati.getComponentInfo
+      case pati: PrimitiveArrayTypeInfo[_] => pati.getComponentType
+    }
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
+    val defaultValue = primitiveDefaultValue(resultType)
+
+    val arrayLengthCode = if (codeGenerator.nullCheck) {
+      s"${array.nullTerm} ? 0 : ${array.resultTerm}.length"
+    } else {
+      s"${array.resultTerm}.length"
+    }
+
+    val arrayAccessCode = array.resultType match {
+      case oati: ObjectArrayTypeInfo[_, _] =>
+        // generate unboxing code
+        val unboxing = codeGenerator.generateInputFieldUnboxing(
+          oati.getComponentInfo,
+          s"${array.resultTerm}[0]")
+
+        s"""
+          |${array.code}
+          |${if (codeGenerator.nullCheck) s"boolean $nullTerm;" else "" }
+          |$resultTypeTerm $resultTerm;
+          |switch ($arrayLengthCode) {
+          |  case 0:
+          |    ${if (codeGenerator.nullCheck) s"$nullTerm = true;" else "" }
+          |    $resultTerm = $defaultValue;
+          |    break;
+          |  case 1:
+          |    ${unboxing.code}
+          |    ${if (codeGenerator.nullCheck) s"$nullTerm = ${unboxing.nullTerm};" else "" }
+          |    $resultTerm = ${unboxing.resultTerm};
+          |    break;
+          |  default:
+          |    throw new RuntimeException("Array has more than one element.");
+          |}
+          |""".stripMargin
+
+      case pati: PrimitiveArrayTypeInfo[_] =>
+        s"""
+          |${array.code}
+          |${if (codeGenerator.nullCheck) s"boolean $nullTerm;" else "" }
+          |$resultTypeTerm $resultTerm;
+          |switch ($arrayLengthCode) {
+          |  case 0:
+          |    ${if (codeGenerator.nullCheck) s"$nullTerm = true;" else "" }
+          |    $resultTerm = $defaultValue;
+          |    break;
+          |  case 1:
+          |    ${if (codeGenerator.nullCheck) s"$nullTerm = false;" else "" }
+          |    $resultTerm = ${array.resultTerm}[0];
+          |    break;
+          |  default:
+          |    throw new RuntimeException("Array has more than one element.");
+          |}
+          |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, arrayAccessCode, resultType)
+  }
+
+  def generateArrayCardinality(
+      nullCheck: Boolean,
+      array: GeneratedExpression)
+    : GeneratedExpression = {
+
+    generateUnaryOperatorIfNotNull(nullCheck, INT_TYPE_INFO, array) {
+      (operandTerm) => s"${array.resultTerm}.length"
+    }
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  private def generateUnaryOperatorIfNotNull(
+      nullCheck: Boolean,
+      resultType: TypeInformation[_],
+      operand: GeneratedExpression)
+      (expr: (String) => String)
+    : GeneratedExpression = {
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
+    val defaultValue = primitiveDefaultValue(resultType)
+
+    val operatorCode = if (nullCheck) {
+      s"""
+        |${operand.code}
+        |$resultTypeTerm $resultTerm;
+        |boolean $nullTerm;
+        |if (!${operand.nullTerm}) {
+        |  $resultTerm = ${expr(operand.resultTerm)};
+        |  $nullTerm = false;
+        |}
+        |else {
+        |  $resultTerm = $defaultValue;
+        |  $nullTerm = true;
+        |}
+        |""".stripMargin
+    }
+    else {
+      s"""
+        |${operand.code}
+        |$resultTypeTerm $resultTerm = ${expr(operand.resultTerm)};
+        |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, operatorCode, resultType)
+  }
+
+  private def generateOperatorIfNotNull(
+      nullCheck: Boolean,
+      resultType: TypeInformation[_],
+      left: GeneratedExpression,
+      right: GeneratedExpression)
+      (expr: (String, String) => String)
+    : GeneratedExpression = {
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
+    val defaultValue = primitiveDefaultValue(resultType)
+
+    val resultCode = if (nullCheck) {
+      s"""
+        |${left.code}
+        |${right.code}
+        |boolean $nullTerm = ${left.nullTerm} || ${right.nullTerm};
+        |$resultTypeTerm $resultTerm;
+        |if ($nullTerm) {
+        |  $resultTerm = $defaultValue;
+        |}
+        |else {
+        |  $resultTerm = ${expr(left.resultTerm, right.resultTerm)};
+        |}
+        |""".stripMargin
+    }
+    else {
+      s"""
+        |${left.code}
+        |${right.code}
+        |$resultTypeTerm $resultTerm = ${expr(left.resultTerm, right.resultTerm)};
+        |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, resultCode, resultType)
+  }
+
+  private def internalExprCasting(
+      expr: GeneratedExpression,
+      typeInfo: TypeInformation[_])
+    : GeneratedExpression = {
+    GeneratedExpression(expr.resultTerm, expr.nullTerm, expr.code, typeInfo)
+  }
+
+  private def arithOpToDecMethod(operator: String): String = operator match {
+    case "+" => "add"
+    case "-" => "subtract"
+    case "*" => "multiply"
+    case "/" => "divide"
+    case "%" => "remainder"
+    case _ => throw new CodeGenException("Unsupported decimal arithmetic operator.")
+  }
+
+  private def numericCasting(
+      operandType: TypeInformation[_],
+      resultType: TypeInformation[_])
+    : (String) => String = {
+
+    def decToPrimMethod(targetType: TypeInformation[_]): String = targetType match {
+      case BYTE_TYPE_INFO => "byteValueExact"
+      case SHORT_TYPE_INFO => "shortValueExact"
+      case INT_TYPE_INFO => "intValueExact"
+      case LONG_TYPE_INFO => "longValueExact"
+      case FLOAT_TYPE_INFO => "floatValue"
+      case DOUBLE_TYPE_INFO => "doubleValue"
+      case _ => throw new CodeGenException("Unsupported decimal casting type.")
+    }
+
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
+    // no casting necessary
+    if (operandType == resultType) {
+      (operandTerm) => s"$operandTerm"
+    }
+    // result type is decimal but numeric operand is not
+    else if (isDecimal(resultType) && !isDecimal(operandType) && isNumeric(operandType)) {
+      (operandTerm) =>
+        s"java.math.BigDecimal.valueOf((${superPrimitive(operandType)}) $operandTerm)"
+    }
+    // numeric result type is not decimal but operand is
+    else if (isNumeric(resultType) && !isDecimal(resultType) && isDecimal(operandType) ) {
+      (operandTerm) => s"$operandTerm.${decToPrimMethod(resultType)}()"
+    }
+    // result type and operand type are numeric but not decimal
+    else if (isNumeric(operandType) && isNumeric(resultType)
+        && !isDecimal(operandType) && !isDecimal(resultType)) {
+      (operandTerm) => s"(($resultTypeTerm) $operandTerm)"
+    }
+    else {
+      throw new CodeGenException(s"Unsupported casting from $operandType to $resultType.")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala
new file mode 100644
index 0000000..50c569f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/TableFunctionCallGen.scala
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.codegen.calls
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.codegen.CodeGenUtils._
+import org.apache.flink.table.codegen.GeneratedExpression.NEVER_NULL
+import org.apache.flink.table.codegen.{CodeGenException, CodeGenerator, GeneratedExpression}
+import org.apache.flink.table.functions.TableFunction
+import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
+
+/**
+  * Generates a call to user-defined [[TableFunction]].
+  *
+  * @param tableFunction user-defined [[TableFunction]] that might be overloaded
+  * @param signature actual signature with which the function is called
+  * @param returnType actual return type required by the surrounding
+  */
+class TableFunctionCallGen(
+    tableFunction: TableFunction[_],
+    signature: Seq[TypeInformation[_]],
+    returnType: TypeInformation[_])
+  extends CallGenerator {
+
+  override def generate(
+      codeGenerator: CodeGenerator,
+      operands: Seq[GeneratedExpression])
+    : GeneratedExpression = {
+    // determine function signature
+    val matchingSignature = getSignature(tableFunction, signature)
+      .getOrElse(throw new CodeGenException("No matching signature found."))
+
+    // convert parameters for function (output boxing)
+    val parameters = matchingSignature
+        .zip(operands)
+        .map { case (paramClass, operandExpr) =>
+          if (paramClass.isPrimitive) {
+            operandExpr
+          } else {
+            val boxedTypeTerm = boxedTypeTermForTypeInfo(operandExpr.resultType)
+            val boxedExpr = codeGenerator.generateOutputFieldBoxing(operandExpr)
+            val exprOrNull: String = if (codeGenerator.nullCheck) {
+              s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}"
+            } else {
+              boxedExpr.resultTerm
+            }
+            boxedExpr.copy(resultTerm = exprOrNull)
+          }
+        }
+
+    // generate function call
+    val functionReference = codeGenerator.addReusableFunction(tableFunction)
+    val functionCallCode =
+      s"""
+        |${parameters.map(_.code).mkString("\n")}
+        |$functionReference.clear();
+        |$functionReference.eval(${parameters.map(_.resultTerm).mkString(", ")});
+        |""".stripMargin
+
+    // has no result
+    GeneratedExpression(
+      functionReference,
+      NEVER_NULL,
+      functionCallCode,
+      returnType)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/TrimCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/TrimCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/TrimCallGen.scala
new file mode 100644
index 0000000..9d50bf9
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/TrimCallGen.scala
@@ -0,0 +1,52 @@
+/*
+ * 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.codegen.calls
+
+import org.apache.calcite.sql.fun.SqlTrimFunction.Flag.{BOTH, LEADING, TRAILING}
+import org.apache.calcite.util.BuiltInMethod
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO
+import org.apache.flink.table.codegen.CodeGenUtils._
+import org.apache.flink.table.codegen.calls.CallGenerator._
+import org.apache.flink.table.codegen.{CodeGenerator, GeneratedExpression}
+
+/**
+  * Generates a TRIM function call.
+  *
+  * First operand: trim mode (see [[org.apache.calcite.sql.fun.SqlTrimFunction.Flag]])
+  * Second operand: String to be removed
+  * Third operand: String to be trimmed
+  */
+class TrimCallGen extends CallGenerator {
+
+  override def generate(
+      codeGenerator: CodeGenerator,
+      operands: Seq[GeneratedExpression])
+    : GeneratedExpression = {
+    generateCallIfArgsNotNull(codeGenerator.nullCheck, STRING_TYPE_INFO, operands) {
+      (terms) =>
+        val leading = compareEnum(terms.head, BOTH) || compareEnum(terms.head, LEADING)
+        val trailing = compareEnum(terms.head, BOTH) || compareEnum(terms.head, TRAILING)
+        s"""
+          |${qualifyMethod(BuiltInMethod.TRIM.method)}(
+          |  $leading, $trailing, ${terms(1)}, ${terms(2)})
+          |""".stripMargin
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/generated.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/generated.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/generated.scala
new file mode 100644
index 0000000..0d60dc1
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/generated.scala
@@ -0,0 +1,43 @@
+/*
+ * 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.codegen
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+
+/**
+  * Describes a generated expression.
+  *
+  * @param resultTerm term to access the result of the expression
+  * @param nullTerm boolean term that indicates if expression is null
+  * @param code code necessary to produce resultTerm and nullTerm
+  * @param resultType type of the resultTerm
+  */
+case class GeneratedExpression(
+    resultTerm: String,
+    nullTerm: String,
+    code: String,
+    resultType: TypeInformation[_])
+
+object GeneratedExpression {
+  val ALWAYS_NULL = "true"
+  val NEVER_NULL = "false"
+  val NO_CODE = ""
+}
+
+case class GeneratedFunction[T](name: String, returnType: TypeInformation[Any], code: String)

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/package.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/package.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/package.scala
new file mode 100644
index 0000000..743f846
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/package.scala
@@ -0,0 +1,25 @@
+/*
+ * 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
+
+package object codegen {
+  // Used in ExpressionCodeGenerator because Scala 2.10 reflection is not thread safe. We might
+  // have several parallel expression operators in one TaskManager, therefore we need to guard
+  // these operations.
+  object ReflectionLock
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala
new file mode 100644
index 0000000..6d16722
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/StreamSQLExample.scala
@@ -0,0 +1,76 @@
+/*
+ * 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.examples.scala
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
+import org.apache.flink.table.api.TableEnvironment
+
+/**
+  * Simple example for demonstrating the use of SQL on a Stream Table.
+  *
+  * This example shows how to:
+  *  - Convert DataStreams to Tables
+  *  - Register a Table under a name
+  *  - Run a StreamSQL query on the registered Table
+  *
+  */
+object StreamSQLExample {
+
+  // *************************************************************************
+  //     PROGRAM
+  // *************************************************************************
+
+  def main(args: Array[String]): Unit = {
+
+    // set up execution environment
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val orderA: DataStream[Order] = env.fromCollection(Seq(
+      Order(1L, "beer", 3),
+      Order(1L, "diaper", 4),
+      Order(3L, "rubber", 2)))
+
+    val orderB: DataStream[Order] = env.fromCollection(Seq(
+      Order(2L, "pen", 3),
+      Order(2L, "rubber", 3),
+      Order(4L, "beer", 1)))
+
+    // register the DataStreams under the name "OrderA" and "OrderB"
+    tEnv.registerDataStream("OrderA", orderA, 'user, 'product, 'amount)
+    tEnv.registerDataStream("OrderB", orderB, 'user, 'product, 'amount)
+
+    // union the two tables
+    val result = tEnv.sql(
+      "SELECT * FROM OrderA WHERE amount > 2 UNION ALL " +
+        "SELECT * FROM OrderB WHERE amount < 2")
+
+    result.toDataStream[Order].print()
+
+    env.execute()
+  }
+
+  // *************************************************************************
+  //     USER DATA TYPES
+  // *************************************************************************
+
+  case class Order(user: Long, product: String, amount: Int)
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/StreamTableExample.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/StreamTableExample.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/StreamTableExample.scala
new file mode 100644
index 0000000..6c1467f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/StreamTableExample.scala
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.examples.scala
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
+import org.apache.flink.table.api.TableEnvironment
+
+/**
+  * Simple example for demonstrating the use of Table API on a Stream Table.
+  *
+  * This example shows how to:
+  *  - Convert DataStreams to Tables
+  *  - Apply union, select, and filter operations
+  *
+  */
+object StreamTableExample {
+
+  // *************************************************************************
+  //     PROGRAM
+  // *************************************************************************
+
+  def main(args: Array[String]): Unit = {
+
+    // set up execution environment
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val orderA = env.fromCollection(Seq(
+      Order(1L, "beer", 3),
+      Order(1L, "diaper", 4),
+      Order(3L, "rubber", 2))).toTable(tEnv)
+
+    val orderB = env.fromCollection(Seq(
+      Order(2L, "pen", 3),
+      Order(2L, "rubber", 3),
+      Order(4L, "beer", 1))).toTable(tEnv)
+
+    // union the two tables
+    val result: DataStream[Order] = orderA.unionAll(orderB)
+      .select('user, 'product, 'amount)
+      .where('amount > 2)
+      .toDataStream[Order]
+
+    result.print()
+
+    env.execute()
+  }
+
+  // *************************************************************************
+  //     USER DATA TYPES
+  // *************************************************************************
+
+  case class Order(user: Long, product: String, amount: Int)
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/TPCHQuery3Table.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/TPCHQuery3Table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/TPCHQuery3Table.scala
new file mode 100644
index 0000000..74afb06
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/TPCHQuery3Table.scala
@@ -0,0 +1,180 @@
+/*
+ * 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.examples.scala
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+
+/**
+  * This program implements a modified version of the TPC-H query 3. The
+  * example demonstrates how to assign names to fields by extending the Tuple class.
+  * The original query can be found at
+  * [http://www.tpc.org/tpch/spec/tpch2.16.0.pdf](http://www.tpc.org/tpch/spec/tpch2.16.0.pdf)
+  * (page 29).
+  *
+  * This program implements the following SQL equivalent:
+  *
+  * {{{
+  * SELECT
+  *      l_orderkey,
+  *      SUM(l_extendedprice*(1-l_discount)) AS revenue,
+  *      o_orderdate,
+  *      o_shippriority
+  * FROM customer,
+  *      orders,
+  *      lineitem
+  * WHERE
+  *      c_mktsegment = '[SEGMENT]'
+  *      AND c_custkey = o_custkey
+  *      AND l_orderkey = o_orderkey
+  *      AND o_orderdate < date '[DATE]'
+  *      AND l_shipdate > date '[DATE]'
+  * GROUP BY
+  *      l_orderkey,
+  *      o_orderdate,
+  *      o_shippriority
+  * ORDER BY
+  *      revenue desc,
+  *      o_orderdate;
+  * }}}
+  *
+  * Input files are plain text CSV files using the pipe character ('|') as field separator
+  * as generated by the TPC-H data generator which is available at
+  * [http://www.tpc.org/tpch/](a href="http://www.tpc.org/tpch/).
+  *
+  * Usage:
+  * {{{
+  * TPCHQuery3Expression <lineitem-csv path> <customer-csv path> <orders-csv path> <result path>
+  * }}}
+  *
+  * This example shows how to:
+  *  - Convert DataSets to Tables
+  *  - Use Table API expressions
+  *
+  */
+object TPCHQuery3Table {
+
+  // *************************************************************************
+  //     PROGRAM
+  // *************************************************************************
+
+  def main(args: Array[String]) {
+    if (!parseParameters(args)) {
+      return
+    }
+
+    // set filter date
+    val date = "1995-03-12".toDate
+
+    // get execution environment
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val lineitems = getLineitemDataSet(env)
+      .toTable(tEnv, 'id, 'extdPrice, 'discount, 'shipDate)
+      .filter('shipDate.toDate > date)
+
+    val customers = getCustomerDataSet(env)
+      .toTable(tEnv, 'id, 'mktSegment)
+      .filter('mktSegment === "AUTOMOBILE")
+
+    val orders = getOrdersDataSet(env)
+      .toTable(tEnv, 'orderId, 'custId, 'orderDate, 'shipPrio)
+      .filter('orderDate.toDate < date)
+
+    val items =
+      orders.join(customers)
+        .where('custId === 'id)
+        .select('orderId, 'orderDate, 'shipPrio)
+      .join(lineitems)
+        .where('orderId === 'id)
+        .select(
+          'orderId,
+          'extdPrice * (1.0f.toExpr - 'discount) as 'revenue,
+          'orderDate,
+          'shipPrio)
+
+    val result = items
+      .groupBy('orderId, 'orderDate, 'shipPrio)
+      .select('orderId, 'revenue.sum as 'revenue, 'orderDate, 'shipPrio)
+      .orderBy('revenue.desc, 'orderDate.asc)
+
+    // emit result
+    result.writeAsCsv(outputPath, "\n", "|")
+
+    // execute program
+    env.execute("Scala TPCH Query 3 (Table API Expression) Example")
+  }
+  
+  // *************************************************************************
+  //     USER DATA TYPES
+  // *************************************************************************
+  
+  case class Lineitem(id: Long, extdPrice: Double, discount: Double, shipDate: String)
+  case class Customer(id: Long, mktSegment: String)
+  case class Order(orderId: Long, custId: Long, orderDate: String, shipPrio: Long)
+
+  // *************************************************************************
+  //     UTIL METHODS
+  // *************************************************************************
+  
+  private var lineitemPath: String = _
+  private var customerPath: String = _
+  private var ordersPath: String = _
+  private var outputPath: String = _
+
+  private def parseParameters(args: Array[String]): Boolean = {
+    if (args.length == 4) {
+      lineitemPath = args(0)
+      customerPath = args(1)
+      ordersPath = args(2)
+      outputPath = args(3)
+      true
+    } else {
+      System.err.println("This program expects data from the TPC-H benchmark as input data.\n" +
+          " Due to legal restrictions, we can not ship generated data.\n" +
+          " You can find the TPC-H data generator at http://www.tpc.org/tpch/.\n" +
+          " Usage: TPCHQuery3 <lineitem-csv path> <customer-csv path> " +
+                             "<orders-csv path> <result path>")
+      false
+    }
+  }
+
+  private def getLineitemDataSet(env: ExecutionEnvironment): DataSet[Lineitem] = {
+    env.readCsvFile[Lineitem](
+        lineitemPath,
+        fieldDelimiter = "|",
+        includedFields = Array(0, 5, 6, 10) )
+  }
+
+  private def getCustomerDataSet(env: ExecutionEnvironment): DataSet[Customer] = {
+    env.readCsvFile[Customer](
+        customerPath,
+        fieldDelimiter = "|",
+        includedFields = Array(0, 6) )
+  }
+
+  private def getOrdersDataSet(env: ExecutionEnvironment): DataSet[Order] = {
+    env.readCsvFile[Order](
+        ordersPath,
+        fieldDelimiter = "|",
+        includedFields = Array(0, 1, 4, 7) )
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/WordCountSQL.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/WordCountSQL.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/WordCountSQL.scala
new file mode 100644
index 0000000..a8b8268
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/WordCountSQL.scala
@@ -0,0 +1,62 @@
+/*
+ * 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.examples.scala
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+
+/**
+  * Simple example that shows how the Batch SQL API is used in Scala.
+  *
+  * This example shows how to:
+  *  - Convert DataSets to Tables
+  *  - Register a Table under a name
+  *  - Run a SQL query on the registered Table
+  *
+  */
+object WordCountSQL {
+
+  // *************************************************************************
+  //     PROGRAM
+  // *************************************************************************
+
+  def main(args: Array[String]): Unit = {
+
+    // set up execution environment
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val input = env.fromElements(WC("hello", 1), WC("hello", 1), WC("ciao", 1))
+
+    // register the DataSet as table "WordCount"
+    tEnv.registerDataSet("WordCount", input, 'word, 'frequency)
+
+    // run a SQL query on the Table and retrieve the result as a new Table
+    val table = tEnv.sql("SELECT word, SUM(frequency) FROM WordCount GROUP BY word")
+
+    table.toDataSet[WC].print()
+  }
+
+  // *************************************************************************
+  //     USER DATA TYPES
+  // *************************************************************************
+
+  case class WC(word: String, frequency: Long)
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/WordCountTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/WordCountTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/WordCountTable.scala
new file mode 100644
index 0000000..75ea8ce
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/examples/scala/WordCountTable.scala
@@ -0,0 +1,62 @@
+/*
+ * 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.examples.scala
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+
+/**
+  * Simple example for demonstrating the use of the Table API for a Word Count in Scala.
+  *
+  * This example shows how to:
+  *  - Convert DataSets to Tables
+  *  - Apply group, aggregate, select, and filter operations
+  *
+  */
+object WordCountTable {
+
+  // *************************************************************************
+  //     PROGRAM
+  // *************************************************************************
+
+  def main(args: Array[String]): Unit = {
+
+    // set up execution environment
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val input = env.fromElements(WC("hello", 1), WC("hello", 1), WC("ciao", 1))
+    val expr = input.toTable(tEnv)
+    val result = expr
+      .groupBy('word)
+      .select('word, 'frequency.sum as 'frequency)
+      .filter('frequency === 2)
+      .toDataSet[WC]
+
+    result.print()
+  }
+
+  // *************************************************************************
+  //     USER DATA TYPES
+  // *************************************************************************
+
+  case class WC(word: String, frequency: Long)
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/Expression.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/Expression.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/Expression.scala
new file mode 100644
index 0000000..14d899d
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/Expression.scala
@@ -0,0 +1,88 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.tools.RelBuilder
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.plan.TreeNode
+import org.apache.flink.table.validate.{ValidationResult, ValidationSuccess}
+
+abstract class Expression extends TreeNode[Expression] {
+  /**
+    * Returns the [[TypeInformation]] for evaluating this expression.
+    * It is sometimes not available until the expression is valid.
+    */
+  private[flink] def resultType: TypeInformation[_]
+
+  /**
+    * One pass validation of the expression tree in post order.
+    */
+  private[flink] lazy val valid: Boolean = childrenValid && validateInput().isSuccess
+
+  private[flink] def childrenValid: Boolean = children.forall(_.valid)
+
+  /**
+    * Check input data types, inputs number or other properties specified by this expression.
+    * Return `ValidationSuccess` if it pass the check,
+    * or `ValidationFailure` with supplement message explaining the error.
+    * Note: we should only call this method until `childrenValid == true`
+    */
+  private[flink] def validateInput(): ValidationResult = ValidationSuccess
+
+  /**
+    * Convert Expression to its counterpart in Calcite, i.e. RexNode
+    */
+  private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode =
+    throw new UnsupportedOperationException(
+      s"${this.getClass.getName} cannot be transformed to RexNode"
+    )
+
+  private[flink] def checkEquals(other: Expression): Boolean = {
+    if (this.getClass != other.getClass) {
+      false
+    } else {
+      def checkEquality(elements1: Seq[Any], elements2: Seq[Any]): Boolean = {
+        elements1.length == elements2.length && elements1.zip(elements2).forall {
+          case (e1: Expression, e2: Expression) => e1.checkEquals(e2)
+          case (t1: Seq[_], t2: Seq[_]) => checkEquality(t1, t2)
+          case (i1, i2) => i1 == i2
+        }
+      }
+      val elements1 = this.productIterator.toSeq
+      val elements2 = other.productIterator.toSeq
+      checkEquality(elements1, elements2)
+    }
+  }
+}
+
+abstract class BinaryExpression extends Expression {
+  private[flink] def left: Expression
+  private[flink] def right: Expression
+  private[flink] def children = Seq(left, right)
+}
+
+abstract class UnaryExpression extends Expression {
+  private[flink] def child: Expression
+  private[flink] def children = Seq(child)
+}
+
+abstract class LeafExpression extends Expression {
+  private[flink] val children = Nil
+}


[26/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala
new file mode 100644
index 0000000..9c6eea8
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkPlannerImpl.scala
@@ -0,0 +1,178 @@
+/*
+ * 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.util
+
+import com.google.common.collect.ImmutableList
+import org.apache.calcite.jdbc.CalciteSchema
+import org.apache.calcite.plan.RelOptTable.ViewExpander
+import org.apache.calcite.plan._
+import org.apache.calcite.prepare.CalciteCatalogReader
+import org.apache.calcite.rel.RelRoot
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex.RexBuilder
+import org.apache.calcite.schema.SchemaPlus
+import org.apache.calcite.sql.parser.{SqlParser, SqlParseException => CSqlParseException}
+import org.apache.calcite.sql.validate.SqlValidator
+import org.apache.calcite.sql.{SqlNode, SqlOperatorTable}
+import org.apache.calcite.sql2rel.{RelDecorrelator, SqlRexConvertletTable, SqlToRelConverter}
+import org.apache.calcite.tools.{FrameworkConfig, RelConversionException}
+import org.apache.flink.table.api.{SqlParserException, TableException, ValidationException}
+
+import scala.collection.JavaConversions._
+
+/**
+  * NOTE: this is heavily inspired by Calcite's PlannerImpl.
+  * We need it in order to share the planner between the Table API relational plans
+  * and the SQL relation plans that are created by the Calcite parser.
+  * The main difference is that we do not create a new RelOptPlanner in the ready() method.
+  */
+class FlinkPlannerImpl(
+    config: FrameworkConfig,
+    planner: RelOptPlanner,
+    typeFactory: FlinkTypeFactory) {
+
+  val operatorTable: SqlOperatorTable = config.getOperatorTable
+  /** Holds the trait definitions to be registered with planner. May be null. */
+  val traitDefs: ImmutableList[RelTraitDef[_ <: RelTrait]] = config.getTraitDefs
+  val parserConfig: SqlParser.Config = config.getParserConfig
+  val convertletTable: SqlRexConvertletTable = config.getConvertletTable
+  val defaultSchema: SchemaPlus = config.getDefaultSchema
+
+  var validator: FlinkCalciteSqlValidator = _
+  var validatedSqlNode: SqlNode = _
+  var root: RelRoot = _
+
+  private def ready() {
+    if (this.traitDefs != null) {
+      planner.clearRelTraitDefs()
+      for (traitDef <- this.traitDefs) {
+        planner.addRelTraitDef(traitDef)
+      }
+    }
+  }
+
+  def parse(sql: String): SqlNode = {
+    try {
+      ready()
+      val parser: SqlParser = SqlParser.create(sql, parserConfig)
+      val sqlNode: SqlNode = parser.parseStmt
+      sqlNode
+    } catch {
+      case e: CSqlParseException =>
+        throw SqlParserException(s"SQL parse failed. ${e.getMessage}", e)
+    }
+  }
+
+  def validate(sqlNode: SqlNode): SqlNode = {
+    validator = new FlinkCalciteSqlValidator(operatorTable, createCatalogReader, typeFactory)
+    validator.setIdentifierExpansion(true)
+    try {
+      validatedSqlNode = validator.validate(sqlNode)
+    }
+    catch {
+      case e: RuntimeException =>
+        throw new ValidationException(s"SQL validation failed. ${e.getMessage}", e)
+    }
+    validatedSqlNode
+  }
+
+  def rel(sql: SqlNode): RelRoot = {
+    try {
+      assert(validatedSqlNode != null)
+      val rexBuilder: RexBuilder = createRexBuilder
+      val cluster: RelOptCluster = RelOptCluster.create(planner, rexBuilder)
+      val config = SqlToRelConverter.configBuilder()
+        .withTrimUnusedFields(false).withConvertTableAccess(false).build()
+      val sqlToRelConverter: SqlToRelConverter = new SqlToRelConverter(
+        new ViewExpanderImpl, validator, createCatalogReader, cluster, convertletTable, config)
+      root = sqlToRelConverter.convertQuery(validatedSqlNode, false, true)
+      // we disable automatic flattening in order to let composite types pass without modification
+      // we might enable it again once Calcite has better support for structured types
+      // root = root.withRel(sqlToRelConverter.flattenTypes(root.rel, true))
+      root = root.withRel(RelDecorrelator.decorrelateQuery(root.rel))
+      root
+    } catch {
+      case e: RelConversionException => throw TableException(e.getMessage)
+    }
+  }
+
+  /** Implements [[org.apache.calcite.plan.RelOptTable.ViewExpander]]
+    * interface for [[org.apache.calcite.tools.Planner]]. */
+  class ViewExpanderImpl extends ViewExpander {
+
+    override def expandView(
+        rowType: RelDataType,
+        queryString: String,
+        schemaPath: util.List[String],
+        viewPath: util.List[String]): RelRoot = {
+
+      val parser: SqlParser = SqlParser.create(queryString, parserConfig)
+      var sqlNode: SqlNode = null
+      try {
+        sqlNode = parser.parseQuery
+      }
+      catch {
+        case e: CSqlParseException =>
+          throw SqlParserException(s"SQL parse failed. ${e.getMessage}", e)
+      }
+      val catalogReader: CalciteCatalogReader = createCatalogReader.withSchemaPath(schemaPath)
+      val validator: SqlValidator =
+        new FlinkCalciteSqlValidator(operatorTable, catalogReader, typeFactory)
+      validator.setIdentifierExpansion(true)
+      val validatedSqlNode: SqlNode = validator.validate(sqlNode)
+      val rexBuilder: RexBuilder = createRexBuilder
+      val cluster: RelOptCluster = RelOptCluster.create(planner, rexBuilder)
+      val config: SqlToRelConverter.Config = SqlToRelConverter.configBuilder
+        .withTrimUnusedFields(false).withConvertTableAccess(false).build
+      val sqlToRelConverter: SqlToRelConverter = new SqlToRelConverter(
+        new ViewExpanderImpl, validator, catalogReader, cluster, convertletTable, config)
+      root = sqlToRelConverter.convertQuery(validatedSqlNode, true, false)
+      root = root.withRel(sqlToRelConverter.flattenTypes(root.rel, true))
+      root = root.withRel(RelDecorrelator.decorrelateQuery(root.rel))
+      FlinkPlannerImpl.this.root
+    }
+  }
+
+  private def createCatalogReader: CalciteCatalogReader = {
+    val rootSchema: SchemaPlus = FlinkPlannerImpl.rootSchema(defaultSchema)
+    new CalciteCatalogReader(
+      CalciteSchema.from(rootSchema),
+      parserConfig.caseSensitive,
+      CalciteSchema.from(defaultSchema).path(null),
+      typeFactory)
+  }
+
+  private def createRexBuilder: RexBuilder = {
+    new RexBuilder(typeFactory)
+  }
+
+}
+
+object FlinkPlannerImpl {
+  private def rootSchema(schema: SchemaPlus): SchemaPlus = {
+    if (schema.getParentSchema == null) {
+      schema
+    }
+    else {
+      rootSchema(schema.getParentSchema)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala
new file mode 100644
index 0000000..8465ec6
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkRelBuilder.scala
@@ -0,0 +1,103 @@
+/*
+ * 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.util.Collections
+
+import org.apache.calcite.plan.volcano.VolcanoPlanner
+import java.lang.Iterable
+
+import org.apache.calcite.jdbc.CalciteSchema
+import org.apache.calcite.plan._
+import org.apache.calcite.prepare.CalciteCatalogReader
+import org.apache.calcite.rel.logical.LogicalAggregate
+import org.apache.calcite.rex.RexBuilder
+import org.apache.calcite.tools.RelBuilder.{AggCall, GroupKey}
+import org.apache.calcite.tools.{FrameworkConfig, RelBuilder}
+import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.expressions.WindowProperty
+import org.apache.flink.table.plan.logical.LogicalWindow
+import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate
+
+/**
+  * Flink specific [[RelBuilder]] that changes the default type factory to a [[FlinkTypeFactory]].
+  */
+class FlinkRelBuilder(
+    context: Context,
+    relOptCluster: RelOptCluster,
+    relOptSchema: RelOptSchema)
+  extends RelBuilder(
+    context,
+    relOptCluster,
+    relOptSchema) {
+
+  def getPlanner: RelOptPlanner = cluster.getPlanner
+
+  def getCluster: RelOptCluster = relOptCluster
+
+  override def getTypeFactory: FlinkTypeFactory =
+    super.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+
+  def aggregate(
+      window: LogicalWindow,
+      groupKey: GroupKey,
+      namedProperties: Seq[NamedWindowProperty],
+      aggCalls: Iterable[AggCall])
+    : RelBuilder = {
+    // build logical aggregate
+    val aggregate = super.aggregate(groupKey, aggCalls).build().asInstanceOf[LogicalAggregate]
+
+    // build logical window aggregate from it
+    push(LogicalWindowAggregate.create(window, namedProperties, aggregate))
+    this
+  }
+
+}
+
+object FlinkRelBuilder {
+
+  def create(config: FrameworkConfig): FlinkRelBuilder = {
+
+    // create Flink type factory
+    val typeSystem = config.getTypeSystem
+    val typeFactory = new FlinkTypeFactory(typeSystem)
+
+    // create context instances with Flink type factory
+    val planner = new VolcanoPlanner(config.getCostFactory, Contexts.empty())
+    planner.setExecutor(config.getExecutor)
+    planner.addRelTraitDef(ConventionTraitDef.INSTANCE)
+    val cluster = RelOptCluster.create(planner, new RexBuilder(typeFactory))
+    val calciteSchema = CalciteSchema.from(config.getDefaultSchema)
+    val relOptSchema = new CalciteCatalogReader(
+      calciteSchema,
+      config.getParserConfig.caseSensitive(),
+      Collections.emptyList(),
+      typeFactory)
+
+    new FlinkRelBuilder(config.getContext, cluster, relOptSchema)
+  }
+
+  /**
+    * Information necessary to create a window aggregate.
+    *
+    * Similar to [[RelBuilder.AggCall]] or [[RelBuilder.GroupKey]].
+    */
+  case class NamedWindowProperty(name: String, property: WindowProperty)
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
new file mode 100644
index 0000000..f3e2f91
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeFactory.scala
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.calcite
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeSystem}
+import org.apache.calcite.sql.SqlIntervalQualifier
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{NothingTypeInfo, PrimitiveArrayTypeInfo, SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
+import org.apache.flink.api.java.typeutils.ValueTypeInfo._
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.plan.schema.{CompositeRelDataType, GenericRelDataType}
+import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
+import org.apache.flink.table.typeutils.TypeCheckUtils.isSimple
+import org.apache.flink.table.plan.schema.ArrayRelDataType
+import org.apache.flink.table.calcite.FlinkTypeFactory.typeInfoToSqlTypeName
+
+import scala.collection.mutable
+
+/**
+  * Flink specific type factory that represents the interface between Flink's [[TypeInformation]]
+  * and Calcite's [[RelDataType]].
+  */
+class FlinkTypeFactory(typeSystem: RelDataTypeSystem) extends JavaTypeFactoryImpl(typeSystem) {
+
+  // NOTE: for future data types it might be necessary to
+  // override more methods of RelDataTypeFactoryImpl
+
+  private val seenTypes = mutable.HashMap[TypeInformation[_], RelDataType]()
+
+  def createTypeFromTypeInfo(typeInfo: TypeInformation[_]): RelDataType = {
+    // simple type can be converted to SQL types and vice versa
+    if (isSimple(typeInfo)) {
+      val sqlType = typeInfoToSqlTypeName(typeInfo)
+      sqlType match {
+
+        case INTERVAL_YEAR_MONTH =>
+          createSqlIntervalType(
+            new SqlIntervalQualifier(TimeUnit.YEAR, TimeUnit.MONTH, SqlParserPos.ZERO))
+
+        case INTERVAL_DAY_SECOND =>
+          createSqlIntervalType(
+            new SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.SECOND, SqlParserPos.ZERO))
+
+        case _ =>
+          createSqlType(sqlType)
+      }
+    }
+    // advanced types require specific RelDataType
+    // for storing the original TypeInformation
+    else {
+      seenTypes.getOrElseUpdate(typeInfo, canonize(createAdvancedType(typeInfo)))
+    }
+  }
+
+  /**
+    * Creates a struct type with the input fieldNames and input fieldTypes using FlinkTypeFactory
+    *
+    * @param fieldNames field names
+    * @param fieldTypes field types, every element is Flink's [[TypeInformation]]
+    * @return a struct type with the input fieldNames and input fieldTypes
+    */
+  def buildRowDataType(
+      fieldNames: Array[String],
+      fieldTypes: Array[TypeInformation[_]])
+    : RelDataType = {
+    val rowDataTypeBuilder = builder
+    fieldNames
+      .zip(fieldTypes)
+      .foreach { f =>
+        rowDataTypeBuilder.add(f._1, createTypeFromTypeInfo(f._2)).nullable(true)
+      }
+    rowDataTypeBuilder.build
+  }
+
+  override def createSqlType(typeName: SqlTypeName, precision: Int): RelDataType = {
+    // it might happen that inferred VARCHAR types overflow as we set them to Int.MaxValue
+    // always set those to default value
+    if (typeName == VARCHAR && precision < 0) {
+      createSqlType(typeName, getTypeSystem.getDefaultPrecision(typeName))
+    } else {
+      super.createSqlType(typeName, precision)
+    }
+  }
+
+  override def createArrayType(elementType: RelDataType, maxCardinality: Long): RelDataType =
+    new ArrayRelDataType(
+      ObjectArrayTypeInfo.getInfoFor(FlinkTypeFactory.toTypeInfo(elementType)),
+      elementType,
+      true)
+
+  private def createAdvancedType(typeInfo: TypeInformation[_]): RelDataType = typeInfo match {
+    case ct: CompositeType[_] =>
+      new CompositeRelDataType(ct, this)
+
+    case pa: PrimitiveArrayTypeInfo[_] =>
+      new ArrayRelDataType(pa, createTypeFromTypeInfo(pa.getComponentType), false)
+
+    case oa: ObjectArrayTypeInfo[_, _] =>
+      new ArrayRelDataType(oa, createTypeFromTypeInfo(oa.getComponentInfo), true)
+
+    case ti: TypeInformation[_] =>
+      new GenericRelDataType(typeInfo, getTypeSystem.asInstanceOf[FlinkTypeSystem])
+
+    case ti@_ =>
+      throw TableException(s"Unsupported type information: $ti")
+  }
+
+  override def createTypeWithNullability(
+    relDataType: RelDataType,
+    nullable: Boolean)
+  : RelDataType = relDataType match {
+    case composite: CompositeRelDataType =>
+      // at the moment we do not care about nullability
+      composite
+    case _ =>
+      super.createTypeWithNullability(relDataType, nullable)
+  }
+}
+
+object FlinkTypeFactory {
+
+  private def typeInfoToSqlTypeName(typeInfo: TypeInformation[_]): SqlTypeName = typeInfo match {
+      case BOOLEAN_TYPE_INFO => BOOLEAN
+      case BYTE_TYPE_INFO => TINYINT
+      case SHORT_TYPE_INFO => SMALLINT
+      case INT_TYPE_INFO => INTEGER
+      case LONG_TYPE_INFO => BIGINT
+      case FLOAT_TYPE_INFO => FLOAT
+      case DOUBLE_TYPE_INFO => DOUBLE
+      case STRING_TYPE_INFO => VARCHAR
+      case BIG_DEC_TYPE_INFO => DECIMAL
+
+      // temporal types
+      case SqlTimeTypeInfo.DATE => DATE
+      case SqlTimeTypeInfo.TIME => TIME
+      case SqlTimeTypeInfo.TIMESTAMP => TIMESTAMP
+      case TimeIntervalTypeInfo.INTERVAL_MONTHS => INTERVAL_YEAR_MONTH
+      case TimeIntervalTypeInfo.INTERVAL_MILLIS => INTERVAL_DAY_SECOND
+
+      case CHAR_TYPE_INFO | CHAR_VALUE_TYPE_INFO =>
+        throw TableException("Character type is not supported.")
+
+      case _@t =>
+        throw TableException(s"Type is not supported: $t")
+  }
+
+  def toTypeInfo(relDataType: RelDataType): TypeInformation[_] = relDataType.getSqlTypeName match {
+    case BOOLEAN => BOOLEAN_TYPE_INFO
+    case TINYINT => BYTE_TYPE_INFO
+    case SMALLINT => SHORT_TYPE_INFO
+    case INTEGER => INT_TYPE_INFO
+    case BIGINT => LONG_TYPE_INFO
+    case FLOAT => FLOAT_TYPE_INFO
+    case DOUBLE => DOUBLE_TYPE_INFO
+    case VARCHAR | CHAR => STRING_TYPE_INFO
+    case DECIMAL => BIG_DEC_TYPE_INFO
+
+    // temporal types
+    case DATE => SqlTimeTypeInfo.DATE
+    case TIME => SqlTimeTypeInfo.TIME
+    case TIMESTAMP => SqlTimeTypeInfo.TIMESTAMP
+    case typeName if YEAR_INTERVAL_TYPES.contains(typeName) => TimeIntervalTypeInfo.INTERVAL_MONTHS
+    case typeName if DAY_INTERVAL_TYPES.contains(typeName) => TimeIntervalTypeInfo.INTERVAL_MILLIS
+
+    case NULL =>
+      throw TableException("Type NULL is not supported. Null values must have a supported type.")
+
+    // symbol for special flags e.g. TRIM's BOTH, LEADING, TRAILING
+    // are represented as integer
+    case SYMBOL => INT_TYPE_INFO
+
+    // extract encapsulated TypeInformation
+    case ANY if relDataType.isInstanceOf[GenericRelDataType] =>
+      val genericRelDataType = relDataType.asInstanceOf[GenericRelDataType]
+      genericRelDataType.typeInfo
+
+    case ROW if relDataType.isInstanceOf[CompositeRelDataType] =>
+      val compositeRelDataType = relDataType.asInstanceOf[CompositeRelDataType]
+      compositeRelDataType.compositeType
+
+    // ROW and CURSOR for UDTF case, whose type info will never be used, just a placeholder
+    case ROW | CURSOR => new NothingTypeInfo
+
+    case ARRAY if relDataType.isInstanceOf[ArrayRelDataType] =>
+      val arrayRelDataType = relDataType.asInstanceOf[ArrayRelDataType]
+      arrayRelDataType.typeInfo
+
+    case _@t =>
+      throw TableException(s"Type is not supported: $t")
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala
new file mode 100644
index 0000000..5935297
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkTypeSystem.scala
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.calcite
+
+import org.apache.calcite.rel.`type`.RelDataTypeSystemImpl
+import org.apache.calcite.sql.`type`.SqlTypeName
+
+/**
+  * Custom type system for Flink.
+  */
+class FlinkTypeSystem extends RelDataTypeSystemImpl {
+
+  // we cannot use Int.MaxValue because of an overflow in Calcites type inference logic
+  // half should be enough for all use cases
+  override def getMaxNumericScale: Int = Int.MaxValue / 2
+
+  // we cannot use Int.MaxValue because of an overflow in Calcites type inference logic
+  // half should be enough for all use cases
+  override def getMaxNumericPrecision: Int = Int.MaxValue / 2
+
+  override def getDefaultPrecision(typeName: SqlTypeName): Int = typeName match {
+
+    // by default all VARCHARs can have the Java default length
+    case SqlTypeName.VARCHAR =>
+      Int.MaxValue
+
+    // we currenty support only timestamps with milliseconds precision
+    case SqlTypeName.TIMESTAMP =>
+      3
+
+    case _ =>
+      super.getDefaultPrecision(typeName)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenException.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenException.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenException.scala
new file mode 100644
index 0000000..1f2e9a9
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenException.scala
@@ -0,0 +1,24 @@
+/*
+ * 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.codegen
+
+/**
+  * Exception for all errors occurring during code generation.
+  */
+class CodeGenException(msg: String) extends RuntimeException(msg)

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala
new file mode 100644
index 0000000..f8885a2
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenUtils.scala
@@ -0,0 +1,292 @@
+/*
+ * 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.codegen
+
+import java.lang.reflect.{Field, Method}
+import java.util.concurrent.atomic.AtomicInteger
+
+import org.apache.calcite.util.BuiltInMethod
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo._
+import org.apache.flink.api.common.typeinfo.{FractionalTypeInfo, SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo, TypeExtractor}
+import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
+import org.apache.flink.table.typeutils.{TimeIntervalTypeInfo, TypeCheckUtils}
+
+object CodeGenUtils {
+
+  private val nameCounter = new AtomicInteger
+
+  def newName(name: String): String = {
+    s"$name$$${nameCounter.getAndIncrement}"
+  }
+
+  // when casting we first need to unbox Primitives, for example,
+  // float a = 1.0f;
+  // byte b = (byte) a;
+  // works, but for boxed types we need this:
+  // Float a = 1.0f;
+  // Byte b = (byte)(float) a;
+  def primitiveTypeTermForTypeInfo(tpe: TypeInformation[_]): String = tpe match {
+    case INT_TYPE_INFO => "int"
+    case LONG_TYPE_INFO => "long"
+    case SHORT_TYPE_INFO => "short"
+    case BYTE_TYPE_INFO => "byte"
+    case FLOAT_TYPE_INFO => "float"
+    case DOUBLE_TYPE_INFO => "double"
+    case BOOLEAN_TYPE_INFO => "boolean"
+    case CHAR_TYPE_INFO => "char"
+
+    // From PrimitiveArrayTypeInfo we would get class "int[]", scala reflections
+    // does not seem to like this, so we manually give the correct type here.
+    case INT_PRIMITIVE_ARRAY_TYPE_INFO => "int[]"
+    case LONG_PRIMITIVE_ARRAY_TYPE_INFO => "long[]"
+    case SHORT_PRIMITIVE_ARRAY_TYPE_INFO => "short[]"
+    case BYTE_PRIMITIVE_ARRAY_TYPE_INFO => "byte[]"
+    case FLOAT_PRIMITIVE_ARRAY_TYPE_INFO => "float[]"
+    case DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO => "double[]"
+    case BOOLEAN_PRIMITIVE_ARRAY_TYPE_INFO => "boolean[]"
+    case CHAR_PRIMITIVE_ARRAY_TYPE_INFO => "char[]"
+
+    // internal primitive representation of time points
+    case SqlTimeTypeInfo.DATE => "int"
+    case SqlTimeTypeInfo.TIME => "int"
+    case SqlTimeTypeInfo.TIMESTAMP => "long"
+
+    // internal primitive representation of time intervals
+    case TimeIntervalTypeInfo.INTERVAL_MONTHS => "int"
+    case TimeIntervalTypeInfo.INTERVAL_MILLIS => "long"
+
+    case _ =>
+      tpe.getTypeClass.getCanonicalName
+  }
+
+  def boxedTypeTermForTypeInfo(tpe: TypeInformation[_]): String = tpe match {
+    // From PrimitiveArrayTypeInfo we would get class "int[]", scala reflections
+    // does not seem to like this, so we manually give the correct type here.
+    case INT_PRIMITIVE_ARRAY_TYPE_INFO => "int[]"
+    case LONG_PRIMITIVE_ARRAY_TYPE_INFO => "long[]"
+    case SHORT_PRIMITIVE_ARRAY_TYPE_INFO => "short[]"
+    case BYTE_PRIMITIVE_ARRAY_TYPE_INFO => "byte[]"
+    case FLOAT_PRIMITIVE_ARRAY_TYPE_INFO => "float[]"
+    case DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO => "double[]"
+    case BOOLEAN_PRIMITIVE_ARRAY_TYPE_INFO => "boolean[]"
+    case CHAR_PRIMITIVE_ARRAY_TYPE_INFO => "char[]"
+
+    case _ =>
+      tpe.getTypeClass.getCanonicalName
+  }
+
+  def primitiveDefaultValue(tpe: TypeInformation[_]): String = tpe match {
+    case INT_TYPE_INFO => "-1"
+    case LONG_TYPE_INFO => "-1L"
+    case SHORT_TYPE_INFO => "-1"
+    case BYTE_TYPE_INFO => "-1"
+    case FLOAT_TYPE_INFO => "-1.0f"
+    case DOUBLE_TYPE_INFO => "-1.0d"
+    case BOOLEAN_TYPE_INFO => "false"
+    case STRING_TYPE_INFO => "\"\""
+    case CHAR_TYPE_INFO => "'\\0'"
+    case SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIME => "-1"
+    case SqlTimeTypeInfo.TIMESTAMP => "-1L"
+    case TimeIntervalTypeInfo.INTERVAL_MONTHS => "-1"
+    case TimeIntervalTypeInfo.INTERVAL_MILLIS => "-1L"
+
+    case _ => "null"
+  }
+
+  def superPrimitive(typeInfo: TypeInformation[_]): String = typeInfo match {
+    case _: FractionalTypeInfo[_] => "double"
+    case _ => "long"
+  }
+
+  def qualifyMethod(method: Method): String =
+    method.getDeclaringClass.getCanonicalName + "." + method.getName
+
+  def qualifyEnum(enum: Enum[_]): String =
+    enum.getClass.getCanonicalName + "." + enum.name()
+
+  def internalToTimePointCode(resultType: TypeInformation[_], resultTerm: String) =
+    resultType match {
+      case SqlTimeTypeInfo.DATE =>
+        s"${qualifyMethod(BuiltInMethod.INTERNAL_TO_DATE.method)}($resultTerm)"
+      case SqlTimeTypeInfo.TIME =>
+        s"${qualifyMethod(BuiltInMethod.INTERNAL_TO_TIME.method)}($resultTerm)"
+      case SqlTimeTypeInfo.TIMESTAMP =>
+        s"${qualifyMethod(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method)}($resultTerm)"
+    }
+
+  def timePointToInternalCode(resultType: TypeInformation[_], resultTerm: String) =
+    resultType match {
+      case SqlTimeTypeInfo.DATE =>
+        s"${qualifyMethod(BuiltInMethod.DATE_TO_INT.method)}($resultTerm)"
+      case SqlTimeTypeInfo.TIME =>
+        s"${qualifyMethod(BuiltInMethod.TIME_TO_INT.method)}($resultTerm)"
+      case SqlTimeTypeInfo.TIMESTAMP =>
+        s"${qualifyMethod(BuiltInMethod.TIMESTAMP_TO_LONG.method)}($resultTerm)"
+    }
+
+  def compareEnum(term: String, enum: Enum[_]): Boolean = term == qualifyEnum(enum)
+
+  def getEnum(genExpr: GeneratedExpression): Enum[_] = {
+    val split = genExpr.resultTerm.split('.')
+    val value = split.last
+    val clazz = genExpr.resultType.getTypeClass
+    enumValueOf(clazz, value)
+  }
+
+  def enumValueOf[T <: Enum[T]](cls: Class[_], stringValue: String): Enum[_] =
+    Enum.valueOf(cls.asInstanceOf[Class[T]], stringValue).asInstanceOf[Enum[_]]
+
+  // ----------------------------------------------------------------------------------------------
+
+  def requireNumeric(genExpr: GeneratedExpression) =
+    if (!TypeCheckUtils.isNumeric(genExpr.resultType)) {
+      throw new CodeGenException("Numeric expression type expected, but was " +
+        s"'${genExpr.resultType}'.")
+    }
+
+  def requireComparable(genExpr: GeneratedExpression) =
+    if (!TypeCheckUtils.isComparable(genExpr.resultType)) {
+      throw new CodeGenException(s"Comparable type expected, but was '${genExpr.resultType}'.")
+    }
+
+  def requireString(genExpr: GeneratedExpression) =
+    if (!TypeCheckUtils.isString(genExpr.resultType)) {
+      throw new CodeGenException("String expression type expected.")
+    }
+
+  def requireBoolean(genExpr: GeneratedExpression) =
+    if (!TypeCheckUtils.isBoolean(genExpr.resultType)) {
+      throw new CodeGenException("Boolean expression type expected.")
+    }
+
+  def requireTemporal(genExpr: GeneratedExpression) =
+    if (!TypeCheckUtils.isTemporal(genExpr.resultType)) {
+      throw new CodeGenException("Temporal expression type expected.")
+    }
+
+  def requireTimeInterval(genExpr: GeneratedExpression) =
+    if (!TypeCheckUtils.isTimeInterval(genExpr.resultType)) {
+      throw new CodeGenException("Interval expression type expected.")
+    }
+
+  def requireArray(genExpr: GeneratedExpression) =
+    if (!TypeCheckUtils.isArray(genExpr.resultType)) {
+      throw new CodeGenException("Array expression type expected.")
+    }
+
+  def requireInteger(genExpr: GeneratedExpression) =
+    if (!TypeCheckUtils.isInteger(genExpr.resultType)) {
+      throw new CodeGenException("Integer expression type expected.")
+    }
+
+  // ----------------------------------------------------------------------------------------------
+
+  def isReference(genExpr: GeneratedExpression): Boolean = isReference(genExpr.resultType)
+
+  def isReference(typeInfo: TypeInformation[_]): Boolean = typeInfo match {
+    case INT_TYPE_INFO
+         | LONG_TYPE_INFO
+         | SHORT_TYPE_INFO
+         | BYTE_TYPE_INFO
+         | FLOAT_TYPE_INFO
+         | DOUBLE_TYPE_INFO
+         | BOOLEAN_TYPE_INFO
+         | CHAR_TYPE_INFO => false
+    case _ => true
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  sealed abstract class FieldAccessor
+
+  case class ObjectFieldAccessor(field: Field) extends FieldAccessor
+
+  case class ObjectGenericFieldAccessor(name: String) extends FieldAccessor
+
+  case class ObjectPrivateFieldAccessor(field: Field) extends FieldAccessor
+
+  case class ObjectMethodAccessor(methodName: String) extends FieldAccessor
+
+  case class ProductAccessor(i: Int) extends FieldAccessor
+
+  def fieldAccessorFor(compType: CompositeType[_], index: Int): FieldAccessor = {
+    compType match {
+      case ri: RowTypeInfo =>
+        ProductAccessor(index)
+
+      case cc: CaseClassTypeInfo[_] =>
+        ObjectMethodAccessor(cc.getFieldNames()(index))
+
+      case javaTup: TupleTypeInfo[_] =>
+        ObjectGenericFieldAccessor("f" + index)
+
+      case pt: PojoTypeInfo[_] =>
+        val fieldName = pt.getFieldNames()(index)
+        getFieldAccessor(pt.getTypeClass, fieldName)
+
+      case _ => throw new CodeGenException("Unsupported composite type.")
+    }
+  }
+
+  def getFieldAccessor(clazz: Class[_], fieldName: String): FieldAccessor = {
+    val field = TypeExtractor.getDeclaredField(clazz, fieldName)
+    if (field.isAccessible) {
+      ObjectFieldAccessor(field)
+    }
+    else {
+      ObjectPrivateFieldAccessor(field)
+    }
+  }
+
+  def isFieldPrimitive(field: Field): Boolean = field.getType.isPrimitive
+
+  def reflectiveFieldReadAccess(fieldTerm: String, field: Field, objectTerm: String): String =
+    field.getType match {
+      case java.lang.Integer.TYPE => s"$fieldTerm.getInt($objectTerm)"
+      case java.lang.Long.TYPE => s"$fieldTerm.getLong($objectTerm)"
+      case java.lang.Short.TYPE => s"$fieldTerm.getShort($objectTerm)"
+      case java.lang.Byte.TYPE => s"$fieldTerm.getByte($objectTerm)"
+      case java.lang.Float.TYPE => s"$fieldTerm.getFloat($objectTerm)"
+      case java.lang.Double.TYPE => s"$fieldTerm.getDouble($objectTerm)"
+      case java.lang.Boolean.TYPE => s"$fieldTerm.getBoolean($objectTerm)"
+      case java.lang.Character.TYPE => s"$fieldTerm.getChar($objectTerm)"
+      case _ => s"(${field.getType.getCanonicalName}) $fieldTerm.get($objectTerm)"
+    }
+
+  def reflectiveFieldWriteAccess(
+      fieldTerm: String,
+      field: Field,
+      objectTerm: String,
+      valueTerm: String)
+    : String =
+    field.getType match {
+      case java.lang.Integer.TYPE => s"$fieldTerm.setInt($objectTerm, $valueTerm)"
+      case java.lang.Long.TYPE => s"$fieldTerm.setLong($objectTerm, $valueTerm)"
+      case java.lang.Short.TYPE => s"$fieldTerm.setShort($objectTerm, $valueTerm)"
+      case java.lang.Byte.TYPE => s"$fieldTerm.setByte($objectTerm, $valueTerm)"
+      case java.lang.Float.TYPE => s"$fieldTerm.setFloat($objectTerm, $valueTerm)"
+      case java.lang.Double.TYPE => s"$fieldTerm.setDouble($objectTerm, $valueTerm)"
+      case java.lang.Boolean.TYPE => s"$fieldTerm.setBoolean($objectTerm, $valueTerm)"
+      case java.lang.Character.TYPE => s"$fieldTerm.setChar($objectTerm, $valueTerm)"
+      case _ => s"$fieldTerm.set($objectTerm, $valueTerm)"
+    }
+}


[23/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala
new file mode 100644
index 0000000..48dbce6
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionParser.scala
@@ -0,0 +1,489 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.avatica.util.DateTimeUtils.{MILLIS_PER_DAY, MILLIS_PER_HOUR, MILLIS_PER_MINUTE, MILLIS_PER_SECOND}
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.table.api.ExpressionParserException
+import org.apache.flink.table.expressions.ExpressionUtils.{toMilliInterval, toMonthInterval}
+import org.apache.flink.table.expressions.TimeIntervalUnit.TimeIntervalUnit
+import org.apache.flink.table.expressions.TimePointUnit.TimePointUnit
+import org.apache.flink.table.expressions.TrimMode.TrimMode
+import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
+
+import scala.language.implicitConversions
+import scala.util.parsing.combinator.{JavaTokenParsers, PackratParsers}
+
+/**
+ * Parser for expressions inside a String. This parses exactly the same expressions that
+ * would be accepted by the Scala Expression DSL.
+ *
+ * See [[org.apache.flink.table.api.scala.ImplicitExpressionConversions]] and
+ * [[org.apache.flink.table.api.scala.ImplicitExpressionOperations]] for the constructs
+ * available in the Scala Expression DSL. This parser must be kept in sync with the Scala DSL
+ * lazy valined in the above files.
+ */
+object ExpressionParser extends JavaTokenParsers with PackratParsers {
+  case class Keyword(key: String)
+
+  // Convert the keyword into an case insensitive Parser
+  implicit def keyword2Parser(kw: Keyword): Parser[String] = {
+    ("""(?i)\Q""" + kw.key + """\E""").r
+  }
+
+  // Keyword
+
+  lazy val ARRAY: Keyword = Keyword("Array")
+  lazy val AS: Keyword = Keyword("as")
+  lazy val COUNT: Keyword = Keyword("count")
+  lazy val AVG: Keyword = Keyword("avg")
+  lazy val MIN: Keyword = Keyword("min")
+  lazy val MAX: Keyword = Keyword("max")
+  lazy val SUM: Keyword = Keyword("sum")
+  lazy val START: Keyword = Keyword("start")
+  lazy val END: Keyword = Keyword("end")
+  lazy val CAST: Keyword = Keyword("cast")
+  lazy val NULL: Keyword = Keyword("Null")
+  lazy val IF: Keyword = Keyword("?")
+  lazy val ASC: Keyword = Keyword("asc")
+  lazy val DESC: Keyword = Keyword("desc")
+  lazy val TO_DATE: Keyword = Keyword("toDate")
+  lazy val TO_TIME: Keyword = Keyword("toTime")
+  lazy val TO_TIMESTAMP: Keyword = Keyword("toTimestamp")
+  lazy val TRIM: Keyword = Keyword("trim")
+  lazy val EXTRACT: Keyword = Keyword("extract")
+  lazy val FLOOR: Keyword = Keyword("floor")
+  lazy val CEIL: Keyword = Keyword("ceil")
+  lazy val YEARS: Keyword = Keyword("years")
+  lazy val YEAR: Keyword = Keyword("year")
+  lazy val MONTHS: Keyword = Keyword("months")
+  lazy val MONTH: Keyword = Keyword("month")
+  lazy val DAYS: Keyword = Keyword("days")
+  lazy val DAY: Keyword = Keyword("day")
+  lazy val HOURS: Keyword = Keyword("hours")
+  lazy val HOUR: Keyword = Keyword("hour")
+  lazy val MINUTES: Keyword = Keyword("minutes")
+  lazy val MINUTE: Keyword = Keyword("minute")
+  lazy val SECONDS: Keyword = Keyword("seconds")
+  lazy val SECOND: Keyword = Keyword("second")
+  lazy val MILLIS: Keyword = Keyword("millis")
+  lazy val MILLI: Keyword = Keyword("milli")
+  lazy val ROWS: Keyword = Keyword("rows")
+  lazy val STAR: Keyword = Keyword("*")
+  lazy val GET: Keyword = Keyword("get")
+  lazy val FLATTEN: Keyword = Keyword("flatten")
+
+  def functionIdent: ExpressionParser.Parser[String] =
+    not(ARRAY) ~ not(AS) ~ not(COUNT) ~ not(AVG) ~ not(MIN) ~ not(MAX) ~
+      not(SUM) ~ not(START) ~ not(END)~ not(CAST) ~ not(NULL) ~
+      not(IF) ~> super.ident
+
+  // symbols
+
+  lazy val timeIntervalUnit: PackratParser[Expression] = TimeIntervalUnit.values map {
+    case unit: TimeIntervalUnit => literal(unit.toString) ^^^ unit.toExpr
+  } reduceLeft(_ | _)
+
+  lazy val timePointUnit: PackratParser[Expression] = TimePointUnit.values map {
+    case unit: TimePointUnit => literal(unit.toString) ^^^ unit.toExpr
+  } reduceLeft(_ | _)
+
+  lazy val trimMode: PackratParser[Expression] = TrimMode.values map {
+    case mode: TrimMode => literal(mode.toString) ^^^ mode.toExpr
+  } reduceLeft(_ | _)
+
+  // data types
+
+  lazy val dataType: PackratParser[TypeInformation[_]] =
+    "BYTE" ^^ { ti => BasicTypeInfo.BYTE_TYPE_INFO } |
+      "SHORT" ^^ { ti => BasicTypeInfo.SHORT_TYPE_INFO } |
+      "INTERVAL_MONTHS" ^^ {
+        ti => TimeIntervalTypeInfo.INTERVAL_MONTHS.asInstanceOf[TypeInformation[_]]
+      } |
+      "INTERVAL_MILLIS" ^^ {
+        ti => TimeIntervalTypeInfo.INTERVAL_MILLIS.asInstanceOf[TypeInformation[_]]
+      } |
+      "INT" ^^ { ti => BasicTypeInfo.INT_TYPE_INFO } |
+      "LONG" ^^ { ti => BasicTypeInfo.LONG_TYPE_INFO } |
+      "FLOAT" ^^ { ti => BasicTypeInfo.FLOAT_TYPE_INFO } |
+      "DOUBLE" ^^ { ti => BasicTypeInfo.DOUBLE_TYPE_INFO } |
+      ("BOOLEAN" | "BOOL") ^^ { ti => BasicTypeInfo.BOOLEAN_TYPE_INFO } |
+      "STRING" ^^ { ti => BasicTypeInfo.STRING_TYPE_INFO } |
+      "DATE" ^^ { ti => SqlTimeTypeInfo.DATE.asInstanceOf[TypeInformation[_]] } |
+      "TIMESTAMP" ^^ { ti => SqlTimeTypeInfo.TIMESTAMP } |
+      "TIME" ^^ { ti => SqlTimeTypeInfo.TIME } |
+      "DECIMAL" ^^ { ti => BasicTypeInfo.BIG_DEC_TYPE_INFO }
+
+  // Literals
+
+  // same as floatingPointNumber but we do not allow trailing dot "12.d" or "2."
+  lazy val floatingPointNumberFlink: Parser[String] =
+    """-?(\d+(\.\d+)?|\d*\.\d+)([eE][+-]?\d+)?[fFdD]?""".r
+
+  lazy val numberLiteral: PackratParser[Expression] =
+    (wholeNumber <~ ("l" | "L")) ^^ { n => Literal(n.toLong) } |
+      (decimalNumber <~ ("p" | "P")) ^^ { n => Literal(BigDecimal(n)) } |
+      (floatingPointNumberFlink | decimalNumber) ^^ {
+        n =>
+          if (n.matches("""-?\d+""")) {
+            Literal(n.toInt)
+          } else if (n.endsWith("f") || n.endsWith("F")) {
+            Literal(n.toFloat)
+          } else {
+            Literal(n.toDouble)
+          }
+      }
+
+  lazy val singleQuoteStringLiteral: Parser[Expression] =
+    ("'" + """([^'\p{Cntrl}\\]|\\[\\'"bfnrt]|\\u[a-fA-F0-9]{4})*""" + "'").r ^^ {
+      str => Literal(str.substring(1, str.length - 1))
+    }
+
+  lazy val stringLiteralFlink: PackratParser[Expression] = super.stringLiteral ^^ {
+    str => Literal(str.substring(1, str.length - 1))
+  }
+
+  lazy val boolLiteral: PackratParser[Expression] = ("true" | "false") ^^ {
+    str => Literal(str.toBoolean)
+  }
+
+  lazy val nullLiteral: PackratParser[Expression] = NULL ~ "(" ~> dataType <~ ")" ^^ {
+    dt => Null(dt)
+  }
+
+  lazy val literalExpr: PackratParser[Expression] =
+    numberLiteral | stringLiteralFlink | singleQuoteStringLiteral | boolLiteral | nullLiteral
+
+  lazy val fieldReference: PackratParser[NamedExpression] = (STAR | ident) ^^ {
+    sym => UnresolvedFieldReference(sym)
+  }
+
+  lazy val atom: PackratParser[Expression] =
+    ( "(" ~> expression <~ ")" ) | literalExpr | fieldReference
+
+  // suffix operators
+
+  lazy val suffixSum: PackratParser[Expression] =
+    composite <~ "." ~ SUM ~ opt("()") ^^ { e => Sum(e) }
+
+  lazy val suffixMin: PackratParser[Expression] =
+    composite <~ "." ~ MIN ~ opt("()") ^^ { e => Min(e) }
+
+  lazy val suffixMax: PackratParser[Expression] =
+    composite <~ "." ~ MAX ~ opt("()") ^^ { e => Max(e) }
+
+  lazy val suffixCount: PackratParser[Expression] =
+    composite <~ "." ~ COUNT ~ opt("()") ^^ { e => Count(e) }
+
+  lazy val suffixAvg: PackratParser[Expression] =
+    composite <~ "." ~ AVG ~ opt("()") ^^ { e => Avg(e) }
+
+  lazy val suffixStart: PackratParser[Expression] =
+    composite <~ "." ~ START ~ opt("()") ^^ { e => WindowStart(e) }
+
+  lazy val suffixEnd: PackratParser[Expression] =
+    composite <~ "." ~ END ~ opt("()") ^^ { e => WindowEnd(e) }
+
+  lazy val suffixCast: PackratParser[Expression] =
+    composite ~ "." ~ CAST ~ "(" ~ dataType ~ ")" ^^ {
+    case e ~ _ ~ _ ~ _ ~ dt ~ _ => Cast(e, dt)
+  }
+
+  lazy val suffixAs: PackratParser[Expression] =
+    composite ~ "." ~ AS ~ "(" ~ rep1sep(fieldReference, ",") ~ ")" ^^ {
+    case e ~ _ ~ _ ~ _ ~ target ~ _ => Alias(e, target.head.name, target.tail.map(_.name))
+  }
+
+  lazy val suffixTrim = composite ~ "." ~ TRIM ~ "(" ~ trimMode ~ "," ~ expression ~ ")" ^^ {
+    case operand ~ _ ~ _ ~ _ ~ mode ~ _ ~ trimCharacter ~ _ => Trim(mode, trimCharacter, operand)
+  }
+
+  lazy val suffixTrimWithoutArgs = composite <~ "." ~ TRIM ~ opt("()") ^^ {
+    e => Trim(TrimMode.BOTH, TrimConstants.TRIM_DEFAULT_CHAR, e)
+  }
+
+  lazy val suffixIf: PackratParser[Expression] =
+    composite ~ "." ~ IF ~ "(" ~ expression ~ "," ~ expression ~ ")" ^^ {
+    case condition ~ _ ~ _ ~ _ ~ ifTrue ~ _ ~ ifFalse ~ _ => If(condition, ifTrue, ifFalse)
+  }
+
+  lazy val suffixExtract = composite ~ "." ~ EXTRACT ~ "(" ~ timeIntervalUnit ~ ")" ^^ {
+    case operand ~ _  ~ _ ~ _ ~ unit ~ _ => Extract(unit, operand)
+  }
+
+  lazy val suffixFloor = composite ~ "." ~ FLOOR ~ "(" ~ timeIntervalUnit ~ ")" ^^ {
+    case operand ~ _  ~ _ ~ _ ~ unit ~ _ => TemporalFloor(unit, operand)
+  }
+
+  lazy val suffixCeil = composite ~ "." ~ CEIL ~ "(" ~ timeIntervalUnit ~ ")" ^^ {
+    case operand ~ _  ~ _ ~ _ ~ unit ~ _ => TemporalCeil(unit, operand)
+  }
+
+  lazy val suffixFunctionCall =
+    composite ~ "." ~ functionIdent ~ "(" ~ repsep(expression, ",") ~ ")" ^^ {
+    case operand ~ _ ~ name ~ _ ~ args ~ _ => Call(name.toUpperCase, operand :: args)
+  }
+
+  lazy val suffixFunctionCallOneArg = composite ~ "." ~ functionIdent ^^ {
+    case operand ~ _ ~ name => Call(name.toUpperCase, Seq(operand))
+  }
+
+  lazy val suffixAsc : PackratParser[Expression] =
+    atom <~ "." ~ ASC ~ opt("()") ^^ { e => Asc(e) }
+
+  lazy val suffixDesc : PackratParser[Expression] =
+    atom <~ "." ~ DESC ~ opt("()") ^^ { e => Desc(e) }
+
+  lazy val suffixToDate: PackratParser[Expression] =
+    composite <~ "." ~ TO_DATE ~ opt("()") ^^ { e => Cast(e, SqlTimeTypeInfo.DATE) }
+
+  lazy val suffixToTimestamp: PackratParser[Expression] =
+    composite <~ "." ~ TO_TIMESTAMP ~ opt("()") ^^ { e => Cast(e, SqlTimeTypeInfo.TIMESTAMP) }
+
+  lazy val suffixToTime: PackratParser[Expression] =
+    composite <~ "." ~ TO_TIME ~ opt("()") ^^ { e => Cast(e, SqlTimeTypeInfo.TIME) }
+
+  lazy val suffixTimeInterval : PackratParser[Expression] =
+    composite ~ "." ~ (YEARS | MONTHS | DAYS | HOURS | MINUTES | SECONDS | MILLIS |
+      YEAR | MONTH | DAY | HOUR | MINUTE | SECOND | MILLI) ^^ {
+
+    case expr ~ _ ~ (YEARS.key | YEAR.key) => toMonthInterval(expr, 12)
+
+    case expr ~ _ ~ (MONTHS.key | MONTH.key) => toMonthInterval(expr, 1)
+
+    case expr ~ _ ~ (DAYS.key | DAY.key) => toMilliInterval(expr, MILLIS_PER_DAY)
+
+    case expr ~ _ ~ (HOURS.key | HOUR.key) => toMilliInterval(expr, MILLIS_PER_HOUR)
+
+    case expr ~ _ ~ (MINUTES.key | MINUTE.key) => toMilliInterval(expr, MILLIS_PER_MINUTE)
+
+    case expr ~ _ ~ (SECONDS.key | SECOND.key) => toMilliInterval(expr, MILLIS_PER_SECOND)
+
+    case expr ~ _ ~ (MILLIS.key | MILLI.key)=> toMilliInterval(expr, 1)
+  }
+
+  lazy val suffixRowInterval : PackratParser[Expression] =
+    composite <~ "." ~ ROWS ^^ { e => ExpressionUtils.toRowInterval(e) }
+
+  lazy val suffixGet: PackratParser[Expression] =
+    composite ~ "." ~ GET ~ "(" ~ literalExpr ~ ")" ^^ {
+      case e ~ _ ~ _ ~ _ ~ index ~ _ =>
+        GetCompositeField(e, index.asInstanceOf[Literal].value)
+  }
+
+  lazy val suffixFlattening: PackratParser[Expression] =
+    composite <~ "." ~ FLATTEN ~ opt("()") ^^ { e => Flattening(e) }
+
+  lazy val suffixed: PackratParser[Expression] =
+    suffixTimeInterval | suffixRowInterval | suffixSum | suffixMin | suffixMax | suffixStart |
+      suffixEnd | suffixCount | suffixAvg | suffixCast | suffixAs | suffixTrim |
+      suffixTrimWithoutArgs | suffixIf | suffixAsc | suffixDesc | suffixToDate |
+      suffixToTimestamp | suffixToTime | suffixExtract | suffixFloor | suffixCeil |
+      suffixGet | suffixFlattening |
+      suffixFunctionCall | suffixFunctionCallOneArg // function call must always be at the end
+
+  // prefix operators
+
+  lazy val prefixArray: PackratParser[Expression] =
+    ARRAY ~ "(" ~> repsep(expression, ",") <~ ")" ^^ { elements => ArrayConstructor(elements) }
+
+  lazy val prefixSum: PackratParser[Expression] =
+    SUM ~ "(" ~> expression <~ ")" ^^ { e => Sum(e) }
+
+  lazy val prefixMin: PackratParser[Expression] =
+    MIN ~ "(" ~> expression <~ ")" ^^ { e => Min(e) }
+
+  lazy val prefixMax: PackratParser[Expression] =
+    MAX ~ "(" ~> expression <~ ")" ^^ { e => Max(e) }
+
+  lazy val prefixCount: PackratParser[Expression] =
+    COUNT ~ "(" ~> expression <~ ")" ^^ { e => Count(e) }
+
+  lazy val prefixAvg: PackratParser[Expression] =
+    AVG ~ "(" ~> expression <~ ")" ^^ { e => Avg(e) }
+
+  lazy val prefixStart: PackratParser[Expression] =
+    START ~ "(" ~> expression <~ ")" ^^ { e => WindowStart(e) }
+
+  lazy val prefixEnd: PackratParser[Expression] =
+    END ~ "(" ~> expression <~ ")" ^^ { e => WindowEnd(e) }
+
+  lazy val prefixCast: PackratParser[Expression] =
+    CAST ~ "(" ~ expression ~ "," ~ dataType ~ ")" ^^ {
+    case _ ~ _ ~ e ~ _ ~ dt ~ _ => Cast(e, dt)
+  }
+
+  lazy val prefixAs: PackratParser[Expression] =
+    AS ~ "(" ~ expression ~ "," ~ rep1sep(fieldReference, ",") ~ ")" ^^ {
+    case _ ~ _ ~ e ~ _ ~ target ~ _ => Alias(e, target.head.name, target.tail.map(_.name))
+  }
+
+  lazy val prefixIf: PackratParser[Expression] =
+      IF ~ "(" ~ expression ~ "," ~ expression ~ "," ~ expression ~ ")" ^^ {
+    case _ ~ _ ~ condition ~ _ ~ ifTrue ~ _ ~ ifFalse ~ _ => If(condition, ifTrue, ifFalse)
+  }
+
+  lazy val prefixFunctionCall = functionIdent ~ "(" ~ repsep(expression, ",") ~ ")" ^^ {
+    case name ~ _ ~ args ~ _ => Call(name.toUpperCase, args)
+  }
+
+  lazy val prefixFunctionCallOneArg = functionIdent ~ "(" ~ expression ~ ")" ^^ {
+    case name ~ _ ~ arg ~ _ => Call(name.toUpperCase, Seq(arg))
+  }
+
+  lazy val prefixTrim = TRIM ~ "(" ~ trimMode ~ "," ~ expression ~ "," ~ expression ~ ")" ^^ {
+    case _ ~ _ ~ mode ~ _ ~ trimCharacter ~ _ ~ operand ~ _ => Trim(mode, trimCharacter, operand)
+  }
+
+  lazy val prefixTrimWithoutArgs = TRIM ~ "(" ~ expression ~ ")" ^^ {
+    case _ ~ _ ~ operand ~ _ => Trim(TrimMode.BOTH, TrimConstants.TRIM_DEFAULT_CHAR, operand)
+  }
+
+  lazy val prefixExtract = EXTRACT ~ "(" ~ expression ~ "," ~ timeIntervalUnit ~ ")" ^^ {
+    case _ ~ _ ~ operand ~ _ ~ unit ~ _ => Extract(unit, operand)
+  }
+
+  lazy val prefixFloor = FLOOR ~ "(" ~ expression ~ "," ~ timeIntervalUnit ~ ")" ^^ {
+    case _ ~ _ ~ operand ~ _ ~ unit ~ _ => TemporalFloor(unit, operand)
+  }
+
+  lazy val prefixCeil = CEIL ~ "(" ~ expression ~ "," ~ timeIntervalUnit ~ ")" ^^ {
+    case _ ~ _ ~ operand ~ _ ~ unit ~ _ => TemporalCeil(unit, operand)
+  }
+
+  lazy val prefixGet: PackratParser[Expression] =
+    GET ~ "(" ~ composite ~ ","  ~ literalExpr ~ ")" ^^ {
+      case _ ~ _ ~ e ~ _ ~ index ~ _ =>
+        GetCompositeField(e, index.asInstanceOf[Literal].value)
+  }
+
+  lazy val prefixFlattening: PackratParser[Expression] =
+    FLATTEN ~ "(" ~> composite <~ ")" ^^ { e => Flattening(e) }
+
+  lazy val prefixed: PackratParser[Expression] =
+    prefixArray | prefixSum | prefixMin | prefixMax | prefixCount | prefixAvg |
+      prefixStart | prefixEnd |
+      prefixCast | prefixAs | prefixTrim | prefixTrimWithoutArgs | prefixIf | prefixExtract |
+      prefixFloor | prefixCeil | prefixGet | prefixFlattening |
+      prefixFunctionCall | prefixFunctionCallOneArg // function call must always be at the end
+
+  // suffix/prefix composite
+
+  lazy val composite: PackratParser[Expression] = suffixed | prefixed | atom |
+    failure("Composite expression expected.")
+
+  // unary ops
+
+  lazy val unaryNot: PackratParser[Expression] = "!" ~> composite ^^ { e => Not(e) }
+
+  lazy val unaryMinus: PackratParser[Expression] = "-" ~> composite ^^ { e => UnaryMinus(e) }
+
+  lazy val unary = composite | unaryNot | unaryMinus |
+    failure("Unary expression expected.")
+
+  // arithmetic
+
+  lazy val product = unary * (
+    "*" ^^^ { (a:Expression, b:Expression) => Mul(a,b) } |
+    "/" ^^^ { (a:Expression, b:Expression) => Div(a,b) } |
+    "%" ^^^ { (a:Expression, b:Expression) => Mod(a,b) } ) |
+    failure("Product expected.")
+
+  lazy val term = product * (
+    "+" ^^^ { (a:Expression, b:Expression) => Plus(a,b) } |
+    "-" ^^^ { (a:Expression, b:Expression) => Minus(a,b) } ) |
+    failure("Term expected.")
+
+  // Comparison
+
+  lazy val equalTo: PackratParser[Expression] = term ~ ("===" | "==" | "=") ~ term ^^ {
+    case l ~ _ ~ r => EqualTo(l, r)
+  }
+
+  lazy val notEqualTo: PackratParser[Expression] = term ~ ("!==" | "!=" | "<>") ~ term ^^ {
+    case l ~ _ ~ r => NotEqualTo(l, r)
+  }
+
+  lazy val greaterThan: PackratParser[Expression] = term ~ ">" ~ term ^^ {
+    case l ~ _ ~ r => GreaterThan(l, r)
+  }
+
+  lazy val greaterThanOrEqual: PackratParser[Expression] = term ~ ">=" ~ term ^^ {
+    case l ~ _ ~ r => GreaterThanOrEqual(l, r)
+  }
+
+  lazy val lessThan: PackratParser[Expression] = term ~ "<" ~ term ^^ {
+    case l ~ _ ~ r => LessThan(l, r)
+  }
+
+  lazy val lessThanOrEqual: PackratParser[Expression] = term ~ "<=" ~ term ^^ {
+    case l ~ _ ~ r => LessThanOrEqual(l, r)
+  }
+
+  lazy val comparison: PackratParser[Expression] =
+    equalTo | notEqualTo |
+    greaterThan | greaterThanOrEqual |
+    lessThan | lessThanOrEqual | term |
+    failure("Comparison expected.")
+
+  // logic
+
+  lazy val logic = comparison * (
+    "&&" ^^^ { (a:Expression, b:Expression) => And(a,b) } |
+    "||" ^^^ { (a:Expression, b:Expression) => Or(a,b) } ) |
+    failure("Logic expected.")
+
+  // alias
+
+  lazy val alias: PackratParser[Expression] = logic ~ AS ~ fieldReference ^^ {
+      case e ~ _ ~ name => Alias(e, name.name)
+  } | logic ~ AS ~ "(" ~ rep1sep(fieldReference, ",") ~ ")" ^^ {
+    case e ~ _ ~ _ ~ names ~ _ => Alias(e, names.head.name, names.tail.map(_.name))
+  } | logic
+
+  lazy val expression: PackratParser[Expression] = alias |
+    failure("Invalid expression.")
+
+  lazy val expressionList: Parser[List[Expression]] = rep1sep(expression, ",")
+
+  def parseExpressionList(expression: String): List[Expression] = {
+    parseAll(expressionList, expression) match {
+      case Success(lst, _) => lst
+
+      case NoSuccess(msg, next) =>
+        throwError(msg, next)
+    }
+  }
+
+  def parseExpression(exprString: String): Expression = {
+    parseAll(expression, exprString) match {
+      case Success(lst, _) => lst
+
+      case NoSuccess(msg, next) =>
+        throwError(msg, next)
+    }
+  }
+
+  private def throwError(msg: String, next: Input): Nothing = {
+    val improvedMsg = msg.replace("string matching regex `\\z'", "End of expression")
+
+    throw ExpressionParserException(
+      s"""Could not parse expression at column ${next.pos.column}: $improvedMsg
+        |${next.pos.longString}""".stripMargin)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionUtils.scala
new file mode 100644
index 0000000..4b5781f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ExpressionUtils.scala
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.expressions
+
+import java.lang.{Boolean => JBoolean, Byte => JByte, Double => JDouble, Float => JFloat, Integer => JInteger, Long => JLong, Short => JShort}
+import java.math.{BigDecimal => JBigDecimal}
+import java.sql.{Date, Time, Timestamp}
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex.{RexBuilder, RexNode}
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo}
+
+object ExpressionUtils {
+
+  private[flink] def toMonthInterval(expr: Expression, multiplier: Int): Expression = expr match {
+    case Literal(value: Int, BasicTypeInfo.INT_TYPE_INFO) =>
+      Literal(value * multiplier, TimeIntervalTypeInfo.INTERVAL_MONTHS)
+    case _ =>
+      Cast(Mul(expr, Literal(multiplier)), TimeIntervalTypeInfo.INTERVAL_MONTHS)
+  }
+
+  private[flink] def toMilliInterval(expr: Expression, multiplier: Long): Expression = expr match {
+    case Literal(value: Int, BasicTypeInfo.INT_TYPE_INFO) =>
+      Literal(value * multiplier, TimeIntervalTypeInfo.INTERVAL_MILLIS)
+    case Literal(value: Long, BasicTypeInfo.LONG_TYPE_INFO) =>
+      Literal(value * multiplier, TimeIntervalTypeInfo.INTERVAL_MILLIS)
+    case _ =>
+      Cast(Mul(expr, Literal(multiplier)), TimeIntervalTypeInfo.INTERVAL_MILLIS)
+  }
+
+  private[flink] def toRowInterval(expr: Expression): Expression = expr match {
+    case Literal(value: Int, BasicTypeInfo.INT_TYPE_INFO) =>
+      Literal(value.toLong, RowIntervalTypeInfo.INTERVAL_ROWS)
+    case Literal(value: Long, BasicTypeInfo.LONG_TYPE_INFO) =>
+      Literal(value, RowIntervalTypeInfo.INTERVAL_ROWS)
+    case _ =>
+      throw new IllegalArgumentException("Invalid value for row interval literal.")
+  }
+
+  private[flink] def convertArray(array: Array[_]): Expression = {
+    def createArray(): Expression = {
+      ArrayConstructor(array.map(Literal(_)))
+    }
+
+    array match {
+      // primitives
+      case _: Array[Boolean] => createArray()
+      case _: Array[Byte] => createArray()
+      case _: Array[Short] => createArray()
+      case _: Array[Int] => createArray()
+      case _: Array[Long] => createArray()
+      case _: Array[Float] => createArray()
+      case _: Array[Double] => createArray()
+
+      // boxed types
+      case _: Array[JBoolean] => createArray()
+      case _: Array[JByte] => createArray()
+      case _: Array[JShort] => createArray()
+      case _: Array[JInteger] => createArray()
+      case _: Array[JLong] => createArray()
+      case _: Array[JFloat] => createArray()
+      case _: Array[JDouble] => createArray()
+
+      // others
+      case _: Array[String] => createArray()
+      case _: Array[JBigDecimal] => createArray()
+      case _: Array[Date] => createArray()
+      case _: Array[Time] => createArray()
+      case _: Array[Timestamp] => createArray()
+      case bda: Array[BigDecimal] => ArrayConstructor(bda.map { bd => Literal(bd.bigDecimal) })
+
+      case _ =>
+        // nested
+        if (array.length > 0 && array.head.isInstanceOf[Array[_]]) {
+          ArrayConstructor(array.map { na => convertArray(na.asInstanceOf[Array[_]]) })
+        } else {
+          throw ValidationException("Unsupported array type.")
+        }
+    }
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // RexNode conversion functions (see org.apache.calcite.sql2rel.StandardConvertletTable)
+  // ----------------------------------------------------------------------------------------------
+
+  /**
+    * Copy of [[org.apache.calcite.sql2rel.StandardConvertletTable#getFactor()]].
+    */
+  private[flink] def getFactor(unit: TimeUnit): JBigDecimal = unit match {
+    case TimeUnit.DAY => java.math.BigDecimal.ONE
+    case TimeUnit.HOUR => TimeUnit.DAY.multiplier
+    case TimeUnit.MINUTE => TimeUnit.HOUR.multiplier
+    case TimeUnit.SECOND => TimeUnit.MINUTE.multiplier
+    case TimeUnit.YEAR => java.math.BigDecimal.ONE
+    case TimeUnit.MONTH => TimeUnit.YEAR.multiplier
+    case _ => throw new IllegalArgumentException("Invalid start unit.")
+  }
+
+  /**
+    * Copy of [[org.apache.calcite.sql2rel.StandardConvertletTable#mod()]].
+    */
+  private[flink] def mod(
+      rexBuilder: RexBuilder,
+      resType: RelDataType,
+      res: RexNode,
+      value: JBigDecimal)
+    : RexNode = {
+    if (value == JBigDecimal.ONE) return res
+    rexBuilder.makeCall(SqlStdOperatorTable.MOD, res, rexBuilder.makeExactLiteral(value, resType))
+  }
+
+  /**
+    * Copy of [[org.apache.calcite.sql2rel.StandardConvertletTable#divide()]].
+    */
+  private[flink] def divide(rexBuilder: RexBuilder, res: RexNode, value: JBigDecimal): RexNode = {
+    if (value == JBigDecimal.ONE) return res
+    if (value.compareTo(JBigDecimal.ONE) < 0 && value.signum == 1) {
+      try {
+        val reciprocal = JBigDecimal.ONE.divide(value, JBigDecimal.ROUND_UNNECESSARY)
+        return rexBuilder.makeCall(
+          SqlStdOperatorTable.MULTIPLY,
+          res,
+          rexBuilder.makeExactLiteral(reciprocal))
+      } catch {
+        case e: ArithmeticException => // ignore
+      }
+    }
+    rexBuilder.makeCall(
+      SqlStdOperatorTable.DIVIDE_INTEGER,
+      res,
+      rexBuilder.makeExactLiteral(value))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/InputTypeSpec.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/InputTypeSpec.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/InputTypeSpec.scala
new file mode 100644
index 0000000..39e1fe2
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/InputTypeSpec.scala
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.expressions
+
+import scala.collection.mutable
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.validate._
+
+/**
+  * Expressions that have specification on its inputs.
+  */
+trait InputTypeSpec extends Expression {
+
+  /**
+    * Input type specification for each child.
+    *
+    * For example, [[Power]] expecting both of the children be of Double Type should use:
+    * {{{
+    *   def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: DOUBLE_TYPE_INFO :: Nil
+    * }}}
+    */
+  private[flink] def expectedTypes: Seq[TypeInformation[_]]
+
+  override private[flink] def validateInput(): ValidationResult = {
+    val typeMismatches = mutable.ArrayBuffer.empty[String]
+    children.zip(expectedTypes).zipWithIndex.foreach { case ((e, tpe), i) =>
+      if (e.resultType != tpe) {
+        typeMismatches += s"expecting $tpe on ${i}th input, get ${e.resultType}"
+      }
+    }
+    if (typeMismatches.isEmpty) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(
+        s"""|$this fails on input type checking: ${typeMismatches.mkString("[", ", ", "]")}.
+            |Operand should be casted to proper type
+            |""".stripMargin)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/aggregations.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/aggregations.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/aggregations.scala
new file mode 100644
index 0000000..b2fca88
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/aggregations.scala
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.tools.RelBuilder
+import org.apache.calcite.tools.RelBuilder.AggCall
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.table.typeutils.TypeCheckUtils
+
+abstract sealed class Aggregation extends UnaryExpression {
+
+  override def toString = s"Aggregate($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode =
+    throw new UnsupportedOperationException("Aggregate cannot be transformed to RexNode")
+
+  /**
+    * Convert Aggregate to its counterpart in Calcite, i.e. AggCall
+    */
+  private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall
+}
+
+case class Sum(child: Expression) extends Aggregation {
+  override def toString = s"sum($child)"
+
+  override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
+    relBuilder.aggregateCall(SqlStdOperatorTable.SUM, false, null, name, child.toRexNode)
+  }
+
+  override private[flink] def resultType = child.resultType
+
+  override private[flink] def validateInput() =
+    TypeCheckUtils.assertNumericExpr(child.resultType, "sum")
+}
+
+case class Min(child: Expression) extends Aggregation {
+  override def toString = s"min($child)"
+
+  override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
+    relBuilder.aggregateCall(SqlStdOperatorTable.MIN, false, null, name, child.toRexNode)
+  }
+
+  override private[flink] def resultType = child.resultType
+
+  override private[flink] def validateInput() =
+    TypeCheckUtils.assertOrderableExpr(child.resultType, "min")
+}
+
+case class Max(child: Expression) extends Aggregation {
+  override def toString = s"max($child)"
+
+  override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
+    relBuilder.aggregateCall(SqlStdOperatorTable.MAX, false, null, name, child.toRexNode)
+  }
+
+  override private[flink] def resultType = child.resultType
+
+  override private[flink] def validateInput() =
+    TypeCheckUtils.assertOrderableExpr(child.resultType, "max")
+}
+
+case class Count(child: Expression) extends Aggregation {
+  override def toString = s"count($child)"
+
+  override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
+    relBuilder.aggregateCall(SqlStdOperatorTable.COUNT, false, null, name, child.toRexNode)
+  }
+
+  override private[flink] def resultType = BasicTypeInfo.LONG_TYPE_INFO
+}
+
+case class Avg(child: Expression) extends Aggregation {
+  override def toString = s"avg($child)"
+
+  override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
+    relBuilder.aggregateCall(SqlStdOperatorTable.AVG, false, null, name, child.toRexNode)
+  }
+
+  override private[flink] def resultType = child.resultType
+
+  override private[flink] def validateInput() =
+    TypeCheckUtils.assertNumericExpr(child.resultType, "avg")
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/arithmetic.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/arithmetic.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/arithmetic.scala
new file mode 100644
index 0000000..ad1af63
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/arithmetic.scala
@@ -0,0 +1,158 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.`type`.IntervalSqlType
+import org.apache.calcite.sql.SqlOperator
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.table.typeutils.TypeCheckUtils._
+import org.apache.flink.table.typeutils.TypeCoercion
+import org.apache.flink.table.validate._
+
+import scala.collection.JavaConversions._
+
+abstract class BinaryArithmetic extends BinaryExpression {
+  private[flink] def sqlOperator: SqlOperator
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(sqlOperator, children.map(_.toRexNode))
+  }
+
+  override private[flink] def resultType: TypeInformation[_] =
+    TypeCoercion.widerTypeOf(left.resultType, right.resultType) match {
+      case Some(t) => t
+      case None =>
+        throw new RuntimeException("This should never happen.")
+    }
+
+  // TODO: tighten this rule once we implemented type coercion rules during validation
+  override private[flink] def validateInput(): ValidationResult = {
+    if (!isNumeric(left.resultType) || !isNumeric(right.resultType)) {
+      ValidationFailure(s"$this requires both operands Numeric, get " +
+        s"$left : ${left.resultType} and $right : ${right.resultType}")
+    } else {
+      ValidationSuccess
+    }
+  }
+}
+
+case class Plus(left: Expression, right: Expression) extends BinaryArithmetic {
+  override def toString = s"($left + $right)"
+
+  private[flink] val sqlOperator = SqlStdOperatorTable.PLUS
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    if(isString(left.resultType)) {
+      val castedRight = Cast(right, BasicTypeInfo.STRING_TYPE_INFO)
+      relBuilder.call(SqlStdOperatorTable.CONCAT, left.toRexNode, castedRight.toRexNode)
+    } else if(isString(right.resultType)) {
+      val castedLeft = Cast(left, BasicTypeInfo.STRING_TYPE_INFO)
+      relBuilder.call(SqlStdOperatorTable.CONCAT, castedLeft.toRexNode, right.toRexNode)
+    } else if (isTimeInterval(left.resultType) && left.resultType == right.resultType) {
+      relBuilder.call(SqlStdOperatorTable.PLUS, left.toRexNode, right.toRexNode)
+    } else if (isTimeInterval(left.resultType) && isTemporal(right.resultType)) {
+      // Calcite has a bug that can't apply INTERVAL + DATETIME (INTERVAL at left)
+      // we manually switch them here
+      relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, right.toRexNode, left.toRexNode)
+    } else if (isTemporal(left.resultType) && isTemporal(right.resultType)) {
+      relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, left.toRexNode, right.toRexNode)
+    } else {
+      val castedLeft = Cast(left, resultType)
+      val castedRight = Cast(right, resultType)
+      relBuilder.call(SqlStdOperatorTable.PLUS, castedLeft.toRexNode, castedRight.toRexNode)
+    }
+  }
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (isString(left.resultType) || isString(right.resultType)) {
+      ValidationSuccess
+    } else if (isTimeInterval(left.resultType) && left.resultType == right.resultType) {
+      ValidationSuccess
+    } else if (isTimePoint(left.resultType) && isTimeInterval(right.resultType)) {
+      ValidationSuccess
+    } else if (isTimeInterval(left.resultType) && isTimePoint(right.resultType)) {
+      ValidationSuccess
+    } else if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(
+        s"$this requires Numeric, String, Intervals of same type, " +
+        s"or Interval and a time point input, " +
+        s"get $left : ${left.resultType} and $right : ${right.resultType}")
+    }
+  }
+}
+
+case class UnaryMinus(child: Expression) extends UnaryExpression {
+  override def toString = s"-($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.UNARY_MINUS, child.toRexNode)
+  }
+
+  override private[flink] def resultType = child.resultType
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (isNumeric(child.resultType)) {
+      ValidationSuccess
+    } else if (isTimeInterval(child.resultType)) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(s"$this requires Numeric, or Interval input, get ${child.resultType}")
+    }
+  }
+}
+
+case class Minus(left: Expression, right: Expression) extends BinaryArithmetic {
+  override def toString = s"($left - $right)"
+
+  private[flink] val sqlOperator = SqlStdOperatorTable.MINUS
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (isTimeInterval(left.resultType) && left.resultType == right.resultType) {
+      ValidationSuccess
+    } else if (isTimePoint(left.resultType) && isTimeInterval(right.resultType)) {
+      ValidationSuccess
+    } else if (isTimeInterval(left.resultType) && isTimePoint(right.resultType)) {
+      ValidationSuccess
+    } else {
+      super.validateInput()
+    }
+  }
+}
+
+case class Div(left: Expression, right: Expression) extends BinaryArithmetic {
+  override def toString = s"($left / $right)"
+
+  private[flink] val sqlOperator = SqlStdOperatorTable.DIVIDE
+}
+
+case class Mul(left: Expression, right: Expression) extends BinaryArithmetic {
+  override def toString = s"($left * $right)"
+
+  private[flink] val sqlOperator = SqlStdOperatorTable.MULTIPLY
+}
+
+case class Mod(left: Expression, right: Expression) extends BinaryArithmetic {
+  override def toString = s"($left % $right)"
+
+  private[flink] val sqlOperator = SqlStdOperatorTable.MOD
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/array.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/array.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/array.scala
new file mode 100644
index 0000000..b087b61
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/array.scala
@@ -0,0 +1,146 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo.INT_TYPE_INFO
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, PrimitiveArrayTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
+import org.apache.flink.table.calcite.FlinkRelBuilder
+import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
+
+import scala.collection.JavaConverters._
+
+case class ArrayConstructor(elements: Seq[Expression]) extends Expression {
+
+  override private[flink] def children: Seq[Expression] = elements
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    val relDataType = relBuilder
+      .asInstanceOf[FlinkRelBuilder]
+      .getTypeFactory
+      .createTypeFromTypeInfo(resultType)
+    val values = elements.map(_.toRexNode).toList.asJava
+    relBuilder
+      .getRexBuilder
+      .makeCall(relDataType, SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR, values)
+  }
+
+  override def toString = s"array(${elements.mkString(", ")})"
+
+  override private[flink] def resultType = ObjectArrayTypeInfo.getInfoFor(elements.head.resultType)
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (elements.isEmpty) {
+      return ValidationFailure("Empty arrays are not supported yet.")
+    }
+    val elementType = elements.head.resultType
+    if (!elements.forall(_.resultType == elementType)) {
+      ValidationFailure("Not all elements of the array have the same type.")
+    } else {
+      ValidationSuccess
+    }
+  }
+}
+
+case class ArrayElementAt(array: Expression, index: Expression) extends Expression {
+
+  override private[flink] def children: Seq[Expression] = Seq(array, index)
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder
+      .getRexBuilder
+      .makeCall(SqlStdOperatorTable.ITEM, array.toRexNode, index.toRexNode)
+  }
+
+  override def toString = s"($array).at($index)"
+
+  override private[flink] def resultType = array.resultType match {
+    case oati: ObjectArrayTypeInfo[_, _] => oati.getComponentInfo
+    case pati: PrimitiveArrayTypeInfo[_] => pati.getComponentType
+  }
+
+  override private[flink] def validateInput(): ValidationResult = {
+    array.resultType match {
+      case _: ObjectArrayTypeInfo[_, _] | _: PrimitiveArrayTypeInfo[_] =>
+        if (index.resultType == INT_TYPE_INFO) {
+          // check for common user mistake
+          index match {
+            case Literal(value: Int, INT_TYPE_INFO) if value < 1 =>
+              ValidationFailure(
+                s"Array element access needs an index starting at 1 but was $value.")
+            case _ => ValidationSuccess
+          }
+        } else {
+          ValidationFailure(
+            s"Array element access needs an integer index but was '${index.resultType}'.")
+        }
+      case other@_ => ValidationFailure(s"Array expected but was '$other'.")
+    }
+  }
+}
+
+case class ArrayCardinality(array: Expression) extends Expression {
+
+  override private[flink] def children: Seq[Expression] = Seq(array)
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder
+      .getRexBuilder
+      .makeCall(SqlStdOperatorTable.CARDINALITY, array.toRexNode)
+  }
+
+  override def toString = s"($array).cardinality()"
+
+  override private[flink] def resultType = BasicTypeInfo.INT_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    array.resultType match {
+      case _: ObjectArrayTypeInfo[_, _] | _: PrimitiveArrayTypeInfo[_] => ValidationSuccess
+      case other@_ => ValidationFailure(s"Array expected but was '$other'.")
+    }
+  }
+}
+
+case class ArrayElement(array: Expression) extends Expression {
+
+  override private[flink] def children: Seq[Expression] = Seq(array)
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder
+      .getRexBuilder
+      .makeCall(SqlStdOperatorTable.ELEMENT, array.toRexNode)
+  }
+
+  override def toString = s"($array).element()"
+
+  override private[flink] def resultType = array.resultType match {
+    case oati: ObjectArrayTypeInfo[_, _] => oati.getComponentInfo
+    case pati: PrimitiveArrayTypeInfo[_] => pati.getComponentType
+  }
+
+  override private[flink] def validateInput(): ValidationResult = {
+    array.resultType match {
+      case _: ObjectArrayTypeInfo[_, _] | _: PrimitiveArrayTypeInfo[_] => ValidationSuccess
+      case other@_ => ValidationFailure(s"Array expected but was '$other'.")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
new file mode 100644
index 0000000..ef2cf4e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
@@ -0,0 +1,160 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.functions.{ScalarFunction, TableFunction}
+import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
+import org.apache.flink.table.plan.logical.{LogicalNode, LogicalTableFunctionCall}
+import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
+import org.apache.flink.table.api.{UnresolvedException, ValidationException}
+
+/**
+  * General expression for unresolved function calls. The function can be a built-in
+  * scalar function or a user-defined scalar function.
+  */
+case class Call(functionName: String, args: Seq[Expression]) extends Expression {
+
+  override private[flink] def children: Seq[Expression] = args
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    throw UnresolvedException(s"trying to convert UnresolvedFunction $functionName to RexNode")
+  }
+
+  override def toString = s"\\$functionName(${args.mkString(", ")})"
+
+  override private[flink] def resultType =
+    throw UnresolvedException(s"calling resultType on UnresolvedFunction $functionName")
+
+  override private[flink] def validateInput(): ValidationResult =
+    ValidationFailure(s"Unresolved function call: $functionName")
+}
+
+/**
+  * Expression for calling a user-defined scalar functions.
+  *
+  * @param scalarFunction scalar function to be called (might be overloaded)
+  * @param parameters actual parameters that determine target evaluation method
+  */
+case class ScalarFunctionCall(
+    scalarFunction: ScalarFunction,
+    parameters: Seq[Expression])
+  extends Expression {
+
+  private var foundSignature: Option[Array[Class[_]]] = None
+
+  override private[flink] def children: Seq[Expression] = parameters
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    val typeFactory = relBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+    relBuilder.call(
+      createScalarSqlFunction(
+        scalarFunction.getClass.getCanonicalName,
+        scalarFunction,
+        typeFactory),
+      parameters.map(_.toRexNode): _*)
+  }
+
+  override def toString =
+    s"${scalarFunction.getClass.getCanonicalName}(${parameters.mkString(", ")})"
+
+  override private[flink] def resultType = getResultType(scalarFunction, foundSignature.get)
+
+  override private[flink] def validateInput(): ValidationResult = {
+    val signature = children.map(_.resultType)
+    // look for a signature that matches the input types
+    foundSignature = getSignature(scalarFunction, signature)
+    if (foundSignature.isEmpty) {
+      ValidationFailure(s"Given parameters do not match any signature. \n" +
+        s"Actual: ${signatureToString(signature)} \n" +
+        s"Expected: ${signaturesToString(scalarFunction)}")
+    } else {
+      ValidationSuccess
+    }
+  }
+}
+
+/**
+  *
+  * Expression for calling a user-defined table function with actual parameters.
+  *
+  * @param functionName function name
+  * @param tableFunction user-defined table function
+  * @param parameters actual parameters of function
+  * @param resultType type information of returned table
+  */
+case class TableFunctionCall(
+    functionName: String,
+    tableFunction: TableFunction[_],
+    parameters: Seq[Expression],
+    resultType: TypeInformation[_])
+  extends Expression {
+
+  private var aliases: Option[Seq[String]] = None
+
+  override private[flink] def children: Seq[Expression] = parameters
+
+  /**
+    * Assigns an alias for this table function's returned fields that the following operator
+    * can refer to.
+    *
+    * @param aliasList alias for this table function's returned fields
+    * @return this table function call
+    */
+  private[flink] def as(aliasList: Option[Seq[String]]): TableFunctionCall = {
+    this.aliases = aliasList
+    this
+  }
+
+  /**
+    * Converts an API class to a logical node for planning.
+    */
+  private[flink] def toLogicalTableFunctionCall(child: LogicalNode): LogicalTableFunctionCall = {
+    val originNames = getFieldInfo(resultType)._1
+
+    // determine the final field names
+    val fieldNames = if (aliases.isDefined) {
+      val aliasList = aliases.get
+      if (aliasList.length != originNames.length) {
+        throw ValidationException(
+          s"List of column aliases must have same degree as table; " +
+            s"the returned table of function '$functionName' has ${originNames.length} " +
+            s"columns (${originNames.mkString(",")}), " +
+            s"whereas alias list has ${aliasList.length} columns")
+      } else {
+        aliasList.toArray
+      }
+    } else {
+      originNames
+    }
+
+    LogicalTableFunctionCall(
+      functionName,
+      tableFunction,
+      parameters,
+      resultType,
+      fieldNames,
+      child)
+  }
+
+  override def toString =
+    s"${tableFunction.getClass.getCanonicalName}(${parameters.mkString(", ")})"
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/cast.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/cast.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/cast.scala
new file mode 100644
index 0000000..312bf12
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/cast.scala
@@ -0,0 +1,53 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.typeutils.TypeCoercion
+import org.apache.flink.table.validate._
+
+case class Cast(child: Expression, resultType: TypeInformation[_]) extends UnaryExpression {
+
+  override def toString = s"$child.cast($resultType)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    val typeFactory = relBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+    relBuilder
+      .getRexBuilder
+      // we use abstract cast here because RelBuilder.cast() has to many side effects
+      .makeAbstractCast(
+        typeFactory.createTypeFromTypeInfo(resultType),
+        child.toRexNode)
+  }
+
+  override private[flink] def makeCopy(anyRefs: Array[AnyRef]): this.type = {
+    val child: Expression = anyRefs.head.asInstanceOf[Expression]
+    copy(child, resultType).asInstanceOf[this.type]
+  }
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (TypeCoercion.canCast(child.resultType, resultType)) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(s"Unsupported cast from ${child.resultType} to $resultType")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/comparison.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/comparison.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/comparison.scala
new file mode 100644
index 0000000..0c7e57c
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/comparison.scala
@@ -0,0 +1,160 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.SqlOperator
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.table.typeutils.TypeCheckUtils.{isComparable, isNumeric}
+import org.apache.flink.table.validate._
+
+import scala.collection.JavaConversions._
+
+abstract class BinaryComparison extends BinaryExpression {
+  private[flink] def sqlOperator: SqlOperator
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(sqlOperator, children.map(_.toRexNode))
+  }
+
+  override private[flink] def resultType = BOOLEAN_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult =
+    (left.resultType, right.resultType) match {
+      case (lType, rType) if isNumeric(lType) && isNumeric(rType) => ValidationSuccess
+      case (lType, rType) if isComparable(lType) && lType == rType => ValidationSuccess
+      case (lType, rType) =>
+        ValidationFailure(
+          s"Comparison is only supported for numeric types and " +
+            s"comparable types of same type, got $lType and $rType")
+    }
+}
+
+case class EqualTo(left: Expression, right: Expression) extends BinaryComparison {
+  override def toString = s"$left === $right"
+
+  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.EQUALS
+
+  override private[flink] def validateInput(): ValidationResult =
+    (left.resultType, right.resultType) match {
+      case (lType, rType) if isNumeric(lType) && isNumeric(rType) => ValidationSuccess
+      case (lType, rType) if lType == rType => ValidationSuccess
+      case (lType, rType) =>
+        ValidationFailure(s"Equality predicate on incompatible types: $lType and $rType")
+    }
+}
+
+case class NotEqualTo(left: Expression, right: Expression) extends BinaryComparison {
+  override def toString = s"$left !== $right"
+
+  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.NOT_EQUALS
+
+  override private[flink] def validateInput(): ValidationResult =
+    (left.resultType, right.resultType) match {
+      case (lType, rType) if isNumeric(lType) && isNumeric(rType) => ValidationSuccess
+      case (lType, rType) if lType == rType => ValidationSuccess
+      case (lType, rType) =>
+        ValidationFailure(s"Inequality predicate on incompatible types: $lType and $rType")
+    }
+}
+
+case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison {
+  override def toString = s"$left > $right"
+
+  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.GREATER_THAN
+}
+
+case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
+  override def toString = s"$left >= $right"
+
+  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.GREATER_THAN_OR_EQUAL
+}
+
+case class LessThan(left: Expression, right: Expression) extends BinaryComparison {
+  override def toString = s"$left < $right"
+
+  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.LESS_THAN
+}
+
+case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
+  override def toString = s"$left <= $right"
+
+  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.LESS_THAN_OR_EQUAL
+}
+
+case class IsNull(child: Expression) extends UnaryExpression {
+  override def toString = s"($child).isNull"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.isNull(child.toRexNode)
+  }
+
+  override private[flink] def resultType = BOOLEAN_TYPE_INFO
+}
+
+case class IsNotNull(child: Expression) extends UnaryExpression {
+  override def toString = s"($child).isNotNull"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.isNotNull(child.toRexNode)
+  }
+
+  override private[flink] def resultType = BOOLEAN_TYPE_INFO
+}
+
+case class IsTrue(child: Expression) extends UnaryExpression {
+  override def toString = s"($child).isTrue"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.IS_TRUE, child.toRexNode)
+  }
+
+  override private[flink] def resultType = BOOLEAN_TYPE_INFO
+}
+
+case class IsFalse(child: Expression) extends UnaryExpression {
+  override def toString = s"($child).isFalse"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.IS_FALSE, child.toRexNode)
+  }
+
+  override private[flink] def resultType = BOOLEAN_TYPE_INFO
+}
+
+case class IsNotTrue(child: Expression) extends UnaryExpression {
+  override def toString = s"($child).isNotTrue"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.IS_NOT_TRUE, child.toRexNode)
+  }
+
+  override private[flink] def resultType = BOOLEAN_TYPE_INFO
+}
+
+case class IsNotFalse(child: Expression) extends UnaryExpression {
+  override def toString = s"($child).isNotFalse"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.IS_NOT_FALSE, child.toRexNode)
+  }
+
+  override private[flink] def resultType = BOOLEAN_TYPE_INFO
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/composite.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/composite.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/composite.scala
new file mode 100644
index 0000000..2f3fdb1
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/composite.scala
@@ -0,0 +1,106 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.table.api.UnresolvedException
+import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
+
+/**
+  * Flattening of composite types. All flattenings are resolved into
+  * `GetCompositeField` expressions.
+  */
+case class Flattening(child: Expression) extends UnaryExpression {
+
+  override def toString = s"$child.flatten()"
+
+  override private[flink] def resultType: TypeInformation[_] =
+    throw UnresolvedException(s"Invalid call to on ${this.getClass}.")
+
+  override private[flink] def validateInput(): ValidationResult =
+    ValidationFailure(s"Unresolved flattening of $child")
+}
+
+case class GetCompositeField(child: Expression, key: Any) extends UnaryExpression {
+
+  private var fieldIndex: Option[Int] = None
+
+  override def toString = s"$child.get($key)"
+
+  override private[flink] def validateInput(): ValidationResult = {
+    // check for composite type
+    if (!child.resultType.isInstanceOf[CompositeType[_]]) {
+      return ValidationFailure(s"Cannot access field of non-composite type '${child.resultType}'.")
+    }
+    val compositeType = child.resultType.asInstanceOf[CompositeType[_]]
+
+    // check key
+    key match {
+      case name: String =>
+        val index = compositeType.getFieldIndex(name)
+        if (index < 0) {
+          ValidationFailure(s"Field name '$name' could not be found.")
+        } else {
+          fieldIndex = Some(index)
+          ValidationSuccess
+        }
+      case index: Int =>
+        if (index >= compositeType.getArity) {
+          ValidationFailure(s"Field index '$index' exceeds arity.")
+        } else {
+          fieldIndex = Some(index)
+          ValidationSuccess
+        }
+      case _ =>
+        ValidationFailure(s"Invalid key '$key'.")
+    }
+  }
+
+  override private[flink] def resultType: TypeInformation[_] =
+    child.resultType.asInstanceOf[CompositeType[_]].getTypeAt(fieldIndex.get)
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder
+      .getRexBuilder
+      .makeFieldAccess(child.toRexNode, fieldIndex.get)
+  }
+
+  override private[flink] def makeCopy(anyRefs: Array[AnyRef]): this.type = {
+    val child: Expression = anyRefs.head.asInstanceOf[Expression]
+    copy(child, key).asInstanceOf[this.type]
+  }
+
+  /**
+    * Gives a meaningful alias if possible (e.g. a$mypojo$field).
+    */
+  private[flink] def aliasName(): Option[String] = child match {
+    case gcf: GetCompositeField =>
+      val alias = gcf.aliasName()
+      if (alias.isDefined) {
+        Some(s"${alias.get}$$$key")
+      } else {
+        None
+      }
+    case c: ResolvedFieldReference => Some(s"${c.name}$$$key")
+    case _ => None
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
new file mode 100644
index 0000000..299a850
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/fieldExpression.scala
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.table.api.{UnresolvedException, ValidationException}
+import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
+
+trait NamedExpression extends Expression {
+  private[flink] def name: String
+  private[flink] def toAttribute: Attribute
+}
+
+abstract class Attribute extends LeafExpression with NamedExpression {
+  override private[flink] def toAttribute: Attribute = this
+
+  private[flink] def withName(newName: String): Attribute
+}
+
+case class UnresolvedFieldReference(name: String) extends Attribute {
+
+  override def toString = "\"" + name
+
+  override private[flink] def withName(newName: String): Attribute =
+    UnresolvedFieldReference(newName)
+
+  override private[flink] def resultType: TypeInformation[_] =
+    throw UnresolvedException(s"Calling resultType on ${this.getClass}.")
+
+  override private[flink] def validateInput(): ValidationResult =
+    ValidationFailure(s"Unresolved reference $name.")
+}
+
+case class ResolvedFieldReference(
+    name: String,
+    resultType: TypeInformation[_]) extends Attribute {
+
+  override def toString = s"'$name"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.field(name)
+  }
+
+  override private[flink] def withName(newName: String): Attribute = {
+    if (newName == name) {
+      this
+    } else {
+      ResolvedFieldReference(newName, resultType)
+    }
+  }
+}
+
+case class Alias(child: Expression, name: String, extraNames: Seq[String] = Seq())
+    extends UnaryExpression with NamedExpression {
+
+  override def toString = s"$child as '$name"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.alias(child.toRexNode, name)
+  }
+
+  override private[flink] def resultType: TypeInformation[_] = child.resultType
+
+  override private[flink] def makeCopy(anyRefs: Array[AnyRef]): this.type = {
+    val child: Expression = anyRefs.head.asInstanceOf[Expression]
+    copy(child, name, extraNames).asInstanceOf[this.type]
+  }
+
+  override private[flink] def toAttribute: Attribute = {
+    if (valid) {
+      ResolvedFieldReference(name, child.resultType)
+    } else {
+      UnresolvedFieldReference(name)
+    }
+  }
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (name == "*") {
+      ValidationFailure("Alias can not accept '*' as name.")
+    } else if (extraNames.nonEmpty) {
+      ValidationFailure("Invalid call to Alias with multiple names.")
+    } else {
+      ValidationSuccess
+    }
+  }
+}
+
+case class UnresolvedAlias(child: Expression) extends UnaryExpression with NamedExpression {
+
+  override private[flink] def name: String =
+    throw UnresolvedException("Invalid call to name on UnresolvedAlias")
+
+  override private[flink] def toAttribute: Attribute =
+    throw UnresolvedException("Invalid call to toAttribute on UnresolvedAlias")
+
+  override private[flink] def resultType: TypeInformation[_] =
+    throw UnresolvedException("Invalid call to resultType on UnresolvedAlias")
+
+  override private[flink] lazy val valid = false
+}
+
+case class RowtimeAttribute() extends Attribute {
+  override private[flink] def withName(newName: String): Attribute = {
+    if (newName == "rowtime") {
+      this
+    } else {
+      throw new ValidationException("Cannot rename streaming rowtime attribute.")
+    }
+  }
+
+  override private[flink] def name: String = "rowtime"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    throw new UnsupportedOperationException("A rowtime attribute can not be used solely.")
+  }
+
+  override private[flink] def resultType: TypeInformation[_] = BasicTypeInfo.LONG_TYPE_INFO
+}
+
+case class WindowReference(name: String) extends Attribute {
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode =
+    throw new UnsupportedOperationException("A window reference can not be used solely.")
+
+  override private[flink] def resultType: TypeInformation[_] =
+    throw new UnsupportedOperationException("A window reference has no result type.")
+
+  override private[flink] def withName(newName: String): Attribute = {
+    if (newName == name) {
+      this
+    } else {
+      throw new ValidationException("Cannot rename window reference.")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala
new file mode 100644
index 0000000..ccdfc2d
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/literals.scala
@@ -0,0 +1,120 @@
+/*
+ * 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.expressions
+
+import java.sql.{Date, Time, Timestamp}
+import java.util.{Calendar, TimeZone}
+
+import org.apache.calcite.avatica.util.TimeUnit
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.SqlIntervalQualifier
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo}
+
+object Literal {
+  private[flink] def apply(l: Any): Literal = l match {
+    case i: Int => Literal(i, BasicTypeInfo.INT_TYPE_INFO)
+    case s: Short => Literal(s, BasicTypeInfo.SHORT_TYPE_INFO)
+    case b: Byte => Literal(b, BasicTypeInfo.BYTE_TYPE_INFO)
+    case l: Long => Literal(l, BasicTypeInfo.LONG_TYPE_INFO)
+    case d: Double => Literal(d, BasicTypeInfo.DOUBLE_TYPE_INFO)
+    case f: Float => Literal(f, BasicTypeInfo.FLOAT_TYPE_INFO)
+    case str: String => Literal(str, BasicTypeInfo.STRING_TYPE_INFO)
+    case bool: Boolean => Literal(bool, BasicTypeInfo.BOOLEAN_TYPE_INFO)
+    case javaDec: java.math.BigDecimal => Literal(javaDec, BasicTypeInfo.BIG_DEC_TYPE_INFO)
+    case scalaDec: scala.math.BigDecimal =>
+      Literal(scalaDec.bigDecimal, BasicTypeInfo.BIG_DEC_TYPE_INFO)
+    case sqlDate: Date => Literal(sqlDate, SqlTimeTypeInfo.DATE)
+    case sqlTime: Time => Literal(sqlTime, SqlTimeTypeInfo.TIME)
+    case sqlTimestamp: Timestamp => Literal(sqlTimestamp, SqlTimeTypeInfo.TIMESTAMP)
+  }
+}
+
+case class Literal(value: Any, resultType: TypeInformation[_]) extends LeafExpression {
+  override def toString = resultType match {
+    case _: BasicTypeInfo[_] => value.toString
+    case _@SqlTimeTypeInfo.DATE => value.toString + ".toDate"
+    case _@SqlTimeTypeInfo.TIME => value.toString + ".toTime"
+    case _@SqlTimeTypeInfo.TIMESTAMP => value.toString + ".toTimestamp"
+    case _@TimeIntervalTypeInfo.INTERVAL_MILLIS => value.toString + ".millis"
+    case _@TimeIntervalTypeInfo.INTERVAL_MONTHS => value.toString + ".months"
+    case _@RowIntervalTypeInfo.INTERVAL_ROWS => value.toString + ".rows"
+    case _ => s"Literal($value, $resultType)"
+  }
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    resultType match {
+      case BasicTypeInfo.BIG_DEC_TYPE_INFO =>
+        val bigDecValue = value.asInstanceOf[java.math.BigDecimal]
+        val decType = relBuilder.getTypeFactory.createSqlType(SqlTypeName.DECIMAL)
+        relBuilder.getRexBuilder.makeExactLiteral(bigDecValue, decType)
+
+      // date/time
+      case SqlTimeTypeInfo.DATE =>
+        relBuilder.getRexBuilder.makeDateLiteral(dateToCalendar)
+      case SqlTimeTypeInfo.TIME =>
+        relBuilder.getRexBuilder.makeTimeLiteral(dateToCalendar, 0)
+      case SqlTimeTypeInfo.TIMESTAMP =>
+        relBuilder.getRexBuilder.makeTimestampLiteral(dateToCalendar, 3)
+
+      case TimeIntervalTypeInfo.INTERVAL_MONTHS =>
+        val interval = java.math.BigDecimal.valueOf(value.asInstanceOf[Int])
+        val intervalQualifier = new SqlIntervalQualifier(
+          TimeUnit.YEAR,
+          TimeUnit.MONTH,
+          SqlParserPos.ZERO)
+        relBuilder.getRexBuilder.makeIntervalLiteral(interval, intervalQualifier)
+
+      case TimeIntervalTypeInfo.INTERVAL_MILLIS =>
+        val interval = java.math.BigDecimal.valueOf(value.asInstanceOf[Long])
+        val intervalQualifier = new SqlIntervalQualifier(
+          TimeUnit.DAY,
+          TimeUnit.SECOND,
+          SqlParserPos.ZERO)
+        relBuilder.getRexBuilder.makeIntervalLiteral(interval, intervalQualifier)
+
+      case _ => relBuilder.literal(value)
+    }
+  }
+
+  private def dateToCalendar: Calendar = {
+    val date = value.asInstanceOf[java.util.Date]
+    val cal = Calendar.getInstance()
+    val t = date.getTime
+    // according to Calcite's SqlFunctions.internalToXXX methods
+    cal.setTimeInMillis(t + TimeZone.getDefault.getOffset(t))
+    cal
+  }
+}
+
+case class Null(resultType: TypeInformation[_]) extends LeafExpression {
+  override def toString = s"null"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    val rexBuilder = relBuilder.getRexBuilder
+    val typeFactory = relBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+    rexBuilder
+      .makeCast(
+        typeFactory.createTypeFromTypeInfo(resultType),
+        rexBuilder.constantNull())
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/logic.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/logic.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/logic.scala
new file mode 100644
index 0000000..dfe00cc
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/logic.scala
@@ -0,0 +1,107 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.tools.RelBuilder
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.table.validate._
+
+abstract class BinaryPredicate extends BinaryExpression {
+  override private[flink] def resultType = BasicTypeInfo.BOOLEAN_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (left.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO &&
+        right.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(s"$this only accepts children of Boolean type, " +
+        s"get $left : ${left.resultType} and $right : ${right.resultType}")
+    }
+  }
+}
+
+case class Not(child: Expression) extends UnaryExpression {
+
+  override def toString = s"!($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.not(child.toRexNode)
+  }
+
+  override private[flink] def resultType = BasicTypeInfo.BOOLEAN_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (child.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(s"Not operator requires a boolean expression as input, " +
+        s"but $child is of type ${child.resultType}")
+    }
+  }
+}
+
+case class And(left: Expression, right: Expression) extends BinaryPredicate {
+
+  override def toString = s"$left && $right"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.and(left.toRexNode, right.toRexNode)
+  }
+}
+
+case class Or(left: Expression, right: Expression) extends BinaryPredicate {
+
+  override def toString = s"$left || $right"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.or(left.toRexNode, right.toRexNode)
+  }
+}
+
+case class If(
+    condition: Expression,
+    ifTrue: Expression,
+    ifFalse: Expression)
+  extends Expression {
+  private[flink] def children = Seq(condition, ifTrue, ifFalse)
+
+  override private[flink] def resultType = ifTrue.resultType
+
+  override def toString = s"($condition)? $ifTrue : $ifFalse"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    val c = condition.toRexNode
+    val t = ifTrue.toRexNode
+    val f = ifFalse.toRexNode
+    relBuilder.call(SqlStdOperatorTable.CASE, c, t, f)
+  }
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (condition.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO &&
+        ifTrue.resultType == ifFalse.resultType) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(
+        s"If should have boolean condition and same type of ifTrue and ifFalse, get " +
+          s"(${condition.resultType}, ${ifTrue.resultType}, ${ifFalse.resultType})")
+    }
+  }
+}


[42/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
[FLINK-4704] [table] Refactor package structure of flink-table.

This closes #2958.


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

Branch: refs/heads/master
Commit: ffe9ec8ee1cf73867018e70dd1d35cb1efa267c3
Parents: 48ef46a
Author: Anton Mushin <an...@epam.com>
Authored: Wed Dec 7 16:42:42 2016 +0400
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Dec 16 16:41:20 2016 +0100

----------------------------------------------------------------------
 docs/dev/table_api.md                           |   14 +-
 .../kafka/Kafka010JsonTableSource.java          |    2 +-
 .../connectors/kafka/Kafka010TableSource.java   |    2 +-
 .../kafka/Kafka08JsonTableSource.java           |    2 +-
 .../connectors/kafka/Kafka08TableSource.java    |    2 +-
 .../kafka/Kafka09JsonTableSource.java           |    2 +-
 .../connectors/kafka/Kafka09TableSource.java    |    2 +-
 .../connectors/kafka/KafkaJsonTableSource.java  |    2 +-
 .../connectors/kafka/KafkaTableSink.java        |    2 +-
 .../connectors/kafka/KafkaTableSource.java      |    2 +-
 .../flink/api/java/table/package-info.java      |   63 -
 .../apache/flink/api/table/explain/Node.java    |  145 --
 .../flink/api/table/explain/PlanJsonParser.java |  152 --
 .../flink/examples/java/WordCountSQL.java       |   87 -
 .../flink/examples/java/WordCountTable.java     |   85 -
 .../flink/table/api/java/package-info.java      |   65 +
 .../flink/table/examples/java/WordCountSQL.java |   87 +
 .../table/examples/java/WordCountTable.java     |   85 +
 .../org/apache/flink/table/explain/Node.java    |  145 ++
 .../flink/table/explain/PlanJsonParser.java     |  152 ++
 .../api/java/table/BatchTableEnvironment.scala  |  181 ---
 .../api/java/table/StreamTableEnvironment.scala |  183 ---
 .../flink/api/java/table/groupWindows.scala     |   84 -
 .../api/scala/table/BatchTableEnvironment.scala |  154 --
 .../api/scala/table/DataSetConversions.scala    |   63 -
 .../api/scala/table/DataStreamConversions.scala |   63 -
 .../scala/table/StreamTableEnvironment.scala    |  156 --
 .../api/scala/table/TableConversions.scala      |   62 -
 .../flink/api/scala/table/expressionDsl.scala   |  691 --------
 .../flink/api/scala/table/groupWindows.scala    |   85 -
 .../apache/flink/api/scala/table/package.scala  |   90 --
 .../flink/api/table/BatchTableEnvironment.scala |  311 ----
 .../apache/flink/api/table/CalciteConfig.scala  |  161 --
 .../api/table/FlinkCalciteSqlValidator.scala    |   47 -
 .../flink/api/table/FlinkPlannerImpl.scala      |  177 --
 .../flink/api/table/FlinkRelBuilder.scala       |  103 --
 .../flink/api/table/FlinkTypeFactory.scala      |  212 ---
 .../flink/api/table/FlinkTypeSystem.scala       |   51 -
 .../api/table/StreamTableEnvironment.scala      |  351 ----
 .../apache/flink/api/table/TableConfig.scala    |  107 --
 .../flink/api/table/TableEnvironment.scala      |  537 ------
 .../org/apache/flink/api/table/Types.scala      |   45 -
 .../api/table/codegen/CodeGenException.scala    |   24 -
 .../flink/api/table/codegen/CodeGenUtils.scala  |  293 ----
 .../flink/api/table/codegen/CodeGenerator.scala | 1522 ------------------
 .../flink/api/table/codegen/Compiler.scala      |   41 -
 .../api/table/codegen/ExpressionReducer.scala   |  119 --
 .../flink/api/table/codegen/Indenter.scala      |   54 -
 .../table/codegen/calls/BuiltInMethods.scala    |   39 -
 .../api/table/codegen/calls/CallGenerator.scala |   74 -
 .../codegen/calls/CurrentTimePointCallGen.scala |   58 -
 .../table/codegen/calls/FloorCeilCallGen.scala  |   73 -
 .../table/codegen/calls/FunctionGenerator.scala |  369 -----
 .../api/table/codegen/calls/MethodCallGen.scala |   44 -
 .../codegen/calls/MultiTypeMethodCallGen.scala  |   44 -
 .../table/codegen/calls/NotCallGenerator.scala  |   37 -
 .../codegen/calls/ScalarFunctionCallGen.scala   |   95 --
 .../table/codegen/calls/ScalarOperators.scala   | 1025 ------------
 .../codegen/calls/TableFunctionCallGen.scala    |   83 -
 .../api/table/codegen/calls/TrimCallGen.scala   |   52 -
 .../flink/api/table/codegen/generated.scala     |   43 -
 .../flink/api/table/codegen/package.scala       |   25 -
 .../org/apache/flink/api/table/exceptions.scala |   73 -
 .../api/table/expressions/Expression.scala      |   88 -
 .../table/expressions/ExpressionParser.scala    |  489 ------
 .../api/table/expressions/ExpressionUtils.scala |  154 --
 .../api/table/expressions/InputTypeSpec.scala   |   57 -
 .../api/table/expressions/aggregations.scala    |  100 --
 .../api/table/expressions/arithmetic.scala      |  158 --
 .../flink/api/table/expressions/array.scala     |  146 --
 .../flink/api/table/expressions/call.scala      |  159 --
 .../flink/api/table/expressions/cast.scala      |   53 -
 .../api/table/expressions/comparison.scala      |  160 --
 .../flink/api/table/expressions/composite.scala |  106 --
 .../api/table/expressions/fieldExpression.scala |  152 --
 .../flink/api/table/expressions/literals.scala  |  120 --
 .../flink/api/table/expressions/logic.scala     |  107 --
 .../api/table/expressions/mathExpressions.scala |  129 --
 .../flink/api/table/expressions/ordering.scala  |   54 -
 .../flink/api/table/expressions/package.scala   |   29 -
 .../table/expressions/stringExpressions.scala   |  279 ----
 .../flink/api/table/expressions/symbols.scala   |  132 --
 .../flink/api/table/expressions/time.scala      |  377 -----
 .../table/expressions/windowProperties.scala    |   57 -
 .../api/table/functions/ScalarFunction.scala    |  100 --
 .../api/table/functions/TableFunction.scala     |  138 --
 .../table/functions/UserDefinedFunction.scala   |   27 -
 .../table/functions/utils/MathFunctions.scala   |   29 -
 .../functions/utils/ScalarSqlFunction.scala     |  182 ---
 .../functions/utils/TableSqlFunction.scala      |  120 --
 .../utils/UserDefinedFunctionUtils.scala        |  338 ----
 .../org/apache/flink/api/table/package.scala    |   34 -
 .../api/table/plan/ProjectionTranslator.scala   |  263 ---
 .../flink/api/table/plan/cost/DataSetCost.scala |  150 --
 .../table/plan/cost/DataSetCostFactory.scala    |   47 -
 .../api/table/plan/logical/LogicalNode.scala    |  162 --
 .../api/table/plan/logical/LogicalWindow.scala  |   36 -
 .../api/table/plan/logical/Resolvable.scala     |   43 -
 .../api/table/plan/logical/groupWindows.scala   |  258 ---
 .../api/table/plan/logical/operators.scala      |  692 --------
 .../logical/rel/LogicalWindowAggregate.scala    |  114 --
 .../api/table/plan/nodes/FlinkAggregate.scala   |   68 -
 .../flink/api/table/plan/nodes/FlinkCalc.scala  |  172 --
 .../api/table/plan/nodes/FlinkCorrelate.scala   |  163 --
 .../flink/api/table/plan/nodes/FlinkRel.scala   |  103 --
 .../table/plan/nodes/dataset/BatchScan.scala    |   97 --
 .../nodes/dataset/BatchTableSourceScan.scala    |   71 -
 .../plan/nodes/dataset/DataSetAggregate.scala   |  166 --
 .../table/plan/nodes/dataset/DataSetCalc.scala  |  136 --
 .../plan/nodes/dataset/DataSetConvention.scala  |   47 -
 .../plan/nodes/dataset/DataSetCorrelate.scala   |  139 --
 .../plan/nodes/dataset/DataSetIntersect.scala   |  133 --
 .../table/plan/nodes/dataset/DataSetJoin.scala  |  229 ---
 .../table/plan/nodes/dataset/DataSetMinus.scala |  145 --
 .../table/plan/nodes/dataset/DataSetRel.scala   |   74 -
 .../table/plan/nodes/dataset/DataSetScan.scala  |   64 -
 .../nodes/dataset/DataSetSingleRowJoin.scala    |  193 ---
 .../table/plan/nodes/dataset/DataSetSort.scala  |  194 ---
 .../table/plan/nodes/dataset/DataSetUnion.scala |  104 --
 .../plan/nodes/dataset/DataSetValues.scala      |  111 --
 .../nodes/datastream/DataStreamAggregate.scala  |  358 ----
 .../plan/nodes/datastream/DataStreamCalc.scala  |  104 --
 .../nodes/datastream/DataStreamConvention.scala |   47 -
 .../nodes/datastream/DataStreamCorrelate.scala  |  134 --
 .../plan/nodes/datastream/DataStreamRel.scala   |   45 -
 .../plan/nodes/datastream/DataStreamScan.scala  |   64 -
 .../plan/nodes/datastream/DataStreamUnion.scala |   75 -
 .../nodes/datastream/DataStreamValues.scala     |   97 --
 .../plan/nodes/datastream/StreamScan.scala      |  107 --
 .../datastream/StreamTableSourceScan.scala      |   62 -
 .../rules/EnumerableToLogicalTableScan.scala    |   49 -
 .../api/table/plan/rules/FlinkRuleSets.scala    |  161 --
 .../dataSet/BatchTableSourceScanRule.scala      |   73 -
 .../rules/dataSet/DataSetAggregateRule.scala    |   79 -
 .../DataSetAggregateWithNullValuesRule.scala    |   97 --
 .../plan/rules/dataSet/DataSetCalcRule.scala    |   52 -
 .../rules/dataSet/DataSetCorrelateRule.scala    |   91 --
 .../rules/dataSet/DataSetIntersectRule.scala    |   54 -
 .../plan/rules/dataSet/DataSetJoinRule.scala    |   73 -
 .../plan/rules/dataSet/DataSetMinusRule.scala   |   56 -
 .../plan/rules/dataSet/DataSetScanRule.scala    |   66 -
 .../dataSet/DataSetSingleRowJoinRule.scala      |   82 -
 .../plan/rules/dataSet/DataSetSortRule.scala    |   54 -
 .../plan/rules/dataSet/DataSetUnionRule.scala   |   64 -
 .../plan/rules/dataSet/DataSetValuesRule.scala  |   51 -
 ...ushProjectIntoBatchTableSourceScanRule.scala |   84 -
 .../datastream/DataStreamAggregateRule.scala    |   78 -
 .../rules/datastream/DataStreamCalcRule.scala   |   53 -
 .../datastream/DataStreamCorrelateRule.scala    |   91 --
 .../rules/datastream/DataStreamScanRule.scala   |   64 -
 .../rules/datastream/DataStreamUnionRule.scala  |   53 -
 .../rules/datastream/DataStreamValuesRule.scala |   51 -
 .../datastream/StreamTableSourceScanRule.scala  |   76 -
 .../rules/util/RexProgramProjectExtractor.scala |  120 --
 .../table/plan/schema/ArrayRelDataType.scala    |   53 -
 .../plan/schema/CompositeRelDataType.scala      |   83 -
 .../api/table/plan/schema/DataSetTable.scala    |   51 -
 .../api/table/plan/schema/DataStreamTable.scala |   28 -
 .../api/table/plan/schema/FlinkTable.scala      |   66 -
 .../plan/schema/FlinkTableFunctionImpl.scala    |   84 -
 .../table/plan/schema/GenericRelDataType.scala  |   53 -
 .../flink/api/table/plan/schema/RelTable.scala  |   46 -
 .../table/plan/schema/TableSourceTable.scala    |   30 -
 .../table/runtime/CountPartitionFunction.scala  |   40 -
 .../api/table/runtime/FlatJoinRunner.scala      |   52 -
 .../flink/api/table/runtime/FlatMapRunner.scala |   52 -
 .../runtime/IntersectCoGroupFunction.scala      |   42 -
 .../api/table/runtime/LimitFilterFunction.scala |   64 -
 .../api/table/runtime/MapJoinLeftRunner.scala   |   33 -
 .../api/table/runtime/MapJoinRightRunner.scala  |   33 -
 .../flink/api/table/runtime/MapRunner.scala     |   51 -
 .../api/table/runtime/MapSideJoinRunner.scala   |   51 -
 .../table/runtime/MinusCoGroupFunction.scala    |   47 -
 .../api/table/runtime/aggregate/Aggregate.scala |   96 --
 .../AggregateAllTimeWindowFunction.scala        |   52 -
 .../aggregate/AggregateAllWindowFunction.scala  |   41 -
 .../aggregate/AggregateMapFunction.scala        |   62 -
 .../AggregateReduceCombineFunction.scala        |   82 -
 .../AggregateReduceGroupFunction.scala          |   92 --
 .../aggregate/AggregateTimeWindowFunction.scala |   57 -
 .../table/runtime/aggregate/AggregateUtil.scala |  593 -------
 .../aggregate/AggregateWindowFunction.scala     |   46 -
 .../table/runtime/aggregate/AvgAggregate.scala  |  296 ----
 .../runtime/aggregate/CountAggregate.scala      |   55 -
 ...rementalAggregateAllTimeWindowFunction.scala |   68 -
 .../IncrementalAggregateAllWindowFunction.scala |   79 -
 .../IncrementalAggregateReduceFunction.scala    |   63 -
 ...IncrementalAggregateTimeWindowFunction.scala |   69 -
 .../IncrementalAggregateWindowFunction.scala    |   81 -
 .../table/runtime/aggregate/MaxAggregate.scala  |  171 --
 .../table/runtime/aggregate/MinAggregate.scala  |  170 --
 .../table/runtime/aggregate/SumAggregate.scala  |  130 --
 .../aggregate/TimeWindowPropertyCollector.scala |   54 -
 .../table/runtime/io/ValuesInputFormat.scala    |   53 -
 .../flink/api/table/sinks/BatchTableSink.scala  |   31 -
 .../flink/api/table/sinks/CsvTableSink.scala    |   86 -
 .../flink/api/table/sinks/StreamTableSink.scala |   31 -
 .../flink/api/table/sinks/TableSink.scala       |   58 -
 .../flink/api/table/sinks/TableSinkBase.scala   |   67 -
 .../api/table/sources/BatchTableSource.scala    |   36 -
 .../api/table/sources/CsvTableSource.scala      |  123 --
 .../table/sources/ProjectableTableSource.scala  |   38 -
 .../api/table/sources/StreamTableSource.scala   |   37 -
 .../flink/api/table/sources/TableSource.scala   |   41 -
 .../org/apache/flink/api/table/table.scala      |  922 -----------
 .../apache/flink/api/table/trees/TreeNode.scala |  115 --
 .../api/table/typeutils/InternalTypeInfo.scala  |   81 -
 .../table/typeutils/RowIntervalTypeInfo.scala   |   36 -
 .../table/typeutils/TimeIntervalTypeInfo.scala  |  113 --
 .../api/table/typeutils/TypeCheckUtils.scala    |   94 --
 .../api/table/typeutils/TypeCoercion.scala      |  121 --
 .../api/table/typeutils/TypeConverter.scala     |  155 --
 .../api/table/validate/FunctionCatalog.scala    |  320 ----
 .../api/table/validate/ValidationResult.scala   |   53 -
 .../org/apache/flink/api/table/windows.scala    |  390 -----
 .../flink/examples/scala/StreamSQLExample.scala |   76 -
 .../examples/scala/StreamTableExample.scala     |   72 -
 .../flink/examples/scala/TPCHQuery3Table.scala  |  180 ---
 .../flink/examples/scala/WordCountSQL.scala     |   62 -
 .../flink/examples/scala/WordCountTable.scala   |   62 -
 .../flink/table/api/BatchTableEnvironment.scala |  312 ++++
 .../table/api/StreamTableEnvironment.scala      |  352 ++++
 .../apache/flink/table/api/TableConfig.scala    |  109 ++
 .../flink/table/api/TableEnvironment.scala      |  538 +++++++
 .../org/apache/flink/table/api/Types.scala      |   45 +
 .../org/apache/flink/table/api/exceptions.scala |   73 +
 .../table/api/java/BatchTableEnvironment.scala  |  181 +++
 .../table/api/java/StreamTableEnvironment.scala |  183 +++
 .../flink/table/api/java/groupWindows.scala     |   84 +
 .../org/apache/flink/table/api/package.scala    |   34 +
 .../table/api/scala/BatchTableEnvironment.scala |  154 ++
 .../table/api/scala/DataSetConversions.scala    |   63 +
 .../table/api/scala/DataStreamConversions.scala |   63 +
 .../api/scala/StreamTableEnvironment.scala      |  155 ++
 .../table/api/scala/TableConversions.scala      |   61 +
 .../flink/table/api/scala/expressionDsl.scala   |  691 ++++++++
 .../flink/table/api/scala/groupWindows.scala    |   85 +
 .../apache/flink/table/api/scala/package.scala  |   93 ++
 .../org/apache/flink/table/api/table.scala      |  924 +++++++++++
 .../org/apache/flink/table/api/windows.scala    |  390 +++++
 .../flink/table/calcite/CalciteConfig.scala     |  161 ++
 .../calcite/FlinkCalciteSqlValidator.scala      |   47 +
 .../flink/table/calcite/FlinkPlannerImpl.scala  |  178 ++
 .../flink/table/calcite/FlinkRelBuilder.scala   |  103 ++
 .../flink/table/calcite/FlinkTypeFactory.scala  |  214 +++
 .../flink/table/calcite/FlinkTypeSystem.scala   |   51 +
 .../flink/table/codegen/CodeGenException.scala  |   24 +
 .../flink/table/codegen/CodeGenUtils.scala      |  292 ++++
 .../flink/table/codegen/CodeGenerator.scala     | 1522 ++++++++++++++++++
 .../apache/flink/table/codegen/Compiler.scala   |   41 +
 .../flink/table/codegen/ExpressionReducer.scala |  120 ++
 .../apache/flink/table/codegen/Indenter.scala   |   54 +
 .../table/codegen/calls/BuiltInMethods.scala    |   39 +
 .../table/codegen/calls/CallGenerator.scala     |   74 +
 .../codegen/calls/CurrentTimePointCallGen.scala |   58 +
 .../table/codegen/calls/FloorCeilCallGen.scala  |   73 +
 .../table/codegen/calls/FunctionGenerator.scala |  369 +++++
 .../table/codegen/calls/MethodCallGen.scala     |   44 +
 .../codegen/calls/MultiTypeMethodCallGen.scala  |   44 +
 .../table/codegen/calls/NotCallGenerator.scala  |   37 +
 .../codegen/calls/ScalarFunctionCallGen.scala   |   95 ++
 .../table/codegen/calls/ScalarOperators.scala   | 1025 ++++++++++++
 .../codegen/calls/TableFunctionCallGen.scala    |   83 +
 .../flink/table/codegen/calls/TrimCallGen.scala |   52 +
 .../apache/flink/table/codegen/generated.scala  |   43 +
 .../apache/flink/table/codegen/package.scala    |   25 +
 .../table/examples/scala/StreamSQLExample.scala |   76 +
 .../examples/scala/StreamTableExample.scala     |   72 +
 .../table/examples/scala/TPCHQuery3Table.scala  |  180 +++
 .../table/examples/scala/WordCountSQL.scala     |   62 +
 .../table/examples/scala/WordCountTable.scala   |   62 +
 .../flink/table/expressions/Expression.scala    |   88 +
 .../table/expressions/ExpressionParser.scala    |  489 ++++++
 .../table/expressions/ExpressionUtils.scala     |  154 ++
 .../flink/table/expressions/InputTypeSpec.scala |   57 +
 .../flink/table/expressions/aggregations.scala  |  100 ++
 .../flink/table/expressions/arithmetic.scala    |  158 ++
 .../apache/flink/table/expressions/array.scala  |  146 ++
 .../apache/flink/table/expressions/call.scala   |  160 ++
 .../apache/flink/table/expressions/cast.scala   |   53 +
 .../flink/table/expressions/comparison.scala    |  160 ++
 .../flink/table/expressions/composite.scala     |  106 ++
 .../table/expressions/fieldExpression.scala     |  152 ++
 .../flink/table/expressions/literals.scala      |  120 ++
 .../apache/flink/table/expressions/logic.scala  |  107 ++
 .../table/expressions/mathExpressions.scala     |  129 ++
 .../flink/table/expressions/ordering.scala      |   54 +
 .../flink/table/expressions/package.scala       |   29 +
 .../table/expressions/stringExpressions.scala   |  279 ++++
 .../flink/table/expressions/symbols.scala       |  132 ++
 .../apache/flink/table/expressions/time.scala   |  377 +++++
 .../table/expressions/windowProperties.scala    |   58 +
 .../flink/table/functions/ScalarFunction.scala  |  100 ++
 .../flink/table/functions/TableFunction.scala   |  138 ++
 .../table/functions/UserDefinedFunction.scala   |   27 +
 .../table/functions/utils/MathFunctions.scala   |   29 +
 .../functions/utils/ScalarSqlFunction.scala     |  183 +++
 .../functions/utils/TableSqlFunction.scala      |  120 ++
 .../utils/UserDefinedFunctionUtils.scala        |  341 ++++
 .../flink/table/plan/ProjectionTranslator.scala |  263 +++
 .../org/apache/flink/table/plan/TreeNode.scala  |  115 ++
 .../flink/table/plan/cost/DataSetCost.scala     |  150 ++
 .../table/plan/cost/DataSetCostFactory.scala    |   47 +
 .../flink/table/plan/logical/LogicalNode.scala  |  162 ++
 .../table/plan/logical/LogicalWindow.scala      |   36 +
 .../flink/table/plan/logical/Resolvable.scala   |   43 +
 .../flink/table/plan/logical/groupWindows.scala |  258 +++
 .../flink/table/plan/logical/operators.scala    |  694 ++++++++
 .../logical/rel/LogicalWindowAggregate.scala    |  114 ++
 .../flink/table/plan/nodes/FlinkAggregate.scala |   69 +
 .../flink/table/plan/nodes/FlinkCalc.scala      |  172 ++
 .../flink/table/plan/nodes/FlinkCorrelate.scala |  163 ++
 .../flink/table/plan/nodes/FlinkRel.scala       |  103 ++
 .../table/plan/nodes/dataset/BatchScan.scala    |   97 ++
 .../nodes/dataset/BatchTableSourceScan.scala    |   72 +
 .../plan/nodes/dataset/DataSetAggregate.scala   |  167 ++
 .../table/plan/nodes/dataset/DataSetCalc.scala  |  136 ++
 .../plan/nodes/dataset/DataSetConvention.scala  |   47 +
 .../plan/nodes/dataset/DataSetCorrelate.scala   |  139 ++
 .../plan/nodes/dataset/DataSetIntersect.scala   |  133 ++
 .../table/plan/nodes/dataset/DataSetJoin.scala  |  229 +++
 .../table/plan/nodes/dataset/DataSetMinus.scala |  145 ++
 .../table/plan/nodes/dataset/DataSetRel.scala   |   71 +
 .../table/plan/nodes/dataset/DataSetScan.scala  |   64 +
 .../nodes/dataset/DataSetSingleRowJoin.scala    |  193 +++
 .../table/plan/nodes/dataset/DataSetSort.scala  |  194 +++
 .../table/plan/nodes/dataset/DataSetUnion.scala |  104 ++
 .../plan/nodes/dataset/DataSetValues.scala      |  111 ++
 .../nodes/datastream/DataStreamAggregate.scala  |  359 +++++
 .../plan/nodes/datastream/DataStreamCalc.scala  |  104 ++
 .../nodes/datastream/DataStreamConvention.scala |   47 +
 .../nodes/datastream/DataStreamCorrelate.scala  |  134 ++
 .../plan/nodes/datastream/DataStreamRel.scala   |   45 +
 .../plan/nodes/datastream/DataStreamScan.scala  |   64 +
 .../plan/nodes/datastream/DataStreamUnion.scala |   75 +
 .../nodes/datastream/DataStreamValues.scala     |   97 ++
 .../plan/nodes/datastream/StreamScan.scala      |  107 ++
 .../datastream/StreamTableSourceScan.scala      |   63 +
 .../rules/EnumerableToLogicalTableScan.scala    |   49 +
 .../flink/table/plan/rules/FlinkRuleSets.scala  |  161 ++
 .../dataSet/BatchTableSourceScanRule.scala      |   73 +
 .../rules/dataSet/DataSetAggregateRule.scala    |   80 +
 .../DataSetAggregateWithNullValuesRule.scala    |   97 ++
 .../plan/rules/dataSet/DataSetCalcRule.scala    |   52 +
 .../rules/dataSet/DataSetCorrelateRule.scala    |   91 ++
 .../rules/dataSet/DataSetIntersectRule.scala    |   54 +
 .../plan/rules/dataSet/DataSetJoinRule.scala    |   73 +
 .../plan/rules/dataSet/DataSetMinusRule.scala   |   56 +
 .../plan/rules/dataSet/DataSetScanRule.scala    |   66 +
 .../dataSet/DataSetSingleRowJoinRule.scala      |   82 +
 .../plan/rules/dataSet/DataSetSortRule.scala    |   54 +
 .../plan/rules/dataSet/DataSetUnionRule.scala   |   64 +
 .../plan/rules/dataSet/DataSetValuesRule.scala  |   51 +
 ...ushProjectIntoBatchTableSourceScanRule.scala |   84 +
 .../datastream/DataStreamAggregateRule.scala    |   78 +
 .../rules/datastream/DataStreamCalcRule.scala   |   53 +
 .../datastream/DataStreamCorrelateRule.scala    |   92 ++
 .../rules/datastream/DataStreamScanRule.scala   |   64 +
 .../rules/datastream/DataStreamUnionRule.scala  |   53 +
 .../rules/datastream/DataStreamValuesRule.scala |   51 +
 .../datastream/StreamTableSourceScanRule.scala  |   76 +
 .../rules/util/RexProgramProjectExtractor.scala |  120 ++
 .../table/plan/schema/ArrayRelDataType.scala    |   53 +
 .../plan/schema/CompositeRelDataType.scala      |   83 +
 .../flink/table/plan/schema/DataSetTable.scala  |   51 +
 .../table/plan/schema/DataStreamTable.scala     |   28 +
 .../flink/table/plan/schema/FlinkTable.scala    |   67 +
 .../plan/schema/FlinkTableFunctionImpl.scala    |   85 +
 .../table/plan/schema/GenericRelDataType.scala  |   53 +
 .../flink/table/plan/schema/RelTable.scala      |   46 +
 .../table/plan/schema/TableSourceTable.scala    |   30 +
 .../table/runtime/CountPartitionFunction.scala  |   40 +
 .../flink/table/runtime/FlatJoinRunner.scala    |   52 +
 .../flink/table/runtime/FlatMapRunner.scala     |   52 +
 .../runtime/IntersectCoGroupFunction.scala      |   42 +
 .../table/runtime/LimitFilterFunction.scala     |   64 +
 .../flink/table/runtime/MapJoinLeftRunner.scala |   33 +
 .../table/runtime/MapJoinRightRunner.scala      |   33 +
 .../apache/flink/table/runtime/MapRunner.scala  |   51 +
 .../flink/table/runtime/MapSideJoinRunner.scala |   51 +
 .../table/runtime/MinusCoGroupFunction.scala    |   47 +
 .../table/runtime/aggregate/Aggregate.scala     |   96 ++
 .../AggregateAllTimeWindowFunction.scala        |   51 +
 .../aggregate/AggregateAllWindowFunction.scala  |   41 +
 .../aggregate/AggregateMapFunction.scala        |   62 +
 .../AggregateReduceCombineFunction.scala        |   80 +
 .../AggregateReduceGroupFunction.scala          |   92 ++
 .../aggregate/AggregateTimeWindowFunction.scala |   56 +
 .../table/runtime/aggregate/AggregateUtil.scala |  595 +++++++
 .../aggregate/AggregateWindowFunction.scala     |   46 +
 .../table/runtime/aggregate/AvgAggregate.scala  |  296 ++++
 .../runtime/aggregate/CountAggregate.scala      |   55 +
 ...rementalAggregateAllTimeWindowFunction.scala |   68 +
 .../IncrementalAggregateAllWindowFunction.scala |   79 +
 .../IncrementalAggregateReduceFunction.scala    |   63 +
 ...IncrementalAggregateTimeWindowFunction.scala |   69 +
 .../IncrementalAggregateWindowFunction.scala    |   81 +
 .../table/runtime/aggregate/MaxAggregate.scala  |  171 ++
 .../table/runtime/aggregate/MinAggregate.scala  |  171 ++
 .../table/runtime/aggregate/SumAggregate.scala  |  131 ++
 .../aggregate/TimeWindowPropertyCollector.scala |   54 +
 .../table/runtime/io/ValuesInputFormat.scala    |   53 +
 .../flink/table/sinks/BatchTableSink.scala      |   32 +
 .../apache/flink/table/sinks/CsvTableSink.scala |   86 +
 .../flink/table/sinks/StreamTableSink.scala     |   32 +
 .../apache/flink/table/sinks/TableSink.scala    |   59 +
 .../flink/table/sinks/TableSinkBase.scala       |   68 +
 .../flink/table/sources/BatchTableSource.scala  |   36 +
 .../flink/table/sources/CsvTableSource.scala    |  123 ++
 .../table/sources/ProjectableTableSource.scala  |   38 +
 .../flink/table/sources/StreamTableSource.scala |   37 +
 .../flink/table/sources/TableSource.scala       |   41 +
 .../table/typeutils/InternalTypeInfo.scala      |   81 +
 .../table/typeutils/RowIntervalTypeInfo.scala   |   36 +
 .../table/typeutils/TimeIntervalTypeInfo.scala  |  113 ++
 .../flink/table/typeutils/TypeCheckUtils.scala  |   94 ++
 .../flink/table/typeutils/TypeCoercion.scala    |  121 ++
 .../flink/table/typeutils/TypeConverter.scala   |  156 ++
 .../flink/table/validate/FunctionCatalog.scala  |  320 ++++
 .../flink/table/validate/ValidationResult.scala |   53 +
 .../flink/api/java/batch/ExplainTest.java       |  160 --
 .../api/java/batch/TableEnvironmentITCase.java  |  631 --------
 .../flink/api/java/batch/TableSourceITCase.java |  119 --
 .../flink/api/java/batch/sql/SqlITCase.java     |  141 --
 .../java/batch/table/AggregationsITCase.java    |  380 -----
 .../flink/api/java/batch/table/CalcITCase.java  |  324 ----
 .../api/java/batch/table/CastingITCase.java     |  140 --
 .../flink/api/java/batch/table/JoinITCase.java  |  207 ---
 .../flink/api/java/stream/sql/SqlITCase.java    |  121 --
 .../api/java/stream/utils/StreamTestData.java   |   64 -
 .../flink/table/api/java/batch/ExplainTest.java |  160 ++
 .../api/java/batch/TableEnvironmentITCase.java  |  630 ++++++++
 .../table/api/java/batch/TableSourceITCase.java |  119 ++
 .../table/api/java/batch/sql/SqlITCase.java     |  141 ++
 .../java/batch/table/AggregationsITCase.java    |  380 +++++
 .../table/api/java/batch/table/CalcITCase.java  |  324 ++++
 .../api/java/batch/table/CastingITCase.java     |  140 ++
 .../table/api/java/batch/table/JoinITCase.java  |  207 +++
 .../table/api/java/stream/sql/SqlITCase.java    |  121 ++
 .../api/java/stream/utils/StreamTestData.java   |   64 +
 .../flink/api/scala/batch/ExplainTest.scala     |  122 --
 .../batch/ProjectableTableSourceITCase.scala    |  146 --
 .../scala/batch/TableEnvironmentITCase.scala    |  273 ----
 .../flink/api/scala/batch/TableSinkITCase.scala |   71 -
 .../api/scala/batch/TableSourceITCase.scala     |  176 --
 .../scala/batch/sql/AggregationsITCase.scala    |  301 ----
 .../flink/api/scala/batch/sql/CalcITCase.scala  |  329 ----
 .../flink/api/scala/batch/sql/JoinITCase.scala  |  366 -----
 .../scala/batch/sql/SetOperatorsITCase.scala    |  266 ---
 .../api/scala/batch/sql/SetOperatorsTest.scala  |   83 -
 .../api/scala/batch/sql/SingleRowJoinTest.scala |  152 --
 .../flink/api/scala/batch/sql/SortITCase.scala  |  153 --
 .../scala/batch/sql/TableWithSQLITCase.scala    |  118 --
 .../sql/UserDefinedTableFunctionTest.scala      |  238 ---
 .../scala/batch/table/AggregationsITCase.scala  |  405 -----
 .../api/scala/batch/table/CalcITCase.scala      |  438 -----
 .../api/scala/batch/table/JoinITCase.scala      |  383 -----
 .../scala/batch/table/SetOperatorsITCase.scala  |  320 ----
 .../api/scala/batch/table/SortITCase.scala      |  186 ---
 .../table/UserDefinedTableFunctionTest.scala    |  180 ---
 .../api/scala/batch/utils/SortTestUtils.scala   |   58 -
 .../batch/utils/TableProgramsTestBase.scala     |   60 -
 .../api/scala/stream/ExplainStreamTest.scala    |   74 -
 .../api/scala/stream/TableSinkITCase.scala      |   64 -
 .../api/scala/stream/TableSourceITCase.scala    |  193 ---
 .../flink/api/scala/stream/sql/SqlITCase.scala  |  174 --
 .../sql/UserDefinedTableFunctionTest.scala      |  237 ---
 .../scala/stream/table/AggregationsITCase.scala |  199 ---
 .../api/scala/stream/table/CalcITCase.scala     |  286 ----
 .../scala/stream/table/GroupWindowTest.scala    |  734 ---------
 .../api/scala/stream/table/UnionITCase.scala    |  125 --
 .../scala/stream/table/UnsupportedOpsTest.scala |  112 --
 .../table/UserDefinedTableFunctionTest.scala    |  385 -----
 .../api/scala/stream/utils/StreamITCase.scala   |   49 -
 .../api/scala/stream/utils/StreamTestData.scala |   83 -
 .../flink/api/table/AggregationTest.scala       |  261 ---
 .../api/table/CalciteConfigBuilderTest.scala    |  208 ---
 .../api/table/CompositeFlatteningTest.scala     |  146 --
 .../api/table/ExpressionReductionTest.scala     |  424 -----
 .../flink/api/table/TableEnvironmentTest.scala  |  292 ----
 .../api/table/expressions/ArrayTypeTest.scala   |  360 -----
 .../table/expressions/CompositeAccessTest.scala |  193 ---
 .../api/table/expressions/DecimalTypeTest.scala |  311 ----
 .../expressions/NonDeterministicTests.scala     |   89 -
 .../table/expressions/ScalarFunctionsTest.scala | 1166 --------------
 .../table/expressions/ScalarOperatorsTest.scala |  220 ---
 .../table/expressions/SqlExpressionTest.scala   |  170 --
 .../table/expressions/TemporalTypesTest.scala   |  573 -------
 .../UserDefinedScalarFunctionTest.scala         |  228 ---
 .../expressions/utils/ExpressionTestBase.scala  |  217 ---
 .../utils/UserDefinedScalarFunctions.scala      |  121 --
 .../api/table/plan/FieldProjectionTest.scala    |  317 ----
 .../util/RexProgramProjectExtractorTest.scala   |  120 --
 .../runtime/aggregate/AggregateTestBase.scala   |  110 --
 .../runtime/aggregate/AvgAggregateTest.scala    |  154 --
 .../runtime/aggregate/CountAggregateTest.scala  |   31 -
 .../runtime/aggregate/MaxAggregateTest.scala    |  177 --
 .../runtime/aggregate/MinAggregateTest.scala    |  177 --
 .../runtime/aggregate/SumAggregateTest.scala    |  137 --
 .../dataset/DataSetCorrelateITCase.scala        |  178 --
 .../datastream/DataStreamCorrelateITCase.scala  |   91 --
 .../flink/api/table/utils/TableTestBase.scala   |  229 ---
 .../table/utils/UserDefinedTableFunctions.scala |  117 --
 .../apache/flink/table/AggregationTest.scala    |  261 +++
 .../flink/table/CalciteConfigBuilderTest.scala  |  209 +++
 .../flink/table/CompositeFlatteningTest.scala   |  147 ++
 .../flink/table/ExpressionReductionTest.scala   |  425 +++++
 .../flink/table/TableEnvironmentTest.scala      |  293 ++++
 .../table/api/scala/batch/ExplainTest.scala     |  122 ++
 .../batch/ProjectableTableSourceITCase.scala    |  146 ++
 .../scala/batch/TableEnvironmentITCase.scala    |  273 ++++
 .../table/api/scala/batch/TableSinkITCase.scala |   71 +
 .../api/scala/batch/TableSourceITCase.scala     |  176 ++
 .../scala/batch/sql/AggregationsITCase.scala    |  301 ++++
 .../table/api/scala/batch/sql/CalcITCase.scala  |  329 ++++
 .../table/api/scala/batch/sql/JoinITCase.scala  |  366 +++++
 .../scala/batch/sql/SetOperatorsITCase.scala    |  266 +++
 .../api/scala/batch/sql/SetOperatorsTest.scala  |   83 +
 .../api/scala/batch/sql/SingleRowJoinTest.scala |  152 ++
 .../table/api/scala/batch/sql/SortITCase.scala  |  153 ++
 .../scala/batch/sql/TableWithSQLITCase.scala    |  118 ++
 .../sql/UserDefinedTableFunctionTest.scala      |  238 +++
 .../scala/batch/table/AggregationsITCase.scala  |  405 +++++
 .../api/scala/batch/table/CalcITCase.scala      |  438 +++++
 .../scala/batch/table/FieldProjectionTest.scala |  318 ++++
 .../api/scala/batch/table/JoinITCase.scala      |  383 +++++
 .../scala/batch/table/SetOperatorsITCase.scala  |  320 ++++
 .../api/scala/batch/table/SortITCase.scala      |  186 +++
 .../table/UserDefinedTableFunctionTest.scala    |  179 ++
 .../api/scala/batch/utils/SortTestUtils.scala   |   58 +
 .../batch/utils/TableProgramsTestBase.scala     |   60 +
 .../api/scala/stream/ExplainStreamTest.scala    |   74 +
 .../api/scala/stream/TableSinkITCase.scala      |   63 +
 .../api/scala/stream/TableSourceITCase.scala    |  193 +++
 .../table/api/scala/stream/sql/SqlITCase.scala  |  174 ++
 .../sql/UserDefinedTableFunctionTest.scala      |  237 +++
 .../scala/stream/table/AggregationsITCase.scala |  199 +++
 .../api/scala/stream/table/CalcITCase.scala     |  286 ++++
 .../scala/stream/table/GroupWindowTest.scala    |  735 +++++++++
 .../api/scala/stream/table/UnionITCase.scala    |  125 ++
 .../scala/stream/table/UnsupportedOpsTest.scala |  112 ++
 .../table/UserDefinedTableFunctionTest.scala    |  385 +++++
 .../api/scala/stream/utils/StreamITCase.scala   |   51 +
 .../api/scala/stream/utils/StreamTestData.scala |   83 +
 .../flink/table/expressions/ArrayTypeTest.scala |  360 +++++
 .../table/expressions/CompositeAccessTest.scala |  193 +++
 .../table/expressions/DecimalTypeTest.scala     |  311 ++++
 .../expressions/NonDeterministicTests.scala     |   89 +
 .../table/expressions/ScalarFunctionsTest.scala | 1166 ++++++++++++++
 .../table/expressions/ScalarOperatorsTest.scala |  220 +++
 .../table/expressions/SqlExpressionTest.scala   |  170 ++
 .../table/expressions/TemporalTypesTest.scala   |  573 +++++++
 .../UserDefinedScalarFunctionTest.scala         |  228 +++
 .../expressions/utils/ExpressionTestBase.scala  |  218 +++
 .../utils/UserDefinedScalarFunctions.scala      |  121 ++
 .../util/RexProgramProjectExtractorTest.scala   |  121 ++
 .../runtime/aggregate/AggregateTestBase.scala   |  111 ++
 .../runtime/aggregate/AvgAggregateTest.scala    |  154 ++
 .../runtime/aggregate/CountAggregateTest.scala  |   31 +
 .../runtime/aggregate/MaxAggregateTest.scala    |  177 ++
 .../runtime/aggregate/MinAggregateTest.scala    |  177 ++
 .../runtime/aggregate/SumAggregateTest.scala    |  137 ++
 .../dataset/DataSetCorrelateITCase.scala        |  177 ++
 .../datastream/DataStreamCorrelateITCase.scala  |   91 ++
 .../flink/table/utils/TableTestBase.scala       |  229 +++
 .../table/utils/UserDefinedTableFunctions.scala |  117 ++
 566 files changed, 43213 insertions(+), 43174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/docs/dev/table_api.md
----------------------------------------------------------------------
diff --git a/docs/dev/table_api.md b/docs/dev/table_api.md
index d14b37a..d851c4c 100644
--- a/docs/dev/table_api.md
+++ b/docs/dev/table_api.md
@@ -398,7 +398,7 @@ Please refer to the Javadoc for a full list of supported operations and a descri
 </div>
 
 <div data-lang="scala" markdown="1">
-The Table API is enabled by importing `org.apache.flink.api.scala.table._`. This enables
+The Table API is enabled by importing `org.apache.flink.table.api.scala._`. This enables
 implicit conversions to convert a `DataSet` or `DataStream` to a Table. The following example shows:
 
 - how a `DataSet` is converted to a `Table`,
@@ -407,7 +407,7 @@ implicit conversions to convert a `DataSet` or `DataStream` to a Table. The foll
 
 {% highlight scala %}
 import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
+import org.apache.flink.table.api.scala._
 
 case class WC(word: String, count: Int)
 
@@ -448,7 +448,7 @@ The following example shows how to convert a `DataStream` to a `Table` and filte
 
 {% highlight scala %}
 import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
+import org.apache.flink.table.api.scala._
 
 val env = StreamExecutionEnvironment.getExecutionEnvironment
 val tEnv = TableEnvironment.getTableEnvironment(env)
@@ -479,7 +479,7 @@ A registered table can be accessed from a `TableEnvironment` as follows:
 ### Table API Operators
 
 The Table API features a domain-specific language to execute language-integrated queries on structured data in Scala and Java.
-This section gives a brief overview of the available operators. You can find more details of operators in the [Javadoc]({{site.baseurl}}/api/java/org/apache/flink/api/table/Table.html).
+This section gives a brief overview of the available operators. You can find more details of operators in the [Javadoc]({{site.baseurl}}/api/java/org/apache/flink/table/api/Table.html).
 
 <div class="codetabs" markdown="1">
 <div data-lang="java" markdown="1">
@@ -1451,7 +1451,7 @@ A, ABS, ABSOLUTE, ACTION, ADA, ADD, ADMIN, AFTER, ALL, ALLOCATE, ALLOW, ALTER, A
 Data Types
 ----------
 
-The Table API is built on top of Flink's DataSet and DataStream API. Internally, it also uses Flink's `TypeInformation` to distinguish between types. The Table API does not support all Flink types so far. All supported simple types are listed in `org.apache.flink.api.table.Types`. The following table summarizes the relation between Table API types, SQL types, and the resulting Java class.
+The Table API is built on top of Flink's DataSet and DataStream API. Internally, it also uses Flink's `TypeInformation` to distinguish between types. The Table API does not support all Flink types so far. All supported simple types are listed in `org.apache.flink.table.api.Types`. The following table summarizes the relation between Table API types, SQL types, and the resulting Java class.
 
 | Table API              | SQL                         | Java type              |
 | :--------------------- | :-------------------------- | :--------------------- |
@@ -3814,7 +3814,7 @@ ELEMENT(ARRAY)
 
 If a required scalar function is not contained in the built-in functions, it is possible to define custom, user-defined scalar functions for both the Table API and SQL. A user-defined scalar functions maps zero, one, or multiple scalar values to a new scalar value.
 
-In order to define a scalar function one has to extend the base class `ScalarFunction` in `org.apache.flink.api.table.functions` and implement (one or more) evaluation methods. The behavior of a scalar function is determined by the evaluation method. An evaluation method must be declared publicly and named `eval`. The parameter types and return type of the evaluation method also determine the parameter and return types of the scalar function. Evaluation methods can also be overloaded by implementing multiple methods named `eval`.
+In order to define a scalar function one has to extend the base class `ScalarFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a scalar function is determined by the evaluation method. An evaluation method must be declared publicly and named `eval`. The parameter types and return type of the evaluation method also determine the parameter and return types of the scalar function. Evaluation methods can also be overloaded by implementing multiple methods named `eval`.
 
 The following example snippet shows how to define your own hash code function:
 
@@ -3901,7 +3901,7 @@ object TimestampModifier extends ScalarFunction {
 
 A user-defined table function is implemented similar to a user-defined scalar function but can return a set of values instead of a single value. The returned set of values can consist of multiple columns and multiple rows similar to a standard table. A user-defined table function works on zero, one, or multiple scalar values as input and returns multiple rows as output.
 
-In order to define a table function one has to extend the base class `TableFunction` in `org.apache.flink.api.table.functions` and implement (one or more) evaluation methods. The behavior of a table function is determined by its evaluation methods. An evaluation method must be declared `public` and named `eval`. The `TableFunction` can be overloaded by implementing multiple methods named `eval`. The parameter types of the evaluation methods determine all valid parameters of the table function. The type of the returned table is determined by the generic type of `TableFunction`. Evaluation methods emit output rows using the protected `collect(T)` method.
+In order to define a table function one has to extend the base class `TableFunction` in `org.apache.flink.table.functions` and implement (one or more) evaluation methods. The behavior of a table function is determined by its evaluation methods. An evaluation method must be declared `public` and named `eval`. The `TableFunction` can be overloaded by implementing multiple methods named `eval`. The parameter types of the evaluation methods determine all valid parameters of the table function. The type of the returned table is determined by the generic type of `TableFunction`. Evaluation methods emit output rows using the protected `collect(T)` method.
 
 In the Table API, a table function is used with `.join(Expression)` or `.leftOuterJoin(Expression)` for Scala users and `.join(String)` or `.leftOuterJoin(String)` for Java users. The `join` operator (cross) joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator). The `leftOuterJoin` operator joins each row from the outer table (table on the left of the operator) with all rows produced by the table-valued function (which is on the right side of the operator) and preserves outer rows for which the table function returns an empty table. In SQL use `LATERAL TABLE(<TableFunction>)` with CROSS JOIN and LEFT JOIN with ON TRUE condition (see examples below).
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
index 920d718..bfae1a9 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010JsonTableSource.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.types.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
index 127dafc..e5254c0 100644
--- a/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.10/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka010TableSource.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.types.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
index 6c7d727..27e7e6e 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08JsonTableSource.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.types.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
index 0e3791c..19fd50d 100644
--- a/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.8/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka08TableSource.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.types.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
index dfcba5f..5be09fb 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09JsonTableSource.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.types.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
index f423003..e226074 100644
--- a/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
+++ b/flink-connectors/flink-connector-kafka-0.9/src/main/java/org/apache/flink/streaming/connectors/kafka/Kafka09TableSource.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.types.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 
 import java.util.Properties;

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
index f145509..3cdad0f 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaJsonTableSource.java
@@ -19,7 +19,7 @@
 package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.table.sources.StreamTableSource;
+import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;
 import org.apache.flink.streaming.util.serialization.JsonRowDeserializationSchema;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
index 6c42943..97f5fba 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSink.java
@@ -19,8 +19,8 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.types.Row;
-import org.apache.flink.api.table.sinks.StreamTableSink;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.table.sinks.StreamTableSink;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.connectors.kafka.partitioner.KafkaPartitioner;
 import org.apache.flink.streaming.util.serialization.SerializationSchema;

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
----------------------------------------------------------------------
diff --git a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
index 498e918..9a9c85d 100644
--- a/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
+++ b/flink-connectors/flink-connector-kafka-base/src/main/java/org/apache/flink/streaming/connectors/kafka/KafkaTableSource.java
@@ -20,8 +20,8 @@ package org.apache.flink.streaming.connectors.kafka;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.types.Row;
-import org.apache.flink.api.table.sources.StreamTableSource;
 import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.table.sources.StreamTableSource;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.util.serialization.DeserializationSchema;

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/java/org/apache/flink/api/java/table/package-info.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/api/java/table/package-info.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/api/java/table/package-info.java
deleted file mode 100644
index 1db4cb8..0000000
--- a/flink-libraries/flink-table/src/main/java/org/apache/flink/api/java/table/package-info.java
+++ /dev/null
@@ -1,63 +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.
- */
-
-/**
- * <strong>Table API (Java)</strong><br>
- *
- * A {@link org.apache.flink.api.java.table.BatchTableEnvironment} can be used to create a
- * {@link org.apache.flink.api.table.Table} from a {@link org.apache.flink.api.java.DataSet}.
- * Equivalently, a {@link org.apache.flink.api.java.table.StreamTableEnvironment} can be used to
- * create a {@link org.apache.flink.api.table.Table} from a
- * {@link org.apache.flink.streaming.api.datastream.DataStream}.
- *
- * <p>
- * Tables can be used to perform SQL-like queries on data. Please have
- * a look at {@link org.apache.flink.api.table.Table} to see which operations are supported and
- * how query strings are written.
- *
- * <p>
- * Example:
- *
- * <pre>{@code
- * ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- * BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env);
- *
- * DataSet<WC> input = env.fromElements(
- *   new WC("Hello", 1),
- *   new WC("Ciao", 1),
- *   new WC("Hello", 1));
- *
- * Table table = tEnv.fromDataSet(input);
- *
- * Table filtered = table
- *     .groupBy("word")
- *     .select("word.count as count, word")
- *     .filter("count = 2");
- *
- * DataSet<WC> result = tEnv.toDataSet(filtered, WC.class);
- *
- * result.print();
- * }</pre>
- *
- * <p>
- * As seen above, a {@link org.apache.flink.api.table.Table} can be converted back to the
- * underlying API representation using
- * {@link org.apache.flink.api.java.table.BatchTableEnvironment#toDataSet(org.apache.flink.api.table.Table, java.lang.Class)}
- * or {@link org.apache.flink.api.java.table.StreamTableEnvironment#toDataStream(org.apache.flink.api.table.Table, java.lang.Class)}}.
- */
-package org.apache.flink.api.java.table;

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/java/org/apache/flink/api/table/explain/Node.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/api/table/explain/Node.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/api/table/explain/Node.java
deleted file mode 100644
index 9152260..0000000
--- a/flink-libraries/flink-table/src/main/java/org/apache/flink/api/table/explain/Node.java
+++ /dev/null
@@ -1,145 +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.api.table.explain;
-
-import java.util.List;
-
-public class Node {
-	private int id;
-	private String type;
-	private String pact;
-	private String contents;
-	private int parallelism;
-	private String driver_strategy;
-	private List<Predecessors> predecessors;
-	private List<Global_properties> global_properties;
-	private List<LocalProperty> local_properties;
-	private List<Estimates> estimates;
-	private List<Costs> costs;
-	private List<Compiler_hints> compiler_hints;
-
-	public int getId() {
-		return id;
-	}
-	public String getType() {
-		return type;
-	}
-	public String getPact() {
-		return pact;
-	}
-	public String getContents() {
-		return contents;
-	}
-	public int getParallelism() {
-		return parallelism;
-	}
-	public String getDriver_strategy() {
-		return driver_strategy;
-	}
-	public List<Predecessors> getPredecessors() {
-		return predecessors;
-	}
-	public List<Global_properties> getGlobal_properties() {
-		return global_properties;
-	}
-	public List<LocalProperty> getLocal_properties() {
-		return local_properties;
-	}
-	public List<Estimates> getEstimates() {
-		return estimates;
-	}
-	public List<Costs> getCosts() {
-		return costs;
-	}
-	public List<Compiler_hints> getCompiler_hints() {
-		return compiler_hints;
-	}
-}
-
-class Predecessors {
-	private String ship_strategy;
-	private String exchange_mode;
-
-	public String getShip_strategy() {
-		return ship_strategy;
-	}
-	public String getExchange_mode() {
-		return exchange_mode;
-	}
-}
-
-class Global_properties {
-	private String name;
-	private String value;
-
-	public String getValue() {
-		return value;
-	}
-	public String getName() {
-		return name;
-	}
-}
-
-class LocalProperty {
-	private String name;
-	private String value;
-
-	public String getValue() {
-		return value;
-	}
-	public String getName() {
-		return name;
-	}
-}
-
-class Estimates {
-	private String name;
-	private String value;
-
-	public String getValue() {
-		return value;
-	}
-	public String getName() {
-		return name;
-	}
-}
-
-class Costs {
-	private String name;
-	private String value;
-
-	public String getValue() {
-		return value;
-	}
-	public String getName() {
-		return name;
-	}
-}
-
-class Compiler_hints {
-	private String name;
-	private String value;
-
-	public String getValue() {
-		return value;
-	}
-	public String getName() {
-		return name;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/java/org/apache/flink/api/table/explain/PlanJsonParser.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/api/table/explain/PlanJsonParser.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/api/table/explain/PlanJsonParser.java
deleted file mode 100644
index bd14cd2..0000000
--- a/flink-libraries/flink-table/src/main/java/org/apache/flink/api/table/explain/PlanJsonParser.java
+++ /dev/null
@@ -1,152 +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.api.table.explain;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.DeserializationFeature;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.LinkedHashMap;
-import java.util.List;
-
-public class PlanJsonParser {
-
-	public static String getSqlExecutionPlan(String t, Boolean extended) throws Exception {
-		ObjectMapper objectMapper = new ObjectMapper();
-
-		//not every node is same, ignore the unknown field
-		objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
-
-		PlanTree tree = objectMapper.readValue(t, PlanTree.class);
-		LinkedHashMap<String, Integer> map = new LinkedHashMap<>();
-		StringWriter sw = new StringWriter();
-		PrintWriter pw = new PrintWriter(sw);
-		int tabCount = 0;
-
-		for (int index = 0; index < tree.getNodes().size(); index++) {
-			Node tempNode = tree.getNodes().get(index);
-
-			//input with operation such as join or union is coordinate, keep the same indent 
-			if ((tempNode.getPact().equals("Data Source")) && (map.containsKey(tempNode.getPact()))) {
-				tabCount = map.get(tempNode.getPact());
-			}
-			else {
-				map.put(tempNode.getPact(), tabCount);
-			}
-
-			printTab(tabCount, pw);
-			pw.print("Stage " + tempNode.getId() + " : " + tempNode.getPact() + "\n");
-
-			printTab(tabCount + 1, pw);
-			String content = tempNode.getContents();
-
-			//drop the hashcode of object instance
-			int dele = tempNode.getContents().indexOf("@");
-			if (dele > -1) {
-				content = tempNode.getContents().substring(0, dele);
-			}
-
-			//replace with certain content if node is dataSource to pass
-			//unit tests, because java and scala use different api to
-			//get input element
-			if (tempNode.getPact().equals("Data Source")) {
-				content = "collect elements with CollectionInputFormat";
-			}
-			pw.print("content : " + content + "\n");
-
-			List<Predecessors> predecessors = tempNode.getPredecessors();
-			if (predecessors != null) {
-				printTab(tabCount + 1, pw);
-				pw.print("ship_strategy : " + predecessors.get(0).getShip_strategy() + "\n");
-
-				String mode = predecessors.get(0).getExchange_mode();
-				if (mode != null) {
-					printTab(tabCount + 1, pw);
-					pw.print("exchange_mode : " + mode + "\n");
-				}
-			}
-
-			if (tempNode.getDriver_strategy() != null) {
-				printTab(tabCount + 1, pw);
-				pw.print("driver_strategy : " + tempNode.getDriver_strategy() + "\n");
-			}
-
-			if (tempNode.getGlobal_properties() != null) {
-				printTab(tabCount + 1, pw);
-				pw.print(tempNode.getGlobal_properties().get(0).getName() + " : "
-					+ tempNode.getGlobal_properties().get(0).getValue() + "\n");
-			}
-
-			if (extended) {
-				List<Global_properties> globalProperties = tempNode.getGlobal_properties();
-				for (int i = 1; i < globalProperties.size(); i++) {
-					printTab(tabCount + 1, pw);
-					pw.print(globalProperties.get(i).getName() + " : "
-					+ globalProperties.get(i).getValue() + "\n");
-				}
-
-				List<LocalProperty> localProperties = tempNode.getLocal_properties();
-				for (int i = 0; i < localProperties.size(); i++) {
-					printTab(tabCount + 1, pw);
-					pw.print(localProperties.get(i).getName() + " : "
-					+ localProperties.get(i).getValue() + "\n");
-				}
-
-				List<Estimates> estimates = tempNode.getEstimates();
-				for (int i = 0; i < estimates.size(); i++) {
-					printTab(tabCount + 1, pw);
-					pw.print(estimates.get(i).getName() + " : "
-					+ estimates.get(i).getValue() + "\n");
-				}
-
-				List<Costs> costs = tempNode.getCosts();
-				for (int i = 0; i < costs.size(); i++) {
-					printTab(tabCount + 1, pw);
-					pw.print(costs.get(i).getName() + " : "
-					+ costs.get(i).getValue() + "\n");
-				}
-
-				List<Compiler_hints> compilerHintses = tempNode.getCompiler_hints();
-				for (int i = 0; i < compilerHintses.size(); i++) {
-					printTab(tabCount + 1, pw);
-					pw.print(compilerHintses.get(i).getName() + " : "
-					+ compilerHintses.get(i).getValue() + "\n");
-				}
-			}
-			tabCount++;
-			pw.print("\n");
-		}
-		pw.close();
-		return sw.toString();
-	}
-
-	private static void printTab(int tabCount, PrintWriter pw) {
-		for (int i = 0; i < tabCount; i++)
-			pw.print("\t");
-	}
-}
-
-class PlanTree {
-	private List<Node> nodes;
-
-	public List<Node> getNodes() {
-		return nodes;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/java/org/apache/flink/examples/java/WordCountSQL.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/examples/java/WordCountSQL.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/examples/java/WordCountSQL.java
deleted file mode 100644
index c9043d1..0000000
--- a/flink-libraries/flink-table/src/main/java/org/apache/flink/examples/java/WordCountSQL.java
+++ /dev/null
@@ -1,87 +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.examples.java;
-
-import org.apache.flink.api.table.Table;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.table.BatchTableEnvironment;
-import org.apache.flink.api.table.TableEnvironment;
-
-/**
- * Simple example that shows how the Batch SQL API is used in Java.
- *
- * This example shows how to:
- *  - Convert DataSets to Tables
- *  - Register a Table under a name
- *  - Run a SQL query on the registered Table
- *
- */
-public class WordCountSQL {
-
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception {
-
-		// set up execution environment
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<WC> input = env.fromElements(
-			new WC("Hello", 1),
-			new WC("Ciao", 1),
-			new WC("Hello", 1));
-
-		// register the DataSet as table "WordCount"
-		tEnv.registerDataSet("WordCount", input, "word, frequency");
-
-		// run a SQL query on the Table and retrieve the result as a new Table
-		Table table = tEnv.sql(
-			"SELECT word, SUM(frequency) as frequency FROM WordCount GROUP BY word");
-
-		DataSet<WC> result = tEnv.toDataSet(table, WC.class);
-
-		result.print();
-	}
-
-	// *************************************************************************
-	//     USER DATA TYPES
-	// *************************************************************************
-
-	public static class WC {
-		public String word;
-		public long frequency;
-
-		// public constructor to make it a Flink POJO
-		public WC() {
-		}
-
-		public WC(String word, long frequency) {
-			this.word = word;
-			this.frequency = frequency;
-		}
-
-		@Override
-		public String toString() {
-			return "WC " + word + " " + frequency;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/java/org/apache/flink/examples/java/WordCountTable.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/examples/java/WordCountTable.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/examples/java/WordCountTable.java
deleted file mode 100644
index c339cf1..0000000
--- a/flink-libraries/flink-table/src/main/java/org/apache/flink/examples/java/WordCountTable.java
+++ /dev/null
@@ -1,85 +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.examples.java;
-
-import org.apache.flink.api.table.Table;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.table.BatchTableEnvironment;
-import org.apache.flink.api.table.TableEnvironment;
-
-/**
-  * Simple example for demonstrating the use of the Table API for a Word Count in Java.
-  *
-  * This example shows how to:
-  *  - Convert DataSets to Tables
-  *  - Apply group, aggregate, select, and filter operations
-  *
-  */
-public class WordCountTable {
-
-	// *************************************************************************
-	//     PROGRAM
-	// *************************************************************************
-
-	public static void main(String[] args) throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment();
-		BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<WC> input = env.fromElements(
-				new WC("Hello", 1),
-				new WC("Ciao", 1),
-				new WC("Hello", 1));
-
-		Table table = tEnv.fromDataSet(input);
-
-		Table filtered = table
-				.groupBy("word")
-				.select("word, frequency.sum as frequency")
-				.filter("frequency = 2");
-
-		DataSet<WC> result = tEnv.toDataSet(filtered, WC.class);
-
-		result.print();
-	}
-
-	// *************************************************************************
-	//     USER DATA TYPES
-	// *************************************************************************
-
-	public static class WC {
-		public String word;
-		public long frequency;
-
-		// public constructor to make it a Flink POJO
-		public WC() {
-
-		}
-
-		public WC(String word, long frequency) {
-			this.word = word;
-			this.frequency = frequency;
-		}
-
-		@Override
-		public String toString() {
-			return "WC " + word + " " + frequency;
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/java/org/apache/flink/table/api/java/package-info.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/api/java/package-info.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/api/java/package-info.java
new file mode 100644
index 0000000..2409872
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/api/java/package-info.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * <strong>Table API (Java)</strong><br>
+ *
+ * A {@link org.apache.flink.table.api.java.BatchTableEnvironment} can be used to create a
+ * {@link org.apache.flink.table.api.Table} from a {@link org.apache.flink.api.java.DataSet}.
+ * Equivalently, a {@link org.apache.flink.table.api.java.StreamTableEnvironment} can be used to
+ * create a {@link org.apache.flink.table.api.Table} from a
+ * {@link org.apache.flink.streaming.api.datastream.DataStream}.
+ *
+ * <p>
+ * Tables can be used to perform SQL-like queries on data. Please have
+ * a look at {@link org.apache.flink.table.api.Table} to see which operations are supported and
+ * how query strings are written.
+ *
+ * <p>
+ * Example:
+ *
+ * <pre>{@code
+ * ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ * BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env);
+ *
+ * DataSet<WC> input = env.fromElements(
+ *   new WC("Hello", 1),
+ *   new WC("Ciao", 1),
+ *   new WC("Hello", 1));
+ *
+ * Table table = tEnv.fromDataSet(input);
+ *
+ * Table filtered = table
+ *     .groupBy("word")
+ *     .select("word.count as count, word")
+ *     .filter("count = 2");
+ *
+ * DataSet<WC> result = tEnv.toDataSet(filtered, WC.class);
+ *
+ * result.print();
+ * }</pre>
+ *
+ * <p>
+ * As seen above, a {@link org.apache.flink.table.api.Table} can be converted back to the
+ * underlying API representation using
+ * {@link org.apache.flink.table.api.java.BatchTableEnvironment#toDataSet(Table, java.lang.Class)}
+ * or {@link org.apache.flink.table.api.java.StreamTableEnvironment#toDataStream(Table, java.lang.Class)}}.
+ */
+package org.apache.flink.table.api.java;
+
+import org.apache.flink.table.api.Table;

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/java/org/apache/flink/table/examples/java/WordCountSQL.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/examples/java/WordCountSQL.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/examples/java/WordCountSQL.java
new file mode 100644
index 0000000..e3b345c
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/examples/java/WordCountSQL.java
@@ -0,0 +1,87 @@
+/*
+ * 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.examples.java;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.table.api.TableEnvironment;
+
+/**
+ * Simple example that shows how the Batch SQL API is used in Java.
+ *
+ * This example shows how to:
+ *  - Convert DataSets to Tables
+ *  - Register a Table under a name
+ *  - Run a SQL query on the registered Table
+ *
+ */
+public class WordCountSQL {
+
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+
+		// set up execution environment
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<WC> input = env.fromElements(
+			new WC("Hello", 1),
+			new WC("Ciao", 1),
+			new WC("Hello", 1));
+
+		// register the DataSet as table "WordCount"
+		tEnv.registerDataSet("WordCount", input, "word, frequency");
+
+		// run a SQL query on the Table and retrieve the result as a new Table
+		Table table = tEnv.sql(
+			"SELECT word, SUM(frequency) as frequency FROM WordCount GROUP BY word");
+
+		DataSet<WC> result = tEnv.toDataSet(table, WC.class);
+
+		result.print();
+	}
+
+	// *************************************************************************
+	//     USER DATA TYPES
+	// *************************************************************************
+
+	public static class WC {
+		public String word;
+		public long frequency;
+
+		// public constructor to make it a Flink POJO
+		public WC() {
+		}
+
+		public WC(String word, long frequency) {
+			this.word = word;
+			this.frequency = frequency;
+		}
+
+		@Override
+		public String toString() {
+			return "WC " + word + " " + frequency;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/java/org/apache/flink/table/examples/java/WordCountTable.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/examples/java/WordCountTable.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/examples/java/WordCountTable.java
new file mode 100644
index 0000000..bdb50f5
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/examples/java/WordCountTable.java
@@ -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.examples.java;
+
+import org.apache.flink.table.api.Table;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.table.api.TableEnvironment;
+
+/**
+  * Simple example for demonstrating the use of the Table API for a Word Count in Java.
+  *
+  * This example shows how to:
+  *  - Convert DataSets to Tables
+  *  - Apply group, aggregate, select, and filter operations
+  *
+  */
+public class WordCountTable {
+
+	// *************************************************************************
+	//     PROGRAM
+	// *************************************************************************
+
+	public static void main(String[] args) throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.createCollectionsEnvironment();
+		BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<WC> input = env.fromElements(
+				new WC("Hello", 1),
+				new WC("Ciao", 1),
+				new WC("Hello", 1));
+
+		Table table = tEnv.fromDataSet(input);
+
+		Table filtered = table
+				.groupBy("word")
+				.select("word, frequency.sum as frequency")
+				.filter("frequency = 2");
+
+		DataSet<WC> result = tEnv.toDataSet(filtered, WC.class);
+
+		result.print();
+	}
+
+	// *************************************************************************
+	//     USER DATA TYPES
+	// *************************************************************************
+
+	public static class WC {
+		public String word;
+		public long frequency;
+
+		// public constructor to make it a Flink POJO
+		public WC() {
+
+		}
+
+		public WC(String word, long frequency) {
+			this.word = word;
+			this.frequency = frequency;
+		}
+
+		@Override
+		public String toString() {
+			return "WC " + word + " " + frequency;
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/java/org/apache/flink/table/explain/Node.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/explain/Node.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/explain/Node.java
new file mode 100644
index 0000000..4616728
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/explain/Node.java
@@ -0,0 +1,145 @@
+/*
+ * 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.explain;
+
+import java.util.List;
+
+public class Node {
+	private int id;
+	private String type;
+	private String pact;
+	private String contents;
+	private int parallelism;
+	private String driver_strategy;
+	private List<Predecessors> predecessors;
+	private List<Global_properties> global_properties;
+	private List<LocalProperty> local_properties;
+	private List<Estimates> estimates;
+	private List<Costs> costs;
+	private List<Compiler_hints> compiler_hints;
+
+	public int getId() {
+		return id;
+	}
+	public String getType() {
+		return type;
+	}
+	public String getPact() {
+		return pact;
+	}
+	public String getContents() {
+		return contents;
+	}
+	public int getParallelism() {
+		return parallelism;
+	}
+	public String getDriver_strategy() {
+		return driver_strategy;
+	}
+	public List<Predecessors> getPredecessors() {
+		return predecessors;
+	}
+	public List<Global_properties> getGlobal_properties() {
+		return global_properties;
+	}
+	public List<LocalProperty> getLocal_properties() {
+		return local_properties;
+	}
+	public List<Estimates> getEstimates() {
+		return estimates;
+	}
+	public List<Costs> getCosts() {
+		return costs;
+	}
+	public List<Compiler_hints> getCompiler_hints() {
+		return compiler_hints;
+	}
+}
+
+class Predecessors {
+	private String ship_strategy;
+	private String exchange_mode;
+
+	public String getShip_strategy() {
+		return ship_strategy;
+	}
+	public String getExchange_mode() {
+		return exchange_mode;
+	}
+}
+
+class Global_properties {
+	private String name;
+	private String value;
+
+	public String getValue() {
+		return value;
+	}
+	public String getName() {
+		return name;
+	}
+}
+
+class LocalProperty {
+	private String name;
+	private String value;
+
+	public String getValue() {
+		return value;
+	}
+	public String getName() {
+		return name;
+	}
+}
+
+class Estimates {
+	private String name;
+	private String value;
+
+	public String getValue() {
+		return value;
+	}
+	public String getName() {
+		return name;
+	}
+}
+
+class Costs {
+	private String name;
+	private String value;
+
+	public String getValue() {
+		return value;
+	}
+	public String getName() {
+		return name;
+	}
+}
+
+class Compiler_hints {
+	private String name;
+	private String value;
+
+	public String getValue() {
+		return value;
+	}
+	public String getName() {
+		return name;
+	}
+}


[38/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/FloorCeilCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/FloorCeilCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/FloorCeilCallGen.scala
deleted file mode 100644
index d41e9a7..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/FloorCeilCallGen.scala
+++ /dev/null
@@ -1,73 +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.api.table.codegen.calls
-
-import java.lang.reflect.Method
-
-import org.apache.calcite.avatica.util.TimeUnitRange
-import org.apache.calcite.avatica.util.TimeUnitRange.{MONTH, YEAR}
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{BIG_DEC_TYPE_INFO, DOUBLE_TYPE_INFO, FLOAT_TYPE_INFO}
-import org.apache.flink.api.table.codegen.CodeGenUtils.{getEnum, primitiveTypeTermForTypeInfo, qualifyMethod}
-import org.apache.flink.api.table.codegen.calls.CallGenerator.generateCallIfArgsNotNull
-import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedExpression}
-
-/**
-  * Generates floor/ceil function calls.
-  */
-class FloorCeilCallGen(
-    arithmeticMethod: Method,
-    temporalMethod: Option[Method] = None)
-  extends MultiTypeMethodCallGen(arithmeticMethod) {
-
-  override def generate(
-      codeGenerator: CodeGenerator,
-      operands: Seq[GeneratedExpression])
-    : GeneratedExpression = operands.size match {
-    // arithmetic
-    case 1 =>
-      operands.head.resultType match {
-        case FLOAT_TYPE_INFO | DOUBLE_TYPE_INFO | BIG_DEC_TYPE_INFO =>
-          super.generate(codeGenerator, operands)
-        case _ =>
-          operands.head // no floor/ceil necessary
-      }
-
-    // temporal
-    case 2 =>
-      val operand = operands.head
-      val unit = getEnum(operands(1)).asInstanceOf[TimeUnitRange]
-      val internalType = primitiveTypeTermForTypeInfo(operand.resultType)
-
-      generateCallIfArgsNotNull(codeGenerator.nullCheck, operand.resultType, operands) {
-        (terms) =>
-          unit match {
-            case YEAR | MONTH =>
-              s"""
-                |($internalType) ${qualifyMethod(temporalMethod.get)}(${terms(1)}, ${terms.head})
-                |""".stripMargin
-            case _ =>
-              s"""
-                |${qualifyMethod(arithmeticMethod)}(
-                |  ($internalType) ${terms.head},
-                |  ($internalType) ${unit.startUnit.multiplier.intValue()})
-                |""".stripMargin
-          }
-      }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/FunctionGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/FunctionGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/FunctionGenerator.scala
deleted file mode 100644
index 9b144ba..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/FunctionGenerator.scala
+++ /dev/null
@@ -1,369 +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.api.table.codegen.calls
-
-import java.lang.reflect.Method
-
-import org.apache.calcite.avatica.util.TimeUnitRange
-import org.apache.calcite.sql.SqlOperator
-import org.apache.calcite.sql.fun.SqlStdOperatorTable._
-import org.apache.calcite.sql.fun.SqlTrimFunction
-import org.apache.calcite.util.BuiltInMethod
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.GenericTypeInfo
-import org.apache.flink.api.table.functions.utils.{TableSqlFunction, ScalarSqlFunction}
-
-import scala.collection.mutable
-
-/**
-  * Global hub for user-defined and built-in advanced SQL functions.
-  */
-object FunctionGenerator {
-
-  private val sqlFunctions: mutable.Map[(SqlOperator, Seq[TypeInformation[_]]), CallGenerator] =
-    mutable.Map()
-
-  // ----------------------------------------------------------------------------------------------
-  // String functions
-  // ----------------------------------------------------------------------------------------------
-
-  addSqlFunctionMethod(
-    SUBSTRING,
-    Seq(STRING_TYPE_INFO, INT_TYPE_INFO, INT_TYPE_INFO),
-    STRING_TYPE_INFO,
-    BuiltInMethod.SUBSTRING.method)
-
-  addSqlFunctionMethod(
-    SUBSTRING,
-    Seq(STRING_TYPE_INFO, INT_TYPE_INFO),
-    STRING_TYPE_INFO,
-    BuiltInMethod.SUBSTRING.method)
-
-  addSqlFunction(
-    TRIM,
-    Seq(new GenericTypeInfo(classOf[SqlTrimFunction.Flag]), STRING_TYPE_INFO, STRING_TYPE_INFO),
-    new TrimCallGen())
-
-  addSqlFunctionMethod(
-    CHAR_LENGTH,
-    Seq(STRING_TYPE_INFO),
-    INT_TYPE_INFO,
-    BuiltInMethod.CHAR_LENGTH.method)
-
-  addSqlFunctionMethod(
-    CHARACTER_LENGTH,
-    Seq(STRING_TYPE_INFO),
-    INT_TYPE_INFO,
-    BuiltInMethod.CHAR_LENGTH.method)
-
-  addSqlFunctionMethod(
-    UPPER,
-    Seq(STRING_TYPE_INFO),
-    STRING_TYPE_INFO,
-    BuiltInMethod.UPPER.method)
-
-  addSqlFunctionMethod(
-    LOWER,
-    Seq(STRING_TYPE_INFO),
-    STRING_TYPE_INFO,
-    BuiltInMethod.LOWER.method)
-
-  addSqlFunctionMethod(
-    INITCAP,
-    Seq(STRING_TYPE_INFO),
-    STRING_TYPE_INFO,
-    BuiltInMethod.INITCAP.method)
-
-  addSqlFunctionMethod(
-    LIKE,
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO),
-    BOOLEAN_TYPE_INFO,
-    BuiltInMethod.LIKE.method)
-
-  addSqlFunctionMethod(
-    LIKE,
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO, STRING_TYPE_INFO),
-    BOOLEAN_TYPE_INFO,
-    BuiltInMethods.LIKE_WITH_ESCAPE)
-
-  addSqlFunctionNotMethod(
-    NOT_LIKE,
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO),
-    BuiltInMethod.LIKE.method)
-
-  addSqlFunctionMethod(
-    SIMILAR_TO,
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO),
-    BOOLEAN_TYPE_INFO,
-    BuiltInMethod.SIMILAR.method)
-
-  addSqlFunctionMethod(
-    SIMILAR_TO,
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO, STRING_TYPE_INFO),
-    BOOLEAN_TYPE_INFO,
-    BuiltInMethods.SIMILAR_WITH_ESCAPE)
-
-  addSqlFunctionNotMethod(
-    NOT_SIMILAR_TO,
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO),
-    BuiltInMethod.SIMILAR.method)
-
-  addSqlFunctionMethod(
-    POSITION,
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO),
-    INT_TYPE_INFO,
-    BuiltInMethod.POSITION.method)
-
-  addSqlFunctionMethod(
-    OVERLAY,
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO, INT_TYPE_INFO),
-    STRING_TYPE_INFO,
-    BuiltInMethod.OVERLAY.method)
-
-  addSqlFunctionMethod(
-    OVERLAY,
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO, INT_TYPE_INFO, INT_TYPE_INFO),
-    STRING_TYPE_INFO,
-    BuiltInMethod.OVERLAY.method)
-
-  // ----------------------------------------------------------------------------------------------
-  // Arithmetic functions
-  // ----------------------------------------------------------------------------------------------
-
-  addSqlFunctionMethod(
-    LOG10,
-    Seq(DOUBLE_TYPE_INFO),
-    DOUBLE_TYPE_INFO,
-    BuiltInMethods.LOG10)
-
-  addSqlFunctionMethod(
-    LN,
-    Seq(DOUBLE_TYPE_INFO),
-    DOUBLE_TYPE_INFO,
-    BuiltInMethods.LN)
-
-  addSqlFunctionMethod(
-    EXP,
-    Seq(DOUBLE_TYPE_INFO),
-    DOUBLE_TYPE_INFO,
-    BuiltInMethods.EXP)
-
-  addSqlFunctionMethod(
-    POWER,
-    Seq(DOUBLE_TYPE_INFO, DOUBLE_TYPE_INFO),
-    DOUBLE_TYPE_INFO,
-    BuiltInMethods.POWER)
-
-  addSqlFunctionMethod(
-    POWER,
-    Seq(DOUBLE_TYPE_INFO, BIG_DEC_TYPE_INFO),
-    DOUBLE_TYPE_INFO,
-    BuiltInMethods.POWER_DEC)
-
-  addSqlFunction(
-    ABS,
-    Seq(DOUBLE_TYPE_INFO),
-    new MultiTypeMethodCallGen(BuiltInMethods.ABS))
-
-  addSqlFunction(
-    ABS,
-    Seq(BIG_DEC_TYPE_INFO),
-    new MultiTypeMethodCallGen(BuiltInMethods.ABS_DEC))
-
-  addSqlFunction(
-    FLOOR,
-    Seq(DOUBLE_TYPE_INFO),
-    new FloorCeilCallGen(BuiltInMethod.FLOOR.method))
-
-  addSqlFunction(
-    FLOOR,
-    Seq(BIG_DEC_TYPE_INFO),
-    new FloorCeilCallGen(BuiltInMethod.FLOOR.method))
-
-  addSqlFunction(
-    CEIL,
-    Seq(DOUBLE_TYPE_INFO),
-    new FloorCeilCallGen(BuiltInMethod.CEIL.method))
-
-  addSqlFunction(
-    CEIL,
-    Seq(BIG_DEC_TYPE_INFO),
-    new FloorCeilCallGen(BuiltInMethod.CEIL.method))
-
-  // ----------------------------------------------------------------------------------------------
-  // Temporal functions
-  // ----------------------------------------------------------------------------------------------
-
-  addSqlFunctionMethod(
-    EXTRACT_DATE,
-    Seq(new GenericTypeInfo(classOf[TimeUnitRange]), LONG_TYPE_INFO),
-    LONG_TYPE_INFO,
-    BuiltInMethod.UNIX_DATE_EXTRACT.method)
-
-  addSqlFunctionMethod(
-    EXTRACT_DATE,
-    Seq(new GenericTypeInfo(classOf[TimeUnitRange]), SqlTimeTypeInfo.DATE),
-    LONG_TYPE_INFO,
-    BuiltInMethod.UNIX_DATE_EXTRACT.method)
-
-  addSqlFunction(
-    FLOOR,
-    Seq(SqlTimeTypeInfo.DATE, new GenericTypeInfo(classOf[TimeUnitRange])),
-    new FloorCeilCallGen(
-      BuiltInMethod.FLOOR.method,
-      Some(BuiltInMethod.UNIX_DATE_FLOOR.method)))
-
-  addSqlFunction(
-    FLOOR,
-    Seq(SqlTimeTypeInfo.TIME, new GenericTypeInfo(classOf[TimeUnitRange])),
-    new FloorCeilCallGen(
-      BuiltInMethod.FLOOR.method,
-      Some(BuiltInMethod.UNIX_DATE_FLOOR.method)))
-
-  addSqlFunction(
-    FLOOR,
-    Seq(SqlTimeTypeInfo.TIMESTAMP, new GenericTypeInfo(classOf[TimeUnitRange])),
-    new FloorCeilCallGen(
-      BuiltInMethod.FLOOR.method,
-      Some(BuiltInMethod.UNIX_TIMESTAMP_FLOOR.method)))
-
-  addSqlFunction(
-    CEIL,
-    Seq(SqlTimeTypeInfo.DATE, new GenericTypeInfo(classOf[TimeUnitRange])),
-    new FloorCeilCallGen(
-      BuiltInMethod.CEIL.method,
-      Some(BuiltInMethod.UNIX_DATE_CEIL.method)))
-
-  addSqlFunction(
-    CEIL,
-    Seq(SqlTimeTypeInfo.TIME, new GenericTypeInfo(classOf[TimeUnitRange])),
-    new FloorCeilCallGen(
-      BuiltInMethod.CEIL.method,
-      Some(BuiltInMethod.UNIX_DATE_CEIL.method)))
-
-  addSqlFunction(
-    CEIL,
-    Seq(SqlTimeTypeInfo.TIMESTAMP, new GenericTypeInfo(classOf[TimeUnitRange])),
-    new FloorCeilCallGen(
-      BuiltInMethod.CEIL.method,
-      Some(BuiltInMethod.UNIX_TIMESTAMP_CEIL.method)))
-
-  addSqlFunction(
-    CURRENT_DATE,
-    Seq(),
-    new CurrentTimePointCallGen(SqlTimeTypeInfo.DATE, local = false))
-
-  addSqlFunction(
-    CURRENT_TIME,
-    Seq(),
-    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIME, local = false))
-
-  addSqlFunction(
-    CURRENT_TIMESTAMP,
-    Seq(),
-    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIMESTAMP, local = false))
-
-  addSqlFunction(
-    LOCALTIME,
-    Seq(),
-    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIME, local = true))
-
-  addSqlFunction(
-    LOCALTIMESTAMP,
-    Seq(),
-    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIMESTAMP, local = true))
-
-  // ----------------------------------------------------------------------------------------------
-
-  /**
-    * Returns a [[CallGenerator]] that generates all required code for calling the given
-    * [[SqlOperator]].
-    *
-    * @param sqlOperator SQL operator (might be overloaded)
-    * @param operandTypes actual operand types
-    * @param resultType expected return type
-    * @return [[CallGenerator]]
-    */
-  def getCallGenerator(
-      sqlOperator: SqlOperator,
-      operandTypes: Seq[TypeInformation[_]],
-      resultType: TypeInformation[_])
-    : Option[CallGenerator] = sqlOperator match {
-
-    // user-defined scalar function
-    case ssf: ScalarSqlFunction =>
-      Some(
-        new ScalarFunctionCallGen(
-          ssf.getScalarFunction,
-          operandTypes,
-          resultType
-        )
-      )
-
-    // user-defined table function
-    case tsf: TableSqlFunction =>
-      Some(
-        new TableFunctionCallGen(
-          tsf.getTableFunction,
-          operandTypes,
-          resultType
-        )
-      )
-
-    // built-in scalar function
-    case _ =>
-      sqlFunctions.get((sqlOperator, operandTypes))
-        .orElse(sqlFunctions.find(entry => entry._1._1 == sqlOperator
-          && entry._1._2.length == operandTypes.length
-          && entry._1._2.zip(operandTypes).forall {
-          case (x: BasicTypeInfo[_], y: BasicTypeInfo[_]) => y.shouldAutocastTo(x) || x == y
-          case _ => false
-        }).map(_._2))
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  private def addSqlFunctionMethod(
-      sqlOperator: SqlOperator,
-      operandTypes: Seq[TypeInformation[_]],
-      returnType: TypeInformation[_],
-      method: Method)
-    : Unit = {
-    sqlFunctions((sqlOperator, operandTypes)) = new MethodCallGen(returnType, method)
-  }
-
-  private def addSqlFunctionNotMethod(
-      sqlOperator: SqlOperator,
-      operandTypes: Seq[TypeInformation[_]],
-      method: Method)
-    : Unit = {
-    sqlFunctions((sqlOperator, operandTypes)) =
-      new NotCallGenerator(new MethodCallGen(BOOLEAN_TYPE_INFO, method))
-  }
-
-  private def addSqlFunction(
-      sqlOperator: SqlOperator,
-      operandTypes: Seq[TypeInformation[_]],
-      callGenerator: CallGenerator)
-    : Unit = {
-    sqlFunctions((sqlOperator, operandTypes)) = callGenerator
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/MethodCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/MethodCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/MethodCallGen.scala
deleted file mode 100644
index 376f54a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/MethodCallGen.scala
+++ /dev/null
@@ -1,44 +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.api.table.codegen.calls
-
-import java.lang.reflect.Method
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.codegen.CodeGenUtils.qualifyMethod
-import org.apache.flink.api.table.codegen.calls.CallGenerator.generateCallIfArgsNotNull
-import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedExpression}
-
-/**
-  * Generates a function call by using a [[java.lang.reflect.Method]].
-  */
-class MethodCallGen(returnType: TypeInformation[_], method: Method) extends CallGenerator {
-
-  override def generate(
-      codeGenerator: CodeGenerator,
-      operands: Seq[GeneratedExpression])
-    : GeneratedExpression = {
-    generateCallIfArgsNotNull(codeGenerator.nullCheck, returnType, operands) {
-      (terms) =>
-        s"""
-          |${qualifyMethod(method)}(${terms.mkString(", ")})
-          |""".stripMargin
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/MultiTypeMethodCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/MultiTypeMethodCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/MultiTypeMethodCallGen.scala
deleted file mode 100644
index e9e8f18..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/MultiTypeMethodCallGen.scala
+++ /dev/null
@@ -1,44 +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.api.table.codegen.calls
-
-import java.lang.reflect.Method
-
-import org.apache.flink.api.table.codegen.calls.CallGenerator._
-import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedExpression}
-
-/**
-  * Generates a function call that calls a method which returns the same type that it
-  * takes as first argument.
-  */
-class MultiTypeMethodCallGen(method: Method) extends CallGenerator {
-
-  override def generate(
-      codeGenerator: CodeGenerator,
-      operands: Seq[GeneratedExpression])
-    : GeneratedExpression = {
-    generateCallIfArgsNotNull(codeGenerator.nullCheck, operands.head.resultType, operands) {
-      (operandResultTerms) =>
-        s"""
-          |${method.getDeclaringClass.getCanonicalName}.
-          |  ${method.getName}(${operandResultTerms.mkString(", ")})
-         """.stripMargin
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/NotCallGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/NotCallGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/NotCallGenerator.scala
deleted file mode 100644
index 5cd358a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/NotCallGenerator.scala
+++ /dev/null
@@ -1,37 +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.api.table.codegen.calls
-
-import org.apache.flink.api.table.codegen.calls.ScalarOperators.generateNot
-import org.apache.flink.api.table.codegen.{GeneratedExpression, CodeGenerator}
-
-/**
-  * Inverts the boolean value of a CallGenerator result.
-  */
-class NotCallGenerator(callGenerator: CallGenerator) extends CallGenerator {
-
-  override def generate(
-      codeGenerator: CodeGenerator,
-      operands: Seq[GeneratedExpression])
-    : GeneratedExpression = {
-    val expr = callGenerator.generate(codeGenerator, operands)
-    generateNot(codeGenerator.nullCheck, expr)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctionCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctionCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctionCallGen.scala
deleted file mode 100644
index b6ef8ad..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarFunctionCallGen.scala
+++ /dev/null
@@ -1,95 +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.api.table.codegen.calls
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.codegen.CodeGenUtils._
-import org.apache.flink.api.table.codegen.{CodeGenException, CodeGenerator, GeneratedExpression}
-import org.apache.flink.api.table.functions.ScalarFunction
-import org.apache.flink.api.table.functions.utils.UserDefinedFunctionUtils._
-
-/**
-  * Generates a call to user-defined [[ScalarFunction]].
-  *
-  * @param scalarFunction user-defined [[ScalarFunction]] that might be overloaded
-  * @param signature actual signature with which the function is called
-  * @param returnType actual return type required by the surrounding
-  */
-class ScalarFunctionCallGen(
-    scalarFunction: ScalarFunction,
-    signature: Seq[TypeInformation[_]],
-    returnType: TypeInformation[_])
-  extends CallGenerator {
-
-  override def generate(
-      codeGenerator: CodeGenerator,
-      operands: Seq[GeneratedExpression])
-    : GeneratedExpression = {
-    // determine function signature and result class
-    val matchingSignature = getSignature(scalarFunction, signature)
-      .getOrElse(throw new CodeGenException("No matching signature found."))
-    val resultClass = getResultTypeClass(scalarFunction, matchingSignature)
-
-    // convert parameters for function (output boxing)
-    val parameters = matchingSignature
-        .zip(operands)
-        .map { case (paramClass, operandExpr) =>
-          if (paramClass.isPrimitive) {
-            operandExpr
-          } else {
-            val boxedTypeTerm = boxedTypeTermForTypeInfo(operandExpr.resultType)
-            val boxedExpr = codeGenerator.generateOutputFieldBoxing(operandExpr)
-            val exprOrNull: String = if (codeGenerator.nullCheck) {
-              s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}"
-            } else {
-              boxedExpr.resultTerm
-            }
-            boxedExpr.copy(resultTerm = exprOrNull)
-          }
-        }
-
-    // generate function call
-    val functionReference = codeGenerator.addReusableFunction(scalarFunction)
-    val resultTypeTerm = if (resultClass.isPrimitive) {
-      primitiveTypeTermForTypeInfo(returnType)
-    } else {
-      boxedTypeTermForTypeInfo(returnType)
-    }
-    val resultTerm = newName("result")
-    val functionCallCode =
-      s"""
-        |${parameters.map(_.code).mkString("\n")}
-        |$resultTypeTerm $resultTerm = $functionReference.eval(
-        |  ${parameters.map(_.resultTerm).mkString(", ")});
-        |""".stripMargin
-
-    // convert result of function to internal representation (input unboxing)
-    val resultUnboxing = if (resultClass.isPrimitive) {
-      codeGenerator.generateNonNullLiteral(returnType, resultTerm)
-    } else {
-      codeGenerator.generateInputFieldUnboxing(returnType, resultTerm)
-    }
-    resultUnboxing.copy(code =
-      s"""
-        |$functionCallCode
-        |${resultUnboxing.code}
-        |""".stripMargin
-    )
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarOperators.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarOperators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarOperators.scala
deleted file mode 100644
index 330e2fe..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/ScalarOperators.scala
+++ /dev/null
@@ -1,1025 +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.api.table.codegen.calls
-
-import org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY
-import org.apache.calcite.avatica.util.{DateTimeUtils, TimeUnitRange}
-import org.apache.calcite.util.BuiltInMethod
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.{NumericTypeInfo, PrimitiveArrayTypeInfo, SqlTimeTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
-import org.apache.flink.api.table.codegen.CodeGenUtils._
-import org.apache.flink.api.table.codegen.{CodeGenerator, CodeGenException, GeneratedExpression}
-import org.apache.flink.api.table.typeutils.TimeIntervalTypeInfo
-import org.apache.flink.api.table.typeutils.TypeCheckUtils._
-
-object ScalarOperators {
-
-  def generateStringConcatOperator(
-      nullCheck: Boolean,
-      left: GeneratedExpression,
-      right: GeneratedExpression)
-    : GeneratedExpression = {
-    generateOperatorIfNotNull(nullCheck, STRING_TYPE_INFO, left, right) {
-      (leftTerm, rightTerm) => s"$leftTerm + $rightTerm"
-    }
-  }
-
-  def generateArithmeticOperator(
-      operator: String,
-      nullCheck: Boolean,
-      resultType: TypeInformation[_],
-      left: GeneratedExpression,
-      right: GeneratedExpression)
-    : GeneratedExpression = {
-    val leftCasting = numericCasting(left.resultType, resultType)
-    val rightCasting = numericCasting(right.resultType, resultType)
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
-
-    generateOperatorIfNotNull(nullCheck, resultType, left, right) {
-      (leftTerm, rightTerm) =>
-        if (isDecimal(resultType)) {
-          s"${leftCasting(leftTerm)}.${arithOpToDecMethod(operator)}(${rightCasting(rightTerm)})"
-        } else {
-          s"($resultTypeTerm) (${leftCasting(leftTerm)} $operator ${rightCasting(rightTerm)})"
-        }
-    }
-  }
-
-  def generateUnaryArithmeticOperator(
-      operator: String,
-      nullCheck: Boolean,
-      resultType: TypeInformation[_],
-      operand: GeneratedExpression)
-    : GeneratedExpression = {
-    generateUnaryOperatorIfNotNull(nullCheck, resultType, operand) {
-      (operandTerm) =>
-        if (isDecimal(operand.resultType) && operator == "-") {
-          s"$operandTerm.negate()"
-        } else if (isDecimal(operand.resultType) && operator == "+") {
-          s"$operandTerm"
-        } else {
-          s"$operator($operandTerm)"
-        }
-    }
-  }
-
-  def generateEquals(
-      nullCheck: Boolean,
-      left: GeneratedExpression,
-      right: GeneratedExpression)
-    : GeneratedExpression = {
-    // numeric types
-    if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
-      generateComparison("==", nullCheck, left, right)
-    }
-    // temporal types
-    else if (isTemporal(left.resultType) && left.resultType == right.resultType) {
-      generateComparison("==", nullCheck, left, right)
-    }
-    // array types
-    else if (isArray(left.resultType) && left.resultType == right.resultType) {
-      generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
-        (leftTerm, rightTerm) => s"java.util.Arrays.equals($leftTerm, $rightTerm)"
-      }
-    }
-    // comparable types of same type
-    else if (isComparable(left.resultType) && left.resultType == right.resultType) {
-      generateComparison("==", nullCheck, left, right)
-    }
-    // non comparable types
-    else {
-      generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
-        if (isReference(left)) {
-          (leftTerm, rightTerm) => s"$leftTerm.equals($rightTerm)"
-        }
-        else if (isReference(right)) {
-          (leftTerm, rightTerm) => s"$rightTerm.equals($leftTerm)"
-        }
-        else {
-          throw new CodeGenException(s"Incomparable types: ${left.resultType} and " +
-            s"${right.resultType}")
-        }
-      }
-    }
-  }
-
-  def generateNotEquals(
-      nullCheck: Boolean,
-      left: GeneratedExpression,
-      right: GeneratedExpression)
-    : GeneratedExpression = {
-    // numeric types
-    if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
-      generateComparison("!=", nullCheck, left, right)
-    }
-    // temporal types
-    else if (isTemporal(left.resultType) && left.resultType == right.resultType) {
-      generateComparison("!=", nullCheck, left, right)
-    }
-    // array types
-    else if (isArray(left.resultType) && left.resultType == right.resultType) {
-      generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
-        (leftTerm, rightTerm) => s"!java.util.Arrays.equals($leftTerm, $rightTerm)"
-      }
-    }
-    // comparable types
-    else if (isComparable(left.resultType) && left.resultType == right.resultType) {
-      generateComparison("!=", nullCheck, left, right)
-    }
-    // non-comparable types
-    else {
-      generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
-        if (isReference(left)) {
-          (leftTerm, rightTerm) => s"!($leftTerm.equals($rightTerm))"
-        }
-        else if (isReference(right)) {
-          (leftTerm, rightTerm) => s"!($rightTerm.equals($leftTerm))"
-        }
-        else {
-          throw new CodeGenException(s"Incomparable types: ${left.resultType} and " +
-            s"${right.resultType}")
-        }
-      }
-    }
-  }
-
-  /**
-    * Generates comparison code for numeric types and comparable types of same type.
-    */
-  def generateComparison(
-      operator: String,
-      nullCheck: Boolean,
-      left: GeneratedExpression,
-      right: GeneratedExpression)
-    : GeneratedExpression = {
-    generateOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, left, right) {
-      // left is decimal or both sides are decimal
-      if (isDecimal(left.resultType) && isNumeric(right.resultType)) {
-        (leftTerm, rightTerm) => {
-          val operandCasting = numericCasting(right.resultType, left.resultType)
-          s"$leftTerm.compareTo(${operandCasting(rightTerm)}) $operator 0"
-        }
-      }
-      // right is decimal
-      else if (isNumeric(left.resultType) && isDecimal(right.resultType)) {
-        (leftTerm, rightTerm) => {
-          val operandCasting = numericCasting(left.resultType, right.resultType)
-          s"${operandCasting(leftTerm)}.compareTo($rightTerm) $operator 0"
-        }
-      }
-      // both sides are numeric
-      else if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
-        (leftTerm, rightTerm) => s"$leftTerm $operator $rightTerm"
-      }
-      // both sides are temporal of same type
-      else if (isTemporal(left.resultType) && left.resultType == right.resultType) {
-        (leftTerm, rightTerm) => s"$leftTerm $operator $rightTerm"
-      }
-      // both sides are boolean
-      else if (isBoolean(left.resultType) && left.resultType == right.resultType) {
-        operator match {
-          case "==" | "!=" => (leftTerm, rightTerm) => s"$leftTerm $operator $rightTerm"
-          case _ => throw new CodeGenException(s"Unsupported boolean comparison '$operator'.")
-        }
-      }
-      // both sides are same comparable type
-      else if (isComparable(left.resultType) && left.resultType == right.resultType) {
-        (leftTerm, rightTerm) => s"$leftTerm.compareTo($rightTerm) $operator 0"
-      }
-      else {
-        throw new CodeGenException(s"Incomparable types: ${left.resultType} and " +
-            s"${right.resultType}")
-      }
-    }
-  }
-
-  def generateIsNull(
-      nullCheck: Boolean,
-      operand: GeneratedExpression)
-    : GeneratedExpression = {
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-    val operatorCode = if (nullCheck) {
-      s"""
-        |${operand.code}
-        |boolean $resultTerm = ${operand.nullTerm};
-        |boolean $nullTerm = false;
-        |""".stripMargin
-    }
-    else if (!nullCheck && isReference(operand)) {
-      s"""
-        |${operand.code}
-        |boolean $resultTerm = ${operand.resultTerm} == null;
-        |boolean $nullTerm = false;
-        |""".stripMargin
-    }
-    else {
-      s"""
-        |${operand.code}
-        |boolean $resultTerm = false;
-        |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, operatorCode, BOOLEAN_TYPE_INFO)
-  }
-
-  def generateIsNotNull(
-      nullCheck: Boolean,
-      operand: GeneratedExpression)
-    : GeneratedExpression = {
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-    val operatorCode = if (nullCheck) {
-      s"""
-        |${operand.code}
-        |boolean $resultTerm = !${operand.nullTerm};
-        |boolean $nullTerm = false;
-        |""".stripMargin
-    }
-    else if (!nullCheck && isReference(operand)) {
-      s"""
-        |${operand.code}
-        |boolean $resultTerm = ${operand.resultTerm} != null;
-        |boolean $nullTerm = false;
-        |""".stripMargin
-    }
-    else {
-      s"""
-        |${operand.code}
-        |boolean $resultTerm = true;
-        |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, operatorCode, BOOLEAN_TYPE_INFO)
-  }
-
-  def generateAnd(
-      nullCheck: Boolean,
-      left: GeneratedExpression,
-      right: GeneratedExpression)
-    : GeneratedExpression = {
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-
-    val operatorCode = if (nullCheck) {
-      // Three-valued logic:
-      // no Unknown -> Two-valued logic
-      // True && Unknown -> Unknown
-      // False && Unknown -> False
-      // Unknown && True -> Unknown
-      // Unknown && False -> False
-      // Unknown && Unknown -> Unknown
-      s"""
-        |${left.code}
-        |${right.code}
-        |boolean $resultTerm;
-        |boolean $nullTerm;
-        |if (!${left.nullTerm} && !${right.nullTerm}) {
-        |  $resultTerm = ${left.resultTerm} && ${right.resultTerm};
-        |  $nullTerm = false;
-        |}
-        |else if (!${left.nullTerm} && ${left.resultTerm} && ${right.nullTerm}) {
-        |  $resultTerm = false;
-        |  $nullTerm = true;
-        |}
-        |else if (!${left.nullTerm} && !${left.resultTerm} && ${right.nullTerm}) {
-        |  $resultTerm = false;
-        |  $nullTerm = false;
-        |}
-        |else if (${left.nullTerm} && !${right.nullTerm} && ${right.resultTerm}) {
-        |  $resultTerm = false;
-        |  $nullTerm = true;
-        |}
-        |else if (${left.nullTerm} && !${right.nullTerm} && !${right.resultTerm}) {
-        |  $resultTerm = false;
-        |  $nullTerm = false;
-        |}
-        |else {
-        |  $resultTerm = false;
-        |  $nullTerm = true;
-        |}
-        |""".stripMargin
-    }
-    else {
-      s"""
-        |${left.code}
-        |${right.code}
-        |boolean $resultTerm = ${left.resultTerm} && ${right.resultTerm};
-        |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, operatorCode, BOOLEAN_TYPE_INFO)
-  }
-
-  def generateOr(
-      nullCheck: Boolean,
-      left: GeneratedExpression,
-      right: GeneratedExpression)
-    : GeneratedExpression = {
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-
-    val operatorCode = if (nullCheck) {
-      // Three-valued logic:
-      // no Unknown -> Two-valued logic
-      // True && Unknown -> True
-      // False && Unknown -> Unknown
-      // Unknown && True -> True
-      // Unknown && False -> Unknown
-      // Unknown && Unknown -> Unknown
-      s"""
-        |${left.code}
-        |${right.code}
-        |boolean $resultTerm;
-        |boolean $nullTerm;
-        |if (!${left.nullTerm} && !${right.nullTerm}) {
-        |  $resultTerm = ${left.resultTerm} || ${right.resultTerm};
-        |  $nullTerm = false;
-        |}
-        |else if (!${left.nullTerm} && ${left.resultTerm} && ${right.nullTerm}) {
-        |  $resultTerm = true;
-        |  $nullTerm = false;
-        |}
-        |else if (!${left.nullTerm} && !${left.resultTerm} && ${right.nullTerm}) {
-        |  $resultTerm = false;
-        |  $nullTerm = true;
-        |}
-        |else if (${left.nullTerm} && !${right.nullTerm} && ${right.resultTerm}) {
-        |  $resultTerm = true;
-        |  $nullTerm = false;
-        |}
-        |else if (${left.nullTerm} && !${right.nullTerm} && !${right.resultTerm}) {
-        |  $resultTerm = false;
-        |  $nullTerm = true;
-        |}
-        |else {
-        |  $resultTerm = false;
-        |  $nullTerm = true;
-        |}
-        |""".stripMargin
-    }
-    else {
-      s"""
-        |${left.code}
-        |${right.code}
-        |boolean $resultTerm = ${left.resultTerm} || ${right.resultTerm};
-        |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, operatorCode, BOOLEAN_TYPE_INFO)
-  }
-
-  def generateNot(
-      nullCheck: Boolean,
-      operand: GeneratedExpression)
-    : GeneratedExpression = {
-    // Three-valued logic:
-    // no Unknown -> Two-valued logic
-    // Unknown -> Unknown
-    generateUnaryOperatorIfNotNull(nullCheck, BOOLEAN_TYPE_INFO, operand) {
-      (operandTerm) => s"!($operandTerm)"
-    }
-  }
-
-  def generateIsTrue(operand: GeneratedExpression): GeneratedExpression = {
-    GeneratedExpression(
-      operand.resultTerm, // unknown is always false by default
-      GeneratedExpression.NEVER_NULL,
-      operand.code,
-      BOOLEAN_TYPE_INFO)
-  }
-
-  def generateIsNotTrue(operand: GeneratedExpression): GeneratedExpression = {
-    GeneratedExpression(
-      s"(!${operand.resultTerm})", // unknown is always false by default
-      GeneratedExpression.NEVER_NULL,
-      operand.code,
-      BOOLEAN_TYPE_INFO)
-  }
-
-  def generateIsFalse(operand: GeneratedExpression): GeneratedExpression = {
-    GeneratedExpression(
-      s"(!${operand.resultTerm} && !${operand.nullTerm})",
-      GeneratedExpression.NEVER_NULL,
-      operand.code,
-      BOOLEAN_TYPE_INFO)
-  }
-
-  def generateIsNotFalse(operand: GeneratedExpression): GeneratedExpression = {
-    GeneratedExpression(
-      s"(${operand.resultTerm} || ${operand.nullTerm})",
-      GeneratedExpression.NEVER_NULL,
-      operand.code,
-      BOOLEAN_TYPE_INFO)
-  }
-
-  def generateCast(
-      nullCheck: Boolean,
-      operand: GeneratedExpression,
-      targetType: TypeInformation[_])
-    : GeneratedExpression = (operand.resultType, targetType) match {
-    // identity casting
-    case (fromTp, toTp) if fromTp == toTp =>
-      operand
-
-    // Date/Time/Timestamp -> String
-    case (dtt: SqlTimeTypeInfo[_], STRING_TYPE_INFO) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"${internalToTimePointCode(dtt, operandTerm)}.toString()"
-      }
-
-    // Interval Months -> String
-    case (TimeIntervalTypeInfo.INTERVAL_MONTHS, STRING_TYPE_INFO) =>
-      val method = qualifyMethod(BuiltInMethod.INTERVAL_YEAR_MONTH_TO_STRING.method)
-      val timeUnitRange = qualifyEnum(TimeUnitRange.YEAR_TO_MONTH)
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"$method($operandTerm, $timeUnitRange)"
-      }
-
-    // Interval Millis -> String
-    case (TimeIntervalTypeInfo.INTERVAL_MILLIS, STRING_TYPE_INFO) =>
-      val method = qualifyMethod(BuiltInMethod.INTERVAL_DAY_TIME_TO_STRING.method)
-      val timeUnitRange = qualifyEnum(TimeUnitRange.DAY_TO_SECOND)
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"$method($operandTerm, $timeUnitRange, 3)" // milli second precision
-      }
-
-    // Object array -> String
-    case (_:ObjectArrayTypeInfo[_, _], STRING_TYPE_INFO) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"java.util.Arrays.deepToString($operandTerm)"
-      }
-
-    // Primitive array -> String
-    case (_:PrimitiveArrayTypeInfo[_], STRING_TYPE_INFO) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"java.util.Arrays.toString($operandTerm)"
-      }
-
-    // * (not Date/Time/Timestamp) -> String
-    case (_, STRING_TYPE_INFO) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s""" "" + $operandTerm"""
-      }
-
-    // * -> Character
-    case (_, CHAR_TYPE_INFO) =>
-      throw new CodeGenException("Character type not supported.")
-
-    // String -> NUMERIC TYPE (not Character), Boolean
-    case (STRING_TYPE_INFO, _: NumericTypeInfo[_])
-        | (STRING_TYPE_INFO, BOOLEAN_TYPE_INFO) =>
-      val wrapperClass = targetType.getTypeClass.getCanonicalName
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"$wrapperClass.valueOf($operandTerm)"
-      }
-
-    // String -> BigDecimal
-    case (STRING_TYPE_INFO, BIG_DEC_TYPE_INFO) =>
-      val wrapperClass = targetType.getTypeClass.getCanonicalName
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"new $wrapperClass($operandTerm)"
-      }
-
-    // String -> Date
-    case (STRING_TYPE_INFO, SqlTimeTypeInfo.DATE) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"${qualifyMethod(BuiltInMethod.STRING_TO_DATE.method)}($operandTerm)"
-      }
-
-    // String -> Time
-    case (STRING_TYPE_INFO, SqlTimeTypeInfo.TIME) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"${qualifyMethod(BuiltInMethod.STRING_TO_TIME.method)}($operandTerm)"
-      }
-
-    // String -> Timestamp
-    case (STRING_TYPE_INFO, SqlTimeTypeInfo.TIMESTAMP) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"${qualifyMethod(BuiltInMethod.STRING_TO_TIMESTAMP.method)}" +
-          s"($operandTerm)"
-      }
-
-    // Boolean -> NUMERIC TYPE
-    case (BOOLEAN_TYPE_INFO, nti: NumericTypeInfo[_]) =>
-      val targetTypeTerm = primitiveTypeTermForTypeInfo(nti)
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"($targetTypeTerm) ($operandTerm ? 1 : 0)"
-      }
-
-    // Boolean -> BigDecimal
-    case (BOOLEAN_TYPE_INFO, BIG_DEC_TYPE_INFO) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"$operandTerm ? java.math.BigDecimal.ONE : java.math.BigDecimal.ZERO"
-      }
-
-    // NUMERIC TYPE -> Boolean
-    case (_: NumericTypeInfo[_], BOOLEAN_TYPE_INFO) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"$operandTerm != 0"
-      }
-
-    // BigDecimal -> Boolean
-    case (BIG_DEC_TYPE_INFO, BOOLEAN_TYPE_INFO) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"$operandTerm.compareTo(java.math.BigDecimal.ZERO) != 0"
-      }
-
-    // NUMERIC TYPE, BigDecimal -> NUMERIC TYPE, BigDecimal
-    case (_: NumericTypeInfo[_], _: NumericTypeInfo[_])
-        | (BIG_DEC_TYPE_INFO, _: NumericTypeInfo[_])
-        | (_: NumericTypeInfo[_], BIG_DEC_TYPE_INFO) =>
-      val operandCasting = numericCasting(operand.resultType, targetType)
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"${operandCasting(operandTerm)}"
-      }
-
-    // Date -> Timestamp
-    case (SqlTimeTypeInfo.DATE, SqlTimeTypeInfo.TIMESTAMP) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) =>
-          s"$operandTerm * ${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY"
-      }
-
-    // Timestamp -> Date
-    case (SqlTimeTypeInfo.TIMESTAMP, SqlTimeTypeInfo.DATE) =>
-      val targetTypeTerm = primitiveTypeTermForTypeInfo(targetType)
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) =>
-          s"($targetTypeTerm) ($operandTerm / " +
-            s"${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY)"
-      }
-
-    // Time -> Timestamp
-    case (SqlTimeTypeInfo.TIME, SqlTimeTypeInfo.TIMESTAMP) =>
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) => s"$operandTerm"
-      }
-
-    // Timestamp -> Time
-    case (SqlTimeTypeInfo.TIMESTAMP, SqlTimeTypeInfo.TIME) =>
-      val targetTypeTerm = primitiveTypeTermForTypeInfo(targetType)
-      generateUnaryOperatorIfNotNull(nullCheck, targetType, operand) {
-        (operandTerm) =>
-          s"($targetTypeTerm) ($operandTerm % " +
-            s"${classOf[DateTimeUtils].getCanonicalName}.MILLIS_PER_DAY)"
-      }
-
-    // internal temporal casting
-    // Date -> Integer
-    // Time -> Integer
-    // Timestamp -> Long
-    // Integer -> Date
-    // Integer -> Time
-    // Long -> Timestamp
-    // Integer -> Interval Months
-    // Long -> Interval Millis
-    // Interval Months -> Integer
-    // Interval Millis -> Long
-    case (SqlTimeTypeInfo.DATE, INT_TYPE_INFO) |
-         (SqlTimeTypeInfo.TIME, INT_TYPE_INFO) |
-         (SqlTimeTypeInfo.TIMESTAMP, LONG_TYPE_INFO) |
-         (INT_TYPE_INFO, SqlTimeTypeInfo.DATE) |
-         (INT_TYPE_INFO, SqlTimeTypeInfo.TIME) |
-         (LONG_TYPE_INFO, SqlTimeTypeInfo.TIMESTAMP) |
-         (INT_TYPE_INFO, TimeIntervalTypeInfo.INTERVAL_MONTHS) |
-         (LONG_TYPE_INFO, TimeIntervalTypeInfo.INTERVAL_MILLIS) |
-         (TimeIntervalTypeInfo.INTERVAL_MONTHS, INT_TYPE_INFO) |
-         (TimeIntervalTypeInfo.INTERVAL_MILLIS, LONG_TYPE_INFO) =>
-      internalExprCasting(operand, targetType)
-
-    // internal reinterpretation of temporal types
-    // Date, Time, Interval Months -> Long
-    case  (SqlTimeTypeInfo.DATE, LONG_TYPE_INFO)
-        | (SqlTimeTypeInfo.TIME, LONG_TYPE_INFO)
-        | (TimeIntervalTypeInfo.INTERVAL_MONTHS, LONG_TYPE_INFO) =>
-      internalExprCasting(operand, targetType)
-
-    case (from, to) =>
-      throw new CodeGenException(s"Unsupported cast from '$from' to '$to'.")
-  }
-
-  def generateIfElse(
-      nullCheck: Boolean,
-      operands: Seq[GeneratedExpression],
-      resultType: TypeInformation[_],
-      i: Int = 0)
-    : GeneratedExpression = {
-    // else part
-    if (i == operands.size - 1) {
-      generateCast(nullCheck, operands(i), resultType)
-    }
-    else {
-      // check that the condition is boolean
-      // we do not check for null instead we use the default value
-      // thus null is false
-      requireBoolean(operands(i))
-      val condition = operands(i)
-      val trueAction = generateCast(nullCheck, operands(i + 1), resultType)
-      val falseAction = generateIfElse(nullCheck, operands, resultType, i + 2)
-
-      val resultTerm = newName("result")
-      val nullTerm = newName("isNull")
-      val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
-
-      val operatorCode = if (nullCheck) {
-        s"""
-          |${condition.code}
-          |$resultTypeTerm $resultTerm;
-          |boolean $nullTerm;
-          |if (${condition.resultTerm}) {
-          |  ${trueAction.code}
-          |  $resultTerm = ${trueAction.resultTerm};
-          |  $nullTerm = ${trueAction.nullTerm};
-          |}
-          |else {
-          |  ${falseAction.code}
-          |  $resultTerm = ${falseAction.resultTerm};
-          |  $nullTerm = ${falseAction.nullTerm};
-          |}
-          |""".stripMargin
-      }
-      else {
-        s"""
-          |${condition.code}
-          |$resultTypeTerm $resultTerm;
-          |if (${condition.resultTerm}) {
-          |  ${trueAction.code}
-          |  $resultTerm = ${trueAction.resultTerm};
-          |}
-          |else {
-          |  ${falseAction.code}
-          |  $resultTerm = ${falseAction.resultTerm};
-          |}
-          |""".stripMargin
-      }
-
-      GeneratedExpression(resultTerm, nullTerm, operatorCode, resultType)
-    }
-  }
-
-  def generateTemporalPlusMinus(
-      plus: Boolean,
-      nullCheck: Boolean,
-      left: GeneratedExpression,
-      right: GeneratedExpression)
-    : GeneratedExpression = {
-
-    val op = if (plus) "+" else "-"
-
-    (left.resultType, right.resultType) match {
-      case (l: TimeIntervalTypeInfo[_], r: TimeIntervalTypeInfo[_]) if l == r =>
-        generateArithmeticOperator(op, nullCheck, l, left, right)
-
-      case (SqlTimeTypeInfo.DATE, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-        generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.DATE, left, right) {
-            (l, r) => s"$l $op ((int) ($r / ${MILLIS_PER_DAY}L))"
-        }
-
-      case (SqlTimeTypeInfo.DATE, TimeIntervalTypeInfo.INTERVAL_MONTHS) =>
-        generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.DATE, left, right) {
-            (l, r) => s"${qualifyMethod(BuiltInMethod.ADD_MONTHS.method)}($l, $op($r))"
-        }
-
-      case (SqlTimeTypeInfo.TIME, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-        generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.TIME, left, right) {
-            (l, r) => s"$l $op ((int) ($r))"
-        }
-
-      case (SqlTimeTypeInfo.TIMESTAMP, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
-        generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.TIMESTAMP, left, right) {
-          (l, r) => s"$l $op $r"
-        }
-
-      case (SqlTimeTypeInfo.TIMESTAMP, TimeIntervalTypeInfo.INTERVAL_MONTHS) =>
-        generateOperatorIfNotNull(nullCheck, SqlTimeTypeInfo.TIMESTAMP, left, right) {
-          (l, r) => s"${qualifyMethod(BuiltInMethod.ADD_MONTHS.method)}($l, $op($r))"
-        }
-
-      case _ =>
-        throw new CodeGenException("Unsupported temporal arithmetic.")
-    }
-  }
-
-  def generateUnaryIntervalPlusMinus(
-      plus: Boolean,
-      nullCheck: Boolean,
-      operand: GeneratedExpression)
-    : GeneratedExpression = {
-    val operator = if (plus) "+" else "-"
-    generateUnaryArithmeticOperator(operator, nullCheck, operand.resultType, operand)
-  }
-
-  def generateArray(
-      codeGenerator: CodeGenerator,
-      resultType: TypeInformation[_],
-      elements: Seq[GeneratedExpression])
-    : GeneratedExpression = {
-    val arrayTerm = codeGenerator.addReusableArray(resultType.getTypeClass, elements.size)
-
-    val boxedElements: Seq[GeneratedExpression] = resultType match {
-
-      case oati: ObjectArrayTypeInfo[_, _] =>
-        // we box the elements to also represent null values
-        val boxedTypeTerm = boxedTypeTermForTypeInfo(oati.getComponentInfo)
-
-        elements.map { e =>
-          val boxedExpr = codeGenerator.generateOutputFieldBoxing(e)
-          val exprOrNull: String = if (codeGenerator.nullCheck) {
-            s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}"
-          } else {
-            boxedExpr.resultTerm
-          }
-          boxedExpr.copy(resultTerm = exprOrNull)
-        }
-
-      // no boxing necessary
-      case _: PrimitiveArrayTypeInfo[_] => elements
-    }
-
-    val code = boxedElements
-      .zipWithIndex
-      .map { case (element, idx) =>
-        s"""
-          |${element.code}
-          |$arrayTerm[$idx] = ${element.resultTerm};
-          |""".stripMargin
-      }
-      .mkString("\n")
-
-    GeneratedExpression(arrayTerm, GeneratedExpression.NEVER_NULL, code, resultType)
-  }
-
-  def generateArrayElementAt(
-      codeGenerator: CodeGenerator,
-      array: GeneratedExpression,
-      index: GeneratedExpression)
-    : GeneratedExpression = {
-
-    val resultTerm = newName("result")
-
-    array.resultType match {
-
-      // unbox object array types
-      case oati: ObjectArrayTypeInfo[_, _] =>
-        // get boxed array element
-        val resultTypeTerm = boxedTypeTermForTypeInfo(oati.getComponentInfo)
-
-        val arrayAccessCode = if (codeGenerator.nullCheck) {
-          s"""
-            |${array.code}
-            |${index.code}
-            |$resultTypeTerm $resultTerm = (${array.nullTerm} || ${index.nullTerm}) ?
-            |  null : ${array.resultTerm}[${index.resultTerm} - 1];
-            |""".stripMargin
-        } else {
-          s"""
-            |${array.code}
-            |${index.code}
-            |$resultTypeTerm $resultTerm = ${array.resultTerm}[${index.resultTerm} - 1];
-            |""".stripMargin
-        }
-
-        // generate unbox code
-        val unboxing = codeGenerator.generateInputFieldUnboxing(oati.getComponentInfo, resultTerm)
-
-        unboxing.copy(code =
-          s"""
-            |$arrayAccessCode
-            |${unboxing.code}
-            |""".stripMargin
-        )
-
-      // no unboxing necessary
-      case pati: PrimitiveArrayTypeInfo[_] =>
-        generateOperatorIfNotNull(codeGenerator.nullCheck, pati.getComponentType, array, index) {
-          (leftTerm, rightTerm) => s"$leftTerm[$rightTerm - 1]"
-        }
-    }
-  }
-
-  def generateArrayElement(
-      codeGenerator: CodeGenerator,
-      array: GeneratedExpression)
-    : GeneratedExpression = {
-
-    val nullTerm = newName("isNull")
-    val resultTerm = newName("result")
-    val resultType = array.resultType match {
-      case oati: ObjectArrayTypeInfo[_, _] => oati.getComponentInfo
-      case pati: PrimitiveArrayTypeInfo[_] => pati.getComponentType
-    }
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
-    val defaultValue = primitiveDefaultValue(resultType)
-
-    val arrayLengthCode = if (codeGenerator.nullCheck) {
-      s"${array.nullTerm} ? 0 : ${array.resultTerm}.length"
-    } else {
-      s"${array.resultTerm}.length"
-    }
-
-    val arrayAccessCode = array.resultType match {
-      case oati: ObjectArrayTypeInfo[_, _] =>
-        // generate unboxing code
-        val unboxing = codeGenerator.generateInputFieldUnboxing(
-          oati.getComponentInfo,
-          s"${array.resultTerm}[0]")
-
-        s"""
-          |${array.code}
-          |${if (codeGenerator.nullCheck) s"boolean $nullTerm;" else "" }
-          |$resultTypeTerm $resultTerm;
-          |switch ($arrayLengthCode) {
-          |  case 0:
-          |    ${if (codeGenerator.nullCheck) s"$nullTerm = true;" else "" }
-          |    $resultTerm = $defaultValue;
-          |    break;
-          |  case 1:
-          |    ${unboxing.code}
-          |    ${if (codeGenerator.nullCheck) s"$nullTerm = ${unboxing.nullTerm};" else "" }
-          |    $resultTerm = ${unboxing.resultTerm};
-          |    break;
-          |  default:
-          |    throw new RuntimeException("Array has more than one element.");
-          |}
-          |""".stripMargin
-
-      case pati: PrimitiveArrayTypeInfo[_] =>
-        s"""
-          |${array.code}
-          |${if (codeGenerator.nullCheck) s"boolean $nullTerm;" else "" }
-          |$resultTypeTerm $resultTerm;
-          |switch ($arrayLengthCode) {
-          |  case 0:
-          |    ${if (codeGenerator.nullCheck) s"$nullTerm = true;" else "" }
-          |    $resultTerm = $defaultValue;
-          |    break;
-          |  case 1:
-          |    ${if (codeGenerator.nullCheck) s"$nullTerm = false;" else "" }
-          |    $resultTerm = ${array.resultTerm}[0];
-          |    break;
-          |  default:
-          |    throw new RuntimeException("Array has more than one element.");
-          |}
-          |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, arrayAccessCode, resultType)
-  }
-
-  def generateArrayCardinality(
-      nullCheck: Boolean,
-      array: GeneratedExpression)
-    : GeneratedExpression = {
-
-    generateUnaryOperatorIfNotNull(nullCheck, INT_TYPE_INFO, array) {
-      (operandTerm) => s"${array.resultTerm}.length"
-    }
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  private def generateUnaryOperatorIfNotNull(
-      nullCheck: Boolean,
-      resultType: TypeInformation[_],
-      operand: GeneratedExpression)
-      (expr: (String) => String)
-    : GeneratedExpression = {
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
-    val defaultValue = primitiveDefaultValue(resultType)
-
-    val operatorCode = if (nullCheck) {
-      s"""
-        |${operand.code}
-        |$resultTypeTerm $resultTerm;
-        |boolean $nullTerm;
-        |if (!${operand.nullTerm}) {
-        |  $resultTerm = ${expr(operand.resultTerm)};
-        |  $nullTerm = false;
-        |}
-        |else {
-        |  $resultTerm = $defaultValue;
-        |  $nullTerm = true;
-        |}
-        |""".stripMargin
-    }
-    else {
-      s"""
-        |${operand.code}
-        |$resultTypeTerm $resultTerm = ${expr(operand.resultTerm)};
-        |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, operatorCode, resultType)
-  }
-
-  private def generateOperatorIfNotNull(
-      nullCheck: Boolean,
-      resultType: TypeInformation[_],
-      left: GeneratedExpression,
-      right: GeneratedExpression)
-      (expr: (String, String) => String)
-    : GeneratedExpression = {
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
-    val defaultValue = primitiveDefaultValue(resultType)
-
-    val resultCode = if (nullCheck) {
-      s"""
-        |${left.code}
-        |${right.code}
-        |boolean $nullTerm = ${left.nullTerm} || ${right.nullTerm};
-        |$resultTypeTerm $resultTerm;
-        |if ($nullTerm) {
-        |  $resultTerm = $defaultValue;
-        |}
-        |else {
-        |  $resultTerm = ${expr(left.resultTerm, right.resultTerm)};
-        |}
-        |""".stripMargin
-    }
-    else {
-      s"""
-        |${left.code}
-        |${right.code}
-        |$resultTypeTerm $resultTerm = ${expr(left.resultTerm, right.resultTerm)};
-        |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, resultCode, resultType)
-  }
-
-  private def internalExprCasting(
-      expr: GeneratedExpression,
-      typeInfo: TypeInformation[_])
-    : GeneratedExpression = {
-    GeneratedExpression(expr.resultTerm, expr.nullTerm, expr.code, typeInfo)
-  }
-
-  private def arithOpToDecMethod(operator: String): String = operator match {
-    case "+" => "add"
-    case "-" => "subtract"
-    case "*" => "multiply"
-    case "/" => "divide"
-    case "%" => "remainder"
-    case _ => throw new CodeGenException("Unsupported decimal arithmetic operator.")
-  }
-
-  private def numericCasting(
-      operandType: TypeInformation[_],
-      resultType: TypeInformation[_])
-    : (String) => String = {
-
-    def decToPrimMethod(targetType: TypeInformation[_]): String = targetType match {
-      case BYTE_TYPE_INFO => "byteValueExact"
-      case SHORT_TYPE_INFO => "shortValueExact"
-      case INT_TYPE_INFO => "intValueExact"
-      case LONG_TYPE_INFO => "longValueExact"
-      case FLOAT_TYPE_INFO => "floatValue"
-      case DOUBLE_TYPE_INFO => "doubleValue"
-      case _ => throw new CodeGenException("Unsupported decimal casting type.")
-    }
-
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
-    // no casting necessary
-    if (operandType == resultType) {
-      (operandTerm) => s"$operandTerm"
-    }
-    // result type is decimal but numeric operand is not
-    else if (isDecimal(resultType) && !isDecimal(operandType) && isNumeric(operandType)) {
-      (operandTerm) =>
-        s"java.math.BigDecimal.valueOf((${superPrimitive(operandType)}) $operandTerm)"
-    }
-    // numeric result type is not decimal but operand is
-    else if (isNumeric(resultType) && !isDecimal(resultType) && isDecimal(operandType) ) {
-      (operandTerm) => s"$operandTerm.${decToPrimMethod(resultType)}()"
-    }
-    // result type and operand type are numeric but not decimal
-    else if (isNumeric(operandType) && isNumeric(resultType)
-        && !isDecimal(operandType) && !isDecimal(resultType)) {
-      (operandTerm) => s"(($resultTypeTerm) $operandTerm)"
-    }
-    else {
-      throw new CodeGenException(s"Unsupported casting from $operandType to $resultType.")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/TableFunctionCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/TableFunctionCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/TableFunctionCallGen.scala
deleted file mode 100644
index 37e70e4..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/TableFunctionCallGen.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.api.table.codegen.calls
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.codegen.CodeGenUtils._
-import org.apache.flink.api.table.codegen.GeneratedExpression.NEVER_NULL
-import org.apache.flink.api.table.codegen.{CodeGenException, CodeGenerator, GeneratedExpression}
-import org.apache.flink.api.table.functions.TableFunction
-import org.apache.flink.api.table.functions.utils.UserDefinedFunctionUtils._
-
-/**
-  * Generates a call to user-defined [[TableFunction]].
-  *
-  * @param tableFunction user-defined [[TableFunction]] that might be overloaded
-  * @param signature actual signature with which the function is called
-  * @param returnType actual return type required by the surrounding
-  */
-class TableFunctionCallGen(
-    tableFunction: TableFunction[_],
-    signature: Seq[TypeInformation[_]],
-    returnType: TypeInformation[_])
-  extends CallGenerator {
-
-  override def generate(
-      codeGenerator: CodeGenerator,
-      operands: Seq[GeneratedExpression])
-    : GeneratedExpression = {
-    // determine function signature
-    val matchingSignature = getSignature(tableFunction, signature)
-      .getOrElse(throw new CodeGenException("No matching signature found."))
-
-    // convert parameters for function (output boxing)
-    val parameters = matchingSignature
-        .zip(operands)
-        .map { case (paramClass, operandExpr) =>
-          if (paramClass.isPrimitive) {
-            operandExpr
-          } else {
-            val boxedTypeTerm = boxedTypeTermForTypeInfo(operandExpr.resultType)
-            val boxedExpr = codeGenerator.generateOutputFieldBoxing(operandExpr)
-            val exprOrNull: String = if (codeGenerator.nullCheck) {
-              s"${boxedExpr.nullTerm} ? null : ($boxedTypeTerm) ${boxedExpr.resultTerm}"
-            } else {
-              boxedExpr.resultTerm
-            }
-            boxedExpr.copy(resultTerm = exprOrNull)
-          }
-        }
-
-    // generate function call
-    val functionReference = codeGenerator.addReusableFunction(tableFunction)
-    val functionCallCode =
-      s"""
-        |${parameters.map(_.code).mkString("\n")}
-        |$functionReference.clear();
-        |$functionReference.eval(${parameters.map(_.resultTerm).mkString(", ")});
-        |""".stripMargin
-
-    // has no result
-    GeneratedExpression(
-      functionReference,
-      NEVER_NULL,
-      functionCallCode,
-      returnType)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/TrimCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/TrimCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/TrimCallGen.scala
deleted file mode 100644
index 678016b..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/TrimCallGen.scala
+++ /dev/null
@@ -1,52 +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.api.table.codegen.calls
-
-import org.apache.calcite.sql.fun.SqlTrimFunction.Flag.{BOTH, LEADING, TRAILING}
-import org.apache.calcite.util.BuiltInMethod
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO
-import org.apache.flink.api.table.codegen.CodeGenUtils._
-import org.apache.flink.api.table.codegen.calls.CallGenerator._
-import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedExpression}
-
-/**
-  * Generates a TRIM function call.
-  *
-  * First operand: trim mode (see [[org.apache.calcite.sql.fun.SqlTrimFunction.Flag]])
-  * Second operand: String to be removed
-  * Third operand: String to be trimmed
-  */
-class TrimCallGen extends CallGenerator {
-
-  override def generate(
-      codeGenerator: CodeGenerator,
-      operands: Seq[GeneratedExpression])
-    : GeneratedExpression = {
-    generateCallIfArgsNotNull(codeGenerator.nullCheck, STRING_TYPE_INFO, operands) {
-      (terms) =>
-        val leading = compareEnum(terms.head, BOTH) || compareEnum(terms.head, LEADING)
-        val trailing = compareEnum(terms.head, BOTH) || compareEnum(terms.head, TRAILING)
-        s"""
-          |${qualifyMethod(BuiltInMethod.TRIM.method)}(
-          |  $leading, $trailing, ${terms(1)}, ${terms(2)})
-          |""".stripMargin
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/generated.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/generated.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/generated.scala
deleted file mode 100644
index bb52ad8..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/generated.scala
+++ /dev/null
@@ -1,43 +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.api.table.codegen
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-
-/**
-  * Describes a generated expression.
-  *
-  * @param resultTerm term to access the result of the expression
-  * @param nullTerm boolean term that indicates if expression is null
-  * @param code code necessary to produce resultTerm and nullTerm
-  * @param resultType type of the resultTerm
-  */
-case class GeneratedExpression(
-    resultTerm: String,
-    nullTerm: String,
-    code: String,
-    resultType: TypeInformation[_])
-
-object GeneratedExpression {
-  val ALWAYS_NULL = "true"
-  val NEVER_NULL = "false"
-  val NO_CODE = ""
-}
-
-case class GeneratedFunction[T](name: String, returnType: TypeInformation[Any], code: String)

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/package.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/package.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/package.scala
deleted file mode 100644
index b69ac1c..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/package.scala
+++ /dev/null
@@ -1,25 +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.api.table
-
-package object codegen {
-  // Used in ExpressionCodeGenerator because Scala 2.10 reflection is not thread safe. We might
-  // have several parallel expression operators in one TaskManager, therefore we need to guard
-  // these operations.
-  object ReflectionLock
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/exceptions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/exceptions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/exceptions.scala
deleted file mode 100644
index 0a100dd..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/exceptions.scala
+++ /dev/null
@@ -1,73 +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.api.table
-
-/**
-  * Exception for all errors occurring during expression parsing.
-  */
-case class ExpressionParserException(msg: String) extends RuntimeException(msg)
-
-/**
-  * Exception for all errors occurring during sql parsing.
-  */
-case class SqlParserException(
-    msg: String,
-    cause: Throwable)
-  extends RuntimeException(msg, cause) {
-
-  def this(msg: String) = this(msg, null)
-
-}
-
-/**
-  * General Exception for all errors during table handling.
-  */
-case class TableException(
-    msg: String,
-    cause: Throwable)
-  extends RuntimeException(msg, cause) {
-
-  def this(msg: String) = this(msg, null)
-
-}
-
-object TableException {
-  def apply(msg: String): TableException = new TableException(msg)
-}
-
-/**
-  * Exception for all errors occurring during validation phase.
-  */
-case class ValidationException(
-    msg: String,
-    cause: Throwable)
-  extends RuntimeException(msg, cause) {
-
-  def this(msg: String) = this(msg, null)
-
-}
-
-object ValidationException {
-  def apply(msg: String): ValidationException = new ValidationException(msg)
-}
-
-/**
-  * Exception for unwanted method calling on unresolved expression.
-  */
-case class UnresolvedException(msg: String) extends RuntimeException(msg)

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/Expression.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/Expression.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/Expression.scala
deleted file mode 100644
index c284bd3..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/Expression.scala
+++ /dev/null
@@ -1,88 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.tools.RelBuilder
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.trees.TreeNode
-import org.apache.flink.api.table.validate.{ValidationResult, ValidationSuccess}
-
-abstract class Expression extends TreeNode[Expression] {
-  /**
-    * Returns the [[TypeInformation]] for evaluating this expression.
-    * It is sometimes not available until the expression is valid.
-    */
-  private[flink] def resultType: TypeInformation[_]
-
-  /**
-    * One pass validation of the expression tree in post order.
-    */
-  private[flink] lazy val valid: Boolean = childrenValid && validateInput().isSuccess
-
-  private[flink] def childrenValid: Boolean = children.forall(_.valid)
-
-  /**
-    * Check input data types, inputs number or other properties specified by this expression.
-    * Return `ValidationSuccess` if it pass the check,
-    * or `ValidationFailure` with supplement message explaining the error.
-    * Note: we should only call this method until `childrenValid == true`
-    */
-  private[flink] def validateInput(): ValidationResult = ValidationSuccess
-
-  /**
-    * Convert Expression to its counterpart in Calcite, i.e. RexNode
-    */
-  private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode =
-    throw new UnsupportedOperationException(
-      s"${this.getClass.getName} cannot be transformed to RexNode"
-    )
-
-  private[flink] def checkEquals(other: Expression): Boolean = {
-    if (this.getClass != other.getClass) {
-      false
-    } else {
-      def checkEquality(elements1: Seq[Any], elements2: Seq[Any]): Boolean = {
-        elements1.length == elements2.length && elements1.zip(elements2).forall {
-          case (e1: Expression, e2: Expression) => e1.checkEquals(e2)
-          case (t1: Seq[_], t2: Seq[_]) => checkEquality(t1, t2)
-          case (i1, i2) => i1 == i2
-        }
-      }
-      val elements1 = this.productIterator.toSeq
-      val elements2 = other.productIterator.toSeq
-      checkEquality(elements1, elements2)
-    }
-  }
-}
-
-abstract class BinaryExpression extends Expression {
-  private[flink] def left: Expression
-  private[flink] def right: Expression
-  private[flink] def children = Seq(left, right)
-}
-
-abstract class UnaryExpression extends Expression {
-  private[flink] def child: Expression
-  private[flink] def children = Seq(child)
-}
-
-abstract class LeafExpression extends Expression {
-  private[flink] val children = Nil
-}


[11/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala
deleted file mode 100644
index 195027d..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/JoinITCase.scala
+++ /dev/null
@@ -1,383 +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.api.scala.batch.table
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.expressions.Literal
-import org.apache.flink.api.table.{TableEnvironment, TableException, ValidationException}
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class JoinITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testJoin(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.join(ds2).where('b === 'e).select('c, 'g)
-
-    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n"
-    val results = joinT.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testJoinWithFilter(): Unit = {
-
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.join(ds2).where('b === 'e && 'b < 2).select('c, 'g)
-
-    val expected = "Hi,Hallo\n"
-    val results = joinT.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testJoinWithJoinFilter(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.join(ds2).where('b === 'e && 'a < 6 && 'h < 'b).select('c, 'g)
-
-    val expected = "Hello world, how are you?,Hallo Welt wie\n" +
-      "I am fine.,Hallo Welt wie\n"
-    val results = joinT.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testJoinWithMultipleKeys(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.join(ds2).filter('a === 'd && 'b === 'h).select('c, 'g)
-
-    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" +
-      "Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n"
-    val results = joinT.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testJoinNonExistingKey(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    ds1.join(ds2)
-      // must fail. Field 'foo does not exist
-      .where('foo === 'e)
-      .select('c, 'g)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testJoinWithNonMatchingKeyTypes(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    ds1.join(ds2)
-      // must fail. Field 'a is Int, and 'g is String
-      .where('a === 'g)
-      .select('c, 'g).collect()
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testJoinWithAmbiguousFields(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'c)
-
-    ds1.join(ds2)
-      // must fail. Both inputs share the same field 'c
-      .where('a === 'd)
-      .select('c, 'g)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testNoEqualityJoinPredicate1(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    ds1.join(ds2)
-      // must fail. No equality join predicate
-      .where('d === 'f)
-      .select('c, 'g).collect()
-  }
-
-  @Test(expected = classOf[TableException])
-  def testNoEqualityJoinPredicate2(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    ds1.join(ds2)
-      // must fail. No equality join predicate
-      .where('a < 'd)
-      .select('c, 'g).collect()
-  }
-
-  @Test
-  def testJoinWithAggregation(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.join(ds2).where('a === 'd).select('g.count)
-
-    val expected = "6"
-    val results = joinT.toDataSet[Row] collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testJoinWithGroupedAggregation(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.join(ds2)
-      .where('a === 'd)
-      .groupBy('a, 'd)
-      .select('b.sum, 'g.count)
-
-    val expected = "6,3\n" + "4,2\n" + "1,1"
-    val results = joinT.toDataSet[Row] collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testJoinPushThroughJoin(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-    val ds3 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'j, 'k, 'l)
-
-    val joinT = ds1.join(ds2)
-      .where(Literal(true))
-      .join(ds3)
-      .where('a === 'd && 'e === 'k)
-      .select('a, 'f, 'l)
-
-    val expected = "2,1,Hello\n" + "2,1,Hello world\n" + "1,0,Hi"
-    val results = joinT.toDataSet[Row] collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testJoinWithDisjunctivePred(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.join(ds2).filter('a === 'd && ('b === 'e || 'b === 'e - 10)).select('c, 'g)
-
-    val expected = "Hi,Hallo\n" +
-      "Hello,Hallo Welt\n" +
-      "I am fine.,IJK"
-    val results = joinT.toDataSet[Row] collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testJoinWithExpressionPreds(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.join(ds2).filter('b === 'h + 1 && 'a - 1 === 'd + 2).select('c, 'g)
-
-    val expected = "I am fine.,Hallo Welt\n" +
-      "Luke Skywalker,Hallo Welt wie gehts?\n" +
-      "Luke Skywalker,ABC\n" +
-      "Comment#2,HIJ\n" +
-      "Comment#2,IJK"
-    val results = joinT.toDataSet[Row] collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testJoinTablesFromDifferentEnvs(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv1 = TableEnvironment.getTableEnvironment(env, config)
-    val tEnv2 = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv1, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv2, 'd, 'e, 'f, 'g, 'h)
-
-    // Must fail. Tables are bound to different TableEnvironments.
-    ds1.join(ds2).where('b === 'e).select('c, 'g)
-  }
-
-  @Test
-  def testLeftJoinWithMultipleKeys(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    tEnv.getConfig.setNullCheck(true)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.leftOuterJoin(ds2, 'a === 'd && 'b === 'h).select('c, 'g)
-
-    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" +
-      "Hello world,ABC\n" + "Hello world, how are you?,null\n" + "I am fine.,HIJ\n" +
-      "I am fine.,IJK\n" + "Luke Skywalker,null\n" + "Comment#1,null\n" + "Comment#2,null\n" +
-      "Comment#3,null\n" + "Comment#4,null\n" + "Comment#5,null\n" + "Comment#6,null\n" +
-      "Comment#7,null\n" + "Comment#8,null\n" + "Comment#9,null\n" + "Comment#10,null\n" +
-      "Comment#11,null\n" + "Comment#12,null\n" + "Comment#13,null\n" + "Comment#14,null\n" +
-      "Comment#15,null\n"
-    val results = joinT.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testNoJoinCondition(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    tEnv.getConfig.setNullCheck(true)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds2.leftOuterJoin(ds1, 'b === 'd && 'b < 3).select('c, 'g)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testNoEquiJoin(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    tEnv.getConfig.setNullCheck(true)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds2.leftOuterJoin(ds1, 'b < 'd).select('c, 'g)
-  }
-
-  @Test
-  def testRightJoinWithMultipleKeys(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    tEnv.getConfig.setNullCheck(true)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.rightOuterJoin(ds2, "a = d && b = h").select('c, 'g)
-
-    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "null,Hallo Welt wie\n" +
-      "Hello world,Hallo Welt wie gehts?\n" + "Hello world,ABC\n" + "null,BCD\n" + "null,CDE\n" +
-      "null,DEF\n" + "null,EFG\n" + "null,FGH\n" + "null,GHI\n" + "I am fine.,HIJ\n" +
-      "I am fine.,IJK\n" + "null,JKL\n" + "null,KLM\n"
-    val results = joinT.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testRightJoinWithNotOnlyEquiJoin(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    tEnv.getConfig.setNullCheck(true)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.rightOuterJoin(ds2, "a = d && b < h").select('c, 'g)
-
-    val expected = "Hello world,BCD\n"
-    val results = joinT.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testFullOuterJoinWithMultipleKeys(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-    tEnv.getConfig.setNullCheck(true)
-
-    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
-
-    val joinT = ds1.fullOuterJoin(ds2, 'a === 'd && 'b === 'h).select('c, 'g)
-
-    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "null,Hallo Welt wie\n" +
-      "Hello world,Hallo Welt wie gehts?\n" + "Hello world,ABC\n" + "null,BCD\n" + "null,CDE\n" +
-      "null,DEF\n" + "null,EFG\n" + "null,FGH\n" + "null,GHI\n" + "I am fine.,HIJ\n" +
-      "I am fine.,IJK\n" + "null,JKL\n" + "null,KLM\n" + "Luke Skywalker,null\n" +
-      "Comment#1,null\n" + "Comment#2,null\n" + "Comment#3,null\n" + "Comment#4,null\n" +
-      "Comment#5,null\n" + "Comment#6,null\n" + "Comment#7,null\n" + "Comment#8,null\n" +
-      "Comment#9,null\n" + "Comment#10,null\n" + "Comment#11,null\n" + "Comment#12,null\n" +
-      "Comment#13,null\n" + "Comment#14,null\n" + "Comment#15,null\n" +
-      "Hello world, how are you?,null\n"
-    val results = joinT.toDataSet[Row].collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SetOperatorsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SetOperatorsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SetOperatorsITCase.scala
deleted file mode 100644
index 0d32cb4..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SetOperatorsITCase.scala
+++ /dev/null
@@ -1,320 +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.api.scala.batch.table
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.table.{TableEnvironment, ValidationException}
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-import scala.util.Random
-
-@RunWith(classOf[Parameterized])
-class SetOperatorsITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testUnionAll(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f)
-
-    val unionDs = ds1.unionAll(ds2).select('c)
-
-    val results = unionDs.toDataSet[Row].collect()
-    val expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hi\n" + "Hello\n" + "Hello world\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testUnion(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f)
-
-    val unionDs = ds1.union(ds2).select('c)
-
-    val results = unionDs.toDataSet[Row].collect()
-    val expected = "Hi\n" + "Hello\n" + "Hello world\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testTernaryUnionAll(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds3 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val unionDs = ds1.unionAll(ds2).unionAll(ds3).select('c)
-
-    val results = unionDs.toDataSet[Row].collect()
-    val expected = "Hi\n" + "Hello\n" + "Hello world\n" +
-      "Hi\n" + "Hello\n" + "Hello world\n" +
-      "Hi\n" + "Hello\n" + "Hello world\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testTernaryUnion(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds3 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val unionDs = ds1.union(ds2).union(ds3).select('c)
-
-    val results = unionDs.toDataSet[Row].collect()
-    val expected = "Hi\n" + "Hello\n" + "Hello world\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testUnionDifferentColumnSize(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'd, 'c, 'e)
-
-    // must fail. Union inputs have different column size.
-    ds1.unionAll(ds2)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testUnionDifferentFieldTypes(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
-      .select('a, 'b, 'c)
-
-    // must fail. Union inputs have different field types.
-    ds1.unionAll(ds2)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testUnionTablesFromDifferentEnvs(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv1 = TableEnvironment.getTableEnvironment(env, config)
-    val tEnv2 = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv1, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv2, 'a, 'b, 'c)
-
-    // Must fail. Tables are bound to different TableEnvironments.
-    ds1.unionAll(ds2).select('c)
-  }
-
-  @Test
-  def testMinusAll(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = env.fromElements((1, 1L, "Hi")).toTable(tEnv, 'a, 'b, 'c)
-
-    val minusDs = ds1.unionAll(ds1).unionAll(ds1)
-      .minusAll(ds2.unionAll(ds2)).select('c)
-
-    val results = minusDs.toDataSet[Row].collect()
-    val expected = "Hi\n" +
-      "Hello\n" + "Hello world\n" +
-      "Hello\n" + "Hello world\n" +
-      "Hello\n" + "Hello world\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testMinus(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = env.fromElements((1, 1L, "Hi")).toTable(tEnv, 'a, 'b, 'c)
-
-    val minusDs = ds1.unionAll(ds1).unionAll(ds1)
-      .minus(ds2.unionAll(ds2)).select('c)
-
-    val results = minusDs.toDataSet[Row].collect()
-    val expected = "Hello\n" + "Hello world\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testMinusDifferentFieldTypes(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
-      .select('a, 'b, 'c)
-
-    // must fail. Minus inputs have different field types.
-    ds1.minus(ds2)
-  }
-
-  @Test
-  def testMinusDifferentFieldNames(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = env.fromElements((1, 1L, "Hi")).toTable(tEnv, 'd, 'e, 'f)
-
-    val minusDs = ds1.unionAll(ds1).unionAll(ds1)
-      .minus(ds2.unionAll(ds2)).select('c)
-
-    val results = minusDs.toDataSet[Row].collect()
-    val expected = "Hello\n" + "Hello world\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testMinusAllTablesFromDifferentEnvs(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv1 = TableEnvironment.getTableEnvironment(env, config)
-    val tEnv2 = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv1, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv2, 'a, 'b, 'c)
-
-    // Must fail. Tables are bound to different TableEnvironments.
-    ds1.minusAll(ds2).select('c)
-  }
-
-  @Test
-  def testIntersect(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val data = new mutable.MutableList[(Int, Long, String)]
-    data.+=((1, 1L, "Hi"))
-    data.+=((2, 2L, "Hello"))
-    data.+=((2, 2L, "Hello"))
-    data.+=((3, 2L, "Hello world!"))
-    val ds2 = env.fromCollection(Random.shuffle(data)).toTable(tEnv, 'a, 'b, 'c)
-
-    val intersectDS = ds1.intersect(ds2).select('c).toDataSet[Row]
-
-    val results = intersectDS.collect()
-
-    val expected = "Hi\n" + "Hello\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testIntersectAll(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val data1 = new mutable.MutableList[Int]
-    data1 += (1, 1, 1, 2, 2)
-    val data2 = new mutable.MutableList[Int]
-    data2 += (1, 2, 2, 2, 3)
-    val ds1 = env.fromCollection(data1).toTable(tEnv, 'c)
-    val ds2 = env.fromCollection(data2).toTable(tEnv, 'c)
-
-    val intersectDS = ds1.intersectAll(ds2).select('c).toDataSet[Row]
-
-    val expected = "1\n2\n2"
-    val results = intersectDS.collect()
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testIntersectWithDifferentFieldNames(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'e, 'f, 'g)
-
-    val intersectDs = ds1.intersect(ds2).select('c)
-
-    val results = intersectDs.toDataSet[Row].collect()
-    val expected = "Hi\n" + "Hello\n" + "Hello world\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testIntersectWithDifferentFieldTypes(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
-      .select('a, 'b, 'c)
-
-    // must fail. Intersect inputs have different field types.
-    ds1.intersect(ds2)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testIntersectTablesFromDifferentEnvs(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv1 = TableEnvironment.getTableEnvironment(env, config)
-    val tEnv2 = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv1, 'a, 'b, 'c)
-    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv2, 'a, 'b, 'c)
-
-    // Must fail. Tables are bound to different TableEnvironments.
-    ds1.intersect(ds2).select('c)
-  }
-
-  @Test
-  def testIntersectWithScalarExpression(): Unit = {
-    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      .select('a + 1, 'b, 'c)
-    val ds2 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
-      .select('a + 1, 'b, 'c)
-
-    val intersectDs = ds1.intersect(ds2)
-
-    val results = intersectDs.toDataSet[Row].collect()
-    val expected = "2,1,Hi\n" + "3,2,Hello\n" + "4,2,Hello world\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SortITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SortITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SortITCase.scala
deleted file mode 100644
index b3cc054..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/SortITCase.scala
+++ /dev/null
@@ -1,186 +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.api.scala.batch.table
-
-import org.apache.flink.api.scala.batch.utils.SortTestUtils._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.util.CollectionDataSets
-import org.apache.flink.api.scala.{ExecutionEnvironment, _}
-import org.apache.flink.api.table.{TableEnvironment, ValidationException}
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit._
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-
-@RunWith(classOf[Parameterized])
-class SortITCase(
-    mode: TestExecutionMode,
-    configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  def getExecutionEnvironment = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    env.setParallelism(4)
-    env
-  }
-
-  @Test
-  def testOrderByDesc(): Unit = {
-    val env = getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    val t = ds.toTable(tEnv).orderBy('_1.desc)
-    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
-      - x.productElement(0).asInstanceOf[Int] )
-
-    val expected = sortExpectedly(tupleDataSetStrings)
-    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
-
-    val result = results
-      .filterNot(_.isEmpty)
-      .sortBy(_.head)(Ordering.by(f=> f.toString))
-      .reduceLeft(_ ++ _)
-
-    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
-  }
-
-  @Test
-  def testOrderByAsc(): Unit = {
-    val env = getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    val t = ds.toTable(tEnv).orderBy('_1.asc)
-    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
-      x.productElement(0).asInstanceOf[Int] )
-
-    val expected = sortExpectedly(tupleDataSetStrings)
-    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
-
-    val result = results
-      .filterNot(_.isEmpty)
-      .sortBy(_.head)(Ordering.by(f=> f.toString))
-      .reduceLeft(_ ++ _)
-
-    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
-  }
-
-  @Test
-  def testOrderByMultipleFieldsDifferentDirections(): Unit = {
-    val env = getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    val t = ds.toTable(tEnv).orderBy('_2.asc, '_1.desc)
-    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
-      (x.productElement(1).asInstanceOf[Long], - x.productElement(0).asInstanceOf[Int]) )
-
-    val expected = sortExpectedly(tupleDataSetStrings)
-    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
-
-    val result = results
-      .filterNot(_.isEmpty)
-      .sortBy(_.head)(Ordering.by(f=> f.toString))
-      .reduceLeft(_ ++ _)
-
-    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
-  }
-
-  @Test
-  def testOrderByOffset(): Unit = {
-    val env = getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    val t = ds.toTable(tEnv).orderBy('_1.asc).limit(3)
-    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
-      x.productElement(0).asInstanceOf[Int] )
-
-    val expected = sortExpectedly(tupleDataSetStrings, 3, 21)
-    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
-
-    val result = results
-      .filterNot(_.isEmpty)
-      .sortBy(_.head)(Ordering.by(f=> f.toString))
-      .reduceLeft(_ ++ _)
-
-    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
-  }
-
-  @Test
-  def testOrderByOffsetAndFetch(): Unit = {
-    val env = getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    val t = ds.toTable(tEnv).orderBy('_1.desc).limit(3, 5)
-    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
-      - x.productElement(0).asInstanceOf[Int] )
-
-    val expected = sortExpectedly(tupleDataSetStrings, 3, 8)
-    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
-
-    val result = results
-      .filterNot(_.isEmpty)
-      .sortBy(_.head)(Ordering.by(f=> f.toString))
-      .reduceLeft(_ ++ _)
-
-    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
-  }
-
-  @Test
-  def testOrderByFetch(): Unit = {
-    val env = getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    val t = ds.toTable(tEnv).orderBy('_1.asc).limit(0, 5)
-    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
-      x.productElement(0).asInstanceOf[Int] )
-
-    val expected = sortExpectedly(tupleDataSetStrings, 0, 5)
-    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
-
-    val result = results
-      .filterNot(_.isEmpty)
-      .sortBy(_.head)(Ordering.by(f=> f.toString))
-      .reduceLeft(_ ++ _)
-
-    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testFetchWithoutOrder(): Unit = {
-    val env = getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env, config)
-
-    val ds = CollectionDataSets.get3TupleDataSet(env)
-    val t = ds.toTable(tEnv).limit(0, 5)
-
-    t.toDataSet[Row].collect()
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UserDefinedTableFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UserDefinedTableFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UserDefinedTableFunctionTest.scala
deleted file mode 100644
index 285a181..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/table/UserDefinedTableFunctionTest.scala
+++ /dev/null
@@ -1,180 +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.api.scala.batch.table
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.{DataSet => JDataSet, ExecutionEnvironment => JavaExecutionEnv}
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment => ScalaExecutionEnv, _}
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.utils.TableTestUtil._
-import org.apache.flink.api.table.utils.{PojoTableFunc, TableFunc2, _}
-import org.apache.flink.api.table.{TableEnvironment, Types}
-import org.apache.flink.types.Row
-import org.junit.Test
-import org.mockito.Mockito._
-
-class UserDefinedTableFunctionTest extends TableTestBase {
-
-  @Test
-  def testJavaScalaTableAPIEquality(): Unit = {
-    // mock
-    val ds = mock(classOf[DataSet[Row]])
-    val jDs = mock(classOf[JDataSet[Row]])
-    val typeInfo = new RowTypeInfo(Seq(Types.INT, Types.LONG, Types.STRING): _*)
-    when(ds.javaSet).thenReturn(jDs)
-    when(jDs.getType).thenReturn(typeInfo)
-
-    // Scala environment
-    val env = mock(classOf[ScalaExecutionEnv])
-    val tableEnv = TableEnvironment.getTableEnvironment(env)
-    val in1 = ds.toTable(tableEnv).as('a, 'b, 'c)
-
-    // Java environment
-    val javaEnv = mock(classOf[JavaExecutionEnv])
-    val javaTableEnv = TableEnvironment.getTableEnvironment(javaEnv)
-    val in2 = javaTableEnv.fromDataSet(jDs).as("a, b, c")
-    javaTableEnv.registerTable("MyTable", in2)
-
-    // test cross join
-    val func1 = new TableFunc1
-    javaTableEnv.registerFunction("func1", func1)
-    var scalaTable = in1.join(func1('c) as 's).select('c, 's)
-    var javaTable = in2.join("func1(c).as(s)").select("c, s")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test left outer join
-    scalaTable = in1.leftOuterJoin(func1('c) as 's).select('c, 's)
-    javaTable = in2.leftOuterJoin("as(func1(c), s)").select("c, s")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test overloading
-    scalaTable = in1.join(func1('c, "$") as 's).select('c, 's)
-    javaTable = in2.join("func1(c, '$') as (s)").select("c, s")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test custom result type
-    val func2 = new TableFunc2
-    javaTableEnv.registerFunction("func2", func2)
-    scalaTable = in1.join(func2('c) as ('name, 'len)).select('c, 'name, 'len)
-    javaTable = in2.join("func2(c).as(name, len)").select("c, name, len")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test hierarchy generic type
-    val hierarchy = new HierarchyTableFunction
-    javaTableEnv.registerFunction("hierarchy", hierarchy)
-    scalaTable = in1.join(hierarchy('c) as ('name, 'adult, 'len))
-      .select('c, 'name, 'len, 'adult)
-    javaTable = in2.join("AS(hierarchy(c), name, adult, len)")
-      .select("c, name, len, adult")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test pojo type
-    val pojo = new PojoTableFunc
-    javaTableEnv.registerFunction("pojo", pojo)
-    scalaTable = in1.join(pojo('c))
-      .select('c, 'name, 'age)
-    javaTable = in2.join("pojo(c)")
-      .select("c, name, age")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test with filter
-    scalaTable = in1.join(func2('c) as ('name, 'len))
-      .select('c, 'name, 'len).filter('len > 2)
-    javaTable = in2.join("func2(c) as (name, len)")
-      .select("c, name, len").filter("len > 2")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test with scalar function
-    scalaTable = in1.join(func1('c.substring(2)) as 's)
-      .select('a, 'c, 's)
-    javaTable = in2.join("func1(substring(c, 2)) as (s)")
-      .select("a, c, s")
-    verifyTableEquals(scalaTable, javaTable)
-  }
-
-  @Test
-  def testCrossJoin(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = util.addFunction("func1", new TableFunc1)
-
-    val result1 = table.join(function('c) as 's).select('c, 's)
-
-    val expected1 = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", s"$function($$2)"),
-        term("function", function),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "s")
-    )
-
-    util.verifyTable(result1, expected1)
-
-    // test overloading
-
-    val result2 = table.join(function('c, "$") as 's).select('c, 's)
-
-    val expected2 = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", s"$function($$2, '$$')"),
-        term("function", function),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "s")
-    )
-
-    util.verifyTable(result2, expected2)
-  }
-
-  @Test
-  def testLeftOuterJoin(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = util.addFunction("func1", new TableFunc1)
-
-    val result = table.leftOuterJoin(function('c) as 's).select('c, 's)
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetCorrelate",
-        batchTableNode(0),
-        term("invocation", s"$function($$2)"),
-        term("function", function),
-        term("rowType",
-          "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
-        term("joinType", "LEFT")
-      ),
-      term("select", "c", "s")
-    )
-
-    util.verifyTable(result, expected)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/SortTestUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/SortTestUtils.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/SortTestUtils.scala
deleted file mode 100644
index 8d1f653..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/SortTestUtils.scala
+++ /dev/null
@@ -1,58 +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.api.scala.batch.utils
-
-object SortTestUtils {
-
-  val tupleDataSetStrings = List((1, 1L, "Hi")
-    ,(2, 2L, "Hello")
-    ,(3, 2L, "Hello world")
-    ,(4, 3L, "Hello world, how are you?")
-    ,(5, 3L, "I am fine.")
-    ,(6, 3L, "Luke Skywalker")
-    ,(7, 4L, "Comment#1")
-    ,(8, 4L, "Comment#2")
-    ,(9, 4L, "Comment#3")
-    ,(10, 4L, "Comment#4")
-    ,(11, 5L, "Comment#5")
-    ,(12, 5L, "Comment#6")
-    ,(13, 5L, "Comment#7")
-    ,(14, 5L, "Comment#8")
-    ,(15, 5L, "Comment#9")
-    ,(16, 6L, "Comment#10")
-    ,(17, 6L, "Comment#11")
-    ,(18, 6L, "Comment#12")
-    ,(19, 6L, "Comment#13")
-    ,(20, 6L, "Comment#14")
-    ,(21, 6L, "Comment#15"))
-
-  def sortExpectedly(dataSet: List[Product])
-                    (implicit ordering: Ordering[Product]): String = 
-    sortExpectedly(dataSet, 0, dataSet.length)
-
-  def sortExpectedly(dataSet: List[Product], start: Int, end: Int)
-                    (implicit ordering: Ordering[Product]): String = {
-    dataSet
-      .sorted(ordering)
-      .slice(start, end)
-      .mkString("\n")
-      .replaceAll("[\\(\\)]", "")
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.scala
deleted file mode 100644
index 2ce42d4..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/batch/utils/TableProgramsTestBase.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.api.scala.batch.utils
-
-import java.util
-
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.{EFFICIENT, NO_NULL, TableConfigMode}
-import org.apache.flink.api.table.TableConfig
-import org.apache.flink.test.util.MultipleProgramsTestBase
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConversions._
-
-class TableProgramsTestBase(
-    mode: TestExecutionMode,
-    tableConfigMode: TableConfigMode)
-  extends MultipleProgramsTestBase(mode) {
-
-  def config: TableConfig = {
-    val conf = new TableConfig
-    tableConfigMode match {
-      case NO_NULL =>
-        conf.setNullCheck(false)
-      case EFFICIENT =>
-        conf.setEfficientTypeUsage(true)
-      case _ => // keep default
-    }
-    conf
-  }
-}
-
-object TableProgramsTestBase {
-  case class TableConfigMode(nullCheck: Boolean, efficientTypes: Boolean)
-
-  val DEFAULT = TableConfigMode(nullCheck = true, efficientTypes = false)
-  val NO_NULL = TableConfigMode(nullCheck = false, efficientTypes = false)
-  val EFFICIENT = TableConfigMode(nullCheck = false, efficientTypes = true)
-
-  @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
-  def parameters(): util.Collection[Array[java.lang.Object]] = {
-    Seq[Array[AnyRef]](Array(TestExecutionMode.COLLECTION, DEFAULT))
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/ExplainStreamTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/ExplainStreamTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/ExplainStreamTest.scala
deleted file mode 100644
index 5eebb34..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/ExplainStreamTest.scala
+++ /dev/null
@@ -1,74 +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.api.scala.stream
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.junit.Assert.assertEquals
-import org.junit._
-
-class ExplainStreamTest
-  extends StreamingMultipleProgramsTestBase {
-
-  val testFilePath = ExplainStreamTest.this.getClass.getResource("/").getFile
-
-  @Test
-  def testFilter(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val table = env.fromElements((1, "hello"))
-      .toTable(tEnv, 'a, 'b)
-      .filter("a % 2 = 0")
-
-    val result = replaceString(tEnv.explain(table))
-
-    val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testFilterStream0.out").mkString
-    val expect = replaceString(source)
-    assertEquals(result, expect)
-  }
-
-  @Test
-  def testUnion(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    val table1 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
-    val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
-    val table = table1.unionAll(table2)
-
-    val result = replaceString(tEnv.explain(table))
-
-    val source = scala.io.Source.fromFile(testFilePath +
-      "../../src/test/scala/resources/testUnionStream0.out").mkString
-    val expect = replaceString(source)
-    assertEquals(result, expect)
-  }
-
-  def replaceString(s: String): String = {
-    /* Stage {id} is ignored, because id keeps incrementing in test class
-     * while StreamExecutionEnvironment is up
-     */
-    s.replaceAll("\\r\\n", "\n").replaceAll("Stage \\d+", "")
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala
deleted file mode 100644
index f3eb87c..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSinkITCase.scala
+++ /dev/null
@@ -1,64 +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.api.scala.stream
-
-import java.io.File
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.stream.utils.StreamTestData
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.api.table.sinks.CsvTableSink
-import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.apache.flink.test.util.TestBaseUtils
-
-import org.junit.Test
-
-class TableSinkITCase extends StreamingMultipleProgramsTestBase {
-
-  @Test
-  def testStreamTableSink(): Unit = {
-
-    val tmpFile = File.createTempFile("flink-table-sink-test", ".tmp")
-    tmpFile.deleteOnExit()
-    val path = tmpFile.toURI.toString
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    env.setParallelism(4)
-
-    val input = StreamTestData.get3TupleDataStream(env)
-      .map(x => x).setParallelism(4) // increase DOP to 4
-
-    val results = input.toTable(tEnv, 'a, 'b, 'c)
-      .where('a < 5 || 'a > 17)
-      .select('c, 'b)
-      .writeToSink(new CsvTableSink(path))
-
-    env.execute()
-
-    val expected = Seq(
-      "Hi,1", "Hello,2", "Hello world,2", "Hello world, how are you?,3",
-      "Comment#12,6", "Comment#13,6", "Comment#14,6", "Comment#15,6").mkString("\n")
-
-    TestBaseUtils.compareResultsByLinesInMemory(expected, path)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala
deleted file mode 100644
index 1c93112..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/TableSourceITCase.scala
+++ /dev/null
@@ -1,193 +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.api.scala.stream
-
-import java.io.{File, FileOutputStream, OutputStreamWriter}
-
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.stream.utils.StreamITCase
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.sources.{CsvTableSource, StreamTableSource}
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.types.Row
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.streaming.api.environment
-import org.apache.flink.streaming.api.functions.source.SourceFunction
-import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
-import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.junit.Assert._
-import org.junit.Test
-
-import scala.collection.mutable
-
-class TableSourceITCase extends StreamingMultipleProgramsTestBase {
-
-  @Test
-  def testStreamTableSourceTableAPI(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    tEnv.registerTableSource("MyTestTable", new TestStreamTableSource(33))
-    tEnv.ingest("MyTestTable")
-      .where('amount < 4)
-      .select('amount * 'id, 'name)
-      .toDataStream[Row]
-      .addSink(new StreamITCase.StringSink)
-
-    env.execute()
-
-    val expected = mutable.MutableList(
-      "0,Record_0", "0,Record_16", "0,Record_32",
-      "1,Record_1", "17,Record_17", "36,Record_18",
-      "4,Record_2", "57,Record_19", "9,Record_3")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testStreamTableSourceSQL(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    tEnv.registerTableSource("MyTestTable", new TestStreamTableSource(33))
-    tEnv.sql(
-      "SELECT amount * id, name FROM MyTestTable WHERE amount < 4")
-      .toDataStream[Row]
-      .addSink(new StreamITCase.StringSink)
-
-    env.execute()
-
-    val expected = mutable.MutableList(
-      "0,Record_0", "0,Record_16", "0,Record_32",
-      "1,Record_1", "17,Record_17", "36,Record_18",
-      "4,Record_2", "57,Record_19", "9,Record_3")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testCsvTableSource(): Unit = {
-
-    val csvRecords = Seq(
-      "First#Id#Score#Last",
-      "Mike#1#12.3#Smith",
-      "Bob#2#45.6#Taylor",
-      "Sam#3#7.89#Miller",
-      "Peter#4#0.12#Smith",
-      "% Just a comment",
-      "Liz#5#34.5#Williams",
-      "Sally#6#6.78#Miller",
-      "Alice#7#90.1#Smith",
-      "Kelly#8#2.34#Williams"
-    )
-
-    val tempFile = File.createTempFile("csv-test", "tmp")
-    tempFile.deleteOnExit()
-    val tmpWriter = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8")
-    tmpWriter.write(csvRecords.mkString("$"))
-    tmpWriter.close()
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val csvTable = new CsvTableSource(
-      tempFile.getAbsolutePath,
-      Array("first", "id", "score", "last"),
-      Array(
-        BasicTypeInfo.STRING_TYPE_INFO,
-        BasicTypeInfo.INT_TYPE_INFO,
-        BasicTypeInfo.DOUBLE_TYPE_INFO,
-        BasicTypeInfo.STRING_TYPE_INFO
-      ),
-      fieldDelim = "#",
-      rowDelim = "$",
-      ignoreFirstLine = true,
-      ignoreComments = "%"
-    )
-
-    tEnv.registerTableSource("csvTable", csvTable)
-    tEnv.sql(
-      "SELECT last, score, id FROM csvTable WHERE id < 4 ")
-      .toDataStream[Row]
-      .addSink(new StreamITCase.StringSink)
-
-    env.execute()
-
-    val expected = mutable.MutableList(
-      "Smith,12.3,1",
-      "Taylor,45.6,2",
-      "Miller,7.89,3")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-}
-
-
-class TestStreamTableSource(val numRecords: Int) extends StreamTableSource[Row] {
-
-  val fieldTypes: Array[TypeInformation[_]] = Array(
-    BasicTypeInfo.STRING_TYPE_INFO,
-    BasicTypeInfo.LONG_TYPE_INFO,
-    BasicTypeInfo.INT_TYPE_INFO
-  )
-
-  /** Returns the data of the table as a [[DataStream]]. */
-  override def getDataStream(execEnv: environment.StreamExecutionEnvironment): DataStream[Row] = {
-    execEnv.addSource(new GeneratingSourceFunction(numRecords), getReturnType).setParallelism(1)
-  }
-
-  /** Returns the types of the table fields. */
-  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
-
-  /** Returns the names of the table fields. */
-  override def getFieldsNames: Array[String] = Array("name", "id", "amount")
-
-  /** Returns the [[TypeInformation]] for the return type. */
-  override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*)
-
-  /** Returns the number of fields of the table. */
-  override def getNumberOfFields: Int = 3
-}
-
-class GeneratingSourceFunction(val num: Long) extends SourceFunction[Row] {
-
-  var running = true
-
-  override def run(ctx: SourceContext[Row]): Unit = {
-    var cnt = 0L
-    while(running && cnt < num) {
-      val out = new Row(3)
-      out.setField(0, s"Record_$cnt")
-      out.setField(1, cnt)
-      out.setField(2, (cnt % 16).toInt)
-
-      ctx.collect(out)
-      cnt += 1
-    }
-  }
-
-  override def cancel(): Unit = {
-    running = false
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/SqlITCase.scala
deleted file mode 100644
index c4ca964..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/SqlITCase.scala
+++ /dev/null
@@ -1,174 +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.api.scala.stream.sql
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.stream.utils.{StreamTestData, StreamITCase}
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.types.Row
-import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.junit.Assert._
-import org.junit._
-
-import scala.collection.mutable
-
-class SqlITCase extends StreamingMultipleProgramsTestBase {
-
-  /** test selection **/
-  @Test
-  def testSelectExpressionFromTable(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val sqlQuery = "SELECT a * 2, b - 1 FROM MyTable"
-
-    val t = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("MyTable", t)
-
-    val result = tEnv.sql(sqlQuery).toDataStream[Row]
-    result.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("2,0", "4,1", "6,1")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  /** test filtering with registered table **/
-  @Test
-  def testSimpleFilter(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val sqlQuery = "SELECT * FROM MyTable WHERE a = 3"
-
-    val t = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("MyTable", t)
-
-    val result = tEnv.sql(sqlQuery).toDataStream[Row]
-    result.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("3,2,Hello world")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  /** test filtering with registered datastream **/
-  @Test
-  def testDatastreamFilter(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val sqlQuery = "SELECT * FROM MyTable WHERE _1 = 3"
-
-    val t = StreamTestData.getSmall3TupleDataStream(env)
-    tEnv.registerDataStream("MyTable", t)
-
-    val result = tEnv.sql(sqlQuery).toDataStream[Row]
-    result.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("3,2,Hello world")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  /** test union with registered tables **/
-  @Test
-  def testUnion(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val sqlQuery = "SELECT * FROM T1 " +
-      "UNION ALL " +
-      "SELECT * FROM T2"
-
-    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("T1", t1)
-    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("T2", t2)
-
-    val result = tEnv.sql(sqlQuery).toDataStream[Row]
-    result.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList(
-      "1,1,Hi", "1,1,Hi",
-      "2,2,Hello", "2,2,Hello",
-      "3,2,Hello world", "3,2,Hello world")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  /** test union with filter **/
-  @Test
-  def testUnionWithFilter(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val sqlQuery = "SELECT * FROM T1 WHERE a = 3 " +
-      "UNION ALL " +
-      "SELECT * FROM T2 WHERE a = 2"
-
-    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("T1", t1)
-    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("T2", t2)
-
-    val result = tEnv.sql(sqlQuery).toDataStream[Row]
-    result.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList(
-      "2,2,Hello",
-      "3,2,Hello world")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  /** test union of a table and a datastream **/
-  @Test
-  def testUnionTableWithDataSet(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val sqlQuery = "SELECT c FROM T1 WHERE a = 3 " +
-      "UNION ALL " +
-      "SELECT c FROM T2 WHERE a = 2"
-
-    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
-    tEnv.registerTable("T1", t1)
-    val t2 = StreamTestData.get3TupleDataStream(env)
-    tEnv.registerDataStream("T2", t2, 'a, 'b, 'c)
-
-    val result = tEnv.sql(sqlQuery).toDataStream[Row]
-    result.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("Hello", "Hello world")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/UserDefinedTableFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/UserDefinedTableFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/UserDefinedTableFunctionTest.scala
deleted file mode 100644
index 21629e4..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/sql/UserDefinedTableFunctionTest.scala
+++ /dev/null
@@ -1,237 +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.api.scala.stream.sql
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.utils.{HierarchyTableFunction, PojoTableFunc, TableFunc2}
-import org.apache.flink.api.table.utils._
-import org.apache.flink.api.table.utils.TableTestUtil._
-import org.junit.Test
-
-class UserDefinedTableFunctionTest extends TableTestBase {
-
-  @Test
-  def testCrossJoin(): Unit = {
-    val util = streamTestUtil()
-    val func1 = new TableFunc1
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    util.addFunction("func1", func1)
-
-    val sqlQuery = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(c)) AS T(s)"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", "func1($cor0.c)"),
-        term("function", func1.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "f0 AS s")
-    )
-
-    util.verifySql(sqlQuery, expected)
-
-    // test overloading
-
-    val sqlQuery2 = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(c, '$')) AS T(s)"
-
-    val expected2 = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", "func1($cor0.c, '$')"),
-        term("function", func1.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "f0 AS s")
-    )
-
-    util.verifySql(sqlQuery2, expected2)
-  }
-
-  @Test
-  def testLeftOuterJoin(): Unit = {
-    val util = streamTestUtil()
-    val func1 = new TableFunc1
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    util.addFunction("func1", func1)
-
-    val sqlQuery = "SELECT c, s FROM MyTable LEFT JOIN LATERAL TABLE(func1(c)) AS T(s) ON TRUE"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", "func1($cor0.c)"),
-        term("function", func1.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
-        term("joinType", "LEFT")
-      ),
-      term("select", "c", "f0 AS s")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testCustomType(): Unit = {
-    val util = streamTestUtil()
-    val func2 = new TableFunc2
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    util.addFunction("func2", func2)
-
-    val sqlQuery = "SELECT c, name, len FROM MyTable, LATERAL TABLE(func2(c)) AS T(name, len)"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", "func2($cor0.c)"),
-        term("function", func2.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
-               "VARCHAR(2147483647) f0, INTEGER f1)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "f0 AS name", "f1 AS len")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testHierarchyType(): Unit = {
-    val util = streamTestUtil()
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = new HierarchyTableFunction
-    util.addFunction("hierarchy", function)
-
-    val sqlQuery = "SELECT c, T.* FROM MyTable, LATERAL TABLE(hierarchy(c)) AS T(name, adult, len)"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", "hierarchy($cor0.c)"),
-        term("function", function.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," +
-               " VARCHAR(2147483647) f0, BOOLEAN f1, INTEGER f2)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "f0 AS name", "f1 AS adult", "f2 AS len")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testPojoType(): Unit = {
-    val util = streamTestUtil()
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = new PojoTableFunc
-    util.addFunction("pojo", function)
-
-    val sqlQuery = "SELECT c, name, age FROM MyTable, LATERAL TABLE(pojo(c))"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", "pojo($cor0.c)"),
-        term("function", function.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," +
-               " INTEGER age, VARCHAR(2147483647) name)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "name", "age")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testFilter(): Unit = {
-    val util = streamTestUtil()
-    val func2 = new TableFunc2
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    util.addFunction("func2", func2)
-
-    val sqlQuery = "SELECT c, name, len FROM MyTable, LATERAL TABLE(func2(c)) AS T(name, len) " +
-      "WHERE len > 2"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", "func2($cor0.c)"),
-        term("function", func2.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
-               "VARCHAR(2147483647) f0, INTEGER f1)"),
-        term("joinType", "INNER"),
-        term("condition", ">($1, 2)")
-      ),
-      term("select", "c", "f0 AS name", "f1 AS len")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testScalarFunction(): Unit = {
-    val util = streamTestUtil()
-    val func1 = new TableFunc1
-    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    util.addFunction("func1", func1)
-
-    val sqlQuery = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(SUBSTRING(c, 2))) AS T(s)"
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", "func1(SUBSTRING($cor0.c, 2))"),
-        term("function", func1.getClass.getCanonicalName),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "f0 AS s")
-    )
-
-    util.verifySql(sqlQuery, expected)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/AggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/AggregationsITCase.scala
deleted file mode 100644
index d398556..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/AggregationsITCase.scala
+++ /dev/null
@@ -1,199 +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.api.scala.stream.table
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.stream.table.GroupWindowITCase.TimestampWithEqualWatermark
-import org.apache.flink.api.scala.stream.utils.StreamITCase
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table._
-import org.apache.flink.types.Row
-import org.apache.flink.streaming.api.TimeCharacteristic
-import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks
-import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
-import org.apache.flink.streaming.api.watermark.Watermark
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.junit.Assert._
-import org.junit.Test
-
-import scala.collection.mutable
-
-/**
-  * We only test some aggregations until better testing of constructed DataStream
-  * programs is possible.
-  */
-class AggregationsITCase extends StreamingMultipleProgramsTestBase {
-
-  val data = List(
-    (1L, 1, "Hi"),
-    (2L, 2, "Hello"),
-    (4L, 2, "Hello"),
-    (8L, 3, "Hello world"),
-    (16L, 3, "Hello world"))
-
-  @Test
-  def testProcessingTimeSlidingGroupWindowOverCount(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val stream = env.fromCollection(data)
-    val table = stream.toTable(tEnv, 'long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Slide over 2.rows every 1.rows)
-      .select('string, 'int.count, 'int.avg)
-
-    val results = windowedTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = Seq("Hello world,1,3", "Hello world,2,3", "Hello,1,2", "Hello,2,2", "Hi,1,1")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testEventTimeSessionGroupWindowOverTime(): 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, 'long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Session withGap 7.milli on 'rowtime)
-      .select('string, 'int.count)
-
-    val results = windowedTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = Seq("Hello world,1", "Hello world,1", "Hello,2", "Hi,1")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testAllProcessingTimeTumblingGroupWindowOverCount(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val stream = env.fromCollection(data)
-    val table = stream.toTable(tEnv, 'long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Tumble over 2.rows)
-      .select('int.count)
-
-    val results = windowedTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = Seq("2", "2")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testEventTimeTumblingWindow(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-
-    val stream = env
-      .fromCollection(data)
-      .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
-    val table = stream.toTable(tEnv, 'long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Tumble over 5.milli on 'rowtime as 'w)
-      .select('string, 'int.count, 'int.avg, 'w.start, 'w.end)
-
-    val results = windowedTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = Seq(
-      "Hello world,1,3,1970-01-01 00:00:00.005,1970-01-01 00:00:00.01",
-      "Hello world,1,3,1970-01-01 00:00:00.015,1970-01-01 00:00:00.02",
-      "Hello,2,2,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005",
-      "Hi,1,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testEventTimeSlidingWindow(): 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, 'long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Slide over 10.milli every 5.milli on 'rowtime as 'w)
-      .select('string, 'int.count, 'w.start, 'w.end, 'w.start)
-
-    val results = windowedTable.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = Seq(
-      "Hello world,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.01,1970-01-01 00:00:00.0",
-      "Hello world,1,1970-01-01 00:00:00.005,1970-01-01 00:00:00.015,1970-01-01 00:00:00.005",
-      "Hello world,1,1970-01-01 00:00:00.01,1970-01-01 00:00:00.02,1970-01-01 00:00:00.01",
-      "Hello world,1,1970-01-01 00:00:00.015,1970-01-01 00:00:00.025,1970-01-01 00:00:00.015",
-      "Hello,2,1969-12-31 23:59:59.995,1970-01-01 00:00:00.005,1969-12-31 23:59:59.995",
-      "Hello,2,1970-01-01 00:00:00.0,1970-01-01 00:00:00.01,1970-01-01 00:00:00.0",
-      "Hi,1,1969-12-31 23:59:59.995,1970-01-01 00:00:00.005,1969-12-31 23:59:59.995",
-      "Hi,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.01,1970-01-01 00:00:00.0")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-}
-
-object GroupWindowITCase {
-  class TimestampWithEqualWatermark extends AssignerWithPunctuatedWatermarks[(Long, Int, String)] {
-
-    override def checkAndGetNextWatermark(
-        lastElement: (Long, Int, String),
-        extractedTimestamp: Long)
-      : Watermark = {
-      new Watermark(extractedTimestamp)
-    }
-
-    override def extractTimestamp(
-        element: (Long, Int, String),
-        previousElementTimestamp: Long): Long = {
-      element._1
-    }
-  }
-}


[45/47] flink git commit: [FLINK-5343] [table] Add support to overwrite files with CsvTableSink.

Posted by fh...@apache.org.
[FLINK-5343] [table] Add support to overwrite files with CsvTableSink.

This closes #3011.


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

Branch: refs/heads/master
Commit: cc34c1450486e5596f6af117e77cac6da93fad03
Parents: ffe9ec8
Author: Fabian Hueske <fh...@apache.org>
Authored: Thu Dec 15 11:01:39 2016 +0100
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Dec 16 16:41:21 2016 +0100

----------------------------------------------------------------------
 .../apache/flink/table/sinks/CsvTableSink.scala | 67 +++++++++++++++++---
 1 file changed, 58 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/cc34c145/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
index 9cf76dd..c37ee74 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/sinks/CsvTableSink.scala
@@ -23,33 +23,82 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.DataSet
 import org.apache.flink.types.Row
 import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.core.fs.FileSystem.WriteMode
 import org.apache.flink.streaming.api.datastream.DataStream
 
 /**
   * A simple [[TableSink]] to emit data as CSV files.
   *
   * @param path The output path to write the Table to.
-  * @param fieldDelim The field delimiter, ',' by default.
+  * @param fieldDelim The field delimiter
+  * @param numFiles The number of files to write to
+  * @param writeMode The write mode to specify whether existing files are overwritten or not.
   */
 class CsvTableSink(
     path: String,
-    fieldDelim: String = ",")
+    fieldDelim: Option[String],
+    numFiles: Option[Int],
+    writeMode: Option[WriteMode])
   extends TableSinkBase[Row] with BatchTableSink[Row] with StreamTableSink[Row] {
 
+  /**
+    * A simple [[TableSink]] to emit data as CSV files.
+    *
+    * @param path The output path to write the Table to.
+    * @param fieldDelim The field delimiter, ',' by default.
+    */
+  def this(path: String, fieldDelim: String = ",") {
+    this(path, Some(fieldDelim), None, None)
+  }
+
+  /**
+    * A simple [[TableSink]] to emit data as CSV files.
+    *
+    * @param path The output path to write the Table to.
+    * @param fieldDelim The field delimiter.
+    * @param numFiles The number of files to write to.
+    * @param writeMode The write mode to specify whether existing files are overwritten or not.
+    */
+  def this(path: String, fieldDelim: String, numFiles: Int, writeMode: WriteMode) {
+    this(path, Some(fieldDelim), Some(numFiles), Some(writeMode))
+  }
+
   override def emitDataSet(dataSet: DataSet[Row]): Unit = {
-    dataSet
-      .map(new CsvFormatter(fieldDelim))
-      .writeAsText(path)
+    val csvRows = dataSet.map(new CsvFormatter(fieldDelim.getOrElse(",")))
+
+    if (numFiles.isDefined) {
+      csvRows.setParallelism(numFiles.get)
+    }
+
+    val sink = writeMode match {
+      case None => csvRows.writeAsText(path)
+      case Some(wm) => csvRows.writeAsText(path, wm)
+    }
+
+    if (numFiles.isDefined) {
+      sink.setParallelism(numFiles.get)
+    }
   }
 
   override def emitDataStream(dataStream: DataStream[Row]): Unit = {
-    dataStream
-      .map(new CsvFormatter(fieldDelim))
-      .writeAsText(path)
+    val csvRows = dataStream.map(new CsvFormatter(fieldDelim.getOrElse(",")))
+
+    if (numFiles.isDefined) {
+      csvRows.setParallelism(numFiles.get)
+    }
+
+    val sink = writeMode match {
+      case None => csvRows.writeAsText(path)
+      case Some(wm) => csvRows.writeAsText(path, wm)
+    }
+
+    if (numFiles.isDefined) {
+      sink.setParallelism(numFiles.get)
+    }
   }
 
   override protected def copy: TableSinkBase[Row] = {
-    new CsvTableSink(path, fieldDelim)
+    new CsvTableSink(path, fieldDelim, numFiles, writeMode)
   }
 
   override def getOutputType: TypeInformation[Row] = {


[14/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/sql/SqlITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/sql/SqlITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/sql/SqlITCase.java
deleted file mode 100644
index 53a1a7d..0000000
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/sql/SqlITCase.java
+++ /dev/null
@@ -1,121 +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.api.java.stream.sql;
-
-import org.apache.flink.api.java.table.StreamTableEnvironment;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.scala.stream.utils.StreamITCase;
-import org.apache.flink.types.Row;
-import org.apache.flink.api.table.Table;
-import org.apache.flink.api.table.TableEnvironment;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.api.java.stream.utils.StreamTestData;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-public class SqlITCase extends StreamingMultipleProgramsTestBase {
-
-	@Test
-	public void testSelect() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-		StreamITCase.clear();
-
-		DataStream<Tuple3<Integer, Long, String>> ds = StreamTestData.getSmall3TupleDataSet(env);
-		Table in = tableEnv.fromDataStream(ds, "a,b,c");
-		tableEnv.registerTable("MyTable", in);
-
-		String sqlQuery = "SELECT * FROM MyTable";
-		Table result = tableEnv.sql(sqlQuery);
-
-		DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink());
-		env.execute();
-
-		List<String> expected = new ArrayList<>();
-		expected.add("1,1,Hi");
-		expected.add("2,2,Hello");
-		expected.add("3,2,Hello world");
-
-		StreamITCase.compareWithList(expected);
-	}
-
-	@Test
-	public void testFilter() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-		StreamITCase.clear();
-
-		DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds = StreamTestData.get5TupleDataStream(env);
-		tableEnv.registerDataStream("MyTable", ds, "a, b, c, d, e");
-
-		String sqlQuery = "SELECT a, b, e FROM MyTable WHERE c < 4";
-		Table result = tableEnv.sql(sqlQuery);
-
-		DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink());
-		env.execute();
-
-		List<String> expected = new ArrayList<>();
-		expected.add("1,1,1");
-		expected.add("2,2,2");
-		expected.add("2,3,1");
-		expected.add("3,4,2");
-
-		StreamITCase.compareWithList(expected);
-	}
-
-	@Test
-	public void testUnion() throws Exception {
-		StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
-		StreamTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-		StreamITCase.clear();
-
-		DataStream<Tuple3<Integer, Long, String>> ds1 = StreamTestData.getSmall3TupleDataSet(env);
-		Table t1 = tableEnv.fromDataStream(ds1, "a,b,c");
-		tableEnv.registerTable("T1", t1);
-
-		DataStream<Tuple5<Integer, Long, Integer, String, Long>> ds2 = StreamTestData.get5TupleDataStream(env);
-		tableEnv.registerDataStream("T2", ds2, "a, b, d, c, e");
-
-		String sqlQuery = "SELECT * FROM T1 " +
-							"UNION ALL " +
-							"(SELECT a, b, c FROM T2 WHERE a	< 3)";
-		Table result = tableEnv.sql(sqlQuery);
-
-		DataStream<Row> resultSet = tableEnv.toDataStream(result, Row.class);
-		resultSet.addSink(new StreamITCase.StringSink());
-		env.execute();
-
-		List<String> expected = new ArrayList<>();
-		expected.add("1,1,Hi");
-		expected.add("2,2,Hello");
-		expected.add("3,2,Hello world");
-		expected.add("1,1,Hallo");
-		expected.add("2,2,Hallo Welt");
-		expected.add("2,3,Hallo Welt wie");
-
-		StreamITCase.compareWithList(expected);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/utils/StreamTestData.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/utils/StreamTestData.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/utils/StreamTestData.java
deleted file mode 100644
index 82ebf95..0000000
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/stream/utils/StreamTestData.java
+++ /dev/null
@@ -1,64 +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.api.java.stream.utils;
-
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class StreamTestData {
-
-	public static DataStream<Tuple3<Integer, Long, String>> getSmall3TupleDataSet(StreamExecutionEnvironment env) {
-
-		List<Tuple3<Integer, Long, String>> data = new ArrayList<>();
-		data.add(new Tuple3<>(1, 1L, "Hi"));
-		data.add(new Tuple3<>(2, 2L, "Hello"));
-		data.add(new Tuple3<>(3, 2L, "Hello world"));
-
-		Collections.shuffle(data);
-
-		return env.fromCollection(data);
-	}
-
-	public static DataStream<Tuple5<Integer, Long, Integer, String, Long>> get5TupleDataStream(StreamExecutionEnvironment env) {
-
-		List<Tuple5<Integer, Long, Integer, String, Long>> data = new ArrayList<>();
-		data.add(new Tuple5<>(1, 1L, 0, "Hallo", 1L));
-		data.add(new Tuple5<>(2, 2L, 1, "Hallo Welt", 2L));
-		data.add(new Tuple5<>(2, 3L, 2, "Hallo Welt wie", 1L));
-		data.add(new Tuple5<>(3, 4L, 3, "Hallo Welt wie gehts?", 2L));
-		data.add(new Tuple5<>(3, 5L, 4, "ABC", 2L));
-		data.add(new Tuple5<>(3, 6L, 5, "BCD", 3L));
-		data.add(new Tuple5<>(4, 7L, 6, "CDE", 2L));
-		data.add(new Tuple5<>(4, 8L, 7, "DEF", 1L));
-		data.add(new Tuple5<>(4, 9L, 8, "EFG", 1L));
-		data.add(new Tuple5<>(4, 10L, 9, "FGH", 2L));
-		data.add(new Tuple5<>(5, 11L, 10, "GHI", 1L));
-		data.add(new Tuple5<>(5, 12L, 11, "HIJ", 3L));
-		data.add(new Tuple5<>(5, 13L, 12, "IJK", 3L));
-		data.add(new Tuple5<>(5, 15L, 14, "KLM", 2L));
-		data.add(new Tuple5<>(5, 14L, 13, "JKL", 2L));
-		return env.fromCollection(data);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/ExplainTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/ExplainTest.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/ExplainTest.java
new file mode 100644
index 0000000..114579f
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/ExplainTest.java
@@ -0,0 +1,160 @@
+/*
+ * 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.java.batch;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.test.util.MultipleProgramsTestBase;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.Scanner;
+
+import static org.junit.Assert.assertEquals;
+
+public class ExplainTest extends MultipleProgramsTestBase {
+
+	public ExplainTest() {
+		super(TestExecutionMode.CLUSTER);
+	}
+
+	private static String testFilePath = ExplainTest.class.getResource("/").getFile();
+
+	@Test
+	public void testFilterWithoutExtended() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple2<Integer, String>> input = env.fromElements(new Tuple2<>(1,"d"));
+		Table table = tableEnv
+			.fromDataSet(input, "a, b")
+			.filter("a % 2 = 0");
+
+		String result = tableEnv.explain(table).replaceAll("\\r\\n", "\n");
+		try (Scanner scanner = new Scanner(new File(testFilePath +
+			"../../src/test/scala/resources/testFilter0.out"))){
+			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
+			assertEquals(source, result);
+		}
+	}
+
+	@Test
+	public void testFilterWithExtended() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple2<Integer, String>> input = env.fromElements(new Tuple2<>(1,"d"));
+		Table table = tableEnv
+			.fromDataSet(input, "a, b")
+			.filter("a % 2 = 0");
+
+		String result = tableEnv.explain(table, true).replaceAll("\\r\\n", "\n");
+		try (Scanner scanner = new Scanner(new File(testFilePath +
+			"../../src/test/scala/resources/testFilter1.out"))){
+			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
+			assertEquals(source, result);
+		}
+	}
+
+	@Test
+	public void testJoinWithoutExtended() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple2<Integer, String>> input1 = env.fromElements(new Tuple2<>(1,"d"));
+		DataSet<Tuple2<Integer, String>> input2 = env.fromElements(new Tuple2<>(1,"d"));
+		Table table1 = tableEnv.fromDataSet(input1, "a, b");
+		Table table2 = tableEnv.fromDataSet(input2, "c, d");
+		Table table = table1
+			.join(table2)
+			.where("b = d")
+			.select("a, c");
+
+		String result = tableEnv.explain(table).replaceAll("\\r\\n", "\n");
+		try (Scanner scanner = new Scanner(new File(testFilePath +
+			"../../src/test/scala/resources/testJoin0.out"))){
+			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
+			assertEquals(source, result);
+		}
+	}
+
+	@Test
+	public void testJoinWithExtended() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple2<Integer, String>> input1 = env.fromElements(new Tuple2<>(1,"d"));
+		DataSet<Tuple2<Integer, String>> input2 = env.fromElements(new Tuple2<>(1,"d"));
+		Table table1 = tableEnv.fromDataSet(input1, "a, b");
+		Table table2 = tableEnv.fromDataSet(input2, "c, d");
+		Table table = table1
+			.join(table2)
+			.where("b = d")
+			.select("a, c");
+
+		String result = tableEnv.explain(table, true).replaceAll("\\r\\n", "\n");
+		try (Scanner scanner = new Scanner(new File(testFilePath +
+			"../../src/test/scala/resources/testJoin1.out"))){
+			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
+			assertEquals(source, result);
+		}
+	}
+
+	@Test
+	public void testUnionWithoutExtended() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple2<Integer, String>> input1 = env.fromElements(new Tuple2<>(1,"d"));
+		DataSet<Tuple2<Integer, String>> input2 = env.fromElements(new Tuple2<>(1,"d"));
+		Table table1 = tableEnv.fromDataSet(input1, "count, word");
+		Table table2 = tableEnv.fromDataSet(input2, "count, word");
+		Table table = table1.unionAll(table2);
+
+		String result = tableEnv.explain(table).replaceAll("\\r\\n", "\n");
+		try (Scanner scanner = new Scanner(new File(testFilePath +
+			"../../src/test/scala/resources/testUnion0.out"))){
+			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
+			assertEquals(source, result);
+		}
+	}
+
+	@Test
+	public void testUnionWithExtended() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
+
+		DataSet<Tuple2<Integer, String>> input1 = env.fromElements(new Tuple2<>(1,"d"));
+		DataSet<Tuple2<Integer, String>> input2 = env.fromElements(new Tuple2<>(1,"d"));
+		Table table1 = tableEnv.fromDataSet(input1, "count, word");
+		Table table2 = tableEnv.fromDataSet(input2, "count, word");
+		Table table = table1.unionAll(table2);
+
+		String result = tableEnv.explain(table, true).replaceAll("\\r\\n", "\n");
+		try (Scanner scanner = new Scanner(new File(testFilePath +
+			"../../src/test/scala/resources/testUnion1.out"))){
+			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
+			assertEquals(source, result);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
new file mode 100644
index 0000000..67eb2d1
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableEnvironmentITCase.java
@@ -0,0 +1,630 @@
+/*
+ * 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.java.batch;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.calcite.tools.RuleSets;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import org.apache.flink.types.Row;
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase;
+import org.apache.flink.table.calcite.CalciteConfig;
+import org.apache.flink.table.calcite.CalciteConfigBuilder;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class TableEnvironmentITCase extends TableProgramsTestBase {
+
+	public TableEnvironmentITCase(TestExecutionMode mode, TableConfigMode configMode) {
+		super(mode, configMode);
+	}
+
+	@Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
+	public static Collection<Object[]> parameters() {
+		return Arrays.asList(new Object[][] {
+			{ TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT() },
+			{ TestExecutionMode.COLLECTION, TableProgramsTestBase.EFFICIENT() }
+		});
+	}
+
+	@Test
+	public void testSimpleRegister() throws Exception {
+		final String tableName = "MyTable";
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		tableEnv.registerDataSet(tableName, ds);
+		Table t = tableEnv.scan(tableName);
+
+		Table result = t.select("f0, f1");
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
+				"8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
+				"16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testRegisterWithFields() throws Exception {
+		final String tableName = "MyTable";
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		tableEnv.registerDataSet(tableName, ds, "a, b, c");
+		Table t = tableEnv.scan(tableName);
+
+		Table result = t.select("a, b, c");
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+				"4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+				"7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+				"11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" +
+				"14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" +
+				"17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" +
+				"20,6,Comment#14\n" + "21,6,Comment#15\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = TableException.class)
+	public void testRegisterExistingDatasetTable() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		tableEnv.registerDataSet("MyTable", ds);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 =
+				CollectionDataSets.getSmall5TupleDataSet(env);
+		// Must fail. Name is already used for different table.
+		tableEnv.registerDataSet("MyTable", ds2);
+	}
+
+	@Test(expected = TableException.class)
+	public void testScanUnregisteredTable() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		// Must fail. No table registered under that name.
+		tableEnv.scan("nonRegisteredTable");
+	}
+
+	@Test
+	public void testTableRegister() throws Exception {
+		final String tableName = "MyTable";
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		Table t = tableEnv.fromDataSet(ds);
+		tableEnv.registerTable(tableName, t);
+		Table result = tableEnv.scan(tableName).select("f0, f1").filter("f0 > 7");
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" +
+				"13,5\n" + "14,5\n" + "15,5\n" +
+				"16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = TableException.class)
+	public void testIllegalName() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		Table t = tableEnv.fromDataSet(ds);
+		// Must fail. Table name matches internal name pattern.
+		tableEnv.registerTable("_DataSetTable_42", t);
+	}
+
+	@Test(expected = TableException.class)
+	public void testRegisterTableFromOtherEnv() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv1 = TableEnvironment.getTableEnvironment(env, config());
+		BatchTableEnvironment tableEnv2 = TableEnvironment.getTableEnvironment(env, config());
+
+		Table t = tableEnv1.fromDataSet(CollectionDataSets.get3TupleDataSet(env));
+		// Must fail. Table is bound to different TableEnvironment.
+		tableEnv2.registerTable("MyTable", t);
+	}
+
+	@Test
+	public void testAsFromTuple() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		Table table = tableEnv
+			.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c")
+			.select("a, b, c");
+
+		DataSet<Row> ds = tableEnv.toDataSet(table, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+			"4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+			"7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+			"11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" +
+			"14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" +
+			"17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" +
+			"20,6,Comment#14\n" + "21,6,Comment#15\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAsFromAndToTuple() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		Table table = tableEnv
+			.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c")
+			.select("a, b, c");
+
+		TypeInformation<?> ti = new TupleTypeInfo<Tuple3<Integer, Long, String>>(
+			BasicTypeInfo.INT_TYPE_INFO,
+			BasicTypeInfo.LONG_TYPE_INFO,
+			BasicTypeInfo.STRING_TYPE_INFO);
+
+		DataSet<?> ds = tableEnv.toDataSet(table, ti);
+		List<?> results = ds.collect();
+		String expected = "(1,1,Hi)\n" + "(2,2,Hello)\n" + "(3,2,Hello world)\n" +
+			"(4,3,Hello world, how are you?)\n" + "(5,3,I am fine.)\n" + "(6,3,Luke Skywalker)\n" +
+			"(7,4,Comment#1)\n" + "(8,4,Comment#2)\n" + "(9,4,Comment#3)\n" + "(10,4,Comment#4)\n" +
+			"(11,5,Comment#5)\n" + "(12,5,Comment#6)\n" + "(13,5,Comment#7)\n" +
+			"(14,5,Comment#8)\n" + "(15,5,Comment#9)\n" + "(16,6,Comment#10)\n" +
+			"(17,6,Comment#11)\n" + "(18,6,Comment#12)\n" + "(19,6,Comment#13)\n" +
+			"(20,6,Comment#14)\n" + "(21,6,Comment#15)\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAsFromTupleToPojo() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		List<Tuple4<String, Integer, Double, String>> data = new ArrayList<>();
+		data.add(new Tuple4<>("Rofl", 1, 1.0, "Hi"));
+		data.add(new Tuple4<>("lol", 2, 1.0, "Hi"));
+		data.add(new Tuple4<>("Test me", 4, 3.33, "Hello world"));
+
+		Table table = tableEnv
+			.fromDataSet(env.fromCollection(data), "a, b, c, d")
+			.select("a, b, c, d");
+
+		DataSet<SmallPojo2> ds = tableEnv.toDataSet(table, SmallPojo2.class);
+		List<SmallPojo2> results = ds.collect();
+		String expected = "Rofl,1,1.0,Hi\n" + "lol,2,1.0,Hi\n" + "Test me,4,3.33,Hello world\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAsFromPojo() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		List<SmallPojo> data = new ArrayList<>();
+		data.add(new SmallPojo("Peter", 28, 4000.00, "Sales"));
+		data.add(new SmallPojo("Anna", 56, 10000.00, "Engineering"));
+		data.add(new SmallPojo("Lucy", 42, 6000.00, "HR"));
+
+		Table table = tableEnv
+			.fromDataSet(env.fromCollection(data),
+				"department AS a, " +
+				"age AS b, " +
+				"salary AS c, " +
+				"name AS d")
+			.select("a, b, c, d");
+
+		DataSet<Row> ds = tableEnv.toDataSet(table, Row.class);
+		List<Row> results = ds.collect();
+		String expected =
+			"Sales,28,4000.0,Peter\n" +
+			"Engineering,56,10000.0,Anna\n" +
+			"HR,42,6000.0,Lucy\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAsFromPrivateFieldsPojo() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		List<PrivateSmallPojo> data = new ArrayList<>();
+		data.add(new PrivateSmallPojo("Peter", 28, 4000.00, "Sales"));
+		data.add(new PrivateSmallPojo("Anna", 56, 10000.00, "Engineering"));
+		data.add(new PrivateSmallPojo("Lucy", 42, 6000.00, "HR"));
+
+		Table table = tableEnv
+			.fromDataSet(env.fromCollection(data),
+				"department AS a, " +
+				"age AS b, " +
+				"salary AS c, " +
+				"name AS d")
+			.select("a, b, c, d");
+
+		DataSet<Row> ds = tableEnv.toDataSet(table, Row.class);
+		List<Row> results = ds.collect();
+		String expected =
+			"Sales,28,4000.0,Peter\n" +
+			"Engineering,56,10000.0,Anna\n" +
+			"HR,42,6000.0,Lucy\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAsFromAndToPojo() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		List<SmallPojo> data = new ArrayList<>();
+		data.add(new SmallPojo("Peter", 28, 4000.00, "Sales"));
+		data.add(new SmallPojo("Anna", 56, 10000.00, "Engineering"));
+		data.add(new SmallPojo("Lucy", 42, 6000.00, "HR"));
+
+		Table table = tableEnv
+			.fromDataSet(env.fromCollection(data),
+				"department AS a, " +
+				"age AS b, " +
+				"salary AS c, " +
+				"name AS d")
+			.select("a, b, c, d");
+
+		DataSet<SmallPojo2> ds = tableEnv.toDataSet(table, SmallPojo2.class);
+		List<SmallPojo2> results = ds.collect();
+		String expected =
+			"Sales,28,4000.0,Peter\n" +
+			"Engineering,56,10000.0,Anna\n" +
+			"HR,42,6000.0,Lucy\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAsFromAndToPrivateFieldPojo() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		List<PrivateSmallPojo> data = new ArrayList<>();
+		data.add(new PrivateSmallPojo("Peter", 28, 4000.00, "Sales"));
+		data.add(new PrivateSmallPojo("Anna", 56, 10000.00, "Engineering"));
+		data.add(new PrivateSmallPojo("Lucy", 42, 6000.00, "HR"));
+
+		Table table = tableEnv
+			.fromDataSet(env.fromCollection(data),
+				"department AS a, " +
+				"age AS b, " +
+				"salary AS c, " +
+				"name AS d")
+			.select("a, b, c, d");
+
+		DataSet<PrivateSmallPojo2> ds = tableEnv.toDataSet(table, PrivateSmallPojo2.class);
+		List<PrivateSmallPojo2> results = ds.collect();
+		String expected =
+			"Sales,28,4000.0,Peter\n" +
+			"Engineering,56,10000.0,Anna\n" +
+			"HR,42,6000.0,Lucy\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAsWithPojoAndGenericTypes() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		List<PojoWithGeneric> data = new ArrayList<>();
+		data.add(new PojoWithGeneric("Peter", 28, new HashMap<String, String>(), new ArrayList<String>()));
+		HashMap<String, String> hm1 = new HashMap<>();
+		hm1.put("test1", "test1");
+		data.add(new PojoWithGeneric("Anna", 56, hm1, new ArrayList<String>()));
+		HashMap<String, String> hm2 = new HashMap<>();
+		hm2.put("abc", "cde");
+		data.add(new PojoWithGeneric("Lucy", 42, hm2, new ArrayList<String>()));
+
+		Table table = tableEnv
+			.fromDataSet(env.fromCollection(data),
+				"name AS a, " +
+				"age AS b, " +
+				"generic AS c, " +
+				"generic2 AS d")
+			.select("a, b, c, c as c2, d")
+			.select("a, b, c, c === c2, d");
+
+		DataSet<Row> ds = tableEnv.toDataSet(table, Row.class);
+		List<Row> results = ds.collect();
+		String expected =
+			"Peter,28,{},true,[]\n" +
+			"Anna,56,{test1=test1},true,[]\n" +
+			"Lucy,42,{abc=cde},true,[]\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = TableException.class)
+	public void testAsWithToFewFields() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		// Must fail. Not enough field names specified.
+		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b");
+	}
+
+	@Test(expected = TableException.class)
+	public void testAsWithToManyFields() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		// Must fail. Too many field names specified.
+		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c, d");
+	}
+
+	@Test(expected = TableException.class)
+	public void testAsWithAmbiguousFields() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		// Must fail. Specified field names are not unique.
+		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, b");
+	}
+
+	@Test(expected = TableException.class)
+	public void testAsWithNonFieldReference1() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		// Must fail. as() does only allow field name expressions
+		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a + 1, b, c");
+	}
+
+	@Test(expected = TableException.class)
+	public void testAsWithNonFieldReference2() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		// Must fail. as() does only allow field name expressions
+		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a as foo, b,  c");
+	}
+
+	@Test(expected = TableException.class)
+	public void testNonStaticClassInput() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		// Must fail since class is not static
+		tableEnv.fromDataSet(env.fromElements(new MyNonStatic()), "name");
+	}
+
+	@Test(expected = TableException.class)
+	public void testNonStaticClassOutput() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		// Must fail since class is not static
+		Table t = tableEnv.fromDataSet(env.fromElements(1, 2, 3), "number");
+		tableEnv.toDataSet(t, MyNonStatic.class);
+	}
+
+	@Test(expected = TableException.class)
+	public void testCustomCalciteConfig() {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		CalciteConfig cc = new CalciteConfigBuilder().replaceRuleSet(RuleSets.ofList()).build();
+		tableEnv.getConfig().setCalciteConfig(cc);
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		Table t = tableEnv.fromDataSet(ds);
+		tableEnv.toDataSet(t, Row.class);
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	public class MyNonStatic {
+		public int number;
+	}
+
+	@SuppressWarnings("unused")
+	public static class SmallPojo {
+
+		public SmallPojo() { }
+
+		public SmallPojo(String name, int age, double salary, String department) {
+			this.name = name;
+			this.age = age;
+			this.salary = salary;
+			this.department = department;
+		}
+
+		public String name;
+		public int age;
+		public double salary;
+		public String department;
+	}
+
+	@SuppressWarnings("unused")
+	public static class PojoWithGeneric {
+		public String name;
+		public int age;
+		public HashMap<String, String> generic;
+		public ArrayList<String> generic2;
+
+		public PojoWithGeneric() {
+			// default constructor
+		}
+
+		public PojoWithGeneric(String name, int age, HashMap<String, String> generic,
+				ArrayList<String> generic2) {
+			this.name = name;
+			this.age = age;
+			this.generic = generic;
+			this.generic2 = generic2;
+		}
+
+		@Override
+		public String toString() {
+			return name + "," + age + "," + generic + "," + generic2;
+		}
+	}
+
+	@SuppressWarnings("unused")
+	public static class PrivateSmallPojo {
+
+		public PrivateSmallPojo() { }
+
+		public PrivateSmallPojo(String name, int age, double salary, String department) {
+			this.name = name;
+			this.age = age;
+			this.salary = salary;
+			this.department = department;
+		}
+
+		private String name;
+		private int age;
+		private double salary;
+		private String department;
+
+		public String getName() {
+			return name;
+		}
+
+		public void setName(String name) {
+			this.name = name;
+		}
+
+		public int getAge() {
+			return age;
+		}
+
+		public void setAge(int age) {
+			this.age = age;
+		}
+
+		public double getSalary() {
+			return salary;
+		}
+
+		public void setSalary(double salary) {
+			this.salary = salary;
+		}
+
+		public String getDepartment() {
+			return department;
+		}
+
+		public void setDepartment(String department) {
+			this.department = department;
+		}
+	}
+
+	@SuppressWarnings("unused")
+	public static class SmallPojo2 {
+
+		public SmallPojo2() { }
+
+		public SmallPojo2(String a, int b, double c, String d) {
+			this.a = a;
+			this.b = b;
+			this.c = c;
+			this.d = d;
+		}
+
+		public String a;
+		public int b;
+		public double c;
+		public String d;
+
+		@Override
+		public String toString() {
+			return a + "," + b + "," + c + "," + d;
+		}
+	}
+
+	@SuppressWarnings("unused")
+	public static class PrivateSmallPojo2 {
+
+		public PrivateSmallPojo2() { }
+
+		public PrivateSmallPojo2(String a, int b, double c, String d) {
+			this.a = a;
+			this.b = b;
+			this.c = c;
+			this.d = d;
+		}
+
+		private String a;
+		private int b;
+		private double c;
+		private String d;
+
+		public String getA() {
+			return a;
+		}
+
+		public void setA(String a) {
+			this.a = a;
+		}
+
+		public int getB() {
+			return b;
+		}
+
+		public void setB(int b) {
+			this.b = b;
+		}
+
+		public double getC() {
+			return c;
+		}
+
+		public void setC(double c) {
+			this.c = c;
+		}
+
+		public String getD() {
+			return d;
+		}
+
+		public void setD(String d) {
+			this.d = d;
+		}
+
+		@Override
+		public String toString() {
+			return a + "," + b + "," + c + "," + d;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableSourceITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableSourceITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableSourceITCase.java
new file mode 100644
index 0000000..c822efb
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/TableSourceITCase.java
@@ -0,0 +1,119 @@
+/*
+ * 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.java.batch;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.types.Row;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.table.api.scala.batch.GeneratingInputFormat;
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.sources.BatchTableSource;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class TableSourceITCase extends TableProgramsTestBase {
+
+	public TableSourceITCase(TestExecutionMode mode, TableConfigMode configMode) {
+		super(mode, configMode);
+	}
+
+	@Test
+	public void testBatchTableSourceTableAPI() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		tableEnv.registerTableSource("MyTable", new TestBatchTableSource());
+
+		Table result = tableEnv.scan("MyTable")
+			.where("amount < 4")
+			.select("amount * id, name");
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+
+		String expected = "0,Record_0\n" + "0,Record_16\n" + "0,Record_32\n" + "1,Record_1\n" +
+			"17,Record_17\n" + "36,Record_18\n" + "4,Record_2\n" + "57,Record_19\n" + "9,Record_3\n";
+
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testBatchTableSourceSQL() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		tableEnv.registerTableSource("MyTable", new TestBatchTableSource());
+
+		Table result = tableEnv
+			.sql("SELECT amount * id, name FROM MyTable WHERE amount < 4");
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+
+		String expected = "0,Record_0\n" + "0,Record_16\n" + "0,Record_32\n" + "1,Record_1\n" +
+			"17,Record_17\n" + "36,Record_18\n" + "4,Record_2\n" + "57,Record_19\n" + "9,Record_3\n";
+
+		compareResultAsText(results, expected);
+	}
+
+	public static class TestBatchTableSource implements BatchTableSource<Row> {
+
+		private TypeInformation<?>[] fieldTypes = new TypeInformation<?>[] {
+			BasicTypeInfo.STRING_TYPE_INFO,
+			BasicTypeInfo.LONG_TYPE_INFO,
+			BasicTypeInfo.INT_TYPE_INFO
+		};
+
+		@Override
+		public DataSet<Row> getDataSet(ExecutionEnvironment execEnv) {
+			return execEnv.createInput(new GeneratingInputFormat(33), getReturnType()).setParallelism(1);
+		}
+
+		@Override
+		public int getNumberOfFields() {
+			return 3;
+		}
+
+		@Override
+		public String[] getFieldsNames() {
+			return new String[]{"name", "id", "amount"};
+		}
+
+		@Override
+		public TypeInformation<?>[] getFieldTypes() {
+			return fieldTypes;
+		}
+
+		@Override
+		public TypeInformation<Row> getReturnType() {
+			return new RowTypeInfo(fieldTypes);
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/SqlITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/SqlITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/SqlITCase.java
new file mode 100644
index 0000000..433410c
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/sql/SqlITCase.java
@@ -0,0 +1,141 @@
+/*
+ * 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.java.batch.sql;
+
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.types.Row;
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class SqlITCase extends TableProgramsTestBase {
+
+	public SqlITCase(TestExecutionMode mode, TableConfigMode configMode) {
+		super(mode, configMode);
+	}
+
+	@Test
+	public void testValues() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		String sqlQuery = "VALUES (1, 'Test', TRUE, DATE '1944-02-24', 12.4444444444444445)," +
+			"(2, 'Hello', TRUE, DATE '1944-02-24', 12.666666665)," +
+			"(3, 'World', FALSE, DATE '1944-12-24', 12.54444445)";
+		Table result = tableEnv.sql(sqlQuery);
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+
+		List<Row> results = resultSet.collect();
+		String expected = "3,World,false,1944-12-24,12.5444444500000000\n" +
+			"2,Hello,true,1944-02-24,12.6666666650000000\n" +
+			// Calcite converts to decimals and strings with equal length
+			"1,Test ,true,1944-02-24,12.4444444444444445\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testSelectFromTable() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		Table in = tableEnv.fromDataSet(ds, "a,b,c");
+		tableEnv.registerTable("T", in);
+
+		String sqlQuery = "SELECT a, c FROM T";
+		Table result = tableEnv.sql(sqlQuery);
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "1,Hi\n" + "2,Hello\n" + "3,Hello world\n" +
+			"4,Hello world, how are you?\n" + "5,I am fine.\n" + "6,Luke Skywalker\n" +
+			"7,Comment#1\n" + "8,Comment#2\n" + "9,Comment#3\n" + "10,Comment#4\n" +
+			"11,Comment#5\n" + "12,Comment#6\n" + "13,Comment#7\n" +
+			"14,Comment#8\n" + "15,Comment#9\n" + "16,Comment#10\n" +
+			"17,Comment#11\n" + "18,Comment#12\n" + "19,Comment#13\n" +
+			"20,Comment#14\n" + "21,Comment#15\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testFilterFromDataSet() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		tableEnv.registerDataSet("DataSetTable", ds, "x, y, z");
+
+		String sqlQuery = "SELECT x FROM DataSetTable WHERE z LIKE '%Hello%'";
+		Table result = tableEnv.sql(sqlQuery);
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "2\n" + "3\n" + "4";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAggregation() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		tableEnv.registerDataSet("AggTable", ds, "x, y, z");
+
+		String sqlQuery = "SELECT sum(x), min(x), max(x), count(y), avg(x) FROM AggTable";
+		Table result = tableEnv.sql(sqlQuery);
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "231,1,21,21,11";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testJoin() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
+
+		tableEnv.registerDataSet("t1", ds1, "a, b, c");
+		tableEnv.registerDataSet("t2",ds2, "d, e, f, g, h");
+
+		String sqlQuery = "SELECT c, g FROM t1, t2 WHERE b = e";
+		Table result = tableEnv.sql(sqlQuery);
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n";
+		compareResultAsText(results, expected);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/AggregationsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/AggregationsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/AggregationsITCase.java
new file mode 100644
index 0000000..d37ebb5
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/AggregationsITCase.java
@@ -0,0 +1,380 @@
+/*
+ * 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.java.batch.table;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.operators.Order;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.DataSource;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple5;
+import org.apache.flink.api.java.tuple.Tuple7;
+import org.apache.flink.types.Row;
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.examples.java.WordCountTable.WC;
+import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class AggregationsITCase extends TableProgramsTestBase {
+
+	public AggregationsITCase(TestExecutionMode mode, TableConfigMode configMode){
+		super(mode, configMode);
+	}
+
+	@Test
+	public void testAggregationTypes() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		Table table = tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env));
+
+		Table result = table.select("f0.sum, f0.min, f0.max, f0.count, f0.avg");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "231,1,21,21,11";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testAggregationOnNonExistingField() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		Table table =
+				tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env));
+
+		Table result =
+				table.select("foo.avg");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testWorkingAggregationDataTypes() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSource<Tuple7<Byte, Short, Integer, Long, Float, Double, String>> input =
+				env.fromElements(
+						new Tuple7<>((byte) 1, (short) 1, 1, 1L, 1.0f, 1.0d, "Hello"),
+						new Tuple7<>((byte) 2, (short) 2, 2, 2L, 2.0f, 2.0d, "Ciao"));
+
+		Table table = tableEnv.fromDataSet(input);
+
+		Table result =
+				table.select("f0.avg, f1.avg, f2.avg, f3.avg, f4.avg, f5.avg, f6.count");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "1,1,1,1,1.5,1.5,2";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAggregationWithArithmetic() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSource<Tuple2<Float, String>> input =
+				env.fromElements(
+						new Tuple2<>(1f, "Hello"),
+						new Tuple2<>(2f, "Ciao"));
+
+		Table table =
+				tableEnv.fromDataSet(input);
+
+		Table result =
+				table.select("(f0 + 2).avg + 2, f1.count + 5");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "5.5,7";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAggregationWithTwoCount() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSource<Tuple2<Float, String>> input =
+			env.fromElements(
+				new Tuple2<>(1f, "Hello"),
+				new Tuple2<>(2f, "Ciao"));
+
+		Table table =
+			tableEnv.fromDataSet(input);
+
+		Table result =
+			table.select("f0.count, f1.count");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "2,2";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testNonWorkingDataTypes() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSource<Tuple2<Float, String>> input = env.fromElements(new Tuple2<>(1f, "Hello"));
+
+		Table table =
+				tableEnv.fromDataSet(input);
+
+		Table result =
+				// Must fail. Cannot compute SUM aggregate on String field.
+				table.select("f1.sum");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testNoNestedAggregation() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSource<Tuple2<Float, String>> input = env.fromElements(new Tuple2<>(1f, "Hello"));
+
+		Table table =
+				tableEnv.fromDataSet(input);
+
+		Table result =
+				// Must fail. Aggregation on aggregation not allowed.
+				table.select("f0.sum.sum");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testGroupingOnNonExistentField() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+
+		tableEnv
+			.fromDataSet(input, "a, b, c")
+			// must fail. Field foo is not in input
+			.groupBy("foo")
+			.select("a.avg");
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testGroupingInvalidSelection() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+
+		tableEnv
+			.fromDataSet(input, "a, b, c")
+			.groupBy("a, b")
+			// must fail. Field c is not a grouping key or aggregation
+			.select("c");
+	}
+
+	@Test
+	public void testGroupedAggregate() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		Table result = table
+				.groupBy("b").select("b, a.sum");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testGroupingKeyForwardIfNotUsed() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		Table result = table
+				.groupBy("b").select("a.sum");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "1\n" + "5\n" + "15\n" + "34\n" + "65\n" + "111\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testGroupNoAggregation() throws Exception {
+
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		Table result = table
+			.groupBy("b").select("a.sum as d, b").groupBy("b, d").select("b");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n";
+		List<Row> results = ds.collect();
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testPojoAggregation() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+		DataSet<WC> input = env.fromElements(
+				new WC("Hello", 1),
+				new WC("Ciao", 1),
+				new WC("Hello", 1),
+				new WC("Hola", 1),
+				new WC("Hola", 1));
+
+		Table table = tableEnv.fromDataSet(input);
+
+		Table filtered = table
+				.groupBy("word")
+				.select("word.count as frequency, word")
+				.filter("frequency = 2");
+
+		List<String> result = tableEnv.toDataSet(filtered, WC.class)
+				.map(new MapFunction<WC, String>() {
+					public String map(WC value) throws Exception {
+						return value.word;
+					}
+				}).collect();
+		String expected = "Hello\n" + "Hola";
+		compareResultAsText(result, expected);
+	}
+
+	@Test
+	public void testPojoGrouping() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+
+		DataSet<Tuple3<String, Double, String>> data = env.fromElements(
+			new Tuple3<>("A", 23.0, "Z"),
+			new Tuple3<>("A", 24.0, "Y"),
+			new Tuple3<>("B", 1.0, "Z"));
+
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		Table table = tableEnv
+			.fromDataSet(data, "groupMe, value, name")
+			.select("groupMe, value, name")
+			.where("groupMe != 'B'");
+
+		DataSet<MyPojo> myPojos = tableEnv.toDataSet(table, MyPojo.class);
+
+		DataSet<MyPojo> result = myPojos.groupBy("groupMe")
+			.sortGroup("value", Order.DESCENDING)
+			.first(1);
+
+		List<MyPojo> resultList = result.collect();
+		compareResultAsText(resultList, "A,24.0,Y");
+	}
+
+	@Test
+	public void testDistinct() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		Table distinct = table.select("b").distinct();
+
+		DataSet<Row> ds = tableEnv.toDataSet(distinct, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "1\n" + "2\n" + "3\n"+ "4\n"+ "5\n"+ "6\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testDistinctAfterAggregate() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple5<Integer, Long, Integer, String, Long>> input = CollectionDataSets.get5TupleDataSet(env);
+
+		Table table = tableEnv.fromDataSet(input, "a, b, c, d, e");
+
+		Table distinct = table.groupBy("a, e").select("e").distinct();
+
+		DataSet<Row> ds = tableEnv.toDataSet(distinct, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "1\n" + "2\n" + "3\n";
+		compareResultAsText(results, expected);
+	}
+
+	// --------------------------------------------------------------------------------------------
+
+	public static class MyPojo implements Serializable {
+		private static final long serialVersionUID = 8741918940120107213L;
+
+		public String groupMe;
+		public double value;
+		public String name;
+
+		public MyPojo() {
+			// for serialization
+		}
+
+		public MyPojo(String groupMe, double value, String name) {
+			this.groupMe = groupMe;
+			this.value = value;
+			this.name = name;
+		}
+
+		@Override
+		public String toString() {
+			return groupMe + "," + value + "," + name;
+		}
+	}
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/CalcITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/CalcITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/CalcITCase.java
new file mode 100644
index 0000000..b1ef563
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/CalcITCase.java
@@ -0,0 +1,324 @@
+/*
+ * 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.java.batch.table;
+
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.flink.api.java.operators.DataSource;
+import org.apache.flink.types.Row;
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.functions.ScalarFunction;
+import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.List;
+
+@RunWith(Parameterized.class)
+public class CalcITCase extends TableProgramsTestBase {
+
+	public CalcITCase(TestExecutionMode mode, TableConfigMode configMode){
+		super(mode, configMode);
+	}
+
+	@Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
+	public static Collection<Object[]> parameters() {
+		return Arrays.asList(new Object[][] {
+			{ TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT() },
+			{ TestExecutionMode.COLLECTION, TableProgramsTestBase.NO_NULL() }
+		});
+	}
+
+	@Test
+	public void testSimpleSelectAllWithAs() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		Table in = tableEnv.fromDataSet(ds, "a,b,c");
+
+		Table result = in
+				.select("a, b, c");
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+			"4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+			"7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+			"11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" +
+			"14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" +
+			"17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" +
+			"20,6,Comment#14\n" + "21,6,Comment#15\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testSimpleSelectWithNaming() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		Table in = tableEnv.fromDataSet(ds);
+
+		Table result = in
+				.select("f0 as a, f1 as b")
+				.select("a, b");
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
+				"8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
+				"16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testSimpleSelectRenameAll() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		Table in = tableEnv.fromDataSet(ds);
+
+		Table result = in
+			.select("f0 as a, f1 as b, f2 as c")
+			.select("a, b");
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
+			"8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
+			"16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testSelectInvalidField() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+
+		tableEnv.fromDataSet(ds, "a, b, c")
+			// Must fail. Field foo does not exist
+			.select("a + 1, foo + 2");
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testSelectAmbiguousFieldNames() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+
+		tableEnv.fromDataSet(ds, "a, b, c")
+			// Must fail. Field foo does not exist
+			.select("a + 1 as foo, b + 2 as foo");
+	}
+
+	@Test
+	public void testSelectStar() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
+		Table in = tableEnv.fromDataSet(ds, "a,b,c");
+
+		Table result = in
+			.select("*");
+
+		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = resultSet.collect();
+		String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+		                  "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+		                  "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+		                  "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" +
+		                  "14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" +
+		                  "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" +
+		                  "20,6,Comment#14\n" + "21,6,Comment#15\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAllRejectingFilter() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		Table result = table
+				.filter("false");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testAllPassingFilter() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		Table result = table
+				.filter("true");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+			"4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+			"7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+			"11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" +
+			"14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" +
+			"17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" +
+			"20,6,Comment#14\n" + "21,6,Comment#15\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testFilterOnIntegerTupleField() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		Table result = table
+				.filter(" a % 2 = 0 ");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + "6,3,Luke Skywalker\n" + "8,4," +
+				"Comment#2\n" + "10,4,Comment#4\n" + "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," +
+				"Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testNotEquals() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		Table result = table
+				.filter("!( a % 2 <> 0 ) ");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + "6,3,Luke Skywalker\n" + "8,4," +
+				"Comment#2\n" + "10,4,Comment#4\n" + "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," +
+				"Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testDisjunctivePreds() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		Table result = table
+			.filter("a < 2 || a > 20");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "1,1,Hi\n" + "21,6,Comment#15\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testIntegerBiggerThan128() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = env.fromElements(new Tuple3<>(300, 1L, "Hello"));
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		Table result = table
+			.filter("a = 300 ");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "300,1,Hello\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test(expected = ValidationException.class)
+	public void testFilterInvalidField() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
+		Table table = tableEnv.fromDataSet(input, "a, b, c");
+
+		table
+			// Must fail. Field foo does not exist.
+			.filter("foo = 17");
+	}
+
+	public static class OldHashCode extends ScalarFunction {
+		public int eval(String s) {
+			return -1;
+		}
+	}
+
+	public static class HashCode extends ScalarFunction {
+		public int eval(String s) {
+			return s.hashCode();
+		}
+	}
+
+	@Test
+	public void testUserDefinedScalarFunction() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		tableEnv.registerFunction("hashCode", new OldHashCode());
+		tableEnv.registerFunction("hashCode", new HashCode());
+
+		DataSource<String> input = env.fromElements("a", "b", "c");
+
+		Table table = tableEnv.fromDataSet(input, "text");
+
+		Table result = table.select("text.hashCode()");
+
+		DataSet<Integer> ds = tableEnv.toDataSet(result, Integer.class);
+		List<Integer> results = ds.collect();
+		String expected = "97\n98\n99";
+		compareResultAsText(results, expected);
+	}
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/CastingITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/CastingITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/CastingITCase.java
new file mode 100644
index 0000000..b1bb6e8
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/java/org/apache/flink/table/api/java/batch/table/CastingITCase.java
@@ -0,0 +1,140 @@
+/*
+ * 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.java.batch.table;
+
+import java.util.List;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.operators.DataSource;
+import org.apache.flink.table.api.java.BatchTableEnvironment;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.api.java.tuple.Tuple4;
+import org.apache.flink.api.java.tuple.Tuple6;
+import org.apache.flink.api.java.tuple.Tuple8;
+import org.apache.flink.types.Row;
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+@RunWith(Parameterized.class)
+public class CastingITCase extends TableProgramsTestBase {
+
+	public CastingITCase(TestExecutionMode mode, TableConfigMode configMode){
+		super(mode, configMode);
+	}
+
+	@Test
+	public void testNumericAutocastInArithmetic() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSource<Tuple8<Byte, Short, Integer, Long, Float, Double, Long, Double>> input =
+				env.fromElements(new Tuple8<>((byte) 1, (short) 1, 1, 1L, 1.0f, 1.0d, 1L, 1001.1));
+
+		Table table =
+				tableEnv.fromDataSet(input);
+
+		Table result = table.select("f0 + 1, f1 +" +
+				" 1, f2 + 1L, f3 + 1.0f, f4 + 1.0d, f5 + 1, f6 + 1.0d, f7 + f0");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "2,2,2,2.0,2.0,2.0,2.0,1002.1";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testNumericAutocastInComparison() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSource<Tuple6<Byte, Short, Integer, Long, Float, Double>> input =
+				env.fromElements(
+						new Tuple6<>((byte) 1, (short) 1, 1, 1L, 1.0f, 1.0d),
+						new Tuple6<>((byte) 2, (short) 2, 2, 2L, 2.0f, 2.0d));
+
+		Table table =
+				tableEnv.fromDataSet(input, "a,b,c,d,e,f");
+
+		Table result = table
+				.filter("a > 1 && b > 1 && c > 1L && d > 1.0f && e > 1.0d && f > 1");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "2,2,2,2,2.0,2.0";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testCasting() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSource<Tuple4<Integer, Double, Long, Boolean>> input =
+				env.fromElements(new Tuple4<>(1, 0.0, 1L, true));
+
+		Table table =
+				tableEnv.fromDataSet(input);
+
+		Table result = table.select(
+				// * -> String
+				"f0.cast(STRING), f1.cast(STRING), f2.cast(STRING), f3.cast(STRING)," +
+				// NUMERIC TYPE -> Boolean
+				"f0.cast(BOOL), f1.cast(BOOL), f2.cast(BOOL)," +
+				// NUMERIC TYPE -> NUMERIC TYPE
+				"f0.cast(DOUBLE), f1.cast(INT), f2.cast(SHORT)," +
+				// Boolean -> NUMERIC TYPE
+				"f3.cast(DOUBLE)," +
+				// identity casting
+				"f0.cast(INT), f1.cast(DOUBLE), f2.cast(LONG), f3.cast(BOOL)");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "1,0.0,1,true," +
+			"true,false,true," +
+			"1.0,0,1," +
+			"1.0," +
+			"1,0.0,1,true\n";
+		compareResultAsText(results, expected);
+	}
+
+	@Test
+	public void testCastFromString() throws Exception {
+		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
+
+		DataSource<Tuple3<String, String, String>> input =
+				env.fromElements(new Tuple3<>("1", "true", "2.0"));
+
+		Table table =
+				tableEnv.fromDataSet(input);
+
+		Table result = table.select(
+				"f0.cast(BYTE), f0.cast(SHORT), f0.cast(INT), f0.cast(LONG), f2.cast(DOUBLE), f2.cast(FLOAT), f1.cast(BOOL)");
+
+		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
+		List<Row> results = ds.collect();
+		String expected = "1,1,1,1,2.0,2.0,true\n";
+		compareResultAsText(results, expected);
+	}
+}
+


[15/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/ExplainTest.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/ExplainTest.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/ExplainTest.java
deleted file mode 100644
index 747cd92..0000000
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/ExplainTest.java
+++ /dev/null
@@ -1,160 +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.api.java.batch;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.table.BatchTableEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.table.Table;
-import org.apache.flink.api.table.TableEnvironment;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.junit.Test;
-
-import java.io.File;
-import java.util.Scanner;
-
-import static org.junit.Assert.assertEquals;
-
-public class ExplainTest extends MultipleProgramsTestBase {
-
-	public ExplainTest() {
-		super(TestExecutionMode.CLUSTER);
-	}
-
-	private static String testFilePath = ExplainTest.class.getResource("/").getFile();
-
-	@Test
-	public void testFilterWithoutExtended() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple2<Integer, String>> input = env.fromElements(new Tuple2<>(1,"d"));
-		Table table = tableEnv
-			.fromDataSet(input, "a, b")
-			.filter("a % 2 = 0");
-
-		String result = tableEnv.explain(table).replaceAll("\\r\\n", "\n");
-		try (Scanner scanner = new Scanner(new File(testFilePath +
-			"../../src/test/scala/resources/testFilter0.out"))){
-			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
-			assertEquals(source, result);
-		}
-	}
-
-	@Test
-	public void testFilterWithExtended() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple2<Integer, String>> input = env.fromElements(new Tuple2<>(1,"d"));
-		Table table = tableEnv
-			.fromDataSet(input, "a, b")
-			.filter("a % 2 = 0");
-
-		String result = tableEnv.explain(table, true).replaceAll("\\r\\n", "\n");
-		try (Scanner scanner = new Scanner(new File(testFilePath +
-			"../../src/test/scala/resources/testFilter1.out"))){
-			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
-			assertEquals(source, result);
-		}
-	}
-
-	@Test
-	public void testJoinWithoutExtended() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple2<Integer, String>> input1 = env.fromElements(new Tuple2<>(1,"d"));
-		DataSet<Tuple2<Integer, String>> input2 = env.fromElements(new Tuple2<>(1,"d"));
-		Table table1 = tableEnv.fromDataSet(input1, "a, b");
-		Table table2 = tableEnv.fromDataSet(input2, "c, d");
-		Table table = table1
-			.join(table2)
-			.where("b = d")
-			.select("a, c");
-
-		String result = tableEnv.explain(table).replaceAll("\\r\\n", "\n");
-		try (Scanner scanner = new Scanner(new File(testFilePath +
-			"../../src/test/scala/resources/testJoin0.out"))){
-			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
-			assertEquals(source, result);
-		}
-	}
-
-	@Test
-	public void testJoinWithExtended() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple2<Integer, String>> input1 = env.fromElements(new Tuple2<>(1,"d"));
-		DataSet<Tuple2<Integer, String>> input2 = env.fromElements(new Tuple2<>(1,"d"));
-		Table table1 = tableEnv.fromDataSet(input1, "a, b");
-		Table table2 = tableEnv.fromDataSet(input2, "c, d");
-		Table table = table1
-			.join(table2)
-			.where("b = d")
-			.select("a, c");
-
-		String result = tableEnv.explain(table, true).replaceAll("\\r\\n", "\n");
-		try (Scanner scanner = new Scanner(new File(testFilePath +
-			"../../src/test/scala/resources/testJoin1.out"))){
-			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
-			assertEquals(source, result);
-		}
-	}
-
-	@Test
-	public void testUnionWithoutExtended() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple2<Integer, String>> input1 = env.fromElements(new Tuple2<>(1,"d"));
-		DataSet<Tuple2<Integer, String>> input2 = env.fromElements(new Tuple2<>(1,"d"));
-		Table table1 = tableEnv.fromDataSet(input1, "count, word");
-		Table table2 = tableEnv.fromDataSet(input2, "count, word");
-		Table table = table1.unionAll(table2);
-
-		String result = tableEnv.explain(table).replaceAll("\\r\\n", "\n");
-		try (Scanner scanner = new Scanner(new File(testFilePath +
-			"../../src/test/scala/resources/testUnion0.out"))){
-			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
-			assertEquals(source, result);
-		}
-	}
-
-	@Test
-	public void testUnionWithExtended() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple2<Integer, String>> input1 = env.fromElements(new Tuple2<>(1,"d"));
-		DataSet<Tuple2<Integer, String>> input2 = env.fromElements(new Tuple2<>(1,"d"));
-		Table table1 = tableEnv.fromDataSet(input1, "count, word");
-		Table table2 = tableEnv.fromDataSet(input2, "count, word");
-		Table table = table1.unionAll(table2);
-
-		String result = tableEnv.explain(table, true).replaceAll("\\r\\n", "\n");
-		try (Scanner scanner = new Scanner(new File(testFilePath +
-			"../../src/test/scala/resources/testUnion1.out"))){
-			String source = scanner.useDelimiter("\\A").next().replaceAll("\\r\\n", "\n");
-			assertEquals(source, result);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java
deleted file mode 100644
index 75d964b..0000000
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableEnvironmentITCase.java
+++ /dev/null
@@ -1,631 +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.api.java.batch;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.calcite.tools.RuleSets;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.table.BatchTableEnvironment;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.typeutils.TupleTypeInfo;
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase;
-import org.apache.flink.api.table.CalciteConfig;
-import org.apache.flink.api.table.CalciteConfigBuilder;
-import org.apache.flink.types.Row;
-import org.apache.flink.api.table.Table;
-import org.apache.flink.api.table.TableEnvironment;
-import org.apache.flink.api.table.TableException;
-import org.apache.flink.api.table.ValidationException;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class TableEnvironmentITCase extends TableProgramsTestBase {
-
-	public TableEnvironmentITCase(TestExecutionMode mode, TableConfigMode configMode) {
-		super(mode, configMode);
-	}
-
-	@Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
-	public static Collection<Object[]> parameters() {
-		return Arrays.asList(new Object[][] {
-			{ TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT() },
-			{ TestExecutionMode.COLLECTION, TableProgramsTestBase.EFFICIENT() }
-		});
-	}
-
-	@Test
-	public void testSimpleRegister() throws Exception {
-		final String tableName = "MyTable";
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		tableEnv.registerDataSet(tableName, ds);
-		Table t = tableEnv.scan(tableName);
-
-		Table result = t.select("f0, f1");
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
-				"8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
-				"16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testRegisterWithFields() throws Exception {
-		final String tableName = "MyTable";
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		tableEnv.registerDataSet(tableName, ds, "a, b, c");
-		Table t = tableEnv.scan(tableName);
-
-		Table result = t.select("a, b, c");
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-				"4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-				"7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-				"11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" +
-				"14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" +
-				"17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" +
-				"20,6,Comment#14\n" + "21,6,Comment#15\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = TableException.class)
-	public void testRegisterExistingDatasetTable() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		tableEnv.registerDataSet("MyTable", ds);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 =
-				CollectionDataSets.getSmall5TupleDataSet(env);
-		// Must fail. Name is already used for different table.
-		tableEnv.registerDataSet("MyTable", ds2);
-	}
-
-	@Test(expected = TableException.class)
-	public void testScanUnregisteredTable() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		// Must fail. No table registered under that name.
-		tableEnv.scan("nonRegisteredTable");
-	}
-
-	@Test
-	public void testTableRegister() throws Exception {
-		final String tableName = "MyTable";
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		Table t = tableEnv.fromDataSet(ds);
-		tableEnv.registerTable(tableName, t);
-		Table result = tableEnv.scan(tableName).select("f0, f1").filter("f0 > 7");
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" +
-				"13,5\n" + "14,5\n" + "15,5\n" +
-				"16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = TableException.class)
-	public void testIllegalName() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		Table t = tableEnv.fromDataSet(ds);
-		// Must fail. Table name matches internal name pattern.
-		tableEnv.registerTable("_DataSetTable_42", t);
-	}
-
-	@Test(expected = TableException.class)
-	public void testRegisterTableFromOtherEnv() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv1 = TableEnvironment.getTableEnvironment(env, config());
-		BatchTableEnvironment tableEnv2 = TableEnvironment.getTableEnvironment(env, config());
-
-		Table t = tableEnv1.fromDataSet(CollectionDataSets.get3TupleDataSet(env));
-		// Must fail. Table is bound to different TableEnvironment.
-		tableEnv2.registerTable("MyTable", t);
-	}
-
-	@Test
-	public void testAsFromTuple() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		Table table = tableEnv
-			.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c")
-			.select("a, b, c");
-
-		DataSet<Row> ds = tableEnv.toDataSet(table, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-			"4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-			"7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-			"11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" +
-			"14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" +
-			"17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" +
-			"20,6,Comment#14\n" + "21,6,Comment#15\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAsFromAndToTuple() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		Table table = tableEnv
-			.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c")
-			.select("a, b, c");
-
-		TypeInformation<?> ti = new TupleTypeInfo<Tuple3<Integer, Long, String>>(
-			BasicTypeInfo.INT_TYPE_INFO,
-			BasicTypeInfo.LONG_TYPE_INFO,
-			BasicTypeInfo.STRING_TYPE_INFO);
-
-		DataSet<?> ds = tableEnv.toDataSet(table, ti);
-		List<?> results = ds.collect();
-		String expected = "(1,1,Hi)\n" + "(2,2,Hello)\n" + "(3,2,Hello world)\n" +
-			"(4,3,Hello world, how are you?)\n" + "(5,3,I am fine.)\n" + "(6,3,Luke Skywalker)\n" +
-			"(7,4,Comment#1)\n" + "(8,4,Comment#2)\n" + "(9,4,Comment#3)\n" + "(10,4,Comment#4)\n" +
-			"(11,5,Comment#5)\n" + "(12,5,Comment#6)\n" + "(13,5,Comment#7)\n" +
-			"(14,5,Comment#8)\n" + "(15,5,Comment#9)\n" + "(16,6,Comment#10)\n" +
-			"(17,6,Comment#11)\n" + "(18,6,Comment#12)\n" + "(19,6,Comment#13)\n" +
-			"(20,6,Comment#14)\n" + "(21,6,Comment#15)\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAsFromTupleToPojo() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		List<Tuple4<String, Integer, Double, String>> data = new ArrayList<>();
-		data.add(new Tuple4<>("Rofl", 1, 1.0, "Hi"));
-		data.add(new Tuple4<>("lol", 2, 1.0, "Hi"));
-		data.add(new Tuple4<>("Test me", 4, 3.33, "Hello world"));
-
-		Table table = tableEnv
-			.fromDataSet(env.fromCollection(data), "a, b, c, d")
-			.select("a, b, c, d");
-
-		DataSet<SmallPojo2> ds = tableEnv.toDataSet(table, SmallPojo2.class);
-		List<SmallPojo2> results = ds.collect();
-		String expected = "Rofl,1,1.0,Hi\n" + "lol,2,1.0,Hi\n" + "Test me,4,3.33,Hello world\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAsFromPojo() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		List<SmallPojo> data = new ArrayList<>();
-		data.add(new SmallPojo("Peter", 28, 4000.00, "Sales"));
-		data.add(new SmallPojo("Anna", 56, 10000.00, "Engineering"));
-		data.add(new SmallPojo("Lucy", 42, 6000.00, "HR"));
-
-		Table table = tableEnv
-			.fromDataSet(env.fromCollection(data),
-				"department AS a, " +
-				"age AS b, " +
-				"salary AS c, " +
-				"name AS d")
-			.select("a, b, c, d");
-
-		DataSet<Row> ds = tableEnv.toDataSet(table, Row.class);
-		List<Row> results = ds.collect();
-		String expected =
-			"Sales,28,4000.0,Peter\n" +
-			"Engineering,56,10000.0,Anna\n" +
-			"HR,42,6000.0,Lucy\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAsFromPrivateFieldsPojo() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		List<PrivateSmallPojo> data = new ArrayList<>();
-		data.add(new PrivateSmallPojo("Peter", 28, 4000.00, "Sales"));
-		data.add(new PrivateSmallPojo("Anna", 56, 10000.00, "Engineering"));
-		data.add(new PrivateSmallPojo("Lucy", 42, 6000.00, "HR"));
-
-		Table table = tableEnv
-			.fromDataSet(env.fromCollection(data),
-				"department AS a, " +
-				"age AS b, " +
-				"salary AS c, " +
-				"name AS d")
-			.select("a, b, c, d");
-
-		DataSet<Row> ds = tableEnv.toDataSet(table, Row.class);
-		List<Row> results = ds.collect();
-		String expected =
-			"Sales,28,4000.0,Peter\n" +
-			"Engineering,56,10000.0,Anna\n" +
-			"HR,42,6000.0,Lucy\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAsFromAndToPojo() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		List<SmallPojo> data = new ArrayList<>();
-		data.add(new SmallPojo("Peter", 28, 4000.00, "Sales"));
-		data.add(new SmallPojo("Anna", 56, 10000.00, "Engineering"));
-		data.add(new SmallPojo("Lucy", 42, 6000.00, "HR"));
-
-		Table table = tableEnv
-			.fromDataSet(env.fromCollection(data),
-				"department AS a, " +
-				"age AS b, " +
-				"salary AS c, " +
-				"name AS d")
-			.select("a, b, c, d");
-
-		DataSet<SmallPojo2> ds = tableEnv.toDataSet(table, SmallPojo2.class);
-		List<SmallPojo2> results = ds.collect();
-		String expected =
-			"Sales,28,4000.0,Peter\n" +
-			"Engineering,56,10000.0,Anna\n" +
-			"HR,42,6000.0,Lucy\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAsFromAndToPrivateFieldPojo() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		List<PrivateSmallPojo> data = new ArrayList<>();
-		data.add(new PrivateSmallPojo("Peter", 28, 4000.00, "Sales"));
-		data.add(new PrivateSmallPojo("Anna", 56, 10000.00, "Engineering"));
-		data.add(new PrivateSmallPojo("Lucy", 42, 6000.00, "HR"));
-
-		Table table = tableEnv
-			.fromDataSet(env.fromCollection(data),
-				"department AS a, " +
-				"age AS b, " +
-				"salary AS c, " +
-				"name AS d")
-			.select("a, b, c, d");
-
-		DataSet<PrivateSmallPojo2> ds = tableEnv.toDataSet(table, PrivateSmallPojo2.class);
-		List<PrivateSmallPojo2> results = ds.collect();
-		String expected =
-			"Sales,28,4000.0,Peter\n" +
-			"Engineering,56,10000.0,Anna\n" +
-			"HR,42,6000.0,Lucy\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAsWithPojoAndGenericTypes() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		List<PojoWithGeneric> data = new ArrayList<>();
-		data.add(new PojoWithGeneric("Peter", 28, new HashMap<String, String>(), new ArrayList<String>()));
-		HashMap<String, String> hm1 = new HashMap<>();
-		hm1.put("test1", "test1");
-		data.add(new PojoWithGeneric("Anna", 56, hm1, new ArrayList<String>()));
-		HashMap<String, String> hm2 = new HashMap<>();
-		hm2.put("abc", "cde");
-		data.add(new PojoWithGeneric("Lucy", 42, hm2, new ArrayList<String>()));
-
-		Table table = tableEnv
-			.fromDataSet(env.fromCollection(data),
-				"name AS a, " +
-				"age AS b, " +
-				"generic AS c, " +
-				"generic2 AS d")
-			.select("a, b, c, c as c2, d")
-			.select("a, b, c, c === c2, d");
-
-		DataSet<Row> ds = tableEnv.toDataSet(table, Row.class);
-		List<Row> results = ds.collect();
-		String expected =
-			"Peter,28,{},true,[]\n" +
-			"Anna,56,{test1=test1},true,[]\n" +
-			"Lucy,42,{abc=cde},true,[]\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = TableException.class)
-	public void testAsWithToFewFields() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		// Must fail. Not enough field names specified.
-		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b");
-	}
-
-	@Test(expected = TableException.class)
-	public void testAsWithToManyFields() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		// Must fail. Too many field names specified.
-		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, c, d");
-	}
-
-	@Test(expected = TableException.class)
-	public void testAsWithAmbiguousFields() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		// Must fail. Specified field names are not unique.
-		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a, b, b");
-	}
-
-	@Test(expected = TableException.class)
-	public void testAsWithNonFieldReference1() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		// Must fail. as() does only allow field name expressions
-		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a + 1, b, c");
-	}
-
-	@Test(expected = TableException.class)
-	public void testAsWithNonFieldReference2() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		// Must fail. as() does only allow field name expressions
-		tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env), "a as foo, b,  c");
-	}
-
-	@Test(expected = TableException.class)
-	public void testNonStaticClassInput() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		// Must fail since class is not static
-		tableEnv.fromDataSet(env.fromElements(new MyNonStatic()), "name");
-	}
-
-	@Test(expected = TableException.class)
-	public void testNonStaticClassOutput() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		// Must fail since class is not static
-		Table t = tableEnv.fromDataSet(env.fromElements(1, 2, 3), "number");
-		tableEnv.toDataSet(t, MyNonStatic.class);
-	}
-
-	@Test(expected = TableException.class)
-	public void testCustomCalciteConfig() {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		CalciteConfig cc = new CalciteConfigBuilder().replaceRuleSet(RuleSets.ofList()).build();
-		tableEnv.getConfig().setCalciteConfig(cc);
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		Table t = tableEnv.fromDataSet(ds);
-		tableEnv.toDataSet(t, Row.class);
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	public class MyNonStatic {
-		public int number;
-	}
-
-	@SuppressWarnings("unused")
-	public static class SmallPojo {
-
-		public SmallPojo() { }
-
-		public SmallPojo(String name, int age, double salary, String department) {
-			this.name = name;
-			this.age = age;
-			this.salary = salary;
-			this.department = department;
-		}
-
-		public String name;
-		public int age;
-		public double salary;
-		public String department;
-	}
-
-	@SuppressWarnings("unused")
-	public static class PojoWithGeneric {
-		public String name;
-		public int age;
-		public HashMap<String, String> generic;
-		public ArrayList<String> generic2;
-
-		public PojoWithGeneric() {
-			// default constructor
-		}
-
-		public PojoWithGeneric(String name, int age, HashMap<String, String> generic,
-				ArrayList<String> generic2) {
-			this.name = name;
-			this.age = age;
-			this.generic = generic;
-			this.generic2 = generic2;
-		}
-
-		@Override
-		public String toString() {
-			return name + "," + age + "," + generic + "," + generic2;
-		}
-	}
-
-	@SuppressWarnings("unused")
-	public static class PrivateSmallPojo {
-
-		public PrivateSmallPojo() { }
-
-		public PrivateSmallPojo(String name, int age, double salary, String department) {
-			this.name = name;
-			this.age = age;
-			this.salary = salary;
-			this.department = department;
-		}
-
-		private String name;
-		private int age;
-		private double salary;
-		private String department;
-
-		public String getName() {
-			return name;
-		}
-
-		public void setName(String name) {
-			this.name = name;
-		}
-
-		public int getAge() {
-			return age;
-		}
-
-		public void setAge(int age) {
-			this.age = age;
-		}
-
-		public double getSalary() {
-			return salary;
-		}
-
-		public void setSalary(double salary) {
-			this.salary = salary;
-		}
-
-		public String getDepartment() {
-			return department;
-		}
-
-		public void setDepartment(String department) {
-			this.department = department;
-		}
-	}
-
-	@SuppressWarnings("unused")
-	public static class SmallPojo2 {
-
-		public SmallPojo2() { }
-
-		public SmallPojo2(String a, int b, double c, String d) {
-			this.a = a;
-			this.b = b;
-			this.c = c;
-			this.d = d;
-		}
-
-		public String a;
-		public int b;
-		public double c;
-		public String d;
-
-		@Override
-		public String toString() {
-			return a + "," + b + "," + c + "," + d;
-		}
-	}
-
-	@SuppressWarnings("unused")
-	public static class PrivateSmallPojo2 {
-
-		public PrivateSmallPojo2() { }
-
-		public PrivateSmallPojo2(String a, int b, double c, String d) {
-			this.a = a;
-			this.b = b;
-			this.c = c;
-			this.d = d;
-		}
-
-		private String a;
-		private int b;
-		private double c;
-		private String d;
-
-		public String getA() {
-			return a;
-		}
-
-		public void setA(String a) {
-			this.a = a;
-		}
-
-		public int getB() {
-			return b;
-		}
-
-		public void setB(int b) {
-			this.b = b;
-		}
-
-		public double getC() {
-			return c;
-		}
-
-		public void setC(double c) {
-			this.c = c;
-		}
-
-		public String getD() {
-			return d;
-		}
-
-		public void setD(String d) {
-			this.d = d;
-		}
-
-		@Override
-		public String toString() {
-			return a + "," + b + "," + c + "," + d;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableSourceITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableSourceITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableSourceITCase.java
deleted file mode 100644
index 7538808..0000000
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/TableSourceITCase.java
+++ /dev/null
@@ -1,119 +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.api.java.batch;
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.table.BatchTableEnvironment;
-import org.apache.flink.api.scala.batch.GeneratingInputFormat;
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase;
-import org.apache.flink.types.Row;
-import org.apache.flink.api.table.Table;
-import org.apache.flink.api.table.TableEnvironment;
-import org.apache.flink.api.table.sources.BatchTableSource;
-import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class TableSourceITCase extends TableProgramsTestBase {
-
-	public TableSourceITCase(TestExecutionMode mode, TableConfigMode configMode) {
-		super(mode, configMode);
-	}
-
-	@Test
-	public void testBatchTableSourceTableAPI() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		tableEnv.registerTableSource("MyTable", new TestBatchTableSource());
-
-		Table result = tableEnv.scan("MyTable")
-			.where("amount < 4")
-			.select("amount * id, name");
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-
-		String expected = "0,Record_0\n" + "0,Record_16\n" + "0,Record_32\n" + "1,Record_1\n" +
-			"17,Record_17\n" + "36,Record_18\n" + "4,Record_2\n" + "57,Record_19\n" + "9,Record_3\n";
-
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testBatchTableSourceSQL() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		tableEnv.registerTableSource("MyTable", new TestBatchTableSource());
-
-		Table result = tableEnv
-			.sql("SELECT amount * id, name FROM MyTable WHERE amount < 4");
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-
-		String expected = "0,Record_0\n" + "0,Record_16\n" + "0,Record_32\n" + "1,Record_1\n" +
-			"17,Record_17\n" + "36,Record_18\n" + "4,Record_2\n" + "57,Record_19\n" + "9,Record_3\n";
-
-		compareResultAsText(results, expected);
-	}
-
-	public static class TestBatchTableSource implements BatchTableSource<Row> {
-
-		private TypeInformation<?>[] fieldTypes = new TypeInformation<?>[] {
-			BasicTypeInfo.STRING_TYPE_INFO,
-			BasicTypeInfo.LONG_TYPE_INFO,
-			BasicTypeInfo.INT_TYPE_INFO
-		};
-
-		@Override
-		public DataSet<Row> getDataSet(ExecutionEnvironment execEnv) {
-			return execEnv.createInput(new GeneratingInputFormat(33), getReturnType()).setParallelism(1);
-		}
-
-		@Override
-		public int getNumberOfFields() {
-			return 3;
-		}
-
-		@Override
-		public String[] getFieldsNames() {
-			return new String[]{"name", "id", "amount"};
-		}
-
-		@Override
-		public TypeInformation<?>[] getFieldTypes() {
-			return fieldTypes;
-		}
-
-		@Override
-		public TypeInformation<Row> getReturnType() {
-			return new RowTypeInfo(fieldTypes);
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/sql/SqlITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/sql/SqlITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/sql/SqlITCase.java
deleted file mode 100644
index 89b0d50..0000000
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/sql/SqlITCase.java
+++ /dev/null
@@ -1,141 +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.api.java.batch.sql;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.table.BatchTableEnvironment;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase;
-import org.apache.flink.types.Row;
-import org.apache.flink.api.table.Table;
-import org.apache.flink.api.table.TableEnvironment;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class SqlITCase extends TableProgramsTestBase {
-
-	public SqlITCase(TestExecutionMode mode, TableConfigMode configMode) {
-		super(mode, configMode);
-	}
-
-	@Test
-	public void testValues() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		String sqlQuery = "VALUES (1, 'Test', TRUE, DATE '1944-02-24', 12.4444444444444445)," +
-			"(2, 'Hello', TRUE, DATE '1944-02-24', 12.666666665)," +
-			"(3, 'World', FALSE, DATE '1944-12-24', 12.54444445)";
-		Table result = tableEnv.sql(sqlQuery);
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-
-		List<Row> results = resultSet.collect();
-		String expected = "3,World,false,1944-12-24,12.5444444500000000\n" +
-			"2,Hello,true,1944-02-24,12.6666666650000000\n" +
-			// Calcite converts to decimals and strings with equal length
-			"1,Test ,true,1944-02-24,12.4444444444444445\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testSelectFromTable() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		Table in = tableEnv.fromDataSet(ds, "a,b,c");
-		tableEnv.registerTable("T", in);
-
-		String sqlQuery = "SELECT a, c FROM T";
-		Table result = tableEnv.sql(sqlQuery);
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "1,Hi\n" + "2,Hello\n" + "3,Hello world\n" +
-			"4,Hello world, how are you?\n" + "5,I am fine.\n" + "6,Luke Skywalker\n" +
-			"7,Comment#1\n" + "8,Comment#2\n" + "9,Comment#3\n" + "10,Comment#4\n" +
-			"11,Comment#5\n" + "12,Comment#6\n" + "13,Comment#7\n" +
-			"14,Comment#8\n" + "15,Comment#9\n" + "16,Comment#10\n" +
-			"17,Comment#11\n" + "18,Comment#12\n" + "19,Comment#13\n" +
-			"20,Comment#14\n" + "21,Comment#15\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testFilterFromDataSet() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		tableEnv.registerDataSet("DataSetTable", ds, "x, y, z");
-
-		String sqlQuery = "SELECT x FROM DataSetTable WHERE z LIKE '%Hello%'";
-		Table result = tableEnv.sql(sqlQuery);
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "2\n" + "3\n" + "4";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAggregation() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		tableEnv.registerDataSet("AggTable", ds, "x, y, z");
-
-		String sqlQuery = "SELECT sum(x), min(x), max(x), count(y), avg(x) FROM AggTable";
-		Table result = tableEnv.sql(sqlQuery);
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "231,1,21,21,11";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testJoin() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		tableEnv.registerDataSet("t1", ds1, "a, b, c");
-		tableEnv.registerDataSet("t2",ds2, "d, e, f, g, h");
-
-		String sqlQuery = "SELECT c, g FROM t1, t2 WHERE b = e";
-		Table result = tableEnv.sql(sqlQuery);
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n";
-		compareResultAsText(results, expected);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java
deleted file mode 100644
index 0856a70..0000000
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/AggregationsITCase.java
+++ /dev/null
@@ -1,380 +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.api.java.batch.table;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.DataSource;
-import org.apache.flink.api.java.table.BatchTableEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.java.tuple.Tuple7;
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase;
-import org.apache.flink.types.Row;
-import org.apache.flink.api.table.Table;
-import org.apache.flink.api.table.TableEnvironment;
-import org.apache.flink.api.table.ValidationException;
-import org.apache.flink.examples.java.WordCountTable.WC;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class AggregationsITCase extends TableProgramsTestBase {
-
-	public AggregationsITCase(TestExecutionMode mode, TableConfigMode configMode){
-		super(mode, configMode);
-	}
-
-	@Test
-	public void testAggregationTypes() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		Table table = tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env));
-
-		Table result = table.select("f0.sum, f0.min, f0.max, f0.count, f0.avg");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "231,1,21,21,11";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testAggregationOnNonExistingField() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		Table table =
-				tableEnv.fromDataSet(CollectionDataSets.get3TupleDataSet(env));
-
-		Table result =
-				table.select("foo.avg");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testWorkingAggregationDataTypes() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSource<Tuple7<Byte, Short, Integer, Long, Float, Double, String>> input =
-				env.fromElements(
-						new Tuple7<>((byte) 1, (short) 1, 1, 1L, 1.0f, 1.0d, "Hello"),
-						new Tuple7<>((byte) 2, (short) 2, 2, 2L, 2.0f, 2.0d, "Ciao"));
-
-		Table table = tableEnv.fromDataSet(input);
-
-		Table result =
-				table.select("f0.avg, f1.avg, f2.avg, f3.avg, f4.avg, f5.avg, f6.count");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "1,1,1,1,1.5,1.5,2";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAggregationWithArithmetic() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSource<Tuple2<Float, String>> input =
-				env.fromElements(
-						new Tuple2<>(1f, "Hello"),
-						new Tuple2<>(2f, "Ciao"));
-
-		Table table =
-				tableEnv.fromDataSet(input);
-
-		Table result =
-				table.select("(f0 + 2).avg + 2, f1.count + 5");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "5.5,7";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAggregationWithTwoCount() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSource<Tuple2<Float, String>> input =
-			env.fromElements(
-				new Tuple2<>(1f, "Hello"),
-				new Tuple2<>(2f, "Ciao"));
-
-		Table table =
-			tableEnv.fromDataSet(input);
-
-		Table result =
-			table.select("f0.count, f1.count");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "2,2";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testNonWorkingDataTypes() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSource<Tuple2<Float, String>> input = env.fromElements(new Tuple2<>(1f, "Hello"));
-
-		Table table =
-				tableEnv.fromDataSet(input);
-
-		Table result =
-				// Must fail. Cannot compute SUM aggregate on String field.
-				table.select("f1.sum");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testNoNestedAggregation() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSource<Tuple2<Float, String>> input = env.fromElements(new Tuple2<>(1f, "Hello"));
-
-		Table table =
-				tableEnv.fromDataSet(input);
-
-		Table result =
-				// Must fail. Aggregation on aggregation not allowed.
-				table.select("f0.sum.sum");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testGroupingOnNonExistentField() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-
-		tableEnv
-			.fromDataSet(input, "a, b, c")
-			// must fail. Field foo is not in input
-			.groupBy("foo")
-			.select("a.avg");
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testGroupingInvalidSelection() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-
-		tableEnv
-			.fromDataSet(input, "a, b, c")
-			.groupBy("a, b")
-			// must fail. Field c is not a grouping key or aggregation
-			.select("c");
-	}
-
-	@Test
-	public void testGroupedAggregate() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		Table result = table
-				.groupBy("b").select("b, a.sum");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testGroupingKeyForwardIfNotUsed() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		Table result = table
-				.groupBy("b").select("a.sum");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "1\n" + "5\n" + "15\n" + "34\n" + "65\n" + "111\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testGroupNoAggregation() throws Exception {
-
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		Table result = table
-			.groupBy("b").select("a.sum as d, b").groupBy("b, d").select("b");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		String expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n";
-		List<Row> results = ds.collect();
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testPojoAggregation() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-		DataSet<WC> input = env.fromElements(
-				new WC("Hello", 1),
-				new WC("Ciao", 1),
-				new WC("Hello", 1),
-				new WC("Hola", 1),
-				new WC("Hola", 1));
-
-		Table table = tableEnv.fromDataSet(input);
-
-		Table filtered = table
-				.groupBy("word")
-				.select("word.count as frequency, word")
-				.filter("frequency = 2");
-
-		List<String> result = tableEnv.toDataSet(filtered, WC.class)
-				.map(new MapFunction<WC, String>() {
-					public String map(WC value) throws Exception {
-						return value.word;
-					}
-				}).collect();
-		String expected = "Hello\n" + "Hola";
-		compareResultAsText(result, expected);
-	}
-
-	@Test
-	public void testPojoGrouping() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
-		DataSet<Tuple3<String, Double, String>> data = env.fromElements(
-			new Tuple3<>("A", 23.0, "Z"),
-			new Tuple3<>("A", 24.0, "Y"),
-			new Tuple3<>("B", 1.0, "Z"));
-
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		Table table = tableEnv
-			.fromDataSet(data, "groupMe, value, name")
-			.select("groupMe, value, name")
-			.where("groupMe != 'B'");
-
-		DataSet<MyPojo> myPojos = tableEnv.toDataSet(table, MyPojo.class);
-
-		DataSet<MyPojo> result = myPojos.groupBy("groupMe")
-			.sortGroup("value", Order.DESCENDING)
-			.first(1);
-
-		List<MyPojo> resultList = result.collect();
-		compareResultAsText(resultList, "A,24.0,Y");
-	}
-
-	@Test
-	public void testDistinct() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		Table distinct = table.select("b").distinct();
-
-		DataSet<Row> ds = tableEnv.toDataSet(distinct, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "1\n" + "2\n" + "3\n"+ "4\n"+ "5\n"+ "6\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testDistinctAfterAggregate() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> input = CollectionDataSets.get5TupleDataSet(env);
-
-		Table table = tableEnv.fromDataSet(input, "a, b, c, d, e");
-
-		Table distinct = table.groupBy("a, e").select("e").distinct();
-
-		DataSet<Row> ds = tableEnv.toDataSet(distinct, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "1\n" + "2\n" + "3\n";
-		compareResultAsText(results, expected);
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-	public static class MyPojo implements Serializable {
-		private static final long serialVersionUID = 8741918940120107213L;
-
-		public String groupMe;
-		public double value;
-		public String name;
-
-		public MyPojo() {
-			// for serialization
-		}
-
-		public MyPojo(String groupMe, double value, String name) {
-			this.groupMe = groupMe;
-			this.value = value;
-			this.name = name;
-		}
-
-		@Override
-		public String toString() {
-			return groupMe + "," + value + "," + name;
-		}
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java
deleted file mode 100644
index 1d5c189..0000000
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CalcITCase.java
+++ /dev/null
@@ -1,324 +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.api.java.batch.table;
-
-import java.util.Arrays;
-import java.util.Collection;
-import org.apache.flink.api.java.operators.DataSource;
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase;
-import org.apache.flink.api.table.Table;
-import org.apache.flink.types.Row;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.table.BatchTableEnvironment;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.table.TableEnvironment;
-import org.apache.flink.api.table.ValidationException;
-import org.apache.flink.api.table.functions.ScalarFunction;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import java.util.List;
-
-@RunWith(Parameterized.class)
-public class CalcITCase extends TableProgramsTestBase {
-
-	public CalcITCase(TestExecutionMode mode, TableConfigMode configMode){
-		super(mode, configMode);
-	}
-
-	@Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
-	public static Collection<Object[]> parameters() {
-		return Arrays.asList(new Object[][] {
-			{ TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT() },
-			{ TestExecutionMode.COLLECTION, TableProgramsTestBase.NO_NULL() }
-		});
-	}
-
-	@Test
-	public void testSimpleSelectAllWithAs() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		Table in = tableEnv.fromDataSet(ds, "a,b,c");
-
-		Table result = in
-				.select("a, b, c");
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-			"4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-			"7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-			"11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" +
-			"14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" +
-			"17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" +
-			"20,6,Comment#14\n" + "21,6,Comment#15\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testSimpleSelectWithNaming() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		Table in = tableEnv.fromDataSet(ds);
-
-		Table result = in
-				.select("f0 as a, f1 as b")
-				.select("a, b");
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
-				"8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
-				"16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testSimpleSelectRenameAll() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		Table in = tableEnv.fromDataSet(ds);
-
-		Table result = in
-			.select("f0 as a, f1 as b, f2 as c")
-			.select("a, b");
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
-			"8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
-			"16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testSelectInvalidField() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-
-		tableEnv.fromDataSet(ds, "a, b, c")
-			// Must fail. Field foo does not exist
-			.select("a + 1, foo + 2");
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testSelectAmbiguousFieldNames() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-
-		tableEnv.fromDataSet(ds, "a, b, c")
-			// Must fail. Field foo does not exist
-			.select("a + 1 as foo, b + 2 as foo");
-	}
-
-	@Test
-	public void testSelectStar() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> ds = CollectionDataSets.get3TupleDataSet(env);
-		Table in = tableEnv.fromDataSet(ds, "a,b,c");
-
-		Table result = in
-			.select("*");
-
-		DataSet<Row> resultSet = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = resultSet.collect();
-		String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-		                  "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-		                  "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-		                  "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" +
-		                  "14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" +
-		                  "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" +
-		                  "20,6,Comment#14\n" + "21,6,Comment#15\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAllRejectingFilter() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		Table result = table
-				.filter("false");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testAllPassingFilter() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		Table result = table
-				.filter("true");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
-			"4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
-			"7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
-			"11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" +
-			"14,5,Comment#8\n" + "15,5,Comment#9\n" + "16,6,Comment#10\n" +
-			"17,6,Comment#11\n" + "18,6,Comment#12\n" + "19,6,Comment#13\n" +
-			"20,6,Comment#14\n" + "21,6,Comment#15\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testFilterOnIntegerTupleField() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		Table result = table
-				.filter(" a % 2 = 0 ");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + "6,3,Luke Skywalker\n" + "8,4," +
-				"Comment#2\n" + "10,4,Comment#4\n" + "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," +
-				"Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testNotEquals() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		Table result = table
-				.filter("!( a % 2 <> 0 ) ");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" + "6,3,Luke Skywalker\n" + "8,4," +
-				"Comment#2\n" + "10,4,Comment#4\n" + "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," +
-				"Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testDisjunctivePreds() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		Table result = table
-			.filter("a < 2 || a > 20");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "1,1,Hi\n" + "21,6,Comment#15\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testIntegerBiggerThan128() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = env.fromElements(new Tuple3<>(300, 1L, "Hello"));
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		Table result = table
-			.filter("a = 300 ");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "300,1,Hello\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testFilterInvalidField() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSet<Tuple3<Integer, Long, String>> input = CollectionDataSets.get3TupleDataSet(env);
-		Table table = tableEnv.fromDataSet(input, "a, b, c");
-
-		table
-			// Must fail. Field foo does not exist.
-			.filter("foo = 17");
-	}
-
-	public static class OldHashCode extends ScalarFunction {
-		public int eval(String s) {
-			return -1;
-		}
-	}
-
-	public static class HashCode extends ScalarFunction {
-		public int eval(String s) {
-			return s.hashCode();
-		}
-	}
-
-	@Test
-	public void testUserDefinedScalarFunction() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		tableEnv.registerFunction("hashCode", new OldHashCode());
-		tableEnv.registerFunction("hashCode", new HashCode());
-
-		DataSource<String> input = env.fromElements("a", "b", "c");
-
-		Table table = tableEnv.fromDataSet(input, "text");
-
-		Table result = table.select("text.hashCode()");
-
-		DataSet<Integer> ds = tableEnv.toDataSet(result, Integer.class);
-		List<Integer> results = ds.collect();
-		String expected = "97\n98\n99";
-		compareResultAsText(results, expected);
-	}
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java
deleted file mode 100644
index 1139837..0000000
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/CastingITCase.java
+++ /dev/null
@@ -1,140 +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.api.java.batch.table;
-
-import java.util.List;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.operators.DataSource;
-import org.apache.flink.api.java.table.BatchTableEnvironment;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.tuple.Tuple6;
-import org.apache.flink.api.java.tuple.Tuple8;
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase;
-import org.apache.flink.types.Row;
-import org.apache.flink.api.table.Table;
-import org.apache.flink.api.table.TableEnvironment;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-@RunWith(Parameterized.class)
-public class CastingITCase extends TableProgramsTestBase {
-
-	public CastingITCase(TestExecutionMode mode, TableConfigMode configMode){
-		super(mode, configMode);
-	}
-
-	@Test
-	public void testNumericAutocastInArithmetic() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSource<Tuple8<Byte, Short, Integer, Long, Float, Double, Long, Double>> input =
-				env.fromElements(new Tuple8<>((byte) 1, (short) 1, 1, 1L, 1.0f, 1.0d, 1L, 1001.1));
-
-		Table table =
-				tableEnv.fromDataSet(input);
-
-		Table result = table.select("f0 + 1, f1 +" +
-				" 1, f2 + 1L, f3 + 1.0f, f4 + 1.0d, f5 + 1, f6 + 1.0d, f7 + f0");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "2,2,2,2.0,2.0,2.0,2.0,1002.1";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testNumericAutocastInComparison() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSource<Tuple6<Byte, Short, Integer, Long, Float, Double>> input =
-				env.fromElements(
-						new Tuple6<>((byte) 1, (short) 1, 1, 1L, 1.0f, 1.0d),
-						new Tuple6<>((byte) 2, (short) 2, 2, 2L, 2.0f, 2.0d));
-
-		Table table =
-				tableEnv.fromDataSet(input, "a,b,c,d,e,f");
-
-		Table result = table
-				.filter("a > 1 && b > 1 && c > 1L && d > 1.0f && e > 1.0d && f > 1");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "2,2,2,2,2.0,2.0";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testCasting() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSource<Tuple4<Integer, Double, Long, Boolean>> input =
-				env.fromElements(new Tuple4<>(1, 0.0, 1L, true));
-
-		Table table =
-				tableEnv.fromDataSet(input);
-
-		Table result = table.select(
-				// * -> String
-				"f0.cast(STRING), f1.cast(STRING), f2.cast(STRING), f3.cast(STRING)," +
-				// NUMERIC TYPE -> Boolean
-				"f0.cast(BOOL), f1.cast(BOOL), f2.cast(BOOL)," +
-				// NUMERIC TYPE -> NUMERIC TYPE
-				"f0.cast(DOUBLE), f1.cast(INT), f2.cast(SHORT)," +
-				// Boolean -> NUMERIC TYPE
-				"f3.cast(DOUBLE)," +
-				// identity casting
-				"f0.cast(INT), f1.cast(DOUBLE), f2.cast(LONG), f3.cast(BOOL)");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "1,0.0,1,true," +
-			"true,false,true," +
-			"1.0,0,1," +
-			"1.0," +
-			"1,0.0,1,true\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testCastFromString() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env, config());
-
-		DataSource<Tuple3<String, String, String>> input =
-				env.fromElements(new Tuple3<>("1", "true", "2.0"));
-
-		Table table =
-				tableEnv.fromDataSet(input);
-
-		Table result = table.select(
-				"f0.cast(BYTE), f0.cast(SHORT), f0.cast(INT), f0.cast(LONG), f2.cast(DOUBLE), f2.cast(FLOAT), f1.cast(BOOL)");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "1,1,1,1,2.0,2.0,true\n";
-		compareResultAsText(results, expected);
-	}
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java b/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java
deleted file mode 100644
index 014c127..0000000
--- a/flink-libraries/flink-table/src/test/java/org/apache/flink/api/java/batch/table/JoinITCase.java
+++ /dev/null
@@ -1,207 +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.api.java.batch.table;
-
-import java.util.List;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.table.BatchTableEnvironment;
-import org.apache.flink.api.java.tuple.Tuple3;
-import org.apache.flink.api.java.tuple.Tuple5;
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase;
-import org.apache.flink.types.Row;
-import org.apache.flink.api.table.Table;
-import org.apache.flink.api.table.TableEnvironment;
-import org.apache.flink.api.table.ValidationException;
-import org.apache.flink.test.javaApiOperators.util.CollectionDataSets;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-
-@RunWith(Parameterized.class)
-public class JoinITCase extends TableProgramsTestBase {
-
-	public JoinITCase(TestExecutionMode mode, TableConfigMode configMode){
-		super(mode, configMode);
-	}
-
-	@Test
-	public void testJoin() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
-		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
-
-		Table result = in1.join(in2).where("b === e").select("c, g");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testJoinWithFilter() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
-		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
-
-		Table result = in1.join(in2).where("b === e && b < 2").select("c, g");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "Hi,Hallo\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testJoinWithJoinFilter() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
-		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
-
-		Table result = in1.join(in2).where("b === e && a < 6 && h < b").select("c, g");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "Hello world, how are you?,Hallo Welt wie\n" +
-				"I am fine.,Hallo Welt wie\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test
-	public void testJoinWithMultipleKeys() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.get3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
-		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
-
-		Table result = in1.join(in2).where("a === d && b === h").select("c, g");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" +
-				"Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testJoinNonExistingKey() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
-		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
-
-		// Must fail. Field foo does not exist.
-		in1.join(in2).where("foo === e").select("c, g");
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testJoinWithNonMatchingKeyTypes() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
-		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
-
-		Table result = in1.join(in2)
-			// Must fail. Types of join fields are not compatible (Integer and String)
-			.where("a === g").select("c, g");
-
-		tableEnv.toDataSet(result, Row.class).collect();
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testJoinWithAmbiguousFields() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
-		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, c");
-
-		// Must fail. Join input have overlapping field names.
-		in1.join(in2).where("a === d").select("c, g");
-	}
-
-	@Test
-	public void testJoinWithAggregation() throws Exception {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tableEnv = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		Table in1 = tableEnv.fromDataSet(ds1, "a, b, c");
-		Table in2 = tableEnv.fromDataSet(ds2, "d, e, f, g, h");
-
-		Table result = in1
-				.join(in2).where("a === d").select("g.count");
-
-		DataSet<Row> ds = tableEnv.toDataSet(result, Row.class);
-		List<Row> results = ds.collect();
-		String expected = "6";
-		compareResultAsText(results, expected);
-	}
-
-	@Test(expected = ValidationException.class)
-	public void testJoinTablesFromDifferentEnvs() {
-		ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-		BatchTableEnvironment tEnv1 = TableEnvironment.getTableEnvironment(env);
-		BatchTableEnvironment tEnv2 = TableEnvironment.getTableEnvironment(env);
-
-		DataSet<Tuple3<Integer, Long, String>> ds1 = CollectionDataSets.getSmall3TupleDataSet(env);
-		DataSet<Tuple5<Integer, Long, Integer, String, Long>> ds2 = CollectionDataSets.get5TupleDataSet(env);
-
-		Table in1 = tEnv1.fromDataSet(ds1, "a, b, c");
-		Table in2 = tEnv2.fromDataSet(ds2, "d, e, f, g, h");
-
-		// Must fail. Tables are bound to different TableEnvironments.
-		in1.join(in2).where("a === d").select("g.count");
-	}
-
-}


[05/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/AggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/AggregationsITCase.scala
new file mode 100644
index 0000000..a98c258
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/AggregationsITCase.scala
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.table
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{TableEnvironment, ValidationException}
+import org.apache.flink.table.examples.scala.WordCountTable.{WC => MyWC}
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class AggregationsITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testAggregationTypes(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+      .select('_1.sum, '_1.min, '_1.max, '_1.count, '_1.avg)
+
+    val results = t.toDataSet[Row].collect()
+    val expected = "231,1,21,21,11"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testAggregationOnNonExistingField(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+      // Must fail. Field 'foo does not exist.
+      .select('foo.avg)
+  }
+
+  @Test
+  def testWorkingAggregationDataTypes(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = env.fromElements(
+      (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"),
+      (2: Byte, 2: Short, 2, 2L, 2.0f, 2.0d, "Ciao")).toTable(tEnv)
+      .select('_1.avg, '_2.avg, '_3.avg, '_4.avg, '_5.avg, '_6.avg, '_7.count)
+
+    val expected = "1,1,1,1,1.5,1.5,2"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testProjection(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = env.fromElements(
+      (1: Byte, 1: Short),
+      (2: Byte, 2: Short)).toTable(tEnv)
+      .select('_1.avg, '_1.sum, '_1.count, '_2.avg, '_2.sum)
+
+    val expected = "1,3,2,1,3"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testAggregationWithArithmetic(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = env.fromElements((1f, "Hello"), (2f, "Ciao")).toTable(tEnv)
+      .select(('_1 + 2).avg + 2, '_2.count + 5)
+
+    val expected = "5.5,7"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testAggregationWithTwoCount(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = env.fromElements((1f, "Hello"), (2f, "Ciao")).toTable(tEnv)
+      .select('_1.count, '_2.count)
+
+    val expected = "2,2"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testAggregationAfterProjection(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = env.fromElements(
+      (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"),
+      (2: Byte, 2: Short, 2, 2L, 2.0f, 2.0d, "Ciao")).toTable(tEnv)
+      .select('_1, '_2, '_3)
+      .select('_1.avg, '_2.sum, '_3.count)
+
+    val expected = "1,3,2"
+    val result = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(result.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testNonWorkingAggregationDataTypes(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = env.fromElements(("Hello", 1)).toTable(tEnv)
+      // Must fail. Field '_1 is not a numeric type.
+      .select('_1.sum)
+
+    t.collect()
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testNoNestedAggregations(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = env.fromElements(("Hello", 1)).toTable(tEnv)
+      // Must fail. Sum aggregation can not be chained.
+      .select('_2.sum.sum)
+  }
+
+  @Test
+  def testSQLStyleAggregations(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      .select(
+        """Sum( a) as a1, a.sum as a2,
+          |Min (a) as b1, a.min as b2,
+          |Max (a ) as c1, a.max as c2,
+          |Avg ( a ) as d1, a.avg as d2,
+          |Count(a) as e1, a.count as e2
+        """.stripMargin)
+
+    val expected = "231,231,1,1,21,21,11,11,21,21"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testPojoAggregation(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val input = env.fromElements(
+      MyWC("hello", 1),
+      MyWC("hello", 1),
+      MyWC("ciao", 1),
+      MyWC("hola", 1),
+      MyWC("hola", 1))
+    val expr = input.toTable(tEnv)
+    val result = expr
+      .groupBy('word)
+      .select('word, 'frequency.sum as 'frequency)
+      .filter('frequency === 2)
+      .toDataSet[MyWC]
+
+    val mappedResult = result.map(w => (w.word, w.frequency * 10)).collect()
+    val expected = "(hello,20)\n" + "(hola,20)"
+    TestBaseUtils.compareResultAsText(mappedResult.asJava, expected)
+  }
+
+  @Test
+  def testDistinct(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val distinct = ds.select('b).distinct()
+
+    val expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"
+    val results = distinct.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testDistinctAfterAggregate(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
+    val distinct = ds.groupBy('a, 'e).select('e).distinct()
+
+    val expected = "1\n" + "2\n" + "3\n"
+    val results = distinct.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testGroupingOnNonExistentField(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      // must fail. '_foo not a valid field
+      .groupBy('_foo)
+      .select('a.avg)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testGroupingInvalidSelection(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      .groupBy('a, 'b)
+      // must fail. 'c is not a grouping key or aggregation
+      .select('c)
+  }
+
+  @Test
+  def testGroupedAggregate(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      .groupBy('b)
+      .select('b, 'a.sum)
+
+    val expected = "1,1\n" + "2,5\n" + "3,15\n" + "4,34\n" + "5,65\n" + "6,111\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testGroupingKeyForwardIfNotUsed(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      .groupBy('b)
+      .select('a.sum)
+
+    val expected = "1\n" + "5\n" + "15\n" + "34\n" + "65\n" + "111\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testGroupNoAggregation(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env)
+      .toTable(tEnv, 'a, 'b, 'c)
+      .groupBy('b)
+      .select('a.sum as 'd, 'b)
+      .groupBy('b, 'd)
+      .select('b)
+
+    val expected = "1\n" + "2\n" + "3\n" + "4\n" + "5\n" + "6\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testGroupedAggregateWithLongKeys(): Unit = {
+    // This uses very long keys to force serialized comparison.
+    // With short keys, the normalized key is sufficient.
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = env.fromElements(
+      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2),
+      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2),
+      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2),
+      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2),
+      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhaa", 1, 2),
+      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2),
+      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2),
+      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2),
+      ("hhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhhab", 1, 2))
+      .rebalance().setParallelism(2).toTable(tEnv, 'a, 'b, 'c)
+      .groupBy('a, 'b)
+      .select('c.sum)
+
+    val expected = "10\n" + "8\n"
+    val results = ds.collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testGroupedAggregateWithConstant1(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      .select('a, 4 as 'four, 'b)
+      .groupBy('four, 'a)
+      .select('four, 'b.sum)
+
+    val expected = "4,2\n" + "4,3\n" + "4,5\n" + "4,5\n" + "4,5\n" + "4,6\n" +
+      "4,6\n" + "4,6\n" + "4,3\n" + "4,4\n" + "4,6\n" + "4,1\n" + "4,4\n" +
+      "4,4\n" + "4,5\n" + "4,6\n" + "4,2\n" + "4,3\n" + "4,4\n" + "4,5\n" + "4,6\n"
+    val results = t.toDataSet[Row].collect()
+
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testGroupedAggregateWithConstant2(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+        .select('b, 4 as 'four, 'a)
+        .groupBy('b, 'four)
+        .select('four, 'a.sum)
+
+    val expected = "4,1\n" + "4,5\n" + "4,15\n" + "4,34\n" + "4,65\n" + "4,111\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testGroupedAggregateWithExpression(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
+        .groupBy('e, 'b % 3)
+        .select('c.min, 'e, 'a.avg, 'd.count)
+
+    val expected = "0,1,1,1\n" + "3,2,3,3\n" + "7,1,4,2\n" + "14,2,5,1\n" +
+        "5,3,4,2\n" + "2,1,3,2\n" + "1,2,3,3\n" + "12,3,5,1"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testGroupedAggregateWithFilter(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      .groupBy('b)
+      .select('b, 'a.sum)
+      .where('b === 2)
+
+    val expected = "2,5\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/CalcITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/CalcITCase.scala
new file mode 100644
index 0000000..bc4f4bd
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/CalcITCase.scala
@@ -0,0 +1,438 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.table
+
+import java.sql.{Date, Time, Timestamp}
+import java.util
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{TableEnvironment, TableException, ValidationException}
+import org.apache.flink.table.expressions.Literal
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit.Assert._
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class CalcITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testSimpleSelectAll(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).select('_1, '_2, '_3)
+
+    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
+      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
+      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testSimpleSelectAllWithAs(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('a, 'b, 'c)
+
+    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
+      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
+      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testSimpleSelectWithNaming(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+      .select('_1 as 'a, '_2 as 'b, '_1 as 'c)
+      .select('a, 'b)
+
+    val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
+      "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
+      "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testSimpleSelectRenameAll(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+      .select('_1 as 'a, '_2 as 'b, '_3 as 'c)
+      .select('a, 'b)
+
+    val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
+      "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
+      "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testSelectInvalidFieldFields(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      // must fail. Field 'foo does not exist
+      .select('a, 'foo)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testSelectAmbiguousRenaming(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      // must fail. 'a and 'b are both renamed to 'foo
+      .select('a + 1 as 'foo, 'b + 2 as 'foo).toDataSet[Row].print()
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testSelectAmbiguousRenaming2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      // must fail. 'a and 'b are both renamed to 'a
+      .select('a, 'b as 'a).toDataSet[Row].print()
+  }
+
+  @Test
+  def testSelectStar(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('*)
+
+    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
+      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
+      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testAliasStarException(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    try {
+      CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, '*, 'b, 'c)
+      fail("TableException expected")
+    } catch {
+      case _: TableException => //ignore
+    }
+
+    try {
+      CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+        .select('_1 as '*, '_2 as 'b, '_1 as 'c)
+      fail("ValidationException expected")
+    } catch {
+      case _: ValidationException => //ignore
+    }
+
+    try {
+      CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('*, 'b, 'c)
+      fail("ValidationException expected")
+    } catch {
+      case _: ValidationException => //ignore
+    }
+
+    try {
+      CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c).select('*, 'b)
+      fail("ValidationException expected")
+    } catch {
+      case _: ValidationException => //ignore
+    }
+  }
+
+  @Test
+  def testAllRejectingFilter(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter( Literal(false) )
+
+    val expected = "\n"
+    val results = filterDs.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testAllPassingFilter(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter( Literal(true) )
+    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, " +
+      "how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + "7,4," +
+      "Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + "11,5," +
+      "Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5," +
+      "Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19," +
+      "6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
+    val results = filterDs.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testFilterOnStringTupleField(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val filterDs = ds.filter( 'c.like("%world%") )
+
+    val expected = "3,2,Hello world\n" + "4,3,Hello world, how are you?\n"
+    val results = filterDs.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testFilterOnIntegerTupleField(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter( 'a % 2 === 0 )
+
+    val expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" +
+      "6,3,Luke Skywalker\n" + "8,4," + "Comment#2\n" + "10,4,Comment#4\n" +
+      "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," +
+      "Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n"
+    val results = filterDs.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testNotEquals(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter( 'a % 2 !== 0)
+    val expected = "1,1,Hi\n" + "3,2,Hello world\n" +
+      "5,3,I am fine.\n" + "7,4,Comment#1\n" + "9,4,Comment#3\n" +
+      "11,5,Comment#5\n" + "13,5,Comment#7\n" + "15,5,Comment#9\n" +
+      "17,6,Comment#11\n" + "19,6,Comment#13\n" + "21,6,Comment#15\n"
+    val results = filterDs.collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testDisjunctivePredicate(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter( 'a < 2 || 'a > 20)
+    val expected = "1,1,Hi\n" + "21,6,Comment#15\n"
+    val results = filterDs.collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testConsecutiveFilters(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter('a % 2 !== 0).filter('b % 2 === 0)
+    val expected = "3,2,Hello world\n" + "7,4,Comment#1\n" +
+      "9,4,Comment#3\n" + "17,6,Comment#11\n" +
+      "19,6,Comment#13\n" + "21,6,Comment#15\n"
+    val results = filterDs.collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testFilterBasicType(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.getStringDataSet(env)
+
+    val filterDs = ds.toTable(tEnv, 'a).filter( 'a.like("H%") )
+
+    val expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hello world, how are you?\n"
+    val results = filterDs.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testFilterOnCustomType(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.getCustomTypeDataSet(env)
+    val filterDs = ds.toTable(tEnv, 'myInt as 'i, 'myLong as 'l, 'myString as 's)
+      .filter( 's.like("%a%") )
+
+    val expected = "3,3,Hello world, how are you?\n" + "3,4,I am fine.\n" + "3,5,Luke Skywalker\n"
+    val results = filterDs.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testFilterInvalidFieldName(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+
+    // must fail. Field 'foo does not exist
+    ds.filter( 'foo === 2 )
+  }
+
+  @Test
+  def testSimpleCalc(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+        .select('_1, '_2, '_3)
+        .where('_1 < 7)
+        .select('_1, '_3)
+
+    val expected = "1,Hi\n" + "2,Hello\n" + "3,Hello world\n" +
+      "4,Hello world, how are you?\n" + "5,I am fine.\n" + "6,Luke Skywalker\n"
+      val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testCalcWithTwoFilters(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+        .select('_1, '_2, '_3)
+        .where('_1 < 7 && '_2 === 3)
+        .select('_1, '_3)
+        .where('_1 === 4)
+        .select('_1)
+
+    val expected = "4\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testCalcWithAggregation(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+        .select('_1, '_2, '_3)
+        .where('_1 < 15)
+        .groupBy('_2)
+        .select('_1.min, '_2.count as 'cnt)
+        .where('cnt > 3)
+
+    val expected = "7,4\n" + "11,4\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testCalcJoin(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.select('a, 'b).join(ds2).where('b === 'e).select('a, 'b, 'd, 'e, 'f)
+      .where('b > 1).select('a, 'd).where('d === 2)
+
+    val expected = "2,2\n" + "3,2\n"
+    val results = joinT.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testAdvancedDataTypes(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = env
+      .fromElements((
+        BigDecimal("78.454654654654654").bigDecimal,
+        BigDecimal("4E+9999").bigDecimal,
+        Date.valueOf("1984-07-12"),
+        Time.valueOf("14:34:24"),
+        Timestamp.valueOf("1984-07-12 14:34:24")))
+      .toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
+      .select('a, 'b, 'c, 'd, 'e, BigDecimal("11.2"), BigDecimal("11.2").bigDecimal,
+        Date.valueOf("1984-07-12"), Time.valueOf("14:34:24"),
+        Timestamp.valueOf("1984-07-12 14:34:24"))
+
+    val expected = "78.454654654654654,4E+9999,1984-07-12,14:34:24,1984-07-12 14:34:24.0," +
+      "11.2,11.2,1984-07-12,14:34:24,1984-07-12 14:34:24.0"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+}
+
+object CalcITCase {
+
+  @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
+  def parameters(): util.Collection[Array[java.lang.Object]] = {
+    Seq[Array[AnyRef]](
+      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT),
+      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.NO_NULL)).asJava
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
new file mode 100644
index 0000000..a80e0cb
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/FieldProjectionTest.scala
@@ -0,0 +1,318 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.batch.table
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.expressions.{RowtimeAttribute, Upper, WindowReference}
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.api.scala.batch.table.FieldProjectionTest._
+import org.apache.flink.table.plan.logical.EventTimeTumblingGroupWindow
+import org.apache.flink.table.utils._
+import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+/**
+  * Tests for all the situations when we can do fields projection. Like selecting few fields
+  * from a large field count source.
+  */
+class FieldProjectionTest extends TableTestBase {
+
+  val util: BatchTableTestUtil = batchTestUtil()
+
+  val streamUtil: StreamTableTestUtil = streamTestUtil()
+
+  @Test
+  def testSimpleSelect(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val resultTable = sourceTable.select('a, 'b)
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select", "a", "b")
+    )
+
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testSelectAllFields(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val resultTable1 = sourceTable.select('*)
+    val resultTable2 = sourceTable.select('a, 'b, 'c, 'd)
+
+    val expected = batchTableNode(0)
+
+    util.verifyTable(resultTable1, expected)
+    util.verifyTable(resultTable2, expected)
+  }
+
+  @Test
+  def testSelectAggregation(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val resultTable = sourceTable.select('a.sum, 'b.max)
+
+    val expected = unaryNode(
+      "DataSetAggregate",
+      binaryNode(
+        "DataSetUnion",
+        values(
+          "DataSetValues",
+          tuples(List(null, null)),
+          term("values", "a", "b")
+        ),
+        unaryNode(
+          "DataSetCalc",
+          batchTableNode(0),
+          term("select", "a", "b")
+        ),
+        term("union", "a", "b")
+      ),
+      term("select", "SUM(a) AS TMP_0", "MAX(b) AS TMP_1")
+    )
+
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testSelectFunction(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+
+    util.tEnv.registerFunction("hashCode", MyHashCode)
+
+    val resultTable = sourceTable.select("hashCode(c), b")
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select", s"${MyHashCode.getClass.getCanonicalName}(c) AS _c0", "b")
+    )
+
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testSelectFromGroupedTable(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val resultTable = sourceTable.groupBy('a, 'c).select('a)
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetAggregate",
+        unaryNode(
+          "DataSetCalc",
+          batchTableNode(0),
+          term("select", "a", "c")
+        ),
+        term("groupBy", "a", "c"),
+        term("select", "a", "c")
+      ),
+      term("select", "a")
+    )
+
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testSelectAllFieldsFromGroupedTable(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val resultTable = sourceTable.groupBy('a, 'c).select('a, 'c)
+
+    val expected = unaryNode(
+      "DataSetAggregate",
+      unaryNode(
+        "DataSetCalc",
+        batchTableNode(0),
+        term("select", "a", "c")
+      ),
+      term("groupBy", "a", "c"),
+      term("select", "a", "c")
+    )
+
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testSelectAggregationFromGroupedTable(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val resultTable = sourceTable.groupBy('c).select('a.sum)
+
+    val expected =
+      unaryNode(
+        "DataSetCalc",
+        unaryNode(
+          "DataSetAggregate",
+          unaryNode(
+            "DataSetCalc",
+            batchTableNode(0),
+            term("select", "a", "c")
+          ),
+          term("groupBy", "c"),
+          term("select", "c", "SUM(a) AS TMP_0")
+        ),
+        term("select", "TMP_0 AS TMP_1")
+      )
+
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testSelectFromGroupedTableWithNonTrivialKey(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val resultTable = sourceTable.groupBy(Upper('c) as 'k).select('a.sum)
+
+    val expected =
+      unaryNode(
+        "DataSetCalc",
+        unaryNode(
+          "DataSetAggregate",
+          unaryNode(
+            "DataSetCalc",
+            batchTableNode(0),
+            term("select", "a", "c", "UPPER(c) AS k")
+          ),
+          term("groupBy", "k"),
+          term("select", "k", "SUM(a) AS TMP_0")
+        ),
+        term("select", "TMP_0 AS TMP_1")
+      )
+
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testSelectFromGroupedTableWithFunctionKey(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val resultTable = sourceTable.groupBy(MyHashCode('c) as 'k).select('a.sum)
+
+    val expected =
+      unaryNode(
+        "DataSetCalc",
+        unaryNode(
+          "DataSetAggregate",
+          unaryNode(
+            "DataSetCalc",
+            batchTableNode(0),
+            term("select", "a", "c", s"${MyHashCode.getClass.getCanonicalName}(c) AS k")
+          ),
+          term("groupBy", "k"),
+          term("select", "k", "SUM(a) AS TMP_0")
+        ),
+        term("select", "TMP_0 AS TMP_1")
+      )
+
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testSelectFromStreamingWindow(): Unit = {
+    val sourceTable = streamUtil.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val resultTable = sourceTable
+        .window(Tumble over 5.millis on 'rowtime as 'w)
+        .select(Upper('c).count, 'a.sum)
+
+    val expected =
+      unaryNode(
+        "DataStreamAggregate",
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(0),
+          term("select", "c", "a", "UPPER(c) AS $f2")
+        ),
+        term("window",
+          EventTimeTumblingGroupWindow(
+            Some(WindowReference("w")),
+            RowtimeAttribute(),
+            5.millis)),
+        term("select", "COUNT($f2) AS TMP_0", "SUM(a) AS TMP_1")
+      )
+
+    streamUtil.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testSelectFromStreamingGroupedWindow(): Unit = {
+    val sourceTable = streamUtil.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+    val resultTable = sourceTable
+        .groupBy('b)
+        .window(Tumble over 5.millis on 'rowtime as 'w)
+        .select(Upper('c).count, 'a.sum, 'b)
+
+    val expected = unaryNode(
+        "DataStreamCalc",
+        unaryNode(
+          "DataStreamAggregate",
+          unaryNode(
+            "DataStreamCalc",
+            streamTableNode(0),
+            term("select", "c", "a", "b", "UPPER(c) AS $f3")
+          ),
+          term("groupBy", "b"),
+          term("window",
+            EventTimeTumblingGroupWindow(
+              Some(WindowReference("w")),
+              RowtimeAttribute(),
+              5.millis)),
+          term("select", "b", "COUNT($f3) AS TMP_0", "SUM(a) AS TMP_1")
+        ),
+        term("select", "TMP_0 AS TMP_2", "TMP_1 AS TMP_3", "b")
+    )
+
+    streamUtil.verifyTable(resultTable, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testSelectFromBatchWindow1(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+
+    // time field is selected
+    val resultTable = sourceTable
+        .window(Tumble over 5.millis on 'a as 'w)
+        .select('a.sum, 'c.count)
+
+    val expected = "TODO"
+
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testSelectFromBatchWindow2(): Unit = {
+    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
+
+    // time field is not selected
+    val resultTable = sourceTable
+        .window(Tumble over 5.millis on 'a as 'w)
+        .select('c.count)
+
+    val expected = "TODO"
+
+    util.verifyTable(resultTable, expected)
+  }
+}
+
+object FieldProjectionTest {
+
+  object MyHashCode extends ScalarFunction {
+    def eval(s: String): Int = s.hashCode()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/JoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/JoinITCase.scala
new file mode 100644
index 0000000..ce16ada
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/JoinITCase.scala
@@ -0,0 +1,383 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.table
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{TableEnvironment, TableException, ValidationException}
+import org.apache.flink.table.expressions.Literal
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class JoinITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testJoin(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.join(ds2).where('b === 'e).select('c, 'g)
+
+    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n"
+    val results = joinT.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testJoinWithFilter(): Unit = {
+
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.join(ds2).where('b === 'e && 'b < 2).select('c, 'g)
+
+    val expected = "Hi,Hallo\n"
+    val results = joinT.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testJoinWithJoinFilter(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.join(ds2).where('b === 'e && 'a < 6 && 'h < 'b).select('c, 'g)
+
+    val expected = "Hello world, how are you?,Hallo Welt wie\n" +
+      "I am fine.,Hallo Welt wie\n"
+    val results = joinT.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testJoinWithMultipleKeys(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.join(ds2).filter('a === 'd && 'b === 'h).select('c, 'g)
+
+    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" +
+      "Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n"
+    val results = joinT.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testJoinNonExistingKey(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    ds1.join(ds2)
+      // must fail. Field 'foo does not exist
+      .where('foo === 'e)
+      .select('c, 'g)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testJoinWithNonMatchingKeyTypes(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    ds1.join(ds2)
+      // must fail. Field 'a is Int, and 'g is String
+      .where('a === 'g)
+      .select('c, 'g).collect()
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testJoinWithAmbiguousFields(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'c)
+
+    ds1.join(ds2)
+      // must fail. Both inputs share the same field 'c
+      .where('a === 'd)
+      .select('c, 'g)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testNoEqualityJoinPredicate1(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    ds1.join(ds2)
+      // must fail. No equality join predicate
+      .where('d === 'f)
+      .select('c, 'g).collect()
+  }
+
+  @Test(expected = classOf[TableException])
+  def testNoEqualityJoinPredicate2(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    ds1.join(ds2)
+      // must fail. No equality join predicate
+      .where('a < 'd)
+      .select('c, 'g).collect()
+  }
+
+  @Test
+  def testJoinWithAggregation(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.join(ds2).where('a === 'd).select('g.count)
+
+    val expected = "6"
+    val results = joinT.toDataSet[Row] collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testJoinWithGroupedAggregation(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.join(ds2)
+      .where('a === 'd)
+      .groupBy('a, 'd)
+      .select('b.sum, 'g.count)
+
+    val expected = "6,3\n" + "4,2\n" + "1,1"
+    val results = joinT.toDataSet[Row] collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testJoinPushThroughJoin(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+    val ds3 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'j, 'k, 'l)
+
+    val joinT = ds1.join(ds2)
+      .where(Literal(true))
+      .join(ds3)
+      .where('a === 'd && 'e === 'k)
+      .select('a, 'f, 'l)
+
+    val expected = "2,1,Hello\n" + "2,1,Hello world\n" + "1,0,Hi"
+    val results = joinT.toDataSet[Row] collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testJoinWithDisjunctivePred(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.join(ds2).filter('a === 'd && ('b === 'e || 'b === 'e - 10)).select('c, 'g)
+
+    val expected = "Hi,Hallo\n" +
+      "Hello,Hallo Welt\n" +
+      "I am fine.,IJK"
+    val results = joinT.toDataSet[Row] collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testJoinWithExpressionPreds(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.join(ds2).filter('b === 'h + 1 && 'a - 1 === 'd + 2).select('c, 'g)
+
+    val expected = "I am fine.,Hallo Welt\n" +
+      "Luke Skywalker,Hallo Welt wie gehts?\n" +
+      "Luke Skywalker,ABC\n" +
+      "Comment#2,HIJ\n" +
+      "Comment#2,IJK"
+    val results = joinT.toDataSet[Row] collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testJoinTablesFromDifferentEnvs(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv1 = TableEnvironment.getTableEnvironment(env, config)
+    val tEnv2 = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv1, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv2, 'd, 'e, 'f, 'g, 'h)
+
+    // Must fail. Tables are bound to different TableEnvironments.
+    ds1.join(ds2).where('b === 'e).select('c, 'g)
+  }
+
+  @Test
+  def testLeftJoinWithMultipleKeys(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    tEnv.getConfig.setNullCheck(true)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.leftOuterJoin(ds2, 'a === 'd && 'b === 'h).select('c, 'g)
+
+    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" +
+      "Hello world,ABC\n" + "Hello world, how are you?,null\n" + "I am fine.,HIJ\n" +
+      "I am fine.,IJK\n" + "Luke Skywalker,null\n" + "Comment#1,null\n" + "Comment#2,null\n" +
+      "Comment#3,null\n" + "Comment#4,null\n" + "Comment#5,null\n" + "Comment#6,null\n" +
+      "Comment#7,null\n" + "Comment#8,null\n" + "Comment#9,null\n" + "Comment#10,null\n" +
+      "Comment#11,null\n" + "Comment#12,null\n" + "Comment#13,null\n" + "Comment#14,null\n" +
+      "Comment#15,null\n"
+    val results = joinT.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testNoJoinCondition(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    tEnv.getConfig.setNullCheck(true)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds2.leftOuterJoin(ds1, 'b === 'd && 'b < 3).select('c, 'g)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testNoEquiJoin(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    tEnv.getConfig.setNullCheck(true)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds2.leftOuterJoin(ds1, 'b < 'd).select('c, 'g)
+  }
+
+  @Test
+  def testRightJoinWithMultipleKeys(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    tEnv.getConfig.setNullCheck(true)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.rightOuterJoin(ds2, "a = d && b = h").select('c, 'g)
+
+    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "null,Hallo Welt wie\n" +
+      "Hello world,Hallo Welt wie gehts?\n" + "Hello world,ABC\n" + "null,BCD\n" + "null,CDE\n" +
+      "null,DEF\n" + "null,EFG\n" + "null,FGH\n" + "null,GHI\n" + "I am fine.,HIJ\n" +
+      "I am fine.,IJK\n" + "null,JKL\n" + "null,KLM\n"
+    val results = joinT.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testRightJoinWithNotOnlyEquiJoin(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    tEnv.getConfig.setNullCheck(true)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.rightOuterJoin(ds2, "a = d && b < h").select('c, 'g)
+
+    val expected = "Hello world,BCD\n"
+    val results = joinT.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testFullOuterJoinWithMultipleKeys(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    tEnv.getConfig.setNullCheck(true)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+
+    val joinT = ds1.fullOuterJoin(ds2, 'a === 'd && 'b === 'h).select('c, 'g)
+
+    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "null,Hallo Welt wie\n" +
+      "Hello world,Hallo Welt wie gehts?\n" + "Hello world,ABC\n" + "null,BCD\n" + "null,CDE\n" +
+      "null,DEF\n" + "null,EFG\n" + "null,FGH\n" + "null,GHI\n" + "I am fine.,HIJ\n" +
+      "I am fine.,IJK\n" + "null,JKL\n" + "null,KLM\n" + "Luke Skywalker,null\n" +
+      "Comment#1,null\n" + "Comment#2,null\n" + "Comment#3,null\n" + "Comment#4,null\n" +
+      "Comment#5,null\n" + "Comment#6,null\n" + "Comment#7,null\n" + "Comment#8,null\n" +
+      "Comment#9,null\n" + "Comment#10,null\n" + "Comment#11,null\n" + "Comment#12,null\n" +
+      "Comment#13,null\n" + "Comment#14,null\n" + "Comment#15,null\n" +
+      "Hello world, how are you?,null\n"
+    val results = joinT.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/SetOperatorsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/SetOperatorsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/SetOperatorsITCase.scala
new file mode 100644
index 0000000..e369250
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/SetOperatorsITCase.scala
@@ -0,0 +1,320 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.table
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{TableEnvironment, ValidationException}
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.util.Random
+
+@RunWith(classOf[Parameterized])
+class SetOperatorsITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testUnionAll(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f)
+
+    val unionDs = ds1.unionAll(ds2).select('c)
+
+    val results = unionDs.toDataSet[Row].collect()
+    val expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hi\n" + "Hello\n" + "Hello world\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testUnion(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f)
+
+    val unionDs = ds1.union(ds2).select('c)
+
+    val results = unionDs.toDataSet[Row].collect()
+    val expected = "Hi\n" + "Hello\n" + "Hello world\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testTernaryUnionAll(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds3 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val unionDs = ds1.unionAll(ds2).unionAll(ds3).select('c)
+
+    val results = unionDs.toDataSet[Row].collect()
+    val expected = "Hi\n" + "Hello\n" + "Hello world\n" +
+      "Hi\n" + "Hello\n" + "Hello world\n" +
+      "Hi\n" + "Hello\n" + "Hello world\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testTernaryUnion(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds3 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val unionDs = ds1.union(ds2).union(ds3).select('c)
+
+    val results = unionDs.toDataSet[Row].collect()
+    val expected = "Hi\n" + "Hello\n" + "Hello world\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testUnionDifferentColumnSize(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'd, 'c, 'e)
+
+    // must fail. Union inputs have different column size.
+    ds1.unionAll(ds2)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testUnionDifferentFieldTypes(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
+      .select('a, 'b, 'c)
+
+    // must fail. Union inputs have different field types.
+    ds1.unionAll(ds2)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testUnionTablesFromDifferentEnvs(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv1 = TableEnvironment.getTableEnvironment(env, config)
+    val tEnv2 = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv1, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv2, 'a, 'b, 'c)
+
+    // Must fail. Tables are bound to different TableEnvironments.
+    ds1.unionAll(ds2).select('c)
+  }
+
+  @Test
+  def testMinusAll(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = env.fromElements((1, 1L, "Hi")).toTable(tEnv, 'a, 'b, 'c)
+
+    val minusDs = ds1.unionAll(ds1).unionAll(ds1)
+      .minusAll(ds2.unionAll(ds2)).select('c)
+
+    val results = minusDs.toDataSet[Row].collect()
+    val expected = "Hi\n" +
+      "Hello\n" + "Hello world\n" +
+      "Hello\n" + "Hello world\n" +
+      "Hello\n" + "Hello world\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testMinus(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = env.fromElements((1, 1L, "Hi")).toTable(tEnv, 'a, 'b, 'c)
+
+    val minusDs = ds1.unionAll(ds1).unionAll(ds1)
+      .minus(ds2.unionAll(ds2)).select('c)
+
+    val results = minusDs.toDataSet[Row].collect()
+    val expected = "Hello\n" + "Hello world\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testMinusDifferentFieldTypes(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
+      .select('a, 'b, 'c)
+
+    // must fail. Minus inputs have different field types.
+    ds1.minus(ds2)
+  }
+
+  @Test
+  def testMinusDifferentFieldNames(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = env.fromElements((1, 1L, "Hi")).toTable(tEnv, 'd, 'e, 'f)
+
+    val minusDs = ds1.unionAll(ds1).unionAll(ds1)
+      .minus(ds2.unionAll(ds2)).select('c)
+
+    val results = minusDs.toDataSet[Row].collect()
+    val expected = "Hello\n" + "Hello world\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testMinusAllTablesFromDifferentEnvs(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv1 = TableEnvironment.getTableEnvironment(env, config)
+    val tEnv2 = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv1, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv2, 'a, 'b, 'c)
+
+    // Must fail. Tables are bound to different TableEnvironments.
+    ds1.minusAll(ds2).select('c)
+  }
+
+  @Test
+  def testIntersect(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val data = new mutable.MutableList[(Int, Long, String)]
+    data.+=((1, 1L, "Hi"))
+    data.+=((2, 2L, "Hello"))
+    data.+=((2, 2L, "Hello"))
+    data.+=((3, 2L, "Hello world!"))
+    val ds2 = env.fromCollection(Random.shuffle(data)).toTable(tEnv, 'a, 'b, 'c)
+
+    val intersectDS = ds1.intersect(ds2).select('c).toDataSet[Row]
+
+    val results = intersectDS.collect()
+
+    val expected = "Hi\n" + "Hello\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testIntersectAll(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val data1 = new mutable.MutableList[Int]
+    data1 += (1, 1, 1, 2, 2)
+    val data2 = new mutable.MutableList[Int]
+    data2 += (1, 2, 2, 2, 3)
+    val ds1 = env.fromCollection(data1).toTable(tEnv, 'c)
+    val ds2 = env.fromCollection(data2).toTable(tEnv, 'c)
+
+    val intersectDS = ds1.intersectAll(ds2).select('c).toDataSet[Row]
+
+    val expected = "1\n2\n2"
+    val results = intersectDS.collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testIntersectWithDifferentFieldNames(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'e, 'f, 'g)
+
+    val intersectDs = ds1.intersect(ds2).select('c)
+
+    val results = intersectDs.toDataSet[Row].collect()
+    val expected = "Hi\n" + "Hello\n" + "Hello world\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testIntersectWithDifferentFieldTypes(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
+      .select('a, 'b, 'c)
+
+    // must fail. Intersect inputs have different field types.
+    ds1.intersect(ds2)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testIntersectTablesFromDifferentEnvs(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv1 = TableEnvironment.getTableEnvironment(env, config)
+    val tEnv2 = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv1, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv2, 'a, 'b, 'c)
+
+    // Must fail. Tables are bound to different TableEnvironments.
+    ds1.intersect(ds2).select('c)
+  }
+
+  @Test
+  def testIntersectWithScalarExpression(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      .select('a + 1, 'b, 'c)
+    val ds2 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+      .select('a + 1, 'b, 'c)
+
+    val intersectDs = ds1.intersect(ds2)
+
+    val results = intersectDs.toDataSet[Row].collect()
+    val expected = "2,1,Hi\n" + "3,2,Hello\n" + "4,2,Hello world\n"
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/SortITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/SortITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/SortITCase.scala
new file mode 100644
index 0000000..3cbc2c8
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/SortITCase.scala
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.table
+
+import org.apache.flink.table.api.scala.batch.utils.SortTestUtils._
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.api.scala.{ExecutionEnvironment, _}
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{TableEnvironment, ValidationException}
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class SortITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  def getExecutionEnvironment = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    env.setParallelism(4)
+    env
+  }
+
+  @Test
+  def testOrderByDesc(): Unit = {
+    val env = getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    val t = ds.toTable(tEnv).orderBy('_1.desc)
+    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
+      - x.productElement(0).asInstanceOf[Int] )
+
+    val expected = sortExpectedly(tupleDataSetStrings)
+    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
+
+    val result = results
+      .filterNot(_.isEmpty)
+      .sortBy(_.head)(Ordering.by(f=> f.toString))
+      .reduceLeft(_ ++ _)
+
+    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
+  }
+
+  @Test
+  def testOrderByAsc(): Unit = {
+    val env = getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    val t = ds.toTable(tEnv).orderBy('_1.asc)
+    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
+      x.productElement(0).asInstanceOf[Int] )
+
+    val expected = sortExpectedly(tupleDataSetStrings)
+    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
+
+    val result = results
+      .filterNot(_.isEmpty)
+      .sortBy(_.head)(Ordering.by(f=> f.toString))
+      .reduceLeft(_ ++ _)
+
+    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
+  }
+
+  @Test
+  def testOrderByMultipleFieldsDifferentDirections(): Unit = {
+    val env = getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    val t = ds.toTable(tEnv).orderBy('_2.asc, '_1.desc)
+    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
+      (x.productElement(1).asInstanceOf[Long], - x.productElement(0).asInstanceOf[Int]) )
+
+    val expected = sortExpectedly(tupleDataSetStrings)
+    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
+
+    val result = results
+      .filterNot(_.isEmpty)
+      .sortBy(_.head)(Ordering.by(f=> f.toString))
+      .reduceLeft(_ ++ _)
+
+    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
+  }
+
+  @Test
+  def testOrderByOffset(): Unit = {
+    val env = getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    val t = ds.toTable(tEnv).orderBy('_1.asc).limit(3)
+    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
+      x.productElement(0).asInstanceOf[Int] )
+
+    val expected = sortExpectedly(tupleDataSetStrings, 3, 21)
+    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
+
+    val result = results
+      .filterNot(_.isEmpty)
+      .sortBy(_.head)(Ordering.by(f=> f.toString))
+      .reduceLeft(_ ++ _)
+
+    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
+  }
+
+  @Test
+  def testOrderByOffsetAndFetch(): Unit = {
+    val env = getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    val t = ds.toTable(tEnv).orderBy('_1.desc).limit(3, 5)
+    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
+      - x.productElement(0).asInstanceOf[Int] )
+
+    val expected = sortExpectedly(tupleDataSetStrings, 3, 8)
+    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
+
+    val result = results
+      .filterNot(_.isEmpty)
+      .sortBy(_.head)(Ordering.by(f=> f.toString))
+      .reduceLeft(_ ++ _)
+
+    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
+  }
+
+  @Test
+  def testOrderByFetch(): Unit = {
+    val env = getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    val t = ds.toTable(tEnv).orderBy('_1.asc).limit(0, 5)
+    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
+      x.productElement(0).asInstanceOf[Int] )
+
+    val expected = sortExpectedly(tupleDataSetStrings, 0, 5)
+    val results = t.toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
+
+    val result = results
+      .filterNot(_.isEmpty)
+      .sortBy(_.head)(Ordering.by(f=> f.toString))
+      .reduceLeft(_ ++ _)
+
+    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testFetchWithoutOrder(): Unit = {
+    val env = getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    val t = ds.toTable(tEnv).limit(0, 5)
+
+    t.toDataSet[Row].collect()
+  }
+
+}


[22/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/mathExpressions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/mathExpressions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/mathExpressions.scala
new file mode 100644
index 0000000..b31367c
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/mathExpressions.scala
@@ -0,0 +1,129 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.tools.RelBuilder
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.typeutils.TypeCheckUtils
+import org.apache.flink.table.validate._
+
+case class Abs(child: Expression) extends UnaryExpression {
+  override private[flink] def resultType: TypeInformation[_] = child.resultType
+
+  override private[flink] def validateInput(): ValidationResult =
+    TypeCheckUtils.assertNumericExpr(child.resultType, "Abs")
+
+  override def toString: String = s"abs($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.ABS, child.toRexNode)
+  }
+}
+
+case class Ceil(child: Expression) extends UnaryExpression {
+  override private[flink] def resultType: TypeInformation[_] = LONG_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult =
+    TypeCheckUtils.assertNumericExpr(child.resultType, "Ceil")
+
+  override def toString: String = s"ceil($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.CEIL, child.toRexNode)
+  }
+}
+
+case class Exp(child: Expression) extends UnaryExpression with InputTypeSpec {
+  override private[flink] def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO
+
+  override private[flink] def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: Nil
+
+  override def toString: String = s"exp($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.EXP, child.toRexNode)
+  }
+}
+
+
+case class Floor(child: Expression) extends UnaryExpression {
+  override private[flink] def resultType: TypeInformation[_] = LONG_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult =
+    TypeCheckUtils.assertNumericExpr(child.resultType, "Floor")
+
+  override def toString: String = s"floor($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.FLOOR, child.toRexNode)
+  }
+}
+
+case class Log10(child: Expression) extends UnaryExpression with InputTypeSpec {
+  override private[flink] def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO
+
+  override private[flink] def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: Nil
+
+  override def toString: String = s"log10($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.LOG10, child.toRexNode)
+  }
+}
+
+case class Ln(child: Expression) extends UnaryExpression with InputTypeSpec {
+  override private[flink] def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO
+
+  override private[flink] def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: Nil
+
+  override def toString: String = s"ln($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.LN, child.toRexNode)
+  }
+}
+
+case class Power(left: Expression, right: Expression) extends BinaryExpression with InputTypeSpec {
+  override private[flink] def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO
+
+  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
+    DOUBLE_TYPE_INFO :: DOUBLE_TYPE_INFO :: Nil
+
+  override def toString: String = s"pow($left, $right)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.POWER, left.toRexNode, right.toRexNode)
+  }
+}
+
+case class Sqrt(child: Expression) extends UnaryExpression with InputTypeSpec {
+  override private[flink] def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO
+
+  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
+    Seq(DOUBLE_TYPE_INFO)
+
+  override def toString: String = s"sqrt($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.POWER, child.toRexNode, Literal(0.5).toRexNode)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ordering.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ordering.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ordering.scala
new file mode 100644
index 0000000..7f03827
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/ordering.scala
@@ -0,0 +1,54 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.tools.RelBuilder
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.validate._
+
+abstract class Ordering extends UnaryExpression {
+  override private[flink] def validateInput(): ValidationResult = {
+    if (!child.isInstanceOf[NamedExpression]) {
+      ValidationFailure(s"Sort should only based on field reference")
+    } else {
+      ValidationSuccess
+    }
+  }
+}
+
+case class Asc(child: Expression) extends Ordering {
+  override def toString: String = s"($child).asc"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    child.toRexNode
+  }
+
+  override private[flink] def resultType: TypeInformation[_] = child.resultType
+}
+
+case class Desc(child: Expression) extends Ordering {
+  override def toString: String = s"($child).desc"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.desc(child.toRexNode)
+  }
+
+  override private[flink] def resultType: TypeInformation[_] = child.resultType
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/package.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/package.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/package.scala
new file mode 100644
index 0000000..41e0c9f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/package.scala
@@ -0,0 +1,29 @@
+/*
+ * 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
+
+/**
+ * This package contains the base class of AST nodes and all the expression language AST classes.
+ * Expression trees should not be manually constructed by users. They are implicitly constructed
+ * from the implicit DSL conversions in
+ * [[org.apache.flink.table.api.scala.ImplicitExpressionConversions]] and
+ * [[org.apache.flink.table.api.scala.ImplicitExpressionOperations]]. For the Java API,
+ * expression trees should be generated from a string parser that parses expressions and creates
+ * AST nodes.
+ */
+package object expressions

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/stringExpressions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/stringExpressions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/stringExpressions.scala
new file mode 100644
index 0000000..f4b58cc
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/stringExpressions.scala
@@ -0,0 +1,279 @@
+/*
+ * 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.expressions
+
+import scala.collection.JavaConversions._
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.expressions.TrimMode.TrimMode
+import org.apache.flink.table.validate._
+
+/**
+  * Returns the length of this `str`.
+  */
+case class CharLength(child: Expression) extends UnaryExpression {
+  override private[flink] def resultType: TypeInformation[_] = INT_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (child.resultType == STRING_TYPE_INFO) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(s"CharLength operator requires String input, " +
+        s"but $child is of type ${child.resultType}")
+    }
+  }
+
+  override def toString: String = s"($child).charLength()"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.CHAR_LENGTH, child.toRexNode)
+  }
+}
+
+/**
+  * Returns str with the first letter of each word in uppercase.
+  * All other letters are in lowercase. Words are delimited by white space.
+  */
+case class InitCap(child: Expression) extends UnaryExpression {
+  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (child.resultType == STRING_TYPE_INFO) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(s"InitCap operator requires String input, " + 
+        s"but $child is of type ${child.resultType}")
+    }
+  }
+
+  override def toString: String = s"($child).initCap()"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.INITCAP, child.toRexNode)
+  }
+}
+
+/**
+  * Returns true if `str` matches `pattern`.
+  */
+case class Like(str: Expression, pattern: Expression) extends BinaryExpression {
+  private[flink] def left: Expression = str
+  private[flink] def right: Expression = pattern
+
+  override private[flink] def resultType: TypeInformation[_] = BOOLEAN_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (str.resultType == STRING_TYPE_INFO && pattern.resultType == STRING_TYPE_INFO) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(s"Like operator requires (String, String) input, " +
+        s"but ($str, $pattern) is of type (${str.resultType}, ${pattern.resultType})")
+    }
+  }
+
+  override def toString: String = s"($str).like($pattern)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.LIKE, children.map(_.toRexNode))
+  }
+}
+
+/**
+  * Returns str with all characters changed to lowercase.
+  */
+case class Lower(child: Expression) extends UnaryExpression {
+  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (child.resultType == STRING_TYPE_INFO) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(s"Lower operator requires String input, " +
+        s"but $child is of type ${child.resultType}")
+    }
+  }
+
+  override def toString: String = s"($child).toLowerCase()"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.LOWER, child.toRexNode)
+  }
+}
+
+/**
+  * Returns true if `str` is similar to `pattern`.
+  */
+case class Similar(str: Expression, pattern: Expression) extends BinaryExpression {
+  private[flink] def left: Expression = str
+  private[flink] def right: Expression = pattern
+
+  override private[flink] def resultType: TypeInformation[_] = BOOLEAN_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (str.resultType == STRING_TYPE_INFO && pattern.resultType == STRING_TYPE_INFO) {
+      ValidationSuccess
+    } else {
+      ValidationFailure(s"Similar operator requires (String, String) input, " +
+        s"but ($str, $pattern) is of type (${str.resultType}, ${pattern.resultType})")
+    }
+  }
+
+  override def toString: String = s"($str).similarTo($pattern)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.SIMILAR_TO, children.map(_.toRexNode))
+  }
+}
+
+/**
+  * Returns substring of `str` from `begin`(inclusive) for `length`.
+  */
+case class Substring(
+    str: Expression,
+    begin: Expression,
+    length: Expression) extends Expression with InputTypeSpec {
+
+  def this(str: Expression, begin: Expression) = this(str, begin, CharLength(str))
+
+  override private[flink] def children: Seq[Expression] = str :: begin :: length :: Nil
+
+  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
+
+  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
+    Seq(STRING_TYPE_INFO, INT_TYPE_INFO, INT_TYPE_INFO)
+
+  override def toString: String = s"($str).substring($begin, $length)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.SUBSTRING, children.map(_.toRexNode))
+  }
+}
+
+/**
+  * Trim `trimString` from `str` according to `trimMode`.
+  */
+case class Trim(
+    trimMode: Expression,
+    trimString: Expression,
+    str: Expression) extends Expression {
+
+  override private[flink] def children: Seq[Expression] = trimMode :: trimString :: str :: Nil
+
+  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    trimMode match {
+      case SymbolExpression(_: TrimMode) =>
+        if (trimString.resultType != STRING_TYPE_INFO) {
+          ValidationFailure(s"String expected for trimString, get ${trimString.resultType}")
+        } else if (str.resultType != STRING_TYPE_INFO) {
+          ValidationFailure(s"String expected for str, get ${str.resultType}")
+        } else {
+          ValidationSuccess
+        }
+      case _ => ValidationFailure("TrimMode symbol expected.")
+    }
+  }
+
+  override def toString: String = s"($str).trim($trimMode, $trimString)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.TRIM, children.map(_.toRexNode))
+  }
+}
+
+/**
+  * Enumeration of trim flags.
+  */
+object TrimConstants {
+  val TRIM_DEFAULT_CHAR = Literal(" ")
+}
+
+/**
+  * Returns str with all characters changed to uppercase.
+  */
+case class Upper(child: Expression) extends UnaryExpression with InputTypeSpec {
+
+  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
+
+  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
+    Seq(STRING_TYPE_INFO)
+
+  override def toString: String = s"($child).upperCase()"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.UPPER, child.toRexNode)
+  }
+}
+
+/**
+  * Returns the position of string needle in string haystack.
+  */
+case class Position(needle: Expression, haystack: Expression)
+    extends Expression with InputTypeSpec {
+
+  override private[flink] def children: Seq[Expression] = Seq(needle, haystack)
+
+  override private[flink] def resultType: TypeInformation[_] = INT_TYPE_INFO
+
+  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO)
+
+  override def toString: String = s"($needle).position($haystack)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.POSITION, needle.toRexNode, haystack.toRexNode)
+  }
+}
+
+/**
+  * Replaces a substring of a string with a replacement string.
+  * Starting at a position for a given length.
+  */
+case class Overlay(
+    str: Expression,
+    replacement: Expression,
+    starting: Expression,
+    position: Expression)
+  extends Expression with InputTypeSpec {
+
+  def this(str: Expression, replacement: Expression, starting: Expression) =
+    this(str, replacement, starting, CharLength(replacement))
+
+  override private[flink] def children: Seq[Expression] =
+    Seq(str, replacement, starting, position)
+
+  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
+
+  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO, INT_TYPE_INFO, INT_TYPE_INFO)
+
+  override def toString: String = s"($str).overlay($replacement, $starting, $position)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(
+      SqlStdOperatorTable.OVERLAY,
+      str.toRexNode,
+      replacement.toRexNode,
+      starting.toRexNode,
+      position.toRexNode)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/symbols.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/symbols.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/symbols.scala
new file mode 100644
index 0000000..0d71fb2
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/symbols.scala
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.expressions
+
+import org.apache.calcite.avatica.util.{TimeUnit, TimeUnitRange}
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.sql.fun.SqlTrimFunction
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.TypeInformation
+
+import scala.language.{existentials, implicitConversions}
+
+/**
+  * General expression class to represent a symbol.
+  */
+case class SymbolExpression(symbol: TableSymbol) extends LeafExpression {
+
+  override private[flink] def resultType: TypeInformation[_] =
+    throw new UnsupportedOperationException("This should not happen. A symbol has no result type.")
+
+  def toExpr = this // triggers implicit conversion
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    // dirty hack to pass Java enums to Java from Scala
+    val enum = symbol.enum.asInstanceOf[Enum[T] forSome { type T <: Enum[T] }]
+    relBuilder.getRexBuilder.makeFlag(enum)
+  }
+
+  override def toString: String = s"${symbol.symbols}.${symbol.name}"
+
+}
+
+/**
+  * Symbol that wraps a Calcite symbol in form of a Java enum.
+  */
+trait TableSymbol {
+  def symbols: TableSymbols
+  def name: String
+  def enum: Enum[_]
+}
+
+/**
+  * Enumeration of symbols.
+  */
+abstract class TableSymbols extends Enumeration {
+
+  class TableSymbolValue(e: Enum[_]) extends Val(e.name()) with TableSymbol {
+    override def symbols: TableSymbols = TableSymbols.this
+
+    override def enum: Enum[_] = e
+
+    override def name: String = toString()
+  }
+
+  protected final def Value(enum: Enum[_]): TableSymbolValue = new TableSymbolValue(enum)
+
+  implicit def symbolToExpression(symbol: TableSymbolValue): SymbolExpression =
+    SymbolExpression(symbol)
+
+}
+
+/**
+  * Units for working with time intervals.
+  */
+object TimeIntervalUnit extends TableSymbols {
+
+  type TimeIntervalUnit = TableSymbolValue
+
+  val YEAR = Value(TimeUnitRange.YEAR)
+  val YEAR_TO_MONTH = Value(TimeUnitRange.YEAR_TO_MONTH)
+  val MONTH = Value(TimeUnitRange.MONTH)
+  val DAY = Value(TimeUnitRange.DAY)
+  val DAY_TO_HOUR = Value(TimeUnitRange.DAY_TO_HOUR)
+  val DAY_TO_MINUTE = Value(TimeUnitRange.DAY_TO_MINUTE)
+  val DAY_TO_SECOND = Value(TimeUnitRange.DAY_TO_SECOND)
+  val HOUR = Value(TimeUnitRange.HOUR)
+  val HOUR_TO_MINUTE = Value(TimeUnitRange.HOUR_TO_MINUTE)
+  val HOUR_TO_SECOND = Value(TimeUnitRange.HOUR_TO_SECOND)
+  val MINUTE = Value(TimeUnitRange.MINUTE)
+  val MINUTE_TO_SECOND = Value(TimeUnitRange.MINUTE_TO_SECOND)
+  val SECOND = Value(TimeUnitRange.SECOND)
+
+}
+
+/**
+  * Units for working with time points.
+  */
+object TimePointUnit extends TableSymbols {
+
+  type TimePointUnit = TableSymbolValue
+
+  val YEAR = Value(TimeUnit.YEAR)
+  val MONTH = Value(TimeUnit.MONTH)
+  val DAY = Value(TimeUnit.DAY)
+  val HOUR = Value(TimeUnit.HOUR)
+  val MINUTE = Value(TimeUnit.MINUTE)
+  val SECOND = Value(TimeUnit.SECOND)
+  val QUARTER = Value(TimeUnit.QUARTER)
+  val WEEK = Value(TimeUnit.WEEK)
+  val MILLISECOND = Value(TimeUnit.MILLISECOND)
+  val MICROSECOND = Value(TimeUnit.MICROSECOND)
+
+}
+
+/**
+  * Modes for trimming strings.
+  */
+object TrimMode extends TableSymbols {
+
+  type TrimMode = TableSymbolValue
+
+  val BOTH = Value(SqlTrimFunction.Flag.BOTH)
+  val LEADING = Value(SqlTrimFunction.Flag.LEADING)
+  val TRAILING = Value(SqlTrimFunction.Flag.TRAILING)
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/time.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/time.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/time.scala
new file mode 100644
index 0000000..f09e2ad
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/time.scala
@@ -0,0 +1,377 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.avatica.util.{TimeUnit, TimeUnitRange}
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.table.calcite.FlinkRelBuilder
+import org.apache.flink.table.expressions.ExpressionUtils.{divide, getFactor, mod}
+import org.apache.flink.table.expressions.TimeIntervalUnit.TimeIntervalUnit
+import org.apache.flink.table.typeutils.TypeCheckUtils.isTimeInterval
+import org.apache.flink.table.typeutils.{TimeIntervalTypeInfo, TypeCheckUtils}
+import org.apache.flink.table.validate.{ValidationResult, ValidationFailure, ValidationSuccess}
+
+import scala.collection.JavaConversions._
+
+case class Extract(timeIntervalUnit: Expression, temporal: Expression) extends Expression {
+
+  override private[flink] def children: Seq[Expression] = timeIntervalUnit :: temporal :: Nil
+
+  override private[flink] def resultType: TypeInformation[_] = LONG_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (!TypeCheckUtils.isTemporal(temporal.resultType)) {
+      return ValidationFailure(s"Extract operator requires Temporal input, " +
+        s"but $temporal is of type ${temporal.resultType}")
+    }
+
+    timeIntervalUnit match {
+      case SymbolExpression(TimeIntervalUnit.YEAR)
+           | SymbolExpression(TimeIntervalUnit.MONTH)
+           | SymbolExpression(TimeIntervalUnit.DAY)
+        if temporal.resultType == SqlTimeTypeInfo.DATE
+          || temporal.resultType == SqlTimeTypeInfo.TIMESTAMP
+          || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MILLIS
+          || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MONTHS =>
+        ValidationSuccess
+
+      case SymbolExpression(TimeIntervalUnit.HOUR)
+           | SymbolExpression(TimeIntervalUnit.MINUTE)
+           | SymbolExpression(TimeIntervalUnit.SECOND)
+        if temporal.resultType == SqlTimeTypeInfo.TIME
+          || temporal.resultType == SqlTimeTypeInfo.TIMESTAMP
+          || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MILLIS =>
+        ValidationSuccess
+
+      case _ =>
+        ValidationFailure(s"Extract operator does not support unit '$timeIntervalUnit' for input" +
+          s" of type '${temporal.resultType}'.")
+    }
+  }
+
+  override def toString: String = s"($temporal).extract($timeIntervalUnit)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    // get wrapped Calcite unit
+    val timeUnitRange = timeIntervalUnit
+      .asInstanceOf[SymbolExpression]
+      .symbol
+      .enum
+      .asInstanceOf[TimeUnitRange]
+
+    // convert RexNodes
+    convertExtract(
+      timeIntervalUnit.toRexNode,
+      timeUnitRange,
+      temporal.toRexNode,
+      relBuilder.asInstanceOf[FlinkRelBuilder])
+  }
+
+  /**
+    * Standard conversion of the EXTRACT operator.
+    * Source: [[org.apache.calcite.sql2rel.StandardConvertletTable#convertExtract()]]
+    */
+  private def convertExtract(
+      timeUnitRangeRexNode: RexNode,
+      timeUnitRange: TimeUnitRange,
+      temporal: RexNode,
+      relBuilder: FlinkRelBuilder)
+    : RexNode = {
+
+    // TODO convert this into Table API expressions to make the code more readable
+    val rexBuilder = relBuilder.getRexBuilder
+    val resultType = relBuilder.getTypeFactory().createTypeFromTypeInfo(LONG_TYPE_INFO)
+    var result = rexBuilder.makeReinterpretCast(
+      resultType,
+      temporal,
+      rexBuilder.makeLiteral(false))
+
+    val unit = timeUnitRange.startUnit
+    val sqlTypeName = temporal.getType.getSqlTypeName
+    unit match {
+      case TimeUnit.YEAR | TimeUnit.MONTH | TimeUnit.DAY =>
+        sqlTypeName match {
+          case SqlTypeName.TIMESTAMP =>
+            result = divide(rexBuilder, result, TimeUnit.DAY.multiplier)
+            return rexBuilder.makeCall(
+              resultType,
+              SqlStdOperatorTable.EXTRACT_DATE,
+              Seq(timeUnitRangeRexNode, result))
+
+          case SqlTypeName.DATE =>
+            return rexBuilder.makeCall(
+              resultType,
+              SqlStdOperatorTable.EXTRACT_DATE,
+              Seq(timeUnitRangeRexNode, result))
+
+          case _ => // do nothing
+        }
+
+      case _ => // do nothing
+    }
+
+    result = mod(rexBuilder, resultType, result, getFactor(unit))
+    result = divide(rexBuilder, result, unit.multiplier)
+    result
+  }
+}
+
+abstract class TemporalCeilFloor(
+    timeIntervalUnit: Expression,
+    temporal: Expression)
+  extends Expression {
+
+  override private[flink] def children: Seq[Expression] = timeIntervalUnit :: temporal :: Nil
+
+  override private[flink] def resultType: TypeInformation[_] = temporal.resultType
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (!TypeCheckUtils.isTimePoint(temporal.resultType)) {
+      return ValidationFailure(s"Temporal ceil/floor operator requires Time Point input, " +
+        s"but $temporal is of type ${temporal.resultType}")
+    }
+    val unit = timeIntervalUnit match {
+      case SymbolExpression(u: TimeIntervalUnit) => Some(u)
+      case _ => None
+    }
+    if (unit.isEmpty) {
+      return ValidationFailure(s"Temporal ceil/floor operator requires Time Interval Unit " +
+        s"input, but $timeIntervalUnit is of type ${timeIntervalUnit.resultType}")
+    }
+
+    (unit.get, temporal.resultType) match {
+      case (TimeIntervalUnit.YEAR | TimeIntervalUnit.MONTH,
+          SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIMESTAMP) =>
+        ValidationSuccess
+      case (TimeIntervalUnit.DAY, SqlTimeTypeInfo.TIMESTAMP) =>
+        ValidationSuccess
+      case (TimeIntervalUnit.HOUR | TimeIntervalUnit.MINUTE | TimeIntervalUnit.SECOND,
+          SqlTimeTypeInfo.TIME | SqlTimeTypeInfo.TIMESTAMP) =>
+        ValidationSuccess
+      case _ =>
+        ValidationFailure(s"Temporal ceil/floor operator does not support " +
+          s"unit '$timeIntervalUnit' for input of type '${temporal.resultType}'.")
+    }
+  }
+}
+
+case class TemporalFloor(
+    timeIntervalUnit: Expression,
+    temporal: Expression)
+  extends TemporalCeilFloor(
+    timeIntervalUnit,
+    temporal) {
+
+  override def toString: String = s"($temporal).floor($timeIntervalUnit)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.FLOOR, temporal.toRexNode, timeIntervalUnit.toRexNode)
+  }
+}
+
+case class TemporalCeil(
+    timeIntervalUnit: Expression,
+    temporal: Expression)
+  extends TemporalCeilFloor(
+    timeIntervalUnit,
+    temporal) {
+
+  override def toString: String = s"($temporal).ceil($timeIntervalUnit)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    relBuilder.call(SqlStdOperatorTable.CEIL, temporal.toRexNode, timeIntervalUnit.toRexNode)
+  }
+}
+
+abstract class CurrentTimePoint(
+    targetType: TypeInformation[_],
+    local: Boolean)
+  extends LeafExpression {
+
+  override private[flink] def resultType: TypeInformation[_] = targetType
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (!TypeCheckUtils.isTimePoint(targetType)) {
+      ValidationFailure(s"CurrentTimePoint operator requires Time Point target type, " +
+        s"but get $targetType.")
+    } else if (local && targetType == SqlTimeTypeInfo.DATE) {
+      ValidationFailure(s"Localized CurrentTimePoint operator requires Time or Timestamp target " +
+        s"type, but get $targetType.")
+    } else {
+      ValidationSuccess
+    }
+  }
+
+  override def toString: String = if (local) {
+    s"local$targetType()"
+  } else {
+    s"current$targetType()"
+  }
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    val operator = targetType match {
+      case SqlTimeTypeInfo.TIME if local => SqlStdOperatorTable.LOCALTIME
+      case SqlTimeTypeInfo.TIMESTAMP if local => SqlStdOperatorTable.LOCALTIMESTAMP
+      case SqlTimeTypeInfo.DATE => SqlStdOperatorTable.CURRENT_DATE
+      case SqlTimeTypeInfo.TIME => SqlStdOperatorTable.CURRENT_TIME
+      case SqlTimeTypeInfo.TIMESTAMP => SqlStdOperatorTable.CURRENT_TIMESTAMP
+    }
+    relBuilder.call(operator)
+  }
+}
+
+case class CurrentDate() extends CurrentTimePoint(SqlTimeTypeInfo.DATE, local = false)
+
+case class CurrentTime() extends CurrentTimePoint(SqlTimeTypeInfo.TIME, local = false)
+
+case class CurrentTimestamp() extends CurrentTimePoint(SqlTimeTypeInfo.TIMESTAMP, local = false)
+
+case class LocalTime() extends CurrentTimePoint(SqlTimeTypeInfo.TIME, local = true)
+
+case class LocalTimestamp() extends CurrentTimePoint(SqlTimeTypeInfo.TIMESTAMP, local = true)
+
+/**
+  * Extracts the quarter of a year from a SQL date.
+  */
+case class Quarter(child: Expression) extends UnaryExpression with InputTypeSpec {
+
+  override private[flink] def expectedTypes: Seq[TypeInformation[_]] = Seq(SqlTimeTypeInfo.DATE)
+
+  override private[flink] def resultType: TypeInformation[_] = LONG_TYPE_INFO
+
+  override def toString: String = s"($child).quarter()"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    /**
+      * Standard conversion of the QUARTER operator.
+      * Source: [[org.apache.calcite.sql2rel.StandardConvertletTable#convertQuarter()]]
+      */
+    Plus(
+      Div(
+        Minus(
+          Extract(TimeIntervalUnit.MONTH, child),
+          Literal(1L)),
+        Literal(TimeUnit.QUARTER.multiplier.longValue())),
+      Literal(1L)
+    ).toRexNode
+  }
+}
+
+/**
+  * Determines whether two anchored time intervals overlap.
+  */
+case class TemporalOverlaps(
+    leftTimePoint: Expression,
+    leftTemporal: Expression,
+    rightTimePoint: Expression,
+    rightTemporal: Expression)
+  extends Expression {
+
+  override private[flink] def children: Seq[Expression] =
+    Seq(leftTimePoint, leftTemporal, rightTimePoint, rightTemporal)
+
+  override private[flink] def resultType: TypeInformation[_] = BOOLEAN_TYPE_INFO
+
+  override private[flink] def validateInput(): ValidationResult = {
+    if (!TypeCheckUtils.isTimePoint(leftTimePoint.resultType)) {
+      return ValidationFailure(s"TemporalOverlaps operator requires leftTimePoint to be of type " +
+        s"Time Point, but get ${leftTimePoint.resultType}.")
+    }
+    if (!TypeCheckUtils.isTimePoint(rightTimePoint.resultType)) {
+      return ValidationFailure(s"TemporalOverlaps operator requires rightTimePoint to be of " +
+        s"type Time Point, but get ${rightTimePoint.resultType}.")
+    }
+    if (leftTimePoint.resultType != rightTimePoint.resultType) {
+      return ValidationFailure(s"TemporalOverlaps operator requires leftTimePoint and " +
+        s"rightTimePoint to be of same type.")
+    }
+
+    // leftTemporal is point, then it must be comparable with leftTimePoint
+    if (TypeCheckUtils.isTimePoint(leftTemporal.resultType)) {
+      if (leftTemporal.resultType != leftTimePoint.resultType) {
+        return ValidationFailure(s"TemporalOverlaps operator requires leftTemporal and " +
+          s"leftTimePoint to be of same type if leftTemporal is of type Time Point.")
+      }
+    } else if (!isTimeInterval(leftTemporal.resultType)) {
+      return ValidationFailure(s"TemporalOverlaps operator requires leftTemporal to be of " +
+        s"type Time Point or Time Interval.")
+    }
+
+    // rightTemporal is point, then it must be comparable with rightTimePoint
+    if (TypeCheckUtils.isTimePoint(rightTemporal.resultType)) {
+      if (rightTemporal.resultType != rightTimePoint.resultType) {
+        return ValidationFailure(s"TemporalOverlaps operator requires rightTemporal and " +
+          s"rightTimePoint to be of same type if rightTemporal is of type Time Point.")
+      }
+    } else if (!isTimeInterval(rightTemporal.resultType)) {
+      return ValidationFailure(s"TemporalOverlaps operator requires rightTemporal to be of " +
+        s"type Time Point or Time Interval.")
+    }
+    ValidationSuccess
+  }
+
+  override def toString: String = s"temporalOverlaps(${children.mkString(", ")})"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+    convertOverlaps(
+      leftTimePoint.toRexNode,
+      leftTemporal.toRexNode,
+      rightTimePoint.toRexNode,
+      rightTemporal.toRexNode,
+      relBuilder.asInstanceOf[FlinkRelBuilder])
+  }
+
+  /**
+    * Standard conversion of the OVERLAPS operator.
+    * Source: [[org.apache.calcite.sql2rel.StandardConvertletTable#convertOverlaps()]]
+    */
+  private def convertOverlaps(
+      leftP: RexNode,
+      leftT: RexNode,
+      rightP: RexNode,
+      rightT: RexNode,
+      relBuilder: FlinkRelBuilder)
+    : RexNode = {
+    // leftT = leftP + leftT if leftT is an interval
+    val convLeftT = if (isTimeInterval(leftTemporal.resultType)) {
+        relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, leftP, leftT)
+      } else {
+        leftT
+      }
+    // rightT = rightP + rightT if rightT is an interval
+    val convRightT = if (isTimeInterval(rightTemporal.resultType)) {
+        relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, rightP, rightT)
+      } else {
+        rightT
+      }
+    // leftT >= rightP
+    val leftPred = relBuilder.call(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, convLeftT, rightP)
+    // rightT >= leftP
+    val rightPred = relBuilder.call(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, convRightT, leftP)
+
+    // leftT >= rightP and rightT >= leftP
+    relBuilder.call(SqlStdOperatorTable.AND, leftPred, rightPred)
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/windowProperties.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/windowProperties.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/windowProperties.scala
new file mode 100644
index 0000000..990d928
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/windowProperties.scala
@@ -0,0 +1,58 @@
+/*
+ * 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.expressions
+
+import org.apache.calcite.rex.RexNode
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo
+import org.apache.flink.table.calcite.FlinkRelBuilder
+import FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.validate.{ValidationFailure, ValidationSuccess}
+
+abstract class WindowProperty(child: Expression) extends UnaryExpression {
+
+  override def toString = s"WindowProperty($child)"
+
+  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode =
+    throw new UnsupportedOperationException("WindowProperty cannot be transformed to RexNode.")
+
+  override private[flink] def validateInput() =
+    if (child.isInstanceOf[WindowReference]) {
+      ValidationSuccess
+    } else {
+      ValidationFailure("Child must be a window reference.")
+    }
+
+  private[flink] def toNamedWindowProperty(name: String)(implicit relBuilder: RelBuilder)
+    : NamedWindowProperty = NamedWindowProperty(name, this)
+}
+
+case class WindowStart(child: Expression) extends WindowProperty(child) {
+
+  override private[flink] def resultType = SqlTimeTypeInfo.TIMESTAMP
+
+  override def toString: String = s"start($child)"
+}
+
+case class WindowEnd(child: Expression) extends WindowProperty(child) {
+
+  override private[flink] def resultType = SqlTimeTypeInfo.TIMESTAMP
+
+  override def toString: String = s"end($child)"
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ScalarFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ScalarFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ScalarFunction.scala
new file mode 100644
index 0000000..d01cf68
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/ScalarFunction.scala
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.functions
+
+import org.apache.flink.api.common.functions.InvalidTypesException
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.TypeExtractor
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.expressions.{Expression, ScalarFunctionCall}
+
+/**
+  * Base class for a user-defined scalar function. A user-defined scalar functions maps zero, one,
+  * or multiple scalar values to a new scalar value.
+  *
+  * The behavior of a [[ScalarFunction]] can be defined by implementing a custom evaluation
+  * method. An evaluation method must be declared publicly and named "eval". Evaluation methods
+  * can also be overloaded by implementing multiple methods named "eval".
+  *
+  * User-defined functions must have a default constructor and must be instantiable during runtime.
+  *
+  * By default the result type of an evaluation method is determined by Flink's type extraction
+  * facilities. This is sufficient for basic types or simple POJOs but might be wrong for more
+  * complex, custom, or composite types. In these cases [[TypeInformation]] of the result type
+  * can be manually defined by overriding [[getResultType()]].
+  *
+  * Internally, the Table/SQL API code generation works with primitive values as much as possible.
+  * If a user-defined scalar function should not introduce much overhead during runtime, it is
+  * recommended to declare parameters and result types as primitive types instead of their boxed
+  * classes. DATE/TIME is equal to int, TIMESTAMP is equal to long.
+  */
+abstract class ScalarFunction extends UserDefinedFunction {
+
+  /**
+    * Creates a call to a [[ScalarFunction]] in Scala Table API.
+    *
+    * @param params actual parameters of function
+    * @return [[Expression]] in form of a [[ScalarFunctionCall]]
+    */
+  final def apply(params: Expression*): Expression = {
+    ScalarFunctionCall(this, params)
+  }
+
+  override def toString: String = getClass.getCanonicalName
+
+  // ----------------------------------------------------------------------------------------------
+
+  /**
+    * Returns the result type of the evaluation method with a given signature.
+    *
+    * This method needs to be overriden in case Flink's type extraction facilities are not
+    * sufficient to extract the [[TypeInformation]] based on the return type of the evaluation
+    * method. Flink's type extraction facilities can handle basic types or
+    * simple POJOs but might be wrong for more complex, custom, or composite types.
+    *
+    * @param signature signature of the method the return type needs to be determined
+    * @return [[TypeInformation]] of result type or null if Flink should determine the type
+    */
+  def getResultType(signature: Array[Class[_]]): TypeInformation[_] = null
+
+  /**
+    * Returns [[TypeInformation]] about the operands of the evaluation method with a given
+    * signature.
+    *
+    * In order to perform operand type inference in SQL (especially when NULL is used) it might be
+    * necessary to determine the parameter [[TypeInformation]] of an evaluation method.
+    * By default Flink's type extraction facilities are used for this but might be wrong for
+    * more complex, custom, or composite types.
+    *
+    * @param signature signature of the method the operand types need to be determined
+    * @return [[TypeInformation]] of  operand types
+    */
+  def getParameterTypes(signature: Array[Class[_]]): Array[TypeInformation[_]] = {
+    signature.map { c =>
+      try {
+        TypeExtractor.getForClass(c)
+      } catch {
+        case ite: InvalidTypesException =>
+          throw new ValidationException(
+            s"Parameter types of scalar function '${this.getClass.getCanonicalName}' cannot be " +
+            s"automatically determined. Please provide type information manually.")
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TableFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TableFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TableFunction.scala
new file mode 100644
index 0000000..653793e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/TableFunction.scala
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.functions
+
+import java.util
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.expressions.{Expression, TableFunctionCall}
+
+/**
+  * Base class for a user-defined table function (UDTF). A user-defined table functions works on
+  * zero, one, or multiple scalar values as input and returns multiple rows as output.
+  *
+  * The behavior of a [[TableFunction]] can be defined by implementing a custom evaluation
+  * method. An evaluation method must be declared publicly, not static and named "eval".
+  * Evaluation methods can also be overloaded by implementing multiple methods named "eval".
+  *
+  * User-defined functions must have a default constructor and must be instantiable during runtime.
+  *
+  * By default the result type of an evaluation method is determined by Flink's type extraction
+  * facilities. This is sufficient for basic types or simple POJOs but might be wrong for more
+  * complex, custom, or composite types. In these cases [[TypeInformation]] of the result type
+  * can be manually defined by overriding [[getResultType()]].
+  *
+  * Internally, the Table/SQL API code generation works with primitive values as much as possible.
+  * If a user-defined table function should not introduce much overhead during runtime, it is
+  * recommended to declare parameters and result types as primitive types instead of their boxed
+  * classes. DATE/TIME is equal to int, TIMESTAMP is equal to long.
+  *
+  * Example:
+  *
+  * {{{
+  *
+  *   public class Split extends TableFunction<String> {
+  *
+  *     // implement an "eval" method with as many parameters as you want
+  *     public void eval(String str) {
+  *       for (String s : str.split(" ")) {
+  *         collect(s);   // use collect(...) to emit an output row
+  *       }
+  *     }
+  *
+  *     // you can overload the eval method here ...
+  *   }
+  *
+  *   val tEnv: TableEnvironment = ...
+  *   val table: Table = ...    // schema: [a: String]
+  *
+  *   // for Scala users
+  *   val split = new Split()
+  *   table.join(split('c) as ('s)).select('a, 's)
+  *
+  *   // for Java users
+  *   tEnv.registerFunction("split", new Split())   // register table function first
+  *   table.join("split(a) as (s)").select("a, s")
+  *
+  *   // for SQL users
+  *   tEnv.registerFunction("split", new Split())   // register table function first
+  *   tEnv.sql("SELECT a, s FROM MyTable, LATERAL TABLE(split(a)) as T(s)")
+  *
+  * }}}
+  *
+  * @tparam T The type of the output row
+  */
+abstract class TableFunction[T] extends UserDefinedFunction {
+
+  /**
+    * Creates a call to a [[TableFunction]] in Scala Table API.
+    *
+    * @param params actual parameters of function
+    * @return [[Expression]] in form of a [[TableFunctionCall]]
+    */
+  final def apply(params: Expression*)(implicit typeInfo: TypeInformation[T]): Expression = {
+    val resultType = if (getResultType == null) {
+      typeInfo
+    } else {
+      getResultType
+    }
+    TableFunctionCall(getClass.getSimpleName, this, params, resultType)
+  }
+
+  override def toString: String = getClass.getCanonicalName
+
+  // ----------------------------------------------------------------------------------------------
+
+  private val rows: util.ArrayList[T] = new util.ArrayList[T]()
+
+  /**
+    * Emit an output row.
+    *
+    * @param row the output row
+    */
+  protected def collect(row: T): Unit = {
+    // cache rows for now, maybe immediately process them further
+    rows.add(row)
+  }
+
+  /**
+    * Internal use. Get an iterator of the buffered rows.
+    */
+  def getRowsIterator = rows.iterator()
+
+  /**
+    * Internal use. Clear buffered rows.
+    */
+  def clear() = rows.clear()
+
+  // ----------------------------------------------------------------------------------------------
+
+  /**
+    * Returns the result type of the evaluation method with a given signature.
+    *
+    * This method needs to be overriden in case Flink's type extraction facilities are not
+    * sufficient to extract the [[TypeInformation]] based on the return type of the evaluation
+    * method. Flink's type extraction facilities can handle basic types or
+    * simple POJOs but might be wrong for more complex, custom, or composite types.
+    *
+    * @return [[TypeInformation]] of result type or null if Flink should determine the type
+    */
+  def getResultType: TypeInformation[T] = null
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/UserDefinedFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/UserDefinedFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/UserDefinedFunction.scala
new file mode 100644
index 0000000..b99ab8d
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/UserDefinedFunction.scala
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.functions
+
+/**
+  * Base class for all user-defined functions such as scalar functions, table functions,
+  * or aggregation functions.
+  *
+  * User-defined functions must have a default constructor and must be instantiable during runtime.
+  */
+trait UserDefinedFunction {
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/MathFunctions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/MathFunctions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/MathFunctions.scala
new file mode 100644
index 0000000..64e4bc4
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/MathFunctions.scala
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.functions.utils
+
+import java.math.{BigDecimal => JBigDecimal}
+
+class MathFunctions {}
+
+object MathFunctions {
+  def power(a: Double, b: JBigDecimal): Double = {
+    Math.pow(a, b.doubleValue())
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala
new file mode 100644
index 0000000..da652e0
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/ScalarSqlFunction.scala
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.functions.utils
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.sql._
+import org.apache.calcite.sql.`type`.SqlOperandTypeChecker.Consistency
+import org.apache.calcite.sql.`type`._
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.functions.utils.ScalarSqlFunction.{createOperandTypeChecker, createOperandTypeInference, createReturnTypeInference}
+import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{getResultType, getSignature, getSignatures, signatureToString, signaturesToString}
+
+import scala.collection.JavaConverters._
+
+/**
+  * Calcite wrapper for user-defined scalar functions.
+  *
+  * @param name function name (used by SQL parser)
+  * @param scalarFunction scalar function to be called
+  * @param typeFactory type factory for converting Flink's between Calcite's types
+  */
+class ScalarSqlFunction(
+    name: String,
+    scalarFunction: ScalarFunction,
+    typeFactory: FlinkTypeFactory)
+  extends SqlFunction(
+    new SqlIdentifier(name, SqlParserPos.ZERO),
+    createReturnTypeInference(name, scalarFunction, typeFactory),
+    createOperandTypeInference(scalarFunction, typeFactory),
+    createOperandTypeChecker(name, scalarFunction),
+    null,
+    SqlFunctionCategory.USER_DEFINED_FUNCTION) {
+
+  def getScalarFunction = scalarFunction
+
+}
+
+object ScalarSqlFunction {
+
+  private[flink] def createReturnTypeInference(
+      name: String,
+      scalarFunction: ScalarFunction,
+      typeFactory: FlinkTypeFactory)
+    : SqlReturnTypeInference = {
+    /**
+      * Return type inference based on [[ScalarFunction]] given information.
+      */
+    new SqlReturnTypeInference {
+      override def inferReturnType(opBinding: SqlOperatorBinding): RelDataType = {
+        val parameters = opBinding
+          .collectOperandTypes()
+          .asScala
+          .map { operandType =>
+            if (operandType.getSqlTypeName == SqlTypeName.NULL) {
+              null
+            } else {
+              FlinkTypeFactory.toTypeInfo(operandType)
+            }
+          }
+        val foundSignature = getSignature(scalarFunction, parameters)
+        if (foundSignature.isEmpty) {
+          throw new ValidationException(
+            s"Given parameters of function '$name' do not match any signature. \n" +
+              s"Actual: ${signatureToString(parameters)} \n" +
+              s"Expected: ${signaturesToString(scalarFunction)}")
+        }
+        val resultType = getResultType(scalarFunction, foundSignature.get)
+        typeFactory.createTypeFromTypeInfo(resultType)
+      }
+    }
+  }
+
+  private[flink] def createOperandTypeInference(
+      scalarFunction: ScalarFunction,
+      typeFactory: FlinkTypeFactory)
+    : SqlOperandTypeInference = {
+    /**
+      * Operand type inference based on [[ScalarFunction]] given information.
+      */
+    new SqlOperandTypeInference {
+      override def inferOperandTypes(
+          callBinding: SqlCallBinding,
+          returnType: RelDataType,
+          operandTypes: Array[RelDataType]): Unit = {
+
+        val operandTypeInfo = getOperandTypeInfo(callBinding)
+
+        val foundSignature = getSignature(scalarFunction, operandTypeInfo)
+          .getOrElse(throw new ValidationException(s"Operand types of could not be inferred."))
+
+        val inferredTypes = scalarFunction
+          .getParameterTypes(foundSignature)
+          .map(typeFactory.createTypeFromTypeInfo)
+
+        inferredTypes.zipWithIndex.foreach {
+          case (inferredType, i) =>
+            operandTypes(i) = inferredType
+        }
+      }
+    }
+  }
+
+  private[flink] def createOperandTypeChecker(
+      name: String,
+      scalarFunction: ScalarFunction)
+    : SqlOperandTypeChecker = {
+
+    val signatures = getSignatures(scalarFunction)
+
+    /**
+      * Operand type checker based on [[ScalarFunction]] given information.
+      */
+    new SqlOperandTypeChecker {
+      override def getAllowedSignatures(op: SqlOperator, opName: String): String = {
+        s"$opName[${signaturesToString(scalarFunction)}]"
+      }
+
+      override def getOperandCountRange: SqlOperandCountRange = {
+        val signatureLengths = signatures.map(_.length)
+        SqlOperandCountRanges.between(signatureLengths.min, signatureLengths.max)
+      }
+
+      override def checkOperandTypes(
+          callBinding: SqlCallBinding,
+          throwOnFailure: Boolean)
+        : Boolean = {
+        val operandTypeInfo = getOperandTypeInfo(callBinding)
+
+        val foundSignature = getSignature(scalarFunction, operandTypeInfo)
+
+        if (foundSignature.isEmpty) {
+          if (throwOnFailure) {
+            throw new ValidationException(
+              s"Given parameters of function '$name' do not match any signature. \n" +
+                s"Actual: ${signatureToString(operandTypeInfo)} \n" +
+                s"Expected: ${signaturesToString(scalarFunction)}")
+          } else {
+            false
+          }
+        } else {
+          true
+        }
+      }
+
+      override def isOptional(i: Int): Boolean = false
+
+      override def getConsistency: Consistency = Consistency.NONE
+
+    }
+  }
+
+  private[flink] def getOperandTypeInfo(callBinding: SqlCallBinding): Seq[TypeInformation[_]] = {
+    val operandTypes = for (i <- 0 until callBinding.getOperandCount)
+      yield callBinding.getOperandType(i)
+    operandTypes.map { operandType =>
+      if (operandType.getSqlTypeName == SqlTypeName.NULL) {
+        null
+      } else {
+        FlinkTypeFactory.toTypeInfo(operandType)
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/TableSqlFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/TableSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/TableSqlFunction.scala
new file mode 100644
index 0000000..74f3374
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/TableSqlFunction.scala
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.functions.utils
+
+import com.google.common.base.Predicate
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.sql._
+import org.apache.calcite.sql.`type`._
+import org.apache.calcite.sql.parser.SqlParserPos
+import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction
+import org.apache.calcite.util.Util
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.functions.TableFunction
+import org.apache.flink.table.plan.schema.FlinkTableFunctionImpl
+
+import scala.collection.JavaConverters._
+import java.util
+
+/**
+  * Calcite wrapper for user-defined table functions.
+  */
+class TableSqlFunction(
+    name: String,
+    udtf: TableFunction[_],
+    rowTypeInfo: TypeInformation[_],
+    returnTypeInference: SqlReturnTypeInference,
+    operandTypeInference: SqlOperandTypeInference,
+    operandTypeChecker: SqlOperandTypeChecker,
+    paramTypes: util.List[RelDataType],
+    functionImpl: FlinkTableFunctionImpl[_])
+  extends SqlUserDefinedTableFunction(
+    new SqlIdentifier(name, SqlParserPos.ZERO),
+    returnTypeInference,
+    operandTypeInference,
+    operandTypeChecker,
+    paramTypes,
+    functionImpl) {
+
+  /**
+    * Get the user-defined table function.
+    */
+  def getTableFunction = udtf
+
+  /**
+    * Get the type information of the table returned by the table function.
+    */
+  def getRowTypeInfo = rowTypeInfo
+
+  /**
+    * Get additional mapping information if the returned table type is a POJO
+    * (POJO types have no deterministic field order).
+    */
+  def getPojoFieldMapping = functionImpl.fieldIndexes
+
+}
+
+object TableSqlFunction {
+
+  /**
+    * Util function to create a [[TableSqlFunction]].
+    *
+    * @param name function name (used by SQL parser)
+    * @param udtf user-defined table function to be called
+    * @param rowTypeInfo the row type information generated by the table function
+    * @param typeFactory type factory for converting Flink's between Calcite's types
+    * @param functionImpl Calcite table function schema
+    * @return [[TableSqlFunction]]
+    */
+  def apply(
+    name: String,
+    udtf: TableFunction[_],
+    rowTypeInfo: TypeInformation[_],
+    typeFactory: FlinkTypeFactory,
+    functionImpl: FlinkTableFunctionImpl[_]): TableSqlFunction = {
+
+    val argTypes: util.List[RelDataType] = new util.ArrayList[RelDataType]
+    val typeFamilies: util.List[SqlTypeFamily] = new util.ArrayList[SqlTypeFamily]
+    // derives operands' data types and type families
+    functionImpl.getParameters.asScala.foreach{ o =>
+      val relType: RelDataType = o.getType(typeFactory)
+      argTypes.add(relType)
+      typeFamilies.add(Util.first(relType.getSqlTypeName.getFamily, SqlTypeFamily.ANY))
+    }
+    // derives whether the 'input'th parameter of a method is optional.
+    val optional: Predicate[Integer] = new Predicate[Integer]() {
+      def apply(input: Integer): Boolean = {
+        functionImpl.getParameters.get(input).isOptional
+      }
+    }
+    // create type check for the operands
+    val typeChecker: FamilyOperandTypeChecker = OperandTypes.family(typeFamilies, optional)
+
+    new TableSqlFunction(
+      name,
+      udtf,
+      rowTypeInfo,
+      ReturnTypes.CURSOR,
+      InferTypes.explicit(argTypes),
+      typeChecker,
+      argTypes,
+      functionImpl)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala
new file mode 100644
index 0000000..aa3fab0
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/utils/UserDefinedFunctionUtils.scala
@@ -0,0 +1,341 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.flink.table.functions.utils
+
+import java.lang.reflect.{Method, Modifier}
+import java.sql.{Date, Time, Timestamp}
+
+import com.google.common.primitives.Primitives
+import org.apache.calcite.sql.SqlFunction
+import org.apache.flink.api.common.functions.InvalidTypesException
+import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.api.java.typeutils.TypeExtractor
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.api.{TableException, ValidationException}
+import org.apache.flink.table.functions.{ScalarFunction, TableFunction, UserDefinedFunction}
+import org.apache.flink.table.plan.schema.FlinkTableFunctionImpl
+import org.apache.flink.util.InstantiationUtil
+
+object UserDefinedFunctionUtils {
+
+  /**
+    * Instantiates a user-defined function.
+    */
+  def instantiate[T <: UserDefinedFunction](clazz: Class[T]): T = {
+    val constructor = clazz.getDeclaredConstructor()
+    constructor.setAccessible(true)
+    constructor.newInstance()
+  }
+
+  /**
+    * Checks if a user-defined function can be easily instantiated.
+    */
+  def checkForInstantiation(clazz: Class[_]): Unit = {
+    if (!InstantiationUtil.isPublic(clazz)) {
+      throw ValidationException("Function class is not public.")
+    }
+    else if (!InstantiationUtil.isProperClass(clazz)) {
+      throw ValidationException("Function class is no proper class, it is either abstract," +
+        " an interface, or a primitive type.")
+    }
+    else if (InstantiationUtil.isNonStaticInnerClass(clazz)) {
+      throw ValidationException("The class is an inner class, but not statically accessible.")
+    }
+
+    // check for default constructor (can be private)
+    clazz
+      .getDeclaredConstructors
+      .find(_.getParameterTypes.isEmpty)
+      .getOrElse(throw ValidationException("Function class needs a default constructor."))
+  }
+
+  /**
+    * Check whether this is a Scala object. It is forbidden to use [[TableFunction]] implemented
+    * by a Scala object, since concurrent risks.
+    */
+  def checkNotSingleton(clazz: Class[_]): Unit = {
+    // TODO it is not a good way to check singleton. Maybe improve it further.
+    if (clazz.getFields.map(_.getName) contains "MODULE$") {
+      throw new ValidationException(
+        s"TableFunction implemented by class ${clazz.getCanonicalName} " +
+          s"is a Scala object, it is forbidden since concurrent risks.")
+    }
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // Utilities for eval methods
+  // ----------------------------------------------------------------------------------------------
+
+  /**
+    * Returns signatures matching the given signature of [[TypeInformation]].
+    * Elements of the signature can be null (act as a wildcard).
+    */
+  def getSignature(
+      function: UserDefinedFunction,
+      signature: Seq[TypeInformation[_]])
+    : Option[Array[Class[_]]] = {
+    // We compare the raw Java classes not the TypeInformation.
+    // TypeInformation does not matter during runtime (e.g. within a MapFunction).
+    val actualSignature = typeInfoToClass(signature)
+    val signatures = getSignatures(function)
+
+    signatures
+      // go over all signatures and find one matching actual signature
+      .find { curSig =>
+      // match parameters of signature to actual parameters
+      actualSignature.length == curSig.length &&
+        curSig.zipWithIndex.forall { case (clazz, i) =>
+          parameterTypeEquals(actualSignature(i), clazz)
+        }
+    }
+  }
+
+  /**
+    * Returns eval method matching the given signature of [[TypeInformation]].
+    */
+  def getEvalMethod(
+      function: UserDefinedFunction,
+      signature: Seq[TypeInformation[_]])
+    : Option[Method] = {
+    // We compare the raw Java classes not the TypeInformation.
+    // TypeInformation does not matter during runtime (e.g. within a MapFunction).
+    val actualSignature = typeInfoToClass(signature)
+    val evalMethods = checkAndExtractEvalMethods(function)
+
+    evalMethods
+      // go over all eval methods and find one matching
+      .find { cur =>
+      val signatures = cur.getParameterTypes
+      // match parameters of signature to actual parameters
+      actualSignature.length == signatures.length &&
+        signatures.zipWithIndex.forall { case (clazz, i) =>
+          parameterTypeEquals(actualSignature(i), clazz)
+        }
+    }
+  }
+
+  /**
+    * Extracts "eval" methods and throws a [[ValidationException]] if no implementation
+    * can be found.
+    */
+  def checkAndExtractEvalMethods(function: UserDefinedFunction): Array[Method] = {
+    val methods = function
+      .getClass
+      .getDeclaredMethods
+      .filter { m =>
+        val modifiers = m.getModifiers
+        m.getName == "eval" &&
+          Modifier.isPublic(modifiers) &&
+          !Modifier.isAbstract(modifiers) &&
+          !(function.isInstanceOf[TableFunction[_]] && Modifier.isStatic(modifiers))
+      }
+
+    if (methods.isEmpty) {
+      throw new ValidationException(
+        s"Function class '${function.getClass.getCanonicalName}' does not implement at least " +
+          s"one method named 'eval' which is public, not abstract and " +
+          s"(in case of table functions) not static.")
+    } else {
+      methods
+    }
+  }
+
+  def getSignatures(function: UserDefinedFunction): Array[Array[Class[_]]] = {
+    checkAndExtractEvalMethods(function).map(_.getParameterTypes)
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // Utilities for SQL functions
+  // ----------------------------------------------------------------------------------------------
+
+  /**
+    * Create [[SqlFunction]] for a [[ScalarFunction]]
+ *
+    * @param name function name
+    * @param function scalar function
+    * @param typeFactory type factory
+    * @return the ScalarSqlFunction
+    */
+  def createScalarSqlFunction(
+      name: String,
+      function: ScalarFunction,
+      typeFactory: FlinkTypeFactory)
+    : SqlFunction = {
+    new ScalarSqlFunction(name, function, typeFactory)
+  }
+
+  /**
+    * Create [[SqlFunction]]s for a [[TableFunction]]'s every eval method
+ *
+    * @param name function name
+    * @param tableFunction table function
+    * @param resultType the type information of returned table
+    * @param typeFactory type factory
+    * @return the TableSqlFunction
+    */
+  def createTableSqlFunctions(
+      name: String,
+      tableFunction: TableFunction[_],
+      resultType: TypeInformation[_],
+      typeFactory: FlinkTypeFactory)
+    : Seq[SqlFunction] = {
+    val (fieldNames, fieldIndexes, _) = UserDefinedFunctionUtils.getFieldInfo(resultType)
+    val evalMethods = checkAndExtractEvalMethods(tableFunction)
+
+    evalMethods.map { method =>
+      val function = new FlinkTableFunctionImpl(resultType, fieldIndexes, fieldNames, method)
+      TableSqlFunction(name, tableFunction, resultType, typeFactory, function)
+    }
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // Utilities for scalar functions
+  // ----------------------------------------------------------------------------------------------
+
+  /**
+    * Internal method of [[ScalarFunction#getResultType()]] that does some pre-checking and uses
+    * [[TypeExtractor]] as default return type inference.
+    */
+  def getResultType(
+      function: ScalarFunction,
+      signature: Array[Class[_]])
+    : TypeInformation[_] = {
+    // find method for signature
+    val evalMethod = checkAndExtractEvalMethods(function)
+      .find(m => signature.sameElements(m.getParameterTypes))
+      .getOrElse(throw new ValidationException("Given signature is invalid."))
+
+    val userDefinedTypeInfo = function.getResultType(signature)
+    if (userDefinedTypeInfo != null) {
+      userDefinedTypeInfo
+    } else {
+      try {
+        TypeExtractor.getForClass(evalMethod.getReturnType)
+      } catch {
+        case ite: InvalidTypesException =>
+          throw new ValidationException(
+            s"Return type of scalar function '${function.getClass.getCanonicalName}' cannot be " +
+              s"automatically determined. Please provide type information manually.")
+      }
+    }
+  }
+
+  /**
+    * Returns the return type of the evaluation method matching the given signature.
+    */
+  def getResultTypeClass(
+      function: ScalarFunction,
+      signature: Array[Class[_]])
+    : Class[_] = {
+    // find method for signature
+    val evalMethod = checkAndExtractEvalMethods(function)
+      .find(m => signature.sameElements(m.getParameterTypes))
+      .getOrElse(throw new IllegalArgumentException("Given signature is invalid."))
+    evalMethod.getReturnType
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // Miscellaneous
+  // ----------------------------------------------------------------------------------------------
+
+  /**
+    * Returns field names and field positions for a given [[TypeInformation]].
+    *
+    * Field names are automatically extracted for
+    * [[org.apache.flink.api.common.typeutils.CompositeType]].
+    *
+    * @param inputType The TypeInformation to extract the field names and positions from.
+    * @return A tuple of two arrays holding the field names and corresponding field positions.
+    */
+  def getFieldInfo(inputType: TypeInformation[_])
+    : (Array[String], Array[Int], Array[TypeInformation[_]]) = {
+
+    val fieldNames: Array[String] = inputType match {
+      case t: CompositeType[_] => t.getFieldNames
+      case a: AtomicType[_] => Array("f0")
+      case tpe =>
+        throw new TableException(s"Currently only CompositeType and AtomicType are supported. " +
+          s"Type $tpe lacks explicit field naming")
+    }
+    val fieldIndexes = fieldNames.indices.toArray
+    val fieldTypes: Array[TypeInformation[_]] = fieldNames.map { i =>
+      inputType match {
+        case t: CompositeType[_] => t.getTypeAt(i).asInstanceOf[TypeInformation[_]]
+        case a: AtomicType[_] => a.asInstanceOf[TypeInformation[_]]
+        case tpe =>
+          throw new TableException(s"Currently only CompositeType and AtomicType are supported.")
+      }
+    }
+    (fieldNames, fieldIndexes, fieldTypes)
+  }
+
+  /**
+    * Prints one signature consisting of classes.
+    */
+  def signatureToString(signature: Array[Class[_]]): String =
+  signature.map { clazz =>
+    if (clazz == null) {
+      "null"
+    } else {
+      clazz.getCanonicalName
+    }
+  }.mkString("(", ", ", ")")
+
+  /**
+    * Prints one signature consisting of TypeInformation.
+    */
+  def signatureToString(signature: Seq[TypeInformation[_]]): String = {
+    signatureToString(typeInfoToClass(signature))
+  }
+
+  /**
+    * Prints all eval methods signatures of a class.
+    */
+  def signaturesToString(function: UserDefinedFunction): String = {
+    getSignatures(function).map(signatureToString).mkString(", ")
+  }
+
+  /**
+    * Extracts type classes of [[TypeInformation]] in a null-aware way.
+    */
+  private def typeInfoToClass(typeInfos: Seq[TypeInformation[_]]): Array[Class[_]] =
+  typeInfos.map { typeInfo =>
+    if (typeInfo == null) {
+      null
+    } else {
+      typeInfo.getTypeClass
+    }
+  }.toArray
+
+
+  /**
+    * Compares parameter candidate classes with expected classes. If true, the parameters match.
+    * Candidate can be null (acts as a wildcard).
+    */
+  private def parameterTypeEquals(candidate: Class[_], expected: Class[_]): Boolean =
+  candidate == null ||
+    candidate == expected ||
+    expected.isPrimitive && Primitives.wrap(expected) == candidate ||
+    candidate == classOf[Date] && expected == classOf[Int] ||
+    candidate == classOf[Time] && expected == classOf[Int] ||
+    candidate == classOf[Timestamp] && expected == classOf[Long]
+
+}


[07/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/datastream/DataStreamCorrelateITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/datastream/DataStreamCorrelateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/datastream/DataStreamCorrelateITCase.scala
deleted file mode 100644
index 21fe157..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/datastream/DataStreamCorrelateITCase.scala
+++ /dev/null
@@ -1,91 +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.api.table.runtime.datastream
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.stream.utils.StreamITCase
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.utils.TableFunc0
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.types.Row
-import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.junit.Assert._
-import org.junit.Test
-
-import scala.collection.mutable
-
-class DataStreamCorrelateITCase extends StreamingMultipleProgramsTestBase {
-
-  @Test
-  def testCrossJoin(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.clear
-
-    val t = testData(env).toTable(tEnv).as('a, 'b, 'c)
-    val func0 = new TableFunc0
-
-    val result = t
-      .join(func0('c) as('d, 'e))
-      .select('c, 'd, 'e)
-      .toDataStream[Row]
-
-    result.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("Jack#22,Jack,22", "John#19,John,19", "Anna#44,Anna,44")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testLeftOuterJoin(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.clear
-
-    val t = testData(env).toTable(tEnv).as('a, 'b, 'c)
-    val func0 = new TableFunc0
-
-    val result = t
-      .leftOuterJoin(func0('c) as('d, 'e))
-      .select('c, 'd, 'e)
-      .toDataStream[Row]
-
-    result.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList(
-      "nosharp,null,null", "Jack#22,Jack,22",
-      "John#19,John,19", "Anna#44,Anna,44")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  private def testData(
-    env: StreamExecutionEnvironment)
-  : DataStream[(Int, Long, String)] = {
-
-    val data = new mutable.MutableList[(Int, Long, String)]
-    data.+=((1, 1L, "Jack#22"))
-    data.+=((2, 2L, "John#19"))
-    data.+=((3, 2L, "Anna#44"))
-    data.+=((4, 3L, "nosharp"))
-    env.fromCollection(data)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/TableTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/TableTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/TableTestBase.scala
deleted file mode 100644
index b281dfc..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/TableTestBase.scala
+++ /dev/null
@@ -1,229 +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.api.table.utils
-
-import org.apache.calcite.plan.RelOptUtil
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.{DataSet => JDataSet}
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment}
-import org.apache.flink.api.table.expressions.Expression
-import org.apache.flink.api.table.functions.{ScalarFunction, TableFunction}
-import org.apache.flink.api.table.{Table, TableEnvironment}
-import org.apache.flink.streaming.api.datastream.{DataStream => JDataStream}
-import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment}
-import org.junit.Assert.assertEquals
-import org.mockito.Mockito.{mock, when}
-
-/**
-  * Test base for testing Table API / SQL plans.
-  */
-class TableTestBase {
-
-  def batchTestUtil(): BatchTableTestUtil = {
-    BatchTableTestUtil()
-  }
-
-  def streamTestUtil(): StreamTableTestUtil = {
-    StreamTableTestUtil()
-  }
-
-  def verifyTableEquals(expected: Table, actual: Table): Unit = {
-    assertEquals(
-      "Logical plans do not match",
-      RelOptUtil.toString(expected.getRelNode),
-      RelOptUtil.toString(actual.getRelNode))
-  }
-
-}
-
-abstract class TableTestUtil {
-
-  private var counter = 0
-
-  def addTable[T: TypeInformation](fields: Expression*): Table = {
-    addTable[T](s"Table${counter += 1}", fields: _*)
-  }
-
-  def addTable[T: TypeInformation](name: String, fields: Expression*): Table
-  def addFunction[T: TypeInformation](name: String, function: TableFunction[T]): TableFunction[T]
-  def addFunction(name: String, function: ScalarFunction): Unit
-
-  def verifySql(query: String, expected: String): Unit
-  def verifyTable(resultTable: Table, expected: String): Unit
-
-  // the print methods are for debugging purposes only
-  def printTable(resultTable: Table): Unit
-  def printSql(query: String): Unit
-}
-
-object TableTestUtil {
-
-  // this methods are currently just for simplifying string construction,
-  // we could replace it with logic later
-
-  def unaryNode(node: String, input: String, term: String*): String = {
-    s"""$node(${term.mkString(", ")})
-       |$input
-       |""".stripMargin.stripLineEnd
-  }
-
-  def binaryNode(node: String, left: String, right: String, term: String*): String = {
-    s"""$node(${term.mkString(", ")})
-       |$left
-       |$right
-       |""".stripMargin.stripLineEnd
-  }
-
-  def values(node: String, term: String*): String = {
-    s"$node(${term.mkString(", ")})"
-  }
-
-  def term(term: AnyRef, value: AnyRef*): String = {
-    s"$term=[${value.mkString(", ")}]"
-  }
-
-  def tuples(value:List[AnyRef]*): String={
-    val listValues = value.map( listValue => s"{ ${listValue.mkString(", ")} }")
-    term("tuples","[" + listValues.mkString(", ") + "]")
-  }
-
-  def batchTableNode(idx: Int): String = {
-    s"DataSetScan(table=[[_DataSetTable_$idx]])"
-  }
-
-  def streamTableNode(idx: Int): String = {
-    s"DataStreamScan(table=[[_DataStreamTable_$idx]])"
-  }
-
-}
-
-case class BatchTableTestUtil() extends TableTestUtil {
-
-  val env = mock(classOf[ExecutionEnvironment])
-  val tEnv = TableEnvironment.getTableEnvironment(env)
-
-  def addTable[T: TypeInformation](
-      name: String,
-      fields: Expression*)
-    : Table = {
-    val ds = mock(classOf[DataSet[T]])
-    val jDs = mock(classOf[JDataSet[T]])
-    when(ds.javaSet).thenReturn(jDs)
-    val typeInfo: TypeInformation[T] = implicitly[TypeInformation[T]]
-    when(jDs.getType).thenReturn(typeInfo)
-
-    val t = ds.toTable(tEnv, fields: _*)
-    tEnv.registerTable(name, t)
-    t
-  }
-
-  def addFunction[T: TypeInformation](
-      name: String,
-      function: TableFunction[T])
-    : TableFunction[T] = {
-    tEnv.registerFunction(name, function)
-    function
-  }
-
-  def addFunction(name: String, function: ScalarFunction): Unit = {
-    tEnv.registerFunction(name, function)
-  }
-
-  def verifySql(query: String, expected: String): Unit = {
-    verifyTable(tEnv.sql(query), expected)
-  }
-
-  def verifyTable(resultTable: Table, expected: String): Unit = {
-    val relNode = resultTable.getRelNode
-    val optimized = tEnv.optimize(relNode)
-    val actual = RelOptUtil.toString(optimized)
-    assertEquals(
-      expected.split("\n").map(_.trim).mkString("\n"),
-      actual.split("\n").map(_.trim).mkString("\n"))
-  }
-
-  def printTable(resultTable: Table): Unit = {
-    val relNode = resultTable.getRelNode
-    val optimized = tEnv.optimize(relNode)
-    println(RelOptUtil.toString(optimized))
-  }
-
-  def printSql(query: String): Unit = {
-    printTable(tEnv.sql(query))
-  }
-}
-
-case class StreamTableTestUtil() extends TableTestUtil {
-
-  val env = mock(classOf[StreamExecutionEnvironment])
-  val tEnv = TableEnvironment.getTableEnvironment(env)
-
-  def addTable[T: TypeInformation](
-      name: String,
-      fields: Expression*)
-    : Table = {
-
-    val ds = mock(classOf[DataStream[T]])
-    val jDs = mock(classOf[JDataStream[T]])
-    when(ds.javaStream).thenReturn(jDs)
-    val typeInfo: TypeInformation[T] = implicitly[TypeInformation[T]]
-    when(jDs.getType).thenReturn(typeInfo)
-
-    val t = ds.toTable(tEnv, fields: _*)
-    tEnv.registerTable(name, t)
-    t
-  }
-
-  def addFunction[T: TypeInformation](
-      name: String,
-      function: TableFunction[T])
-    : TableFunction[T] = {
-    tEnv.registerFunction(name, function)
-    function
-  }
-
-  def addFunction(name: String, function: ScalarFunction): Unit = {
-    tEnv.registerFunction(name, function)
-  }
-
-  def verifySql(query: String, expected: String): Unit = {
-    verifyTable(tEnv.sql(query), expected)
-  }
-
-  def verifyTable(resultTable: Table, expected: String): Unit = {
-    val relNode = resultTable.getRelNode
-    val optimized = tEnv.optimize(relNode)
-    val actual = RelOptUtil.toString(optimized)
-    assertEquals(
-      expected.split("\n").map(_.trim).mkString("\n"),
-      actual.split("\n").map(_.trim).mkString("\n"))
-  }
-
-  // the print methods are for debugging purposes only
-  def printTable(resultTable: Table): Unit = {
-    val relNode = resultTable.getRelNode
-    val optimized = tEnv.optimize(relNode)
-    println(RelOptUtil.toString(optimized))
-  }
-
-  def printSql(query: String): Unit = {
-    printTable(tEnv.sql(query))
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/UserDefinedTableFunctions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/UserDefinedTableFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/UserDefinedTableFunctions.scala
deleted file mode 100644
index 4291b29..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/utils/UserDefinedTableFunctions.scala
+++ /dev/null
@@ -1,117 +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.api.table.utils
-
-import java.lang.Boolean
-
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.java.tuple.Tuple3
-import org.apache.flink.types.Row
-import org.apache.flink.api.table.functions.TableFunction
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-
-
-case class SimpleUser(name: String, age: Int)
-
-class TableFunc0 extends TableFunction[SimpleUser] {
-  // make sure input element's format is "<string>#<int>"
-  def eval(user: String): Unit = {
-    if (user.contains("#")) {
-      val splits = user.split("#")
-      collect(SimpleUser(splits(0), splits(1).toInt))
-    }
-  }
-}
-
-class TableFunc1 extends TableFunction[String] {
-  def eval(str: String): Unit = {
-    if (str.contains("#")){
-      str.split("#").foreach(collect)
-    }
-  }
-
-  def eval(str: String, prefix: String): Unit = {
-    if (str.contains("#")) {
-      str.split("#").foreach(s => collect(prefix + s))
-    }
-  }
-}
-
-
-class TableFunc2 extends TableFunction[Row] {
-  def eval(str: String): Unit = {
-    if (str.contains("#")) {
-      str.split("#").foreach({ s =>
-        val row = new Row(2)
-        row.setField(0, s)
-        row.setField(1, s.length)
-        collect(row)
-      })
-    }
-  }
-
-  override def getResultType: TypeInformation[Row] = {
-    new RowTypeInfo(BasicTypeInfo.STRING_TYPE_INFO,
-                    BasicTypeInfo.INT_TYPE_INFO)
-  }
-}
-
-class HierarchyTableFunction extends SplittableTableFunction[Boolean, Integer] {
-  def eval(user: String) {
-    if (user.contains("#")) {
-      val splits = user.split("#")
-      val age = splits(1).toInt
-      collect(new Tuple3[String, Boolean, Integer](splits(0), age >= 20, age))
-    }
-  }
-}
-
-abstract class SplittableTableFunction[A, B] extends TableFunction[Tuple3[String, A, B]] {}
-
-class PojoTableFunc extends TableFunction[PojoUser] {
-  def eval(user: String) {
-    if (user.contains("#")) {
-      val splits = user.split("#")
-      collect(new PojoUser(splits(0), splits(1).toInt))
-    }
-  }
-}
-
-class PojoUser() {
-  var name: String = _
-  var age: Int = 0
-
-  def this(name: String, age: Int) {
-    this()
-    this.name = name
-    this.age = age
-  }
-}
-
-// ----------------------------------------------------------------------------------------------
-// Invalid Table Functions
-// ----------------------------------------------------------------------------------------------
-
-
-// this is used to check whether scala object is forbidden
-object ObjectTableFunction extends TableFunction[Integer] {
-  def eval(a: Int, b: Int): Unit = {
-    collect(a)
-    collect(b)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/AggregationTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/AggregationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/AggregationTest.scala
new file mode 100644
index 0000000..708e007
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/AggregationTest.scala
@@ -0,0 +1,261 @@
+/*
+ * 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
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+/**
+  * Test for testing aggregate plans.
+  */
+class AggregationTest extends TableTestBase {
+
+  @Test
+  def testAggregateQueryBatchSQL(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM MyTable"
+
+    val setValues = unaryNode(
+      "DataSetValues",
+      batchTableNode(0),
+      tuples(List(null,null,null)),
+      term("values","a","b","c")
+    )
+    val union = unaryNode(
+      "DataSetUnion",
+      setValues,
+      term("union","a","b","c")
+    )
+
+    val aggregate = unaryNode(
+      "DataSetAggregate",
+      union,
+      term("select",
+        "AVG(a) AS EXPR$0",
+        "SUM(b) AS EXPR$1",
+        "COUNT(c) AS EXPR$2")
+    )
+    util.verifySql(sqlQuery, aggregate)
+  }
+
+  @Test
+  def testAggregateWithFilterQueryBatchSQL(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM MyTable WHERE a = 1"
+
+    val calcNode = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select", "a", "b", "c"),
+      term("where", "=(a, 1)")
+    )
+
+    val setValues =  unaryNode(
+        "DataSetValues",
+        calcNode,
+        tuples(List(null,null,null)),
+        term("values","a","b","c")
+    )
+
+    val union = unaryNode(
+      "DataSetUnion",
+      setValues,
+      term("union","a","b","c")
+    )
+
+    val aggregate = unaryNode(
+      "DataSetAggregate",
+      union,
+      term("select",
+        "AVG(a) AS EXPR$0",
+        "SUM(b) AS EXPR$1",
+        "COUNT(c) AS EXPR$2")
+    )
+    util.verifySql(sqlQuery, aggregate)
+  }
+
+  @Test
+  def testAggregateGroupQueryBatchSQL(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM MyTable GROUP BY a"
+
+    val aggregate = unaryNode(
+        "DataSetAggregate",
+        batchTableNode(0),
+        term("groupBy", "a"),
+        term("select",
+          "a",
+          "AVG(a) AS EXPR$0",
+          "SUM(b) AS EXPR$1",
+          "COUNT(c) AS EXPR$2")
+    )
+    val expected = unaryNode(
+        "DataSetCalc",
+        aggregate,
+        term("select",
+          "EXPR$0",
+          "EXPR$1",
+          "EXPR$2")
+    )
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testAggregateGroupWithFilterQueryBatchSQL(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM MyTable WHERE a = 1 GROUP BY a"
+
+    val calcNode = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select","a", "b", "c") ,
+      term("where","=(a, 1)")
+    )
+
+    val aggregate = unaryNode(
+        "DataSetAggregate",
+        calcNode,
+        term("groupBy", "a"),
+        term("select",
+          "a",
+          "AVG(a) AS EXPR$0",
+          "SUM(b) AS EXPR$1",
+          "COUNT(c) AS EXPR$2")
+    )
+    val expected = unaryNode(
+        "DataSetCalc",
+        aggregate,
+        term("select",
+          "EXPR$0",
+          "EXPR$1",
+          "EXPR$2")
+    )
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testAggregateGroupWithFilterTableApi(): Unit = {
+
+    val util = batchTestUtil()
+    val sourceTable = util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
+
+    val resultTable = sourceTable.groupBy('a)
+      .select('a, 'a.avg, 'b.sum, 'c.count)
+      .where('a === 1)
+
+    val calcNode = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select", "a", "b", "c"),
+      term("where", "=(a, 1)")
+    )
+
+    val expected = unaryNode(
+      "DataSetAggregate",
+      calcNode,
+      term("groupBy", "a"),
+      term("select",
+        "a",
+        "AVG(a) AS TMP_0",
+        "SUM(b) AS TMP_1",
+        "COUNT(c) AS TMP_2")
+    )
+
+    util.verifyTable(resultTable,expected)
+  }
+
+  @Test
+  def testAggregateTableApi(): Unit = {
+    val util = batchTestUtil()
+    val sourceTable = util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
+    val resultTable = sourceTable.select('a.avg,'b.sum,'c.count)
+
+    val setValues = unaryNode(
+      "DataSetValues",
+      batchTableNode(0),
+      tuples(List(null,null,null)),
+      term("values","a","b","c")
+    )
+    val union = unaryNode(
+      "DataSetUnion",
+      setValues,
+      term("union","a","b","c")
+    )
+
+    val expected = unaryNode(
+      "DataSetAggregate",
+      union,
+      term("select",
+        "AVG(a) AS TMP_0",
+        "SUM(b) AS TMP_1",
+        "COUNT(c) AS TMP_2")
+    )
+    util.verifyTable(resultTable, expected)
+  }
+
+  @Test
+  def testAggregateWithFilterTableApi(): Unit = {
+    val util = batchTestUtil()
+    val sourceTable = util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
+
+    val resultTable = sourceTable.select('a,'b,'c).where('a === 1)
+      .select('a.avg,'b.sum,'c.count)
+
+    val calcNode = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select", "a", "b", "c"),
+      term("where", "=(a, 1)")
+    )
+
+    val setValues =  unaryNode(
+      "DataSetValues",
+      calcNode,
+      tuples(List(null,null,null)),
+      term("values","a","b","c")
+    )
+
+    val union = unaryNode(
+      "DataSetUnion",
+      setValues,
+      term("union","a","b","c")
+    )
+
+    val expected = unaryNode(
+      "DataSetAggregate",
+      union,
+      term("select",
+        "AVG(a) AS TMP_0",
+        "SUM(b) AS TMP_1",
+        "COUNT(c) AS TMP_2")
+    )
+
+    util.verifyTable(resultTable, expected)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
new file mode 100644
index 0000000..09a4c4e
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CalciteConfigBuilderTest.scala
@@ -0,0 +1,209 @@
+/*
+ * 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
+
+import org.apache.calcite.rel.rules.{CalcSplitRule, CalcMergeRule, FilterMergeRule}
+import org.apache.calcite.sql.fun.{SqlStdOperatorTable, OracleSqlOperatorTable}
+import org.apache.calcite.tools.RuleSets
+import org.apache.flink.table.calcite.{CalciteConfigBuilder, CalciteConfig}
+import org.junit.Test
+import org.junit.Assert._
+
+import scala.collection.JavaConverters._
+
+class CalciteConfigBuilderTest {
+
+  @Test
+  def testDefaultRules(): Unit = {
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .build()
+
+    assertEquals(false, cc.replacesRuleSet)
+    assertFalse(cc.getRuleSet.isDefined)
+  }
+
+  @Test
+  def testReplaceRules(): Unit = {
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .replaceRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
+      .build()
+
+    assertEquals(true, cc.replacesRuleSet)
+    assertTrue(cc.getRuleSet.isDefined)
+    val cSet = cc.getRuleSet.get.iterator().asScala.toSet
+    assertEquals(1, cSet.size)
+    assertTrue(cSet.contains(FilterMergeRule.INSTANCE))
+  }
+
+  @Test
+  def testReplaceAddRules(): Unit = {
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .replaceRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
+      .addRuleSet(RuleSets.ofList(CalcMergeRule.INSTANCE, CalcSplitRule.INSTANCE))
+      .build()
+
+    assertEquals(true, cc.replacesRuleSet)
+    assertTrue(cc.getRuleSet.isDefined)
+    val cSet = cc.getRuleSet.get.iterator().asScala.toSet
+    assertEquals(3, cSet.size)
+    assertTrue(cSet.contains(FilterMergeRule.INSTANCE))
+    assertTrue(cSet.contains(CalcMergeRule.INSTANCE))
+    assertTrue(cSet.contains(CalcSplitRule.INSTANCE))
+  }
+
+  @Test
+  def testAddRules(): Unit = {
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .addRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
+      .build()
+
+    assertEquals(false, cc.replacesRuleSet)
+    assertTrue(cc.getRuleSet.isDefined)
+    val cSet = cc.getRuleSet.get.iterator().asScala.toSet
+    assertEquals(1, cSet.size)
+    assertTrue(cSet.contains(FilterMergeRule.INSTANCE))
+  }
+
+  @Test
+  def testAddAddRules(): Unit = {
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .addRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
+      .addRuleSet(RuleSets.ofList(CalcMergeRule.INSTANCE, CalcSplitRule.INSTANCE))
+      .build()
+
+    assertEquals(false, cc.replacesRuleSet)
+    assertTrue(cc.getRuleSet.isDefined)
+    val cSet = cc.getRuleSet.get.iterator().asScala.toSet
+    assertEquals(3, cSet.size)
+    assertTrue(cSet.contains(FilterMergeRule.INSTANCE))
+    assertTrue(cSet.contains(CalcMergeRule.INSTANCE))
+    assertTrue(cSet.contains(CalcSplitRule.INSTANCE))
+  }
+
+  @Test
+  def testDefaultOperatorTable(): Unit = {
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .build()
+
+    assertEquals(false, cc.replacesSqlOperatorTable)
+    assertFalse(cc.getSqlOperatorTable.isDefined)
+  }
+
+  def testReplaceOperatorTable(): Unit = {
+
+    val oracleTable = new OracleSqlOperatorTable
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .replaceSqlOperatorTable(oracleTable)
+      .build()
+
+    val oracleOps = oracleTable.getOperatorList.asScala
+
+    assertEquals(true, cc.replacesSqlOperatorTable)
+    assertTrue(cc.getSqlOperatorTable.isDefined)
+    val ops = cc.getSqlOperatorTable.get.getOperatorList
+      .asScala.toSet
+    assertEquals(oracleOps.size, ops.size)
+    for (o <- oracleOps) {
+      assertTrue(ops.contains(o))
+    }
+  }
+
+  def testReplaceAddOperatorTable(): Unit = {
+
+    val oracleTable = new OracleSqlOperatorTable
+    val stdTable = new SqlStdOperatorTable
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .replaceSqlOperatorTable(oracleTable)
+      .addSqlOperatorTable(stdTable)
+      .build()
+
+    val oracleOps = oracleTable.getOperatorList.asScala
+    val stdOps = stdTable.getOperatorList.asScala
+
+    assertEquals(true, cc.replacesSqlOperatorTable)
+    assertTrue(cc.getSqlOperatorTable.isDefined)
+    val ops = cc.getSqlOperatorTable.get.getOperatorList
+      .asScala.toSet
+    assertEquals(oracleOps.size + stdOps.size, ops.size)
+    for (o <- oracleOps) {
+      assertTrue(ops.contains(o))
+    }
+    for (o <- stdOps) {
+      assertTrue(ops.contains(o))
+    }
+
+  }
+
+  def testAddOperatorTable(): Unit = {
+
+    val oracleTable = new OracleSqlOperatorTable
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .addSqlOperatorTable(oracleTable)
+      .build()
+
+    val oracleOps = oracleTable.getOperatorList.asScala
+
+    assertEquals(false, cc.replacesSqlOperatorTable)
+    assertTrue(cc.getSqlOperatorTable.isDefined)
+    val ops = cc.getSqlOperatorTable.get.getOperatorList
+      .asScala.toSet
+    assertEquals(oracleOps.size, ops.size)
+    for (o <- oracleOps) {
+      assertTrue(ops.contains(o))
+    }
+  }
+
+  def testAddAddOperatorTable(): Unit = {
+
+    val oracleTable = new OracleSqlOperatorTable
+    val stdTable = new SqlStdOperatorTable
+
+    val cc: CalciteConfig = new CalciteConfigBuilder()
+      .addSqlOperatorTable(oracleTable)
+      .addSqlOperatorTable(stdTable)
+      .build()
+
+    val oracleOps = oracleTable.getOperatorList.asScala
+    val stdOps = stdTable.getOperatorList.asScala
+
+    assertEquals(false, cc.replacesSqlOperatorTable)
+    assertTrue(cc.getSqlOperatorTable.isDefined)
+    val ops = cc.getSqlOperatorTable.get.getOperatorList
+      .asScala.toSet
+    assertEquals(oracleOps.size + stdOps.size, ops.size)
+    for (o <- oracleOps) {
+      assertTrue(ops.contains(o))
+    }
+    for (o <- stdOps) {
+      assertTrue(ops.contains(o))
+    }
+
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CompositeFlatteningTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CompositeFlatteningTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CompositeFlatteningTest.scala
new file mode 100644
index 0000000..0055fc2
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/CompositeFlatteningTest.scala
@@ -0,0 +1,147 @@
+/*
+ * 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
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.scala.createTypeInformation
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.CompositeFlatteningTest.{TestCaseClass, giveMeCaseClass}
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+
+class CompositeFlatteningTest extends TableTestBase {
+
+  @Test(expected = classOf[ValidationException])
+  def testDuplicateFlattening(): Unit = {
+    val util = batchTestUtil()
+    val table = util.addTable[((Int, Long), (String, Boolean), String)]("MyTable", 'a, 'b, 'c)
+
+    table.select('a.flatten(), 'a.flatten())
+  }
+
+  @Test
+  def testMultipleFlatteningsTable(): Unit = {
+    val util = batchTestUtil()
+    val table = util.addTable[((Int, Long), (String, Boolean), String)]("MyTable", 'a, 'b, 'c)
+
+    val result = table.select('a.flatten(), 'c, 'b.flatten())
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select",
+        "a._1 AS a$_1",
+        "a._2 AS a$_2",
+        "c",
+        "b._1 AS b$_1",
+        "b._2 AS b$_2"
+      )
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testMultipleFlatteningsSql(): Unit = {
+    val util = batchTestUtil()
+    val table = util.addTable[((Int, Long), (String, Boolean), String)]("MyTable", 'a, 'b, 'c)
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select",
+        "a._1 AS _1",
+        "a._2 AS _2",
+        "c",
+        "b._1 AS _10",
+        "b._2 AS _20"
+      )
+    )
+
+    util.verifySql(
+      "SELECT MyTable.a.*, c, MyTable.b.* FROM MyTable",
+      expected)
+  }
+
+  @Test
+  def testNestedFlattenings(): Unit = {
+    val util = batchTestUtil()
+    val table = util
+      .addTable[((((String, TestCaseClass), Boolean), String), String)]("MyTable", 'a, 'b)
+
+    val result = table.select('a.flatten(), 'b.flatten())
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select",
+        "a._1 AS a$_1",
+        "a._2 AS a$_2",
+        "b"
+      )
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testScalarFunctionAccess(): Unit = {
+    val util = batchTestUtil()
+    val table = util
+      .addTable[(String, Int)]("MyTable", 'a, 'b)
+
+    val result = table.select(
+      giveMeCaseClass().get("my"),
+      giveMeCaseClass().get("clazz"),
+      giveMeCaseClass().flatten())
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select",
+        "org.apache.flink.table.CompositeFlatteningTest.giveMeCaseClass$().my AS _c0",
+        "org.apache.flink.table.CompositeFlatteningTest.giveMeCaseClass$().clazz AS _c1",
+        "org.apache.flink.table.CompositeFlatteningTest.giveMeCaseClass$().my AS _c2",
+        "org.apache.flink.table.CompositeFlatteningTest.giveMeCaseClass$().clazz AS _c3"
+      )
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+}
+
+object CompositeFlatteningTest {
+
+  case class TestCaseClass(my: String, clazz: Int)
+
+  object giveMeCaseClass extends ScalarFunction {
+    def eval(): TestCaseClass = {
+      TestCaseClass("hello", 42)
+    }
+
+    override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = {
+      createTypeInformation[TestCaseClass]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/ExpressionReductionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/ExpressionReductionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/ExpressionReductionTest.scala
new file mode 100644
index 0000000..b660243
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/ExpressionReductionTest.scala
@@ -0,0 +1,425 @@
+/*
+ * 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
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+class ExpressionReductionTest extends TableTestBase {
+
+  @Test
+  def testReduceCalcExpressionForBatchSQL(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT " +
+      "(3+4)+a, " +
+      "b+(1+2), " +
+      "CASE 11 WHEN 1 THEN 'a' ELSE 'b' END, " +
+      "TRIM(BOTH ' STRING '),  " +
+      "'test' || 'string', " +
+      "NULLIF(1, 1), " +
+      "TIMESTAMP '1990-10-14 23:00:00.123' + INTERVAL '10 00:00:01' DAY TO SECOND, " +
+      "EXTRACT(DAY FROM INTERVAL '19 12:10:10.123' DAY TO SECOND(3)),  " +
+      "1 IS NULL, " +
+      "'TEST' LIKE '%EST', " +
+      "FLOOR(2.5), " +
+      "'TEST' IN ('west', 'TEST', 'rest'), " +
+      "CAST(TRUE AS VARCHAR) || 'X'" +
+      "FROM MyTable WHERE a>(1+7)"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select",
+        "+(7, a) AS EXPR$0",
+        "+(b, 3) AS EXPR$1",
+        "'b' AS EXPR$2",
+        "'STRING' AS EXPR$3",
+        "'teststring' AS EXPR$4",
+        "null AS EXPR$5",
+        "1990-10-24 23:00:01 AS EXPR$6",
+        "19 AS EXPR$7",
+        "false AS EXPR$8",
+        "true AS EXPR$9",
+        "2 AS EXPR$10",
+        "true AS EXPR$11",
+        "'trueX' AS EXPR$12"
+      ),
+      term("where", ">(a, 8)")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testReduceProjectExpressionForBatchSQL(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT " +
+      "(3+4)+a, " +
+      "b+(1+2), " +
+      "CASE 11 WHEN 1 THEN 'a' ELSE 'b' END, " +
+      "TRIM(BOTH ' STRING '),  " +
+      "'test' || 'string', " +
+      "NULLIF(1, 1), " +
+      "TIMESTAMP '1990-10-14 23:00:00.123' + INTERVAL '10 00:00:01' DAY TO SECOND, " +
+      "EXTRACT(DAY FROM INTERVAL '19 12:10:10.123' DAY TO SECOND(3)),  " +
+      "1 IS NULL, " +
+      "'TEST' LIKE '%EST', " +
+      "FLOOR(2.5), " +
+      "'TEST' IN ('west', 'TEST', 'rest'), " +
+      "CAST(TRUE AS VARCHAR) || 'X'" +
+      "FROM MyTable"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select",
+        "+(7, a) AS EXPR$0",
+        "+(b, 3) AS EXPR$1",
+        "'b' AS EXPR$2",
+        "'STRING' AS EXPR$3",
+        "'teststring' AS EXPR$4",
+        "null AS EXPR$5",
+        "1990-10-24 23:00:01 AS EXPR$6",
+        "19 AS EXPR$7",
+        "false AS EXPR$8",
+        "true AS EXPR$9",
+        "2 AS EXPR$10",
+        "true AS EXPR$11",
+        "'trueX' AS EXPR$12"
+      )
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testReduceFilterExpressionForBatchSQL(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT " +
+      "*" +
+      "FROM MyTable WHERE a>(1+7)"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select", "a", "b", "c"),
+      term("where", ">(a, 8)")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testReduceCalcExpressionForBatchTableAPI(): Unit = {
+    val util = batchTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val result = table
+      .where('a > (1 + 7))
+      .select((3 + 4).toExpr + 6,
+              (11 === 1) ? ("a", "b"),
+              " STRING ".trim,
+              "test" + "string",
+              "1990-10-14 23:00:00.123".toTimestamp + 10.days + 1.second,
+              1.isNull,
+              "TEST".like("%EST"),
+              2.5.toExpr.floor(),
+              true.cast(Types.STRING) + "X")
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select",
+        "13 AS _c0",
+        "'b' AS _c1",
+        "'STRING' AS _c2",
+        "'teststring' AS _c3",
+        "1990-10-24 23:00:01 AS _c4",
+        "false AS _c5",
+        "true AS _c6",
+        "2E0 AS _c7",
+        "'trueX' AS _c8"
+      ),
+      term("where", ">(a, 8)")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testReduceProjectExpressionForBatchTableAPI(): Unit = {
+    val util = batchTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val result = table
+      .select((3 + 4).toExpr + 6,
+              (11 === 1) ? ("a", "b"),
+              " STRING ".trim,
+              "test" + "string",
+              "1990-10-14 23:00:00.123".toTimestamp + 10.days + 1.second,
+              1.isNull,
+              "TEST".like("%EST"),
+              2.5.toExpr.floor(),
+              true.cast(Types.STRING) + "X")
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select",
+        "13 AS _c0",
+        "'b' AS _c1",
+        "'STRING' AS _c2",
+        "'teststring' AS _c3",
+        "1990-10-24 23:00:01 AS _c4",
+        "false AS _c5",
+        "true AS _c6",
+        "2E0 AS _c7",
+        "'trueX' AS _c8"
+      )
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testReduceFilterExpressionForBatchTableAPI(): Unit = {
+    val util = batchTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val result = table
+      .where('a > (1 + 7))
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      batchTableNode(0),
+      term("select", "a", "b", "c"),
+      term("where", ">(a, 8)")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testReduceCalcExpressionForStreamSQL(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT " +
+      "(3+4)+a, " +
+      "b+(1+2), " +
+      "CASE 11 WHEN 1 THEN 'a' ELSE 'b' END, " +
+      "TRIM(BOTH ' STRING '),  " +
+      "'test' || 'string', " +
+      "NULLIF(1, 1), " +
+      "TIMESTAMP '1990-10-14 23:00:00.123' + INTERVAL '10 00:00:01' DAY TO SECOND, " +
+      "EXTRACT(DAY FROM INTERVAL '19 12:10:10.123' DAY TO SECOND(3)),  " +
+      "1 IS NULL, " +
+      "'TEST' LIKE '%EST', " +
+      "FLOOR(2.5), " +
+      "'TEST' IN ('west', 'TEST', 'rest'), " +
+      "CAST(TRUE AS VARCHAR) || 'X'" +
+      "FROM MyTable WHERE a>(1+7)"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      streamTableNode(0),
+      term("select",
+        "+(7, a) AS EXPR$0",
+        "+(b, 3) AS EXPR$1",
+        "'b' AS EXPR$2",
+        "'STRING' AS EXPR$3",
+        "'teststring' AS EXPR$4",
+        "null AS EXPR$5",
+        "1990-10-24 23:00:01 AS EXPR$6",
+        "19 AS EXPR$7",
+        "false AS EXPR$8",
+        "true AS EXPR$9",
+        "2 AS EXPR$10",
+        "true AS EXPR$11",
+        "'trueX' AS EXPR$12"
+      ),
+      term("where", ">(a, 8)")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testReduceProjectExpressionForStreamSQL(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT " +
+      "(3+4)+a, " +
+      "b+(1+2), " +
+      "CASE 11 WHEN 1 THEN 'a' ELSE 'b' END, " +
+      "TRIM(BOTH ' STRING '),  " +
+      "'test' || 'string', " +
+      "NULLIF(1, 1), " +
+      "TIMESTAMP '1990-10-14 23:00:00.123' + INTERVAL '10 00:00:01' DAY TO SECOND, " +
+      "EXTRACT(DAY FROM INTERVAL '19 12:10:10.123' DAY TO SECOND(3)),  " +
+      "1 IS NULL, " +
+      "'TEST' LIKE '%EST', " +
+      "FLOOR(2.5), " +
+      "'TEST' IN ('west', 'TEST', 'rest'), " +
+      "CAST(TRUE AS VARCHAR) || 'X'" +
+      "FROM MyTable"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      streamTableNode(0),
+      term("select",
+        "+(7, a) AS EXPR$0",
+        "+(b, 3) AS EXPR$1",
+        "'b' AS EXPR$2",
+        "'STRING' AS EXPR$3",
+        "'teststring' AS EXPR$4",
+        "null AS EXPR$5",
+        "1990-10-24 23:00:01 AS EXPR$6",
+        "19 AS EXPR$7",
+        "false AS EXPR$8",
+        "true AS EXPR$9",
+        "2 AS EXPR$10",
+        "true AS EXPR$11",
+        "'trueX' AS EXPR$12"
+      )
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testReduceFilterExpressionForStreamSQL(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT " +
+      "*" +
+      "FROM MyTable WHERE a>(1+7)"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      streamTableNode(0),
+      term("select", "a", "b", "c"),
+      term("where", ">(a, 8)")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testReduceCalcExpressionForStreamTableAPI(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val result = table
+      .where('a > (1 + 7))
+      .select((3 + 4).toExpr + 6,
+              (11 === 1) ? ("a", "b"),
+              " STRING ".trim,
+              "test" + "string",
+              "1990-10-14 23:00:00.123".toTimestamp + 10.days + 1.second,
+              1.isNull,
+              "TEST".like("%EST"),
+              2.5.toExpr.floor(),
+              true.cast(Types.STRING) + "X")
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      streamTableNode(0),
+      term("select",
+        "13 AS _c0",
+        "'b' AS _c1",
+        "'STRING' AS _c2",
+        "'teststring' AS _c3",
+        "1990-10-24 23:00:01 AS _c4",
+        "false AS _c5",
+        "true AS _c6",
+        "2E0 AS _c7",
+        "'trueX' AS _c8"
+      ),
+      term("where", ">(a, 8)")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testReduceProjectExpressionForStreamTableAPI(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val result =  table
+      .select((3 + 4).toExpr + 6,
+              (11 === 1) ? ("a", "b"),
+              " STRING ".trim,
+              "test" + "string",
+              "1990-10-14 23:00:00.123".toTimestamp + 10.days + 1.second,
+              1.isNull,
+              "TEST".like("%EST"),
+              2.5.toExpr.floor(),
+              true.cast(Types.STRING) + "X")
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      streamTableNode(0),
+      term("select",
+        "13 AS _c0",
+        "'b' AS _c1",
+        "'STRING' AS _c2",
+        "'teststring' AS _c3",
+        "1990-10-24 23:00:01 AS _c4",
+        "false AS _c5",
+        "true AS _c6",
+        "2E0 AS _c7",
+        "'trueX' AS _c8"
+      )
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testReduceFilterExpressionForStreamTableAPI(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+    val result = table
+      .where('a > (1 + 7))
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      streamTableNode(0),
+      term("select", "a", "b", "c"),
+      term("where", ">(a, 8)")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
new file mode 100644
index 0000000..b90de97
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/TableEnvironmentTest.scala
@@ -0,0 +1,293 @@
+/*
+ * 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
+
+import org.apache.calcite.tools.RuleSet
+import org.apache.flink.api.scala._
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor}
+import org.apache.flink.table.api.{Table, TableConfig, TableEnvironment, TableException}
+import org.apache.flink.table.expressions.{Alias, UnresolvedFieldReference}
+import org.apache.flink.table.sinks.TableSink
+import org.junit.Test
+import org.junit.Assert.assertEquals
+
+class TableEnvironmentTest {
+
+  val tEnv = new MockTableEnvironment
+
+  val tupleType = new TupleTypeInfo(
+    INT_TYPE_INFO,
+    STRING_TYPE_INFO,
+    DOUBLE_TYPE_INFO)
+
+  val caseClassType = implicitly[TypeInformation[CClass]]
+
+  val pojoType = TypeExtractor.createTypeInfo(classOf[PojoClass])
+
+  val atomicType = INT_TYPE_INFO
+
+  @Test
+  def testGetFieldInfoTuple(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(tupleType)
+
+    fieldInfo._1.zip(Array("f0", "f1", "f2")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test
+  def testGetFieldInfoCClass(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(caseClassType)
+
+    fieldInfo._1.zip(Array("cf1", "cf2", "cf3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test
+  def testGetFieldInfoPojo(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(pojoType)
+
+    fieldInfo._1.zip(Array("pf1", "pf2", "pf3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test(expected = classOf[TableException])
+  def testGetFieldInfoAtomic(): Unit = {
+    tEnv.getFieldInfo(atomicType)
+  }
+
+  @Test
+  def testGetFieldInfoTupleNames(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      tupleType,
+      Array(
+        new UnresolvedFieldReference("name1"),
+        new UnresolvedFieldReference("name2"),
+        new UnresolvedFieldReference("name3")
+    ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test
+  def testGetFieldInfoCClassNames(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      caseClassType,
+      Array(
+        new UnresolvedFieldReference("name1"),
+        new UnresolvedFieldReference("name2"),
+        new UnresolvedFieldReference("name3")
+    ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test(expected = classOf[TableException])
+  def testGetFieldInfoPojoNames1(): Unit = {
+    tEnv.getFieldInfo(
+      pojoType,
+      Array(
+        new UnresolvedFieldReference("name1"),
+        new UnresolvedFieldReference("name2"),
+        new UnresolvedFieldReference("name3")
+      ))
+  }
+
+  @Test
+  def testGetFieldInfoPojoNames2(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      pojoType,
+      Array(
+        new UnresolvedFieldReference("pf3"),
+        new UnresolvedFieldReference("pf1"),
+        new UnresolvedFieldReference("pf2")
+      ))
+
+    fieldInfo._1.zip(Array("pf3", "pf1", "pf2")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test
+  def testGetFieldInfoAtomicName1(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      atomicType,
+      Array(new UnresolvedFieldReference("name"))
+    )
+
+    fieldInfo._1.zip(Array("name")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test(expected = classOf[TableException])
+  def testGetFieldInfoAtomicName2(): Unit = {
+    tEnv.getFieldInfo(
+      atomicType,
+      Array(
+        new UnresolvedFieldReference("name1"),
+        new UnresolvedFieldReference("name2")
+      ))
+  }
+
+  @Test
+  def testGetFieldInfoTupleAlias1(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      tupleType,
+      Array(
+        new Alias(UnresolvedFieldReference("f0"), "name1"),
+        new Alias(UnresolvedFieldReference("f1"), "name2"),
+        new Alias(UnresolvedFieldReference("f2"), "name3")
+      ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test
+  def testGetFieldInfoTupleAlias2(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      tupleType,
+      Array(
+        new Alias(UnresolvedFieldReference("f2"), "name1"),
+        new Alias(UnresolvedFieldReference("f0"), "name2"),
+        new Alias(UnresolvedFieldReference("f1"), "name3")
+      ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test(expected = classOf[TableException])
+  def testGetFieldInfoTupleAlias3(): Unit = {
+    tEnv.getFieldInfo(
+      tupleType,
+      Array(
+        new Alias(UnresolvedFieldReference("xxx"), "name1"),
+        new Alias(UnresolvedFieldReference("yyy"), "name2"),
+        new Alias(UnresolvedFieldReference("zzz"), "name3")
+      ))
+  }
+
+  @Test
+  def testGetFieldInfoCClassAlias1(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      caseClassType,
+      Array(
+        new Alias(new UnresolvedFieldReference("cf1"), "name1"),
+        new Alias(new UnresolvedFieldReference("cf2"), "name2"),
+        new Alias(new UnresolvedFieldReference("cf3"), "name3")
+      ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test
+  def testGetFieldInfoCClassAlias2(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      caseClassType,
+      Array(
+        new Alias(new UnresolvedFieldReference("cf3"), "name1"),
+        new Alias(new UnresolvedFieldReference("cf1"), "name2"),
+        new Alias(new UnresolvedFieldReference("cf2"), "name3")
+      ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test(expected = classOf[TableException])
+  def testGetFieldInfoCClassAlias3(): Unit = {
+    tEnv.getFieldInfo(
+      caseClassType,
+      Array(
+        new Alias(new UnresolvedFieldReference("xxx"), "name1"),
+        new Alias(new UnresolvedFieldReference("yyy"), "name2"),
+        new Alias(new UnresolvedFieldReference("zzz"), "name3")
+      ))
+  }
+
+  @Test
+  def testGetFieldInfoPojoAlias1(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      pojoType,
+      Array(
+        new Alias(new UnresolvedFieldReference("pf1"), "name1"),
+        new Alias(new UnresolvedFieldReference("pf2"), "name2"),
+        new Alias(new UnresolvedFieldReference("pf3"), "name3")
+      ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(0, 1, 2)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test
+  def testGetFieldInfoPojoAlias2(): Unit = {
+    val fieldInfo = tEnv.getFieldInfo(
+      pojoType,
+      Array(
+        new Alias(new UnresolvedFieldReference("pf3"), "name1"),
+        new Alias(new UnresolvedFieldReference("pf1"), "name2"),
+        new Alias(new UnresolvedFieldReference("pf2"), "name3")
+      ))
+
+    fieldInfo._1.zip(Array("name1", "name2", "name3")).foreach(x => assertEquals(x._2, x._1))
+    fieldInfo._2.zip(Array(2, 0, 1)).foreach(x => assertEquals(x._2, x._1))
+  }
+
+  @Test(expected = classOf[TableException])
+  def testGetFieldInfoPojoAlias3(): Unit = {
+    tEnv.getFieldInfo(
+      pojoType,
+      Array(
+        new Alias(new UnresolvedFieldReference("xxx"), "name1"),
+        new Alias(new UnresolvedFieldReference("yyy"), "name2"),
+        new Alias(new UnresolvedFieldReference("zzz"), "name3")
+      ))
+  }
+
+  @Test(expected = classOf[TableException])
+  def testGetFieldInfoAtomicAlias(): Unit = {
+    tEnv.getFieldInfo(
+      atomicType,
+      Array(
+        new Alias(new UnresolvedFieldReference("name1"), "name2")
+      ))
+  }
+
+}
+
+class MockTableEnvironment extends TableEnvironment(new TableConfig) {
+
+  override private[flink] def writeToSink[T](table: Table, sink: TableSink[T]): Unit = ???
+
+  override protected def checkValidTableName(name: String): Unit = ???
+
+  override protected def getBuiltInRuleSet: RuleSet = ???
+
+  override def sql(query: String): Table = ???
+}
+
+case class CClass(cf1: Int, cf2: String, cf3: Double)
+
+class PojoClass(var pf1: Int, var pf2: String, var pf3: Double) {
+  def this() = this(0, "", 0.0)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ExplainTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ExplainTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ExplainTest.scala
new file mode 100644
index 0000000..a323ec9
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ExplainTest.scala
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.test.util.MultipleProgramsTestBase
+import org.junit.Assert.assertEquals
+import org.junit._
+
+class ExplainTest
+  extends MultipleProgramsTestBase(MultipleProgramsTestBase.TestExecutionMode.CLUSTER) {
+
+  val testFilePath = ExplainTest.this.getClass.getResource("/").getFile
+
+  @Test
+  def testFilterWithoutExtended() : Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val table = env.fromElements((1, "hello"))
+      .toTable(tEnv, 'a, 'b)
+      .filter("a % 2 = 0")
+
+    val result = tEnv.explain(table).replaceAll("\\r\\n", "\n")
+    val source = scala.io.Source.fromFile(testFilePath +
+      "../../src/test/scala/resources/testFilter0.out").mkString.replaceAll("\\r\\n", "\n")
+    assertEquals(result, source)
+  }
+
+  @Test
+  def testFilterWithExtended() : Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val table = env.fromElements((1, "hello"))
+      .toTable(tEnv, 'a, 'b)
+      .filter("a % 2 = 0")
+
+    val result = tEnv.explain(table, true).replaceAll("\\r\\n", "\n")
+    val source = scala.io.Source.fromFile(testFilePath +
+      "../../src/test/scala/resources/testFilter1.out").mkString.replaceAll("\\r\\n", "\n")
+    assertEquals(result, source)
+  }
+
+  @Test
+  def testJoinWithoutExtended() : Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val table1 = env.fromElements((1, "hello")).toTable(tEnv, 'a, 'b)
+    val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'c, 'd)
+    val table = table1.join(table2).where("b = d").select("a, c")
+
+    val result = tEnv.explain(table).replaceAll("\\r\\n", "\n")
+    val source = scala.io.Source.fromFile(testFilePath +
+      "../../src/test/scala/resources/testJoin0.out").mkString.replaceAll("\\r\\n", "\n")
+    assertEquals(source, result)
+  }
+
+  @Test
+  def testJoinWithExtended() : Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val table1 = env.fromElements((1, "hello")).toTable(tEnv, 'a, 'b)
+    val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'c, 'd)
+    val table = table1.join(table2).where("b = d").select("a, c")
+
+    val result = tEnv.explain(table, true).replaceAll("\\r\\n", "\n")
+    val source = scala.io.Source.fromFile(testFilePath +
+      "../../src/test/scala/resources/testJoin1.out").mkString.replaceAll("\\r\\n", "\n")
+    assertEquals(source, result)
+  }
+
+  @Test
+  def testUnionWithoutExtended() : Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val table1 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
+    val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
+    val table = table1.unionAll(table2)
+
+    val result = tEnv.explain(table).replaceAll("\\r\\n", "\n")
+    val source = scala.io.Source.fromFile(testFilePath +
+      "../../src/test/scala/resources/testUnion0.out").mkString.replaceAll("\\r\\n", "\n")
+    assertEquals(result, source)
+  }
+
+  @Test
+  def testUnionWithExtended() : Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val table1 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
+    val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
+    val table = table1.unionAll(table2)
+
+    val result = tEnv.explain(table, true).replaceAll("\\r\\n", "\n")
+    val source = scala.io.Source.fromFile(testFilePath +
+      "../../src/test/scala/resources/testUnion1.out").mkString.replaceAll("\\r\\n", "\n")
+    assertEquals(result, source)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala
new file mode 100644
index 0000000..ec4dc59
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/ProjectableTableSourceITCase.scala
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch
+
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.java.{DataSet => JavaSet, ExecutionEnvironment => JavaExecEnv}
+import org.apache.flink.api.scala.ExecutionEnvironment
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.sources.{BatchTableSource, ProjectableTableSource}
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.apache.flink.types.Row
+import org.junit.{Before, Test}
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class ProjectableTableSourceITCase(mode: TestExecutionMode,
+  configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  private val tableName = "MyTable"
+  private var tableEnv: BatchTableEnvironment = null
+
+  @Before
+  def initTableEnv(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    tableEnv = TableEnvironment.getTableEnvironment(env, config)
+    tableEnv.registerTableSource(tableName, new TestProjectableTableSource)
+  }
+
+  @Test
+  def testTableAPI(): Unit = {
+    val results = tableEnv
+                  .scan(tableName)
+                  .where("amount < 4")
+                  .select("id, name")
+                  .collect()
+
+    val expected = Seq(
+      "0,Record_0", "1,Record_1", "2,Record_2", "3,Record_3", "16,Record_16",
+      "17,Record_17", "18,Record_18", "19,Record_19", "32,Record_32").mkString("\n")
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+
+  @Test
+  def testSQL(): Unit = {
+    val results = tableEnv
+                  .sql(s"select id, name from $tableName where amount < 4 ")
+                  .collect()
+
+    val expected = Seq(
+      "0,Record_0", "1,Record_1", "2,Record_2", "3,Record_3", "16,Record_16",
+      "17,Record_17", "18,Record_18", "19,Record_19", "32,Record_32").mkString("\n")
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+}
+
+class TestProjectableTableSource(
+  fieldTypes: Array[TypeInformation[_]],
+  fieldNames: Array[String])
+  extends BatchTableSource[Row] with ProjectableTableSource[Row] {
+
+  def this() = this(
+    fieldTypes = Array(
+      BasicTypeInfo.STRING_TYPE_INFO,
+      BasicTypeInfo.LONG_TYPE_INFO,
+      BasicTypeInfo.INT_TYPE_INFO,
+      BasicTypeInfo.DOUBLE_TYPE_INFO),
+    fieldNames = Array[String]("name", "id", "amount", "price")
+  )
+
+  /** Returns the data of the table as a [[org.apache.flink.api.java.DataSet]]. */
+  override def getDataSet(execEnv: JavaExecEnv): JavaSet[Row] = {
+    execEnv.fromCollection(generateDynamicCollection(33, fieldNames).asJava, getReturnType)
+  }
+
+  /** Returns the types of the table fields. */
+  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
+
+  /** Returns the names of the table fields. */
+  override def getFieldsNames: Array[String] = fieldNames
+
+  /** Returns the [[TypeInformation]] for the return type. */
+  override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*)
+
+  /** Returns the number of fields of the table. */
+  override def getNumberOfFields: Int = fieldNames.length
+
+  override def projectFields(fields: Array[Int]): TestProjectableTableSource = {
+    val projectedFieldTypes = new Array[TypeInformation[_]](fields.length)
+    val projectedFieldNames = new Array[String](fields.length)
+
+    fields.zipWithIndex.foreach(f => {
+      projectedFieldTypes(f._2) = fieldTypes(f._1)
+      projectedFieldNames(f._2) = fieldNames(f._1)
+    })
+    new TestProjectableTableSource(projectedFieldTypes, projectedFieldNames)
+  }
+
+  private def generateDynamicCollection(num: Int, fieldNames: Array[String]): Seq[Row] = {
+    for {cnt <- 0 until num}
+      yield {
+        val row = new Row(fieldNames.length)
+        fieldNames.zipWithIndex.foreach(
+          f =>
+            f._1 match {
+              case "name" =>
+                row.setField(f._2, "Record_" + cnt)
+              case "id" =>
+                row.setField(f._2, cnt.toLong)
+              case "amount" =>
+                row.setField(f._2, cnt.toInt % 16)
+              case "price" =>
+                row.setField(f._2, cnt.toDouble / 3)
+              case _ =>
+                throw new IllegalArgumentException(s"unknown field name $f._1")
+            }
+        )
+        row
+      }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
new file mode 100644
index 0000000..961e575
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableEnvironmentITCase.scala
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch
+
+import java.util
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class TableEnvironmentITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testSimpleRegister(): Unit = {
+
+    val tableName = "MyTable"
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet(tableName, ds)
+    val t = tEnv.scan(tableName).select('_1, '_2, '_3)
+
+    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
+      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
+      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testRegisterWithFields(): Unit = {
+
+    val tableName = "MyTable"
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet(tableName, ds, 'a, 'b, 'c)
+    val t = tEnv.scan(tableName).select('a, 'b)
+
+    val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" +
+      "7,4\n" + "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" +
+      "15,5\n" + "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testRegisterExistingDataSet(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds1)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env)
+    // Must fail. Name is already in use.
+    tEnv.registerDataSet("MyTable", ds2)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testScanUnregisteredTable(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    // Must fail. No table registered under that name.
+    tEnv.scan("someTable")
+  }
+
+  @Test
+  def testTableRegister(): Unit = {
+
+    val tableName = "MyTable"
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    tEnv.registerTable(tableName, t)
+
+    val regT = tEnv.scan(tableName).select('a, 'b).filter('a > 8)
+
+    val expected = "9,4\n" + "10,4\n" +
+      "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" +
+      "15,5\n" + "16,6\n" + "17,6\n" + "18,6\n" +
+      "19,6\n" + "20,6\n" + "21,6\n"
+
+    val results = regT.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testRegisterExistingTable(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+    tEnv.registerTable("MyTable", t1)
+    val t2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv)
+    // Must fail. Name is already in use.
+    tEnv.registerDataSet("MyTable", t2)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testRegisterTableFromOtherEnv(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv1 = TableEnvironment.getTableEnvironment(env, config)
+    val tEnv2 = TableEnvironment.getTableEnvironment(env, config)
+
+    val t1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv1)
+    // Must fail. Table is bound to different TableEnvironment.
+    tEnv2.registerTable("MyTable", t1)
+  }
+
+  @Test
+  def testToTable(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env)
+      .toTable(tEnv, 'a, 'b, 'c)
+      .select('a, 'b, 'c)
+
+    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
+      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
+      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testToTableFromCaseClass(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val data = List(
+      SomeCaseClass("Peter", 28, 4000.00, "Sales"),
+      SomeCaseClass("Anna", 56, 10000.00, "Engineering"),
+      SomeCaseClass("Lucy", 42, 6000.00, "HR"))
+
+    val t =  env.fromCollection(data)
+      .toTable(tEnv, 'a, 'b, 'c, 'd)
+      .select('a, 'b, 'c, 'd)
+
+    val expected: String =
+      "Peter,28,4000.0,Sales\n" +
+      "Anna,56,10000.0,Engineering\n" +
+      "Lucy,42,6000.0,HR\n"
+    val results = t.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testToTableFromAndToCaseClass(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val data = List(
+      SomeCaseClass("Peter", 28, 4000.00, "Sales"),
+      SomeCaseClass("Anna", 56, 10000.00, "Engineering"),
+      SomeCaseClass("Lucy", 42, 6000.00, "HR"))
+
+    val t =  env.fromCollection(data)
+      .toTable(tEnv, 'a, 'b, 'c, 'd)
+      .select('a, 'b, 'c, 'd)
+
+    val expected: String =
+      "SomeCaseClass(Peter,28,4000.0,Sales)\n" +
+      "SomeCaseClass(Anna,56,10000.0,Engineering)\n" +
+      "SomeCaseClass(Lucy,42,6000.0,HR)\n"
+    val results = t.toDataSet[SomeCaseClass].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testToTableWithToFewFields(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    CollectionDataSets.get3TupleDataSet(env)
+      // Must fail. Number of fields does not match.
+      .toTable(tEnv, 'a, 'b)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testToTableWithToManyFields(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    CollectionDataSets.get3TupleDataSet(env)
+      // Must fail. Number of fields does not match.
+      .toTable(tEnv, 'a, 'b, 'c, 'd)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testToTableWithAmbiguousFields(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    CollectionDataSets.get3TupleDataSet(env)
+      // Must fail. Field names not unique.
+      .toTable(tEnv, 'a, 'b, 'b)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testToTableWithNonFieldReference1(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    // Must fail. as() can only have field references
+    CollectionDataSets.get3TupleDataSet(env)
+      .toTable(tEnv, 'a + 1, 'b, 'c)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testToTableWithNonFieldReference2(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    // Must fail. as() can only have field references
+    CollectionDataSets.get3TupleDataSet(env)
+      .toTable(tEnv, 'a as 'foo, 'b, 'c)
+  }
+}
+
+object TableEnvironmentITCase {
+
+  @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
+  def parameters(): util.Collection[Array[java.lang.Object]] = {
+    Seq[Array[AnyRef]](
+      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT),
+      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.EFFICIENT)).asJava
+  }
+}
+
+case class SomeCaseClass(name: String, age: Int, salary: Double, department: String) {
+  def this() { this("", 0, 0.0, "") }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSinkITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSinkITCase.scala
new file mode 100644
index 0000000..8bc7874
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSinkITCase.scala
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch
+
+import java.io.File
+
+import org.apache.flink.api.scala.{ExecutionEnvironment, _}
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.sinks.CsvTableSink
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+
+@RunWith(classOf[Parameterized])
+class TableSinkITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testBatchTableSink(): Unit = {
+
+    val tmpFile = File.createTempFile("flink-table-sink-test", ".tmp")
+    tmpFile.deleteOnExit()
+    val path = tmpFile.toURI.toString
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    env.setParallelism(4)
+
+    val input = CollectionDataSets.get3TupleDataSet(env)
+      .map(x => x).setParallelism(4) // increase DOP to 4
+
+    val results = input.toTable(tEnv, 'a, 'b, 'c)
+      .where('a < 5 || 'a > 17)
+      .select('c, 'b)
+      .writeToSink(new CsvTableSink(path, fieldDelim = "|"))
+
+    env.execute()
+
+    val expected = Seq(
+      "Hi|1", "Hello|2", "Hello world|2", "Hello world, how are you?|3",
+      "Comment#12|6", "Comment#13|6", "Comment#14|6", "Comment#15|6").mkString("\n")
+
+    TestBaseUtils.compareResultsByLinesInMemory(expected, path)
+  }
+
+}


[27/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
new file mode 100644
index 0000000..0634f0b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
@@ -0,0 +1,691 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala
+
+import java.sql.{Date, Time, Timestamp}
+
+import org.apache.calcite.avatica.util.DateTimeUtils._
+import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.table.expressions.ExpressionUtils.{convertArray, toMilliInterval, toMonthInterval, toRowInterval}
+import org.apache.flink.table.expressions.TimeIntervalUnit.TimeIntervalUnit
+import org.apache.flink.table.expressions._
+import java.math.{BigDecimal => JBigDecimal}
+
+import scala.language.implicitConversions
+
+/**
+ * These are all the operations that can be used to construct an [[Expression]] AST for expression
+ * operations.
+ *
+ * These operations must be kept in sync with the parser in
+ * [[org.apache.flink.table.expressions.ExpressionParser]].
+ */
+trait ImplicitExpressionOperations {
+  private[flink] def expr: Expression
+
+  /**
+    * Enables literals on left side of binary expressions.
+    *
+    * e.g. 12.toExpr % 'a
+    *
+    * @return expression
+    */
+  def toExpr: Expression = expr
+
+  def && (other: Expression) = And(expr, other)
+  def || (other: Expression) = Or(expr, other)
+
+  def > (other: Expression) = GreaterThan(expr, other)
+  def >= (other: Expression) = GreaterThanOrEqual(expr, other)
+  def < (other: Expression) = LessThan(expr, other)
+  def <= (other: Expression) = LessThanOrEqual(expr, other)
+
+  def === (other: Expression) = EqualTo(expr, other)
+  def !== (other: Expression) = NotEqualTo(expr, other)
+
+  def unary_! = Not(expr)
+  def unary_- = UnaryMinus(expr)
+
+  def isNull = IsNull(expr)
+  def isNotNull = IsNotNull(expr)
+
+  /**
+    * Returns true if given boolean expression is true. False otherwise (for null and false).
+    */
+  def isTrue = IsTrue(expr)
+
+  /**
+    * Returns true if given boolean expression is false. False otherwise (for null and true).
+    */
+  def isFalse = IsFalse(expr)
+
+  /**
+    * Returns true if given boolean expression is not true (for null and false). False otherwise.
+    */
+  def isNotTrue = IsNotTrue(expr)
+
+  /**
+    * Returns true if given boolean expression is not false (for null and true). False otherwise.
+    */
+  def isNotFalse = IsNotFalse(expr)
+
+  def + (other: Expression) = Plus(expr, other)
+  def - (other: Expression) = Minus(expr, other)
+  def / (other: Expression) = Div(expr, other)
+  def * (other: Expression) = Mul(expr, other)
+  def % (other: Expression) = mod(other)
+
+  def sum = Sum(expr)
+  def min = Min(expr)
+  def max = Max(expr)
+  def count = Count(expr)
+  def avg = Avg(expr)
+
+  def cast(toType: TypeInformation[_]) = Cast(expr, toType)
+
+  /**
+    * Specifies a name for an expression i.e. a field.
+    *
+    * @param name name for one field
+    * @param extraNames additional names if the expression expands to multiple fields
+    * @return field with an alias
+    */
+  def as(name: Symbol, extraNames: Symbol*) = Alias(expr, name.name, extraNames.map(_.name))
+
+  def asc = Asc(expr)
+  def desc = Desc(expr)
+
+  /**
+    * Returns the start time of a window when applied on a window reference.
+    */
+  def start = WindowStart(expr)
+
+  /**
+    * Returns the end time of a window when applied on a window reference.
+    */
+  def end = WindowEnd(expr)
+
+  /**
+    * Ternary conditional operator that decides which of two other expressions should be evaluated
+    * based on a evaluated boolean condition.
+    *
+    * e.g. (42 > 5).?("A", "B") leads to "A"
+    *
+    * @param ifTrue expression to be evaluated if condition holds
+    * @param ifFalse expression to be evaluated if condition does not hold
+    */
+  def ?(ifTrue: Expression, ifFalse: Expression) = {
+    If(expr, ifTrue, ifFalse)
+  }
+
+  // scalar functions
+
+  /**
+    * Calculates the remainder of division the given number by another one.
+    */
+  def mod(other: Expression) = Mod(expr, other)
+
+  /**
+    * Calculates the Euler's number raised to the given power.
+    */
+  def exp() = Exp(expr)
+
+  /**
+    * Calculates the base 10 logarithm of given value.
+    */
+  def log10() = Log10(expr)
+
+  /**
+    * Calculates the natural logarithm of given value.
+    */
+  def ln() = Ln(expr)
+
+  /**
+    * Calculates the given number raised to the power of the other value.
+    */
+  def power(other: Expression) = Power(expr, other)
+
+  /**
+    * Calculates the square root of a given value.
+    */
+  def sqrt() = Sqrt(expr)
+
+  /**
+    * Calculates the absolute value of given value.
+    */
+  def abs() = Abs(expr)
+
+  /**
+    * Calculates the largest integer less than or equal to a given number.
+    */
+  def floor() = Floor(expr)
+
+  /**
+    * Calculates the smallest integer greater than or equal to a given number.
+    */
+  def ceil() = Ceil(expr)
+
+  // String operations
+
+  /**
+    * Creates a substring of the given string at given index for a given length.
+    *
+    * @param beginIndex first character of the substring (starting at 1, inclusive)
+    * @param length number of characters of the substring
+    * @return substring
+    */
+  def substring(beginIndex: Expression, length: Expression) =
+    Substring(expr, beginIndex, length)
+
+  /**
+    * Creates a substring of the given string beginning at the given index to the end.
+    *
+    * @param beginIndex first character of the substring (starting at 1, inclusive)
+    * @return substring
+    */
+  def substring(beginIndex: Expression) =
+    new Substring(expr, beginIndex)
+
+  /**
+    * Removes leading and/or trailing characters from the given string.
+    *
+    * @param removeLeading if true, remove leading characters (default: true)
+    * @param removeTrailing if true, remove trailing characters (default: true)
+    * @param character string containing the character (default: " ")
+    * @return trimmed string
+    */
+  def trim(
+      removeLeading: Boolean = true,
+      removeTrailing: Boolean = true,
+      character: Expression = TrimConstants.TRIM_DEFAULT_CHAR) = {
+    if (removeLeading && removeTrailing) {
+      Trim(TrimMode.BOTH, character, expr)
+    } else if (removeLeading) {
+      Trim(TrimMode.LEADING, character, expr)
+    } else if (removeTrailing) {
+      Trim(TrimMode.TRAILING, character, expr)
+    } else {
+      expr
+    }
+  }
+
+  /**
+    * Returns the length of a string.
+    */
+  def charLength() = CharLength(expr)
+
+  /**
+    * Returns all of the characters in a string in upper case using the rules of
+    * the default locale.
+    */
+  def upperCase() = Upper(expr)
+
+  /**
+    * Returns all of the characters in a string in lower case using the rules of
+    * the default locale.
+    */
+  def lowerCase() = Lower(expr)
+
+  /**
+    * Converts the initial letter of each word in a string to uppercase.
+    * Assumes a string containing only [A-Za-z0-9], everything else is treated as whitespace.
+    */
+  def initCap() = InitCap(expr)
+
+  /**
+    * Returns true, if a string matches the specified LIKE pattern.
+    *
+    * e.g. "Jo_n%" matches all strings that start with "Jo(arbitrary letter)n"
+    */
+  def like(pattern: Expression) = Like(expr, pattern)
+
+  /**
+    * Returns true, if a string matches the specified SQL regex pattern.
+    *
+    * e.g. "A+" matches all strings that consist of at least one A
+    */
+  def similar(pattern: Expression) = Similar(expr, pattern)
+
+  /**
+    * Returns the position of string in an other string starting at 1.
+    * Returns 0 if string could not be found.
+    *
+    * e.g. "a".position("bbbbba") leads to 6
+    */
+  def position(haystack: Expression) = Position(expr, haystack)
+
+  /**
+    * Replaces a substring of string with a string starting at a position (starting at 1).
+    *
+    * e.g. "xxxxxtest".overlay("xxxx", 6) leads to "xxxxxxxxx"
+    */
+  def overlay(newString: Expression, starting: Expression) = new Overlay(expr, newString, starting)
+
+  /**
+    * Replaces a substring of string with a string starting at a position (starting at 1).
+    * The length specifies how many characters should be removed.
+    *
+    * e.g. "xxxxxtest".overlay("xxxx", 6, 2) leads to "xxxxxxxxxst"
+    */
+  def overlay(newString: Expression, starting: Expression, length: Expression) =
+    Overlay(expr, newString, starting, length)
+
+  // Temporal operations
+
+  /**
+    * Parses a date string in the form "yy-mm-dd" to a SQL Date.
+    */
+  def toDate = Cast(expr, SqlTimeTypeInfo.DATE)
+
+  /**
+    * Parses a time string in the form "hh:mm:ss" to a SQL Time.
+    */
+  def toTime = Cast(expr, SqlTimeTypeInfo.TIME)
+
+  /**
+    * Parses a timestamp string in the form "yy-mm-dd hh:mm:ss.fff" to a SQL Timestamp.
+    */
+  def toTimestamp = Cast(expr, SqlTimeTypeInfo.TIMESTAMP)
+
+  /**
+    * Extracts parts of a time point or time interval. Returns the part as a long value.
+    *
+    * e.g. "2006-06-05".toDate.extract(DAY) leads to 5
+    */
+  def extract(timeIntervalUnit: TimeIntervalUnit) = Extract(timeIntervalUnit, expr)
+
+  /**
+    * Returns the quarter of a year from a SQL date.
+    *
+    * e.g. "1994-09-27".toDate.quarter() leads to 3
+    */
+  def quarter() = Quarter(expr)
+
+  /**
+    * Rounds down a time point to the given unit.
+    *
+    * e.g. "12:44:31".toDate.floor(MINUTE) leads to 12:44:00
+    */
+  def floor(timeIntervalUnit: TimeIntervalUnit) = TemporalFloor(timeIntervalUnit, expr)
+
+  /**
+    * Rounds up a time point to the given unit.
+    *
+    * e.g. "12:44:31".toDate.ceil(MINUTE) leads to 12:45:00
+    */
+  def ceil(timeIntervalUnit: TimeIntervalUnit) = TemporalCeil(timeIntervalUnit, expr)
+
+  // Interval types
+
+  /**
+    * Creates an interval of the given number of years.
+    *
+    * @return interval of months
+    */
+  def year = toMonthInterval(expr, 12)
+
+  /**
+    * Creates an interval of the given number of years.
+    *
+    * @return interval of months
+    */
+  def years = year
+
+  /**
+    * Creates an interval of the given number of months.
+    *
+    * @return interval of months
+    */
+  def month = toMonthInterval(expr, 1)
+
+  /**
+    * Creates an interval of the given number of months.
+    *
+    * @return interval of months
+    */
+  def months = month
+
+  /**
+    * Creates an interval of the given number of days.
+    *
+    * @return interval of milliseconds
+    */
+  def day = toMilliInterval(expr, MILLIS_PER_DAY)
+
+  /**
+    * Creates an interval of the given number of days.
+    *
+    * @return interval of milliseconds
+    */
+  def days = day
+
+  /**
+    * Creates an interval of the given number of hours.
+    *
+    * @return interval of milliseconds
+    */
+  def hour = toMilliInterval(expr, MILLIS_PER_HOUR)
+
+  /**
+    * Creates an interval of the given number of hours.
+    *
+    * @return interval of milliseconds
+    */
+  def hours = hour
+
+  /**
+    * Creates an interval of the given number of minutes.
+    *
+    * @return interval of milliseconds
+    */
+  def minute = toMilliInterval(expr, MILLIS_PER_MINUTE)
+
+  /**
+    * Creates an interval of the given number of minutes.
+    *
+    * @return interval of milliseconds
+    */
+  def minutes = minute
+
+  /**
+    * Creates an interval of the given number of seconds.
+    *
+    * @return interval of milliseconds
+    */
+  def second = toMilliInterval(expr, MILLIS_PER_SECOND)
+
+  /**
+    * Creates an interval of the given number of seconds.
+    *
+    * @return interval of milliseconds
+    */
+  def seconds = second
+
+  /**
+    * Creates an interval of the given number of milliseconds.
+    *
+    * @return interval of milliseconds
+    */
+  def milli = toMilliInterval(expr, 1)
+
+  /**
+    * Creates an interval of the given number of milliseconds.
+    *
+    * @return interval of milliseconds
+    */
+  def millis = milli
+
+  // row interval type
+
+  /**
+    * Creates an interval of rows.
+    *
+    * @return interval of rows
+    */
+  def rows = toRowInterval(expr)
+
+  /**
+    * Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by name and
+    * returns it's value.
+    *
+    * @param name name of the field (similar to Flink's field expressions)
+    * @return value of the field
+    */
+  def get(name: String) = GetCompositeField(expr, name)
+
+  /**
+    * Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by index and
+    * returns it's value.
+    *
+    * @param index position of the field
+    * @return value of the field
+    */
+  def get(index: Int) = GetCompositeField(expr, index)
+
+  /**
+    * Converts a Flink composite type (such as Tuple, POJO, etc.) and all of its direct subtypes
+    * into a flat representation where every subtype is a separate field.
+    */
+  def flatten() = Flattening(expr)
+
+  /**
+    * Accesses the element of an array based on an index (starting at 1).
+    *
+    * @param index position of the element (starting at 1)
+    * @return value of the element
+    */
+  def at(index: Expression) = ArrayElementAt(expr, index)
+
+  /**
+    * Returns the number of elements of an array.
+    *
+    * @return number of elements
+    */
+  def cardinality() = ArrayCardinality(expr)
+
+  /**
+    * Returns the sole element of an array with a single element. Returns null if the array is
+    * empty. Throws an exception if the array has more than one element.
+    *
+    * @return the first and only element of an array with a single element
+    */
+  def element() = ArrayElement(expr)
+}
+
+/**
+ * Implicit conversions from Scala Literals to Expression [[Literal]] and from [[Expression]]
+ * to [[ImplicitExpressionOperations]].
+ */
+trait ImplicitExpressionConversions {
+  implicit class WithOperations(e: Expression) extends ImplicitExpressionOperations {
+    def expr = e
+  }
+
+  implicit class UnresolvedFieldExpression(s: Symbol) extends ImplicitExpressionOperations {
+    def expr = UnresolvedFieldReference(s.name)
+  }
+
+  implicit class LiteralLongExpression(l: Long) extends ImplicitExpressionOperations {
+    def expr = Literal(l)
+  }
+
+  implicit class LiteralByteExpression(b: Byte) extends ImplicitExpressionOperations {
+    def expr = Literal(b)
+  }
+
+  implicit class LiteralShortExpression(s: Short) extends ImplicitExpressionOperations {
+    def expr = Literal(s)
+  }
+
+  implicit class LiteralIntExpression(i: Int) extends ImplicitExpressionOperations {
+    def expr = Literal(i)
+  }
+
+  implicit class LiteralFloatExpression(f: Float) extends ImplicitExpressionOperations {
+    def expr = Literal(f)
+  }
+
+  implicit class LiteralDoubleExpression(d: Double) extends ImplicitExpressionOperations {
+    def expr = Literal(d)
+  }
+
+  implicit class LiteralStringExpression(str: String) extends ImplicitExpressionOperations {
+    def expr = Literal(str)
+  }
+
+  implicit class LiteralBooleanExpression(bool: Boolean) extends ImplicitExpressionOperations {
+    def expr = Literal(bool)
+  }
+
+  implicit class LiteralJavaDecimalExpression(javaDecimal: java.math.BigDecimal)
+      extends ImplicitExpressionOperations {
+    def expr = Literal(javaDecimal)
+  }
+
+  implicit class LiteralScalaDecimalExpression(scalaDecimal: scala.math.BigDecimal)
+      extends ImplicitExpressionOperations {
+    def expr = Literal(scalaDecimal.bigDecimal)
+  }
+
+  implicit class LiteralSqlDateExpression(sqlDate: Date) extends ImplicitExpressionOperations {
+    def expr = Literal(sqlDate)
+  }
+
+  implicit class LiteralSqlTimeExpression(sqlTime: Time) extends ImplicitExpressionOperations {
+    def expr = Literal(sqlTime)
+  }
+
+  implicit class LiteralSqlTimestampExpression(sqlTimestamp: Timestamp)
+      extends ImplicitExpressionOperations {
+    def expr = Literal(sqlTimestamp)
+  }
+
+  implicit def symbol2FieldExpression(sym: Symbol): Expression = UnresolvedFieldReference(sym.name)
+  implicit def byte2Literal(b: Byte): Expression = Literal(b)
+  implicit def short2Literal(s: Short): Expression = Literal(s)
+  implicit def int2Literal(i: Int): Expression = Literal(i)
+  implicit def long2Literal(l: Long): Expression = Literal(l)
+  implicit def double2Literal(d: Double): Expression = Literal(d)
+  implicit def float2Literal(d: Float): Expression = Literal(d)
+  implicit def string2Literal(str: String): Expression = Literal(str)
+  implicit def boolean2Literal(bool: Boolean): Expression = Literal(bool)
+  implicit def javaDec2Literal(javaDec: JBigDecimal): Expression = Literal(javaDec)
+  implicit def scalaDec2Literal(scalaDec: BigDecimal): Expression =
+    Literal(scalaDec.bigDecimal)
+  implicit def sqlDate2Literal(sqlDate: Date): Expression = Literal(sqlDate)
+  implicit def sqlTime2Literal(sqlTime: Time): Expression = Literal(sqlTime)
+  implicit def sqlTimestamp2Literal(sqlTimestamp: Timestamp): Expression =
+    Literal(sqlTimestamp)
+  implicit def array2ArrayConstructor(array: Array[_]): Expression = convertArray(array)
+}
+
+// ------------------------------------------------------------------------------------------------
+// Expressions with no parameters
+// ------------------------------------------------------------------------------------------------
+
+// we disable the object checker here as it checks for capital letters of objects
+// but we want that objects look like functions in certain cases e.g. array(1, 2, 3)
+// scalastyle:off object.name
+
+/**
+  * Returns the current SQL date in UTC time zone.
+  */
+object currentDate {
+
+  /**
+    * Returns the current SQL date in UTC time zone.
+    */
+  def apply(): Expression = {
+    CurrentDate()
+  }
+}
+
+/**
+  * Returns the current SQL time in UTC time zone.
+  */
+object currentTime {
+
+  /**
+    * Returns the current SQL time in UTC time zone.
+    */
+  def apply(): Expression = {
+    CurrentTime()
+  }
+}
+
+/**
+  * Returns the current SQL timestamp in UTC time zone.
+  */
+object currentTimestamp {
+
+  /**
+    * Returns the current SQL timestamp in UTC time zone.
+    */
+  def apply(): Expression = {
+    CurrentTimestamp()
+  }
+}
+
+/**
+  * Returns the current SQL time in local time zone.
+  */
+object localTime {
+
+  /**
+    * Returns the current SQL time in local time zone.
+    */
+  def apply(): Expression = {
+    LocalTime()
+  }
+}
+
+/**
+  * Returns the current SQL timestamp in local time zone.
+  */
+object localTimestamp {
+
+  /**
+    * Returns the current SQL timestamp in local time zone.
+    */
+  def apply(): Expression = {
+    LocalTimestamp()
+  }
+}
+
+/**
+  * Determines whether two anchored time intervals overlap. Time point and temporal are
+  * transformed into a range defined by two time points (start, end). The function
+  * evaluates <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
+  *
+  * It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+  *
+  * e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+  */
+object temporalOverlaps {
+
+  /**
+    * Determines whether two anchored time intervals overlap. Time point and temporal are
+    * transformed into a range defined by two time points (start, end).
+    *
+    * It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
+    *
+    * e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
+    */
+  def apply(
+      leftTimePoint: Expression,
+      leftTemporal: Expression,
+      rightTimePoint: Expression,
+      rightTemporal: Expression): Expression = {
+    TemporalOverlaps(leftTimePoint, leftTemporal, rightTimePoint, rightTemporal)
+  }
+}
+
+/**
+  * Creates an array of literals. The array will be an array of objects (not primitives).
+  */
+object array {
+
+  /**
+    * Creates an array of literals. The array will be an array of objects (not primitives).
+    */
+  def apply(head: Expression, tail: Expression*): Expression = {
+    ArrayConstructor(head +: tail.toSeq)
+  }
+}
+
+// scalastyle:on object.name

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/groupWindows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/groupWindows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/groupWindows.scala
new file mode 100644
index 0000000..0e4c1c7
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/groupWindows.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.api.scala
+
+import org.apache.flink.table.expressions.Expression
+import org.apache.flink.table.api.{SessionWindow, SlideWithSize, TumblingWindow}
+
+/**
+  * Helper object for creating a tumbling window. Tumbling windows are consecutive, non-overlapping
+  * windows of a specified fixed length. For example, a tumbling window of 5 minutes size groups
+  * elements in 5 minutes intervals.
+  */
+object Tumble {
+
+  /**
+    * Creates a tumbling window. Tumbling windows are fixed-size, consecutive, non-overlapping
+    * windows. For example, a tumbling window of 5 minutes size groups
+    * elements in 5 minutes intervals.
+    *
+    * @param size the size of the window as time or row-count interval.
+    * @return a tumbling window
+    */
+  def over(size: Expression): TumblingWindow = new TumblingWindow(size)
+}
+
+/**
+  * Helper object for creating a sliding window. Sliding windows have a fixed size and slide by
+  * a specified slide interval. If the slide interval is smaller than the window size, sliding
+  * windows are overlapping. Thus, an element can be assigned to multiple windows.
+  *
+  * For example, a sliding window of size 15 minutes with 5 minutes sliding interval groups elements
+  * of 15 minutes and evaluates every five minutes. Each element is contained in three consecutive
+  * window evaluations.
+  */
+object Slide {
+
+  /**
+    * Creates a sliding window. Sliding windows have a fixed size and slide by
+    * a specified slide interval. If the slide interval is smaller than the window size, sliding
+    * windows are overlapping. Thus, an element can be assigned to multiple windows.
+    *
+    * For example, a sliding window of size 15 minutes with 5 minutes sliding interval groups
+    * elements of 15 minutes and evaluates every five minutes. Each element is contained in three
+    * consecutive
+    *
+    * @param size the size of the window as time or row-count interval
+    * @return a partially specified sliding window
+    */
+  def over(size: Expression): SlideWithSize = new SlideWithSize(size)
+}
+
+/**
+  * Helper object for creating a session window. The boundary of session windows are defined by
+  * intervals of inactivity, i.e., a session window is closes if no event appears for a defined
+  * gap period.
+  */
+object Session {
+
+  /**
+    * Creates a session window. The boundary of session windows are defined by
+    * intervals of inactivity, i.e., a session window is closes if no event appears for a defined
+    * gap period.
+    *
+    * @param gap specifies how long (as interval of milliseconds) to wait for new data before
+    *            closing the session window.
+    * @return a session window
+    */
+  def withGap(gap: Expression): SessionWindow = new SessionWindow(gap)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/package.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/package.scala
new file mode 100644
index 0000000..cd341cb
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/package.scala
@@ -0,0 +1,93 @@
+/*
+ * 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 org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.types.Row
+import org.apache.flink.api.scala._
+import org.apache.flink.api.scala.DataSet
+import org.apache.flink.streaming.api.scala.DataStream
+import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnv}
+import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTableEnv}
+
+import _root_.scala.language.implicitConversions
+import _root_.scala.reflect.ClassTag
+
+/**
+  * == Table API (Scala) ==
+  *
+  * Importing this package with:
+  *
+  * {{{
+  *   import org.apache.flink.table.api.scala._
+  * }}}
+  *
+  * imports implicit conversions for converting a [[DataSet]] and a [[DataStream]] to a
+  * [[Table]]. This can be used to perform SQL-like queries on data. Please have
+  * a look at [[Table]] to see which operations are supported and
+  * [[org.apache.flink.table.api.scala.ImplicitExpressionOperations]] to see how an
+  * expression can be specified.
+  *
+  * When writing a query you can use Scala Symbols to refer to field names. One would
+  * refer to field `a` by writing `'a`. Sometimes it is necessary to manually convert a
+  * Scala literal to an Expression literal, in those cases use `Literal`, as in `Literal(3)`.
+  *
+  * Example:
+  *
+  * {{{
+  *   import org.apache.flink.api.scala._
+  *   import org.apache.flink.table.api.scala._
+  *
+  *   val env = ExecutionEnvironment.getExecutionEnvironment
+  *   val tEnv = TableEnvironment.getTableEnvironment(env)
+  *
+  *   val input: DataSet[(String, Int)] = env.fromElements(("Hello", 2), ("Hello", 5), ("Ciao", 3))
+  *   val result = input
+  *         .toTable(tEnv, 'word, 'count)
+  *         .groupBy('word)
+  *         .select('word, 'count.avg)
+  *
+  *   result.print()
+  * }}}
+  *
+  */
+package object scala extends ImplicitExpressionConversions {
+
+  implicit def table2TableConversions(table: Table): TableConversions = {
+    new TableConversions(table)
+  }
+
+  implicit def dataSet2DataSetConversions[T](set: DataSet[T]): DataSetConversions[T] = {
+    new DataSetConversions[T](set, set.getType())
+  }
+
+  implicit def table2RowDataSet(table: Table): DataSet[Row] = {
+    val tableEnv = table.tableEnv.asInstanceOf[ScalaBatchTableEnv]
+    tableEnv.toDataSet[Row](table)
+  }
+
+  implicit def dataStream2DataStreamConversions[T](set: DataStream[T]): DataStreamConversions[T] = {
+    new DataStreamConversions[T](set, set.dataType.asInstanceOf[CompositeType[T]])
+  }
+
+  implicit def table2RowDataStream(table: Table): DataStream[Row] = {
+    val tableEnv = table.tableEnv.asInstanceOf[ScalaStreamTableEnv]
+    tableEnv.toDataStream[Row](table)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..6322026
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/table.scala
@@ -0,0 +1,924 @@
+/*
+ * 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 org.apache.calcite.rel.RelNode
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.operators.join.JoinType
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.logical.Minus
+import org.apache.flink.table.expressions.{Alias, Asc, Call, Expression, ExpressionParser, Ordering, TableFunctionCall, UnresolvedAlias}
+import org.apache.flink.table.plan.ProjectionTranslator._
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.sinks.TableSink
+
+import _root_.scala.collection.JavaConverters._
+
+/**
+  * A Table is the core component of the Table API.
+  * Similar to how the batch and streaming APIs have DataSet and DataStream,
+  * the Table API is built around [[Table]].
+  *
+  * Use the methods of [[Table]] to transform data. Use [[TableEnvironment]] to convert a [[Table]]
+  * back to a DataSet or DataStream.
+  *
+  * When using Scala a [[Table]] can also be converted using implicit conversions.
+  *
+  * Example:
+  *
+  * {{{
+  *   val env = ExecutionEnvironment.getExecutionEnvironment
+  *   val tEnv = TableEnvironment.getTableEnvironment(env)
+  *
+  *   val set: DataSet[(String, Int)] = ...
+  *   val table = set.toTable(tEnv, 'a, 'b)
+  *   ...
+  *   val table2 = ...
+  *   val set2: DataSet[MyType] = table2.toDataSet[MyType]
+  * }}}
+  *
+  * Operations such as [[join]], [[select]], [[where]] and [[groupBy]] either take arguments
+  * in a Scala DSL or as an expression String. Please refer to the documentation for the expression
+  * syntax.
+  *
+  * @param tableEnv The [[TableEnvironment]] to which the table is bound.
+  * @param logicalPlan logical representation
+  */
+class Table(
+    private[flink] val tableEnv: TableEnvironment,
+    private[flink] val logicalPlan: LogicalNode) {
+
+  def relBuilder = tableEnv.getRelBuilder
+
+  def getRelNode: RelNode = logicalPlan.toRelNode(relBuilder)
+
+  /**
+    * Performs a selection operation. Similar to an SQL SELECT statement. The field expressions
+    * can contain complex expressions and aggregations.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.select('key, 'value.avg + " The average" as 'average)
+    * }}}
+    */
+  def select(fields: Expression*): Table = {
+    val expandedFields = expandProjectList(fields, logicalPlan, tableEnv)
+    val (aggNames, propNames) = extractAggregationsAndProperties(expandedFields, tableEnv)
+    if (propNames.nonEmpty) {
+      throw ValidationException("Window properties can only be used on windowed tables.")
+    }
+
+    if (aggNames.nonEmpty) {
+      val projectsOnAgg = replaceAggregationsAndProperties(
+        expandedFields, tableEnv, aggNames, propNames)
+      val projectFields = extractFieldReferences(expandedFields)
+
+      new Table(tableEnv,
+        Project(projectsOnAgg,
+          Aggregate(Nil, aggNames.map(a => Alias(a._1, a._2)).toSeq,
+            Project(projectFields, logicalPlan).validate(tableEnv)
+          ).validate(tableEnv)
+        ).validate(tableEnv)
+      )
+    } else {
+      new Table(tableEnv,
+        Project(expandedFields.map(UnresolvedAlias), logicalPlan).validate(tableEnv))
+    }
+  }
+
+  /**
+    * Performs a selection operation. Similar to an SQL SELECT statement. The field expressions
+    * can contain complex expressions and aggregations.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.select("key, value.avg + ' The average' as average")
+    * }}}
+    */
+  def select(fields: String): Table = {
+    val fieldExprs = ExpressionParser.parseExpressionList(fields)
+    select(fieldExprs: _*)
+  }
+
+  /**
+    * Renames the fields of the expression result. Use this to disambiguate fields before
+    * joining to operations.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.as('a, 'b)
+    * }}}
+    */
+  def as(fields: Expression*): Table = {
+    new Table(tableEnv, AliasNode(fields, logicalPlan).validate(tableEnv))
+  }
+
+  /**
+    * Renames the fields of the expression result. Use this to disambiguate fields before
+    * joining to operations.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.as("a, b")
+    * }}}
+    */
+  def as(fields: String): Table = {
+    val fieldExprs = ExpressionParser.parseExpressionList(fields)
+    as(fieldExprs: _*)
+  }
+
+  /**
+    * Filters out elements that don't pass the filter predicate. Similar to a SQL WHERE
+    * clause.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.filter('name === "Fred")
+    * }}}
+    */
+  def filter(predicate: Expression): Table = {
+    new Table(tableEnv, Filter(predicate, logicalPlan).validate(tableEnv))
+  }
+
+  /**
+    * Filters out elements that don't pass the filter predicate. Similar to a SQL WHERE
+    * clause.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.filter("name = 'Fred'")
+    * }}}
+    */
+  def filter(predicate: String): Table = {
+    val predicateExpr = ExpressionParser.parseExpression(predicate)
+    filter(predicateExpr)
+  }
+
+  /**
+    * Filters out elements that don't pass the filter predicate. Similar to a SQL WHERE
+    * clause.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.where('name === "Fred")
+    * }}}
+    */
+  def where(predicate: Expression): Table = {
+    filter(predicate)
+  }
+
+  /**
+    * Filters out elements that don't pass the filter predicate. Similar to a SQL WHERE
+    * clause.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.where("name = 'Fred'")
+    * }}}
+    */
+  def where(predicate: String): Table = {
+    filter(predicate)
+  }
+
+  /**
+    * Groups the elements on some grouping keys. Use this before a selection with aggregations
+    * to perform the aggregation on a per-group basis. Similar to a SQL GROUP BY statement.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.groupBy('key).select('key, 'value.avg)
+    * }}}
+    */
+  def groupBy(fields: Expression*): GroupedTable = {
+    new GroupedTable(this, fields)
+  }
+
+  /**
+    * Groups the elements on some grouping keys. Use this before a selection with aggregations
+    * to perform the aggregation on a per-group basis. Similar to a SQL GROUP BY statement.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.groupBy("key").select("key, value.avg")
+    * }}}
+    */
+  def groupBy(fields: String): GroupedTable = {
+    val fieldsExpr = ExpressionParser.parseExpressionList(fields)
+    groupBy(fieldsExpr: _*)
+  }
+
+  /**
+    * Removes duplicate values and returns only distinct (different) values.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.select("key, value").distinct()
+    * }}}
+    */
+  def distinct(): Table = {
+    new Table(tableEnv, Distinct(logicalPlan).validate(tableEnv))
+  }
+
+  /**
+    * Joins two [[Table]]s. Similar to an SQL join. The fields of the two joined
+    * operations must not overlap, use [[as]] to rename fields if necessary. You can use
+    * where and select clauses after a join to further specify the behaviour of the join.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]].
+    *
+    * Example:
+    *
+    * {{{
+    *   left.join(right).where('a === 'b && 'c > 3).select('a, 'b, 'd)
+    * }}}
+    */
+  def join(right: Table): Table = {
+    join(right, None, JoinType.INNER)
+  }
+
+  /**
+    * Joins two [[Table]]s. Similar to an SQL join. The fields of the two joined
+    * operations must not overlap, use [[as]] to rename fields if necessary.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]].
+    *
+    * Example:
+    *
+    * {{{
+    *   left.join(right, "a = b")
+    * }}}
+    */
+  def join(right: Table, joinPredicate: String): Table = {
+    join(right, joinPredicate, JoinType.INNER)
+  }
+
+  /**
+    * Joins two [[Table]]s. Similar to an SQL join. The fields of the two joined
+    * operations must not overlap, use [[as]] to rename fields if necessary.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]].
+    *
+    * Example:
+    *
+    * {{{
+    *   left.join(right, 'a === 'b).select('a, 'b, 'd)
+    * }}}
+    */
+  def join(right: Table, joinPredicate: Expression): Table = {
+    join(right, Some(joinPredicate), JoinType.INNER)
+  }
+
+  /**
+    * Joins two [[Table]]s. Similar to an SQL left outer join. The fields of the two joined
+    * operations must not overlap, use [[as]] to rename fields if necessary.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
+    * have nullCheck enabled.
+    *
+    * Example:
+    *
+    * {{{
+    *   left.leftOuterJoin(right, "a = b").select('a, 'b, 'd)
+    * }}}
+    */
+  def leftOuterJoin(right: Table, joinPredicate: String): Table = {
+    join(right, joinPredicate, JoinType.LEFT_OUTER)
+  }
+
+  /**
+    * Joins two [[Table]]s. Similar to an SQL left outer join. The fields of the two joined
+    * operations must not overlap, use [[as]] to rename fields if necessary.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
+    * have nullCheck enabled.
+    *
+    * Example:
+    *
+    * {{{
+    *   left.leftOuterJoin(right, 'a === 'b).select('a, 'b, 'd)
+    * }}}
+    */
+  def leftOuterJoin(right: Table, joinPredicate: Expression): Table = {
+    join(right, Some(joinPredicate), JoinType.LEFT_OUTER)
+  }
+
+  /**
+    * Joins two [[Table]]s. Similar to an SQL right outer join. The fields of the two joined
+    * operations must not overlap, use [[as]] to rename fields if necessary.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
+    * have nullCheck enabled.
+    *
+    * Example:
+    *
+    * {{{
+    *   left.rightOuterJoin(right, "a = b").select('a, 'b, 'd)
+    * }}}
+    */
+  def rightOuterJoin(right: Table, joinPredicate: String): Table = {
+    join(right, joinPredicate, JoinType.RIGHT_OUTER)
+  }
+
+  /**
+    * Joins two [[Table]]s. Similar to an SQL right outer join. The fields of the two joined
+    * operations must not overlap, use [[as]] to rename fields if necessary.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
+    * have nullCheck enabled.
+    *
+    * Example:
+    *
+    * {{{
+    *   left.rightOuterJoin(right, 'a === 'b).select('a, 'b, 'd)
+    * }}}
+    */
+  def rightOuterJoin(right: Table, joinPredicate: Expression): Table = {
+    join(right, Some(joinPredicate), JoinType.RIGHT_OUTER)
+  }
+
+  /**
+    * Joins two [[Table]]s. Similar to an SQL full outer join. The fields of the two joined
+    * operations must not overlap, use [[as]] to rename fields if necessary.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
+    * have nullCheck enabled.
+    *
+    * Example:
+    *
+    * {{{
+    *   left.fullOuterJoin(right, "a = b").select('a, 'b, 'd)
+    * }}}
+    */
+  def fullOuterJoin(right: Table, joinPredicate: String): Table = {
+    join(right, joinPredicate, JoinType.FULL_OUTER)
+  }
+
+  /**
+    * Joins two [[Table]]s. Similar to an SQL full outer join. The fields of the two joined
+    * operations must not overlap, use [[as]] to rename fields if necessary.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
+    * have nullCheck enabled.
+    *
+    * Example:
+    *
+    * {{{
+    *   left.fullOuterJoin(right, 'a === 'b).select('a, 'b, 'd)
+    * }}}
+    */
+  def fullOuterJoin(right: Table, joinPredicate: Expression): Table = {
+    join(right, Some(joinPredicate), JoinType.FULL_OUTER)
+  }
+
+  private def join(right: Table, joinPredicate: String, joinType: JoinType): Table = {
+    val joinPredicateExpr = ExpressionParser.parseExpression(joinPredicate)
+    join(right, Some(joinPredicateExpr), joinType)
+  }
+
+  private def join(right: Table, joinPredicate: Option[Expression], joinType: JoinType): Table = {
+    // check that right table belongs to the same TableEnvironment
+    if (right.tableEnv != this.tableEnv) {
+      throw new ValidationException("Only tables from the same TableEnvironment can be joined.")
+    }
+    new Table(
+      tableEnv,
+      Join(this.logicalPlan, right.logicalPlan, joinType, joinPredicate, correlated = false)
+        .validate(tableEnv))
+  }
+
+  /**
+    * Minus of two [[Table]]s with duplicate records removed.
+    * Similar to a SQL EXCEPT clause. Minus returns records from the left table that do not
+    * exist in the right table. Duplicate records in the left table are returned
+    * exactly once, i.e., duplicates are removed. Both tables must have identical field types.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]].
+    *
+    * Example:
+    *
+    * {{{
+    *   left.minus(right)
+    * }}}
+    */
+  def minus(right: Table): Table = {
+    // check that right table belongs to the same TableEnvironment
+    if (right.tableEnv != this.tableEnv) {
+      throw new ValidationException("Only tables from the same TableEnvironment can be " +
+        "subtracted.")
+    }
+    new Table(tableEnv, Minus(logicalPlan, right.logicalPlan, all = false)
+      .validate(tableEnv))
+  }
+
+  /**
+    * Minus of two [[Table]]s. Similar to an SQL EXCEPT ALL.
+    * Similar to a SQL EXCEPT ALL clause. MinusAll returns the records that do not exist in
+    * the right table. A record that is present n times in the left table and m times
+    * in the right table is returned (n - m) times, i.e., as many duplicates as are present
+    * in the right table are removed. Both tables must have identical field types.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]].
+    *
+    * Example:
+    *
+    * {{{
+    *   left.minusAll(right)
+    * }}}
+    */
+  def minusAll(right: Table): Table = {
+    // check that right table belongs to the same TableEnvironment
+    if (right.tableEnv != this.tableEnv) {
+      throw new ValidationException("Only tables from the same TableEnvironment can be " +
+        "subtracted.")
+    }
+    new Table(tableEnv, Minus(logicalPlan, right.logicalPlan, all = true)
+      .validate(tableEnv))
+  }
+
+  /**
+    * Unions two [[Table]]s with duplicate records removed.
+    * Similar to an SQL UNION. The fields of the two union operations must fully overlap.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]].
+    *
+    * Example:
+    *
+    * {{{
+    *   left.union(right)
+    * }}}
+    */
+  def union(right: Table): Table = {
+    // check that right table belongs to the same TableEnvironment
+    if (right.tableEnv != this.tableEnv) {
+      throw new ValidationException("Only tables from the same TableEnvironment can be unioned.")
+    }
+    new Table(tableEnv, Union(logicalPlan, right.logicalPlan, all = false).validate(tableEnv))
+  }
+
+  /**
+    * Unions two [[Table]]s. Similar to an SQL UNION ALL. The fields of the two union operations
+    * must fully overlap.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]].
+    *
+    * Example:
+    *
+    * {{{
+    *   left.unionAll(right)
+    * }}}
+    */
+  def unionAll(right: Table): Table = {
+    // check that right table belongs to the same TableEnvironment
+    if (right.tableEnv != this.tableEnv) {
+      throw new ValidationException("Only tables from the same TableEnvironment can be unioned.")
+    }
+    new Table(tableEnv, Union(logicalPlan, right.logicalPlan, all = true).validate(tableEnv))
+  }
+
+  /**
+    * Intersects two [[Table]]s with duplicate records removed. Intersect returns records that
+    * exist in both tables. If a record is present in one or both tables more than once, it is
+    * returned just once, i.e., the resulting table has no duplicate records. Similar to an
+    * SQL INTERSECT. The fields of the two intersect operations must fully overlap.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]].
+    *
+    * Example:
+    *
+    * {{{
+    *   left.intersect(right)
+    * }}}
+    */
+  def intersect(right: Table): Table = {
+    // check that right table belongs to the same TableEnvironment
+    if (right.tableEnv != this.tableEnv) {
+      throw new ValidationException(
+        "Only tables from the same TableEnvironment can be intersected.")
+    }
+    new Table(tableEnv, Intersect(logicalPlan, right.logicalPlan, all = false).validate(tableEnv))
+  }
+
+  /**
+    * Intersects two [[Table]]s. IntersectAll returns records that exist in both tables.
+    * If a record is present in both tables more than once, it is returned as many times as it
+    * is present in both tables, i.e., the resulting table might have duplicate records. Similar
+    * to an SQL INTERSECT ALL. The fields of the two intersect operations must fully overlap.
+    *
+    * Note: Both tables must be bound to the same [[TableEnvironment]].
+    *
+    * Example:
+    *
+    * {{{
+    *   left.intersectAll(right)
+    * }}}
+    */
+  def intersectAll(right: Table): Table = {
+    // check that right table belongs to the same TableEnvironment
+    if (right.tableEnv != this.tableEnv) {
+      throw new ValidationException(
+        "Only tables from the same TableEnvironment can be intersected.")
+    }
+    new Table(tableEnv, Intersect(logicalPlan, right.logicalPlan, all = true).validate(tableEnv))
+  }
+
+  /**
+    * Sorts the given [[Table]]. Similar to SQL ORDER BY.
+    * The resulting Table is globally sorted across all parallel partitions.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.orderBy('name.desc)
+    * }}}
+    */
+  def orderBy(fields: Expression*): Table = {
+    val order: Seq[Ordering] = fields.map {
+      case o: Ordering => o
+      case e => Asc(e)
+    }
+    new Table(tableEnv, Sort(order, logicalPlan).validate(tableEnv))
+  }
+
+  /**
+    * Sorts the given [[Table]]. Similar to SQL ORDER BY.
+    * The resulting Table is sorted globally sorted across all parallel partitions.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.orderBy("name.desc")
+    * }}}
+    */
+  def orderBy(fields: String): Table = {
+    val parsedFields = ExpressionParser.parseExpressionList(fields)
+    orderBy(parsedFields: _*)
+  }
+
+  /**
+    * Limits a sorted result from an offset position.
+    * Similar to a SQL LIMIT clause. Limit is technically part of the Order By operator and
+    * thus must be preceded by it.
+    *
+    * Example:
+    *
+    * {{{
+    *   // returns unlimited number of records beginning with the 4th record
+    *   tab.orderBy('name.desc).limit(3)
+    * }}}
+    *
+    * @param offset number of records to skip
+    */
+  def limit(offset: Int): Table = {
+    new Table(tableEnv, Limit(offset = offset, child = logicalPlan).validate(tableEnv))
+  }
+
+  /**
+    * Limits a sorted result to a specified number of records from an offset position.
+    * Similar to a SQL LIMIT clause. Limit is technically part of the Order By operator and
+    * thus must be preceded by it.
+    *
+    * Example:
+    *
+    * {{{
+    *   // returns 5 records beginning with the 4th record
+    *   tab.orderBy('name.desc).limit(3, 5)
+    * }}}
+    *
+    * @param offset number of records to skip
+    * @param fetch number of records to be returned
+    */
+  def limit(offset: Int, fetch: Int): Table = {
+    new Table(tableEnv, Limit(offset, fetch, logicalPlan).validate(tableEnv))
+  }
+
+  /**
+    * Joins this [[Table]] to a user-defined [[org.apache.calcite.schema.TableFunction]]. Similar
+    * to an SQL cross join, but it works with a table function. It returns rows from the outer
+    * table (table on the left of the operator) that produces matching values from the table
+    * function (which is defined in the expression on the right side of the operator).
+    *
+    * Example:
+    *
+    * {{{
+    *   class MySplitUDTF extends TableFunction[String] {
+    *     def eval(str: String): Unit = {
+    *       str.split("#").foreach(collect)
+    *     }
+    *   }
+    *
+    *   val split = new MySplitUDTF()
+    *   table.join(split('c) as ('s)).select('a,'b,'c,'s)
+    * }}}
+    */
+  def join(udtf: Expression): Table = {
+    joinUdtfInternal(udtf, JoinType.INNER)
+  }
+
+  /**
+    * Joins this [[Table]] to a user-defined [[org.apache.calcite.schema.TableFunction]]. Similar
+    * to an SQL cross join, but it works with a table function. It returns rows from the outer
+    * table (table on the left of the operator) that produces matching values from the table
+    * function (which is defined in the expression on the right side of the operator).
+    *
+    * Example:
+    *
+    * {{{
+    *   class MySplitUDTF extends TableFunction<String> {
+    *     public void eval(String str) {
+    *       str.split("#").forEach(this::collect);
+    *     }
+    *   }
+    *
+    *   TableFunction<String> split = new MySplitUDTF();
+    *   tableEnv.registerFunction("split", split);
+    *
+    *   table.join("split(c) as (s)").select("a, b, c, s");
+    * }}}
+    */
+  def join(udtf: String): Table = {
+    joinUdtfInternal(udtf, JoinType.INNER)
+  }
+
+  /**
+    * Joins this [[Table]] to a user-defined [[org.apache.calcite.schema.TableFunction]]. Similar
+    * to an SQL left outer join with ON TRUE, but it works with a table function. It returns all
+    * the rows from the outer table (table on the left of the operator), and rows that do not match
+    * the condition from the table function (which is defined in the expression on the right
+    * side of the operator). Rows with no matching condition are filled with null values.
+    *
+    * Example:
+    *
+    * {{{
+    *   class MySplitUDTF extends TableFunction[String] {
+    *     def eval(str: String): Unit = {
+    *       str.split("#").foreach(collect)
+    *     }
+    *   }
+    *
+    *   val split = new MySplitUDTF()
+    *   table.leftOuterJoin(split('c) as ('s)).select('a,'b,'c,'s)
+    * }}}
+    */
+  def leftOuterJoin(udtf: Expression): Table = {
+    joinUdtfInternal(udtf, JoinType.LEFT_OUTER)
+  }
+
+  /**
+    * Joins this [[Table]] to a user-defined [[org.apache.calcite.schema.TableFunction]]. Similar
+    * to an SQL left outer join with ON TRUE, but it works with a table function. It returns all
+    * the rows from the outer table (table on the left of the operator), and rows that do not match
+    * the condition from the table function (which is defined in the expression on the right
+    * side of the operator). Rows with no matching condition are filled with null values.
+    *
+    * Example:
+    *
+    * {{{
+    *   class MySplitUDTF extends TableFunction<String> {
+    *     public void eval(String str) {
+    *       str.split("#").forEach(this::collect);
+    *     }
+    *   }
+    *
+    *   TableFunction<String> split = new MySplitUDTF();
+    *   tableEnv.registerFunction("split", split);
+    *
+    *   table.leftOuterJoin("split(c) as (s)").select("a, b, c, s");
+    * }}}
+    */
+  def leftOuterJoin(udtf: String): Table = {
+    joinUdtfInternal(udtf, JoinType.LEFT_OUTER)
+  }
+
+  private def joinUdtfInternal(udtfString: String, joinType: JoinType): Table = {
+    val udtf = ExpressionParser.parseExpression(udtfString)
+    joinUdtfInternal(udtf, joinType)
+  }
+
+  private def joinUdtfInternal(udtf: Expression, joinType: JoinType): Table = {
+    var alias: Option[Seq[String]] = None
+
+    // unwrap an Expression until we get a TableFunctionCall
+    def unwrap(expr: Expression): TableFunctionCall = expr match {
+      case Alias(child, name, extraNames) =>
+        alias = Some(Seq(name) ++ extraNames)
+        unwrap(child)
+      case Call(name, args) =>
+        val function = tableEnv.getFunctionCatalog.lookupFunction(name, args)
+        unwrap(function)
+      case c: TableFunctionCall => c
+      case _ =>
+        throw new TableException(
+          "Cross/Outer Apply operators only accept expressions that define table functions.")
+    }
+
+    val call = unwrap(udtf)
+      .as(alias)
+      .toLogicalTableFunctionCall(this.logicalPlan)
+      .validate(tableEnv)
+
+    new Table(
+      tableEnv,
+      Join(this.logicalPlan, call, joinType, None, correlated = true).validate(tableEnv))
+  }
+
+  /**
+    * 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.StreamTableSink]].
+    *
+    * @param sink The [[TableSink]] to which the [[Table]] is written.
+    * @tparam T The data type that the [[TableSink]] expects.
+    */
+  def writeToSink[T](sink: TableSink[T]): Unit = {
+
+    // get schema information of table
+    val rowType = getRelNode.getRowType
+    val fieldNames: Array[String] = rowType.getFieldNames.asScala.toArray
+    val fieldTypes: Array[TypeInformation[_]] = rowType.getFieldList.asScala
+      .map(field => FlinkTypeFactory.toTypeInfo(field.getType)).toArray
+
+    // configure the table sink
+    val configuredSink = sink.configure(fieldNames, fieldTypes)
+
+    // emit the table to the configured table sink
+    tableEnv.writeToSink(this, configuredSink)
+  }
+
+  /**
+    * Groups the records of a table by assigning them to windows defined by a time or row interval.
+    *
+    * For streaming tables of infinite size, grouping into windows is required to define finite
+    * groups on which group-based aggregates can be computed.
+    *
+    * For batch tables of finite size, windowing essentially provides shortcuts for time-based
+    * groupBy.
+    *
+    * __Note__: window on non-grouped streaming table is a non-parallel operation, i.e., all data
+    * will be processed by a single operator.
+    *
+    * @param groupWindow group-window that specifies how elements are grouped.
+    * @return A windowed table.
+    */
+  def window(groupWindow: GroupWindow): GroupWindowedTable = {
+    if (tableEnv.isInstanceOf[BatchTableEnvironment]) {
+      throw new ValidationException(s"Windows on batch tables are currently not supported.")
+    }
+    new GroupWindowedTable(this, Seq(), groupWindow)
+  }
+}
+
+/**
+  * A table that has been grouped on a set of grouping keys.
+  */
+class GroupedTable(
+  private[flink] val table: Table,
+  private[flink] val groupKey: Seq[Expression]) {
+
+  /**
+    * Performs a selection operation on a grouped table. Similar to an SQL SELECT statement.
+    * The field expressions can contain complex expressions and aggregations.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.groupBy('key).select('key, 'value.avg + " The average" as 'average)
+    * }}}
+    */
+  def select(fields: Expression*): Table = {
+    val (aggNames, propNames) = extractAggregationsAndProperties(fields, table.tableEnv)
+    if (propNames.nonEmpty) {
+      throw ValidationException("Window properties can only be used on windowed tables.")
+    }
+
+    val projectsOnAgg = replaceAggregationsAndProperties(
+      fields, table.tableEnv, aggNames, propNames)
+    val projectFields = extractFieldReferences(fields ++ groupKey)
+
+    new Table(table.tableEnv,
+      Project(projectsOnAgg,
+        Aggregate(groupKey, aggNames.map(a => Alias(a._1, a._2)).toSeq,
+          Project(projectFields, table.logicalPlan).validate(table.tableEnv)
+        ).validate(table.tableEnv)
+      ).validate(table.tableEnv))
+  }
+
+  /**
+    * Performs a selection operation on a grouped table. Similar to an SQL SELECT statement.
+    * The field expressions can contain complex expressions and aggregations.
+    *
+    * Example:
+    *
+    * {{{
+    *   tab.groupBy("key").select("key, value.avg + ' The average' as average")
+    * }}}
+    */
+  def select(fields: String): Table = {
+    val fieldExprs = ExpressionParser.parseExpressionList(fields)
+    select(fieldExprs: _*)
+  }
+
+  /**
+    * Groups the records of a table by assigning them to windows defined by a time or row interval.
+    *
+    * For streaming tables of infinite size, grouping into windows is required to define finite
+    * groups on which group-based aggregates can be computed.
+    *
+    * For batch tables of finite size, windowing essentially provides shortcuts for time-based
+    * groupBy.
+    *
+    * @param groupWindow group-window that specifies how elements are grouped.
+    * @return A windowed table.
+    */
+  def window(groupWindow: GroupWindow): GroupWindowedTable = {
+    if (table.tableEnv.isInstanceOf[BatchTableEnvironment]) {
+      throw new ValidationException(s"Windows on batch tables are currently not supported.")
+    }
+    new GroupWindowedTable(table, groupKey, groupWindow)
+  }
+}
+
+class GroupWindowedTable(
+    private[flink] val table: Table,
+    private[flink] val groupKey: Seq[Expression],
+    private[flink] val window: GroupWindow) {
+
+  /**
+    * Performs a selection operation on a windowed table. Similar to an SQL SELECT statement.
+    * The field expressions can contain complex expressions and aggregations.
+    *
+    * Example:
+    *
+    * {{{
+    *   groupWindowTable.select('key, 'window.start, 'value.avg + " The average" as 'average)
+    * }}}
+    */
+  def select(fields: Expression*): Table = {
+    val (aggNames, propNames) = extractAggregationsAndProperties(fields, table.tableEnv)
+    val projectsOnAgg = replaceAggregationsAndProperties(
+      fields, table.tableEnv, aggNames, propNames)
+
+    val projectFields = (table.tableEnv, window) match {
+      // event time can be arbitrary field in batch environment
+      case (_: BatchTableEnvironment, w: EventTimeWindow) =>
+        extractFieldReferences(fields ++ groupKey ++ Seq(w.timeField))
+      case (_, _) =>
+        extractFieldReferences(fields ++ groupKey)
+    }
+
+    new Table(table.tableEnv,
+      Project(
+        projectsOnAgg,
+        WindowAggregate(
+          groupKey,
+          window.toLogicalWindow,
+          propNames.map(a => Alias(a._1, a._2)).toSeq,
+          aggNames.map(a => Alias(a._1, a._2)).toSeq,
+          Project(projectFields, table.logicalPlan).validate(table.tableEnv)
+        ).validate(table.tableEnv)
+      ).validate(table.tableEnv))
+  }
+
+  /**
+    * Performs a selection operation on a group-windows table. Similar to an SQL SELECT statement.
+    * The field expressions can contain complex expressions and aggregations.
+    *
+    * Example:
+    *
+    * {{{
+    *   groupWindowTable.select("key, window.start, value.avg + ' The average' as average")
+    * }}}
+    */
+  def select(fields: String): Table = {
+    val fieldExprs = ExpressionParser.parseExpressionList(fields)
+    select(fieldExprs: _*)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/windows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/windows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/windows.scala
new file mode 100644
index 0000000..7e4498d
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/windows.scala
@@ -0,0 +1,390 @@
+/*
+ * 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 org.apache.flink.table.expressions.{Expression, ExpressionParser}
+import org.apache.flink.table.plan.logical._
+
+/**
+  * A group-window specification.
+  *
+  * Group-windows group rows based on time or row-count intervals and is therefore essentially a
+  * special type of groupBy. Just like groupBy, group-windows allow to compute aggregates
+  * on groups of elements.
+  *
+  * Infinite streaming tables can only be grouped into time or row intervals. Hence window grouping
+  * is required to apply aggregations on streaming tables.
+  *
+  * For finite batch tables, group-windows provide shortcuts for time-based groupBy.
+  *
+  */
+trait GroupWindow {
+
+  /**
+    * Converts an API class to a logical window for planning.
+    */
+  private[flink] def toLogicalWindow: LogicalWindow
+}
+
+/**
+  * A group-window operating on event-time.
+  *
+  * @param timeField defines the time mode for streaming tables. For batch table it defines the
+  *                  time attribute on which is grouped.
+  */
+abstract class EventTimeWindow(val timeField: Expression) extends GroupWindow {
+
+  protected var name: Option[Expression] = None
+
+  /**
+    * Assigns an alias for this window that the following `select()` clause can refer to in order
+    * to access window properties such as window start or end time.
+    *
+    * @param alias alias for this window
+    * @return this window
+    */
+  def as(alias: Expression): EventTimeWindow = {
+    this.name = Some(alias)
+    this
+  }
+
+  /**
+    * Assigns an alias for this window that the following `select()` clause can refer to in order
+    * to access window properties such as window start or end time.
+    *
+    * @param alias alias for this window
+    * @return this window
+    */
+  def as(alias: String): EventTimeWindow = as(ExpressionParser.parseExpression(alias))
+}
+
+// ------------------------------------------------------------------------------------------------
+// Tumbling group-windows
+// ------------------------------------------------------------------------------------------------
+
+/**
+  * Tumbling group-window.
+  *
+  * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows are
+  * grouped by processing-time.
+  *
+  * @param size the size of the window either as time or row-count interval.
+  */
+class TumblingWindow(size: Expression) extends GroupWindow {
+
+  /**
+    * Tumbling group-window.
+    *
+    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
+    * are grouped by processing-time.
+    *
+    * @param size the size of the window either as time or row-count interval.
+    */
+  def this(size: String) = this(ExpressionParser.parseExpression(size))
+
+  private var alias: Option[Expression] = None
+
+  /**
+    * Specifies the time attribute on which rows are grouped.
+    *
+    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
+    * are grouped by processing-time.
+    *
+    * For batch tables, refer to a timestamp or long attribute.
+    *
+    * @param timeField time mode for streaming tables and time attribute for batch tables
+    * @return a tumbling group-window on event-time
+    */
+  def on(timeField: Expression): TumblingEventTimeWindow =
+    new TumblingEventTimeWindow(alias, timeField, size)
+
+  /**
+    * Specifies the time attribute on which rows are grouped.
+    *
+    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
+    * are grouped by processing-time.
+    *
+    * For batch tables, refer to a timestamp or long attribute.
+    *
+    * @param timeField time mode for streaming tables and time attribute for batch tables
+    * @return a tumbling group-window on event-time
+    */
+  def on(timeField: String): TumblingEventTimeWindow =
+    on(ExpressionParser.parseExpression(timeField))
+
+  /**
+    * Assigns an alias for this window that the following `select()` clause can refer to in order
+    * to access window properties such as window start or end time.
+    *
+    * @param alias alias for this window
+    * @return this window
+    */
+  def as(alias: Expression): TumblingWindow = {
+    this.alias = Some(alias)
+    this
+  }
+
+  /**
+    * Assigns an alias for this window that the following `select()` clause can refer to in order
+    * to access window properties such as window start or end time.
+    *
+    * @param alias alias for this window
+    * @return this window
+    */
+  def as(alias: String): TumblingWindow = as(ExpressionParser.parseExpression(alias))
+
+  override private[flink] def toLogicalWindow: LogicalWindow =
+    ProcessingTimeTumblingGroupWindow(alias, size)
+}
+
+/**
+  * Tumbling group-window on event-time.
+  */
+class TumblingEventTimeWindow(
+    alias: Option[Expression],
+    time: Expression,
+    size: Expression)
+  extends EventTimeWindow(time) {
+
+  override private[flink] def toLogicalWindow: LogicalWindow =
+    EventTimeTumblingGroupWindow(name.orElse(alias), time, size)
+}
+
+// ------------------------------------------------------------------------------------------------
+// Sliding group windows
+// ------------------------------------------------------------------------------------------------
+
+/**
+  * Partially specified sliding window.
+  *
+  * @param size the size of the window either as time or row-count interval.
+  */
+class SlideWithSize(size: Expression) {
+
+  /**
+    * Partially specified sliding window.
+    *
+    * @param size the size of the window either as time or row-count interval.
+    */
+  def this(size: String) = this(ExpressionParser.parseExpression(size))
+
+  /**
+    * Specifies the window's slide as time or row-count interval.
+    *
+    * The slide determines the interval in which windows are started. Hence, sliding windows can
+    * overlap if the slide is smaller than the size of the window.
+    *
+    * For example, you could have windows of size 15 minutes that slide by 3 minutes. With this
+    * 15 minutes worth of elements are grouped every 3 minutes and each row contributes to 5
+    * windows.
+    *
+    * @param slide the slide of the window either as time or row-count interval.
+    * @return a sliding group-window
+    */
+  def every(slide: Expression): SlidingWindow = new SlidingWindow(size, slide)
+
+  /**
+    * Specifies the window's slide as time or row-count interval.
+    *
+    * The slide determines the interval in which windows are started. Hence, sliding windows can
+    * overlap if the slide is smaller than the size of the window.
+    *
+    * For example, you could have windows of size 15 minutes that slide by 3 minutes. With this
+    * 15 minutes worth of elements are grouped every 3 minutes and each row contributes to 5
+    * windows.
+    *
+    * @param slide the slide of the window either as time or row-count interval.
+    * @return a sliding group-window
+    */
+  def every(slide: String): SlidingWindow = every(ExpressionParser.parseExpression(slide))
+}
+
+/**
+  * Sliding group-window.
+  *
+  * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows are
+  * grouped by processing-time.
+  *
+  * @param size the size of the window either as time or row-count interval.
+  */
+class SlidingWindow(
+    size: Expression,
+    slide: Expression)
+  extends GroupWindow {
+
+  private var alias: Option[Expression] = None
+
+  /**
+    * Specifies the time attribute on which rows are grouped.
+    *
+    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
+    * are grouped by processing-time.
+    *
+    * For batch tables, refer to a timestamp or long attribute.
+    *
+    * @param timeField time mode for streaming tables and time attribute for batch tables
+    * @return a sliding group-window on event-time
+    */
+  def on(timeField: Expression): SlidingEventTimeWindow =
+    new SlidingEventTimeWindow(alias, timeField, size, slide)
+
+  /**
+    * Specifies the time attribute on which rows are grouped.
+    *
+    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
+    * are grouped by processing-time.
+    *
+    * For batch tables, refer to a timestamp or long attribute.
+    *
+    * @param timeField time mode for streaming tables and time attribute for batch tables
+    * @return a sliding group-window on event-time
+    */
+  def on(timeField: String): SlidingEventTimeWindow =
+    on(ExpressionParser.parseExpression(timeField))
+
+  /**
+    * Assigns an alias for this window that the following `select()` clause can refer to in order
+    * to access window properties such as window start or end time.
+    *
+    * @param alias alias for this window
+    * @return this window
+    */
+  def as(alias: Expression): SlidingWindow = {
+    this.alias = Some(alias)
+    this
+  }
+
+  /**
+    * Assigns an alias for this window that the following `select()` clause can refer to in order
+    * to access window properties such as window start or end time.
+    *
+    * @param alias alias for this window
+    * @return this window
+    */
+  def as(alias: String): SlidingWindow = as(ExpressionParser.parseExpression(alias))
+
+  override private[flink] def toLogicalWindow: LogicalWindow =
+    ProcessingTimeSlidingGroupWindow(alias, size, slide)
+}
+
+/**
+  * Sliding group-window on event-time.
+  */
+class SlidingEventTimeWindow(
+    alias: Option[Expression],
+    timeField: Expression,
+    size: Expression,
+    slide: Expression)
+  extends EventTimeWindow(timeField) {
+
+  override private[flink] def toLogicalWindow: LogicalWindow =
+    EventTimeSlidingGroupWindow(name.orElse(alias), timeField, size, slide)
+}
+
+// ------------------------------------------------------------------------------------------------
+// Session group windows
+// ------------------------------------------------------------------------------------------------
+
+/**
+  * Session group-window.
+  *
+  * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows are
+  * grouped by processing-time.
+  *
+  * @param gap the time interval of inactivity before a window is closed.
+  */
+class SessionWindow(gap: Expression) extends GroupWindow {
+
+  /**
+    * Session group-window.
+    *
+    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
+    * are grouped by processing-time.
+    *
+    * @param gap the time interval of inactivity before a window is closed.
+    */
+  def this(gap: String) = this(ExpressionParser.parseExpression(gap))
+
+  private var alias: Option[Expression] = None
+
+  /**
+    * Specifies the time attribute on which rows are grouped.
+    *
+    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
+    * are grouped by processing-time.
+    *
+    * For batch tables, refer to a timestamp or long attribute.
+    *
+    * @param timeField time mode for streaming tables and time attribute for batch tables
+    * @return a session group-window on event-time
+    */
+  def on(timeField: Expression): SessionEventTimeWindow =
+    new SessionEventTimeWindow(alias, timeField, gap)
+
+  /**
+    * Specifies the time attribute on which rows are grouped.
+    *
+    * For streaming tables call [[on('rowtime)]] to specify grouping by event-time. Otherwise rows
+    * are grouped by processing-time.
+    *
+    * For batch tables, refer to a timestamp or long attribute.
+    *
+    * @param timeField time mode for streaming tables and time attribute for batch tables
+    * @return a session group-window on event-time
+    */
+  def on(timeField: String): SessionEventTimeWindow =
+    on(ExpressionParser.parseExpression(timeField))
+
+  /**
+    * Assigns an alias for this window that the following `select()` clause can refer to in order
+    * to access window properties such as window start or end time.
+    *
+    * @param alias alias for this window
+    * @return this window
+    */
+  def as(alias: Expression): SessionWindow = {
+    this.alias = Some(alias)
+    this
+  }
+
+  /**
+    * Assigns an alias for this window that the following `select()` clause can refer to in order
+    * to access window properties such as window start or end time.
+    *
+    * @param alias alias for this window
+    * @return this window
+    */
+  def as(alias: String): SessionWindow = as(ExpressionParser.parseExpression(alias))
+
+  override private[flink] def toLogicalWindow: LogicalWindow =
+    ProcessingTimeSessionGroupWindow(alias, gap)
+}
+
+/**
+  * Session group-window on event-time.
+  */
+class SessionEventTimeWindow(
+    alias: Option[Expression],
+    timeField: Expression,
+    gap: Expression)
+  extends EventTimeWindow(timeField) {
+
+  override private[flink] def toLogicalWindow: LogicalWindow =
+    EventTimeSessionGroupWindow(name.orElse(alias), timeField, gap)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
new file mode 100644
index 0000000..f646caf
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.calcite
+
+import org.apache.calcite.plan.RelOptRule
+import org.apache.calcite.sql.SqlOperatorTable
+import org.apache.calcite.sql.parser.SqlParser
+import org.apache.calcite.sql.util.ChainedSqlOperatorTable
+import org.apache.calcite.tools.{RuleSet, RuleSets}
+import org.apache.flink.util.Preconditions
+
+import scala.collection.JavaConverters._
+
+/**
+  * Builder for creating a Calcite configuration.
+  */
+class CalciteConfigBuilder {
+  private var replaceRules: Boolean = false
+  private var ruleSets: List[RuleSet] = Nil
+
+  private var replaceOperatorTable: Boolean = false
+  private var operatorTables: List[SqlOperatorTable] = Nil
+
+  private var replaceSqlParserConfig: Option[SqlParser.Config] = None
+
+  /**
+    * Replaces the built-in rule set with the given rule set.
+    */
+  def replaceRuleSet(replaceRuleSet: RuleSet): CalciteConfigBuilder = {
+    Preconditions.checkNotNull(replaceRuleSet)
+    ruleSets = List(replaceRuleSet)
+    replaceRules = true
+    this
+  }
+
+  /**
+    * Appends the given rule set to the built-in rule set.
+    */
+  def addRuleSet(addedRuleSet: RuleSet): CalciteConfigBuilder = {
+    Preconditions.checkNotNull(addedRuleSet)
+    ruleSets = addedRuleSet :: ruleSets
+    this
+  }
+
+  /**
+    * Replaces the built-in SQL operator table with the given table.
+    */
+  def replaceSqlOperatorTable(replaceSqlOperatorTable: SqlOperatorTable): CalciteConfigBuilder = {
+    Preconditions.checkNotNull(replaceSqlOperatorTable)
+    operatorTables = List(replaceSqlOperatorTable)
+    replaceOperatorTable = true
+    this
+  }
+
+  /**
+    * Appends the given table to the built-in SQL operator table.
+    */
+  def addSqlOperatorTable(addedSqlOperatorTable: SqlOperatorTable): CalciteConfigBuilder = {
+    Preconditions.checkNotNull(addedSqlOperatorTable)
+    this.operatorTables = addedSqlOperatorTable :: this.operatorTables
+    this
+  }
+
+  /**
+    * Replaces the built-in SQL parser configuration with the given configuration.
+    */
+  def replaceSqlParserConfig(sqlParserConfig: SqlParser.Config): CalciteConfigBuilder = {
+    Preconditions.checkNotNull(sqlParserConfig)
+    replaceSqlParserConfig = Some(sqlParserConfig)
+    this
+  }
+
+  private class CalciteConfigImpl(
+      val getRuleSet: Option[RuleSet],
+      val replacesRuleSet: Boolean,
+      val getSqlOperatorTable: Option[SqlOperatorTable],
+      val replacesSqlOperatorTable: Boolean,
+      val getSqlParserConfig: Option[SqlParser.Config])
+    extends CalciteConfig
+
+  /**
+    * Builds a new [[CalciteConfig]].
+    */
+  def build(): CalciteConfig = new CalciteConfigImpl(
+        ruleSets match {
+      case Nil => None
+      case h :: Nil => Some(h)
+      case _ =>
+        // concat rule sets
+        val concatRules = ruleSets.foldLeft(Nil: Iterable[RelOptRule])( (c, r) => r.asScala ++ c)
+        Some(RuleSets.ofList(concatRules.asJava))
+    },
+    this.replaceRules,
+    operatorTables match {
+      case Nil => None
+      case h :: Nil => Some(h)
+      case _ =>
+        // chain operator tables
+        Some(operatorTables.reduce( (x, y) => ChainedSqlOperatorTable.of(x, y)))
+    },
+    this.replaceOperatorTable,
+    replaceSqlParserConfig)
+}
+
+/**
+  * Calcite configuration for defining a custom Calcite configuration for Table and SQL API.
+  */
+trait CalciteConfig {
+  /**
+    * Returns whether this configuration replaces the built-in rule set.
+    */
+  def replacesRuleSet: Boolean
+
+  /**
+    * Returns a custom rule set.
+    */
+  def getRuleSet: Option[RuleSet]
+
+  /**
+    * Returns whether this configuration replaces the built-in SQL operator table.
+    */
+  def replacesSqlOperatorTable: Boolean
+
+  /**
+    * Returns a custom SQL operator table.
+    */
+  def getSqlOperatorTable: Option[SqlOperatorTable]
+
+  /**
+    * Returns a custom SQL parser configuration.
+    */
+  def getSqlParserConfig: Option[SqlParser.Config]
+}
+
+object CalciteConfig {
+
+  val DEFAULT = createBuilder().build()
+
+  /**
+    * Creates a new builder for constructing a [[CalciteConfig]].
+    */
+  def createBuilder(): CalciteConfigBuilder = {
+    new CalciteConfigBuilder
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkCalciteSqlValidator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkCalciteSqlValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkCalciteSqlValidator.scala
new file mode 100644
index 0000000..b4a3c42
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/calcite/FlinkCalciteSqlValidator.scala
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.calcite
+
+import org.apache.calcite.adapter.java.JavaTypeFactory
+import org.apache.calcite.prepare.CalciteCatalogReader
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.sql.validate.{SqlConformance, SqlValidatorImpl}
+import org.apache.calcite.sql.{SqlInsert, SqlOperatorTable}
+
+/**
+ * This is a copy of Calcite's CalciteSqlValidator to use with [[FlinkPlannerImpl]].
+ */
+class FlinkCalciteSqlValidator(
+    opTab: SqlOperatorTable,
+    catalogReader: CalciteCatalogReader,
+    typeFactory: JavaTypeFactory) extends SqlValidatorImpl(
+        opTab, catalogReader, typeFactory, SqlConformance.DEFAULT) {
+
+  override def getLogicalSourceRowType(
+      sourceRowType: RelDataType,
+      insert: SqlInsert): RelDataType = {
+    typeFactory.asInstanceOf[JavaTypeFactory].toSql(sourceRowType)
+  }
+
+  override def getLogicalTargetRowType(
+      targetRowType: RelDataType,
+      insert: SqlInsert): RelDataType = {
+    typeFactory.asInstanceOf[JavaTypeFactory].toSql(targetRowType)
+  }
+}


[25/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..13fe4c3
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/CodeGenerator.scala
@@ -0,0 +1,1522 @@
+/*
+ * 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.codegen
+
+import java.math.{BigDecimal => JBigDecimal}
+
+import org.apache.calcite.avatica.util.DateTimeUtils
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.SqlOperator
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable._
+import org.apache.flink.api.common.functions.{FlatJoinFunction, FlatMapFunction, Function, MapFunction}
+import org.apache.flink.api.common.io.GenericInputFormat
+import org.apache.flink.api.common.typeinfo.{AtomicType, SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.api.java.typeutils.{GenericTypeInfo, PojoTypeInfo, RowTypeInfo, TupleTypeInfo}
+import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.codegen.CodeGenUtils._
+import org.apache.flink.table.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE}
+import org.apache.flink.table.codegen.Indenter.toISC
+import org.apache.flink.table.codegen.calls.FunctionGenerator
+import org.apache.flink.table.codegen.calls.ScalarOperators._
+import org.apache.flink.table.functions.UserDefinedFunction
+import org.apache.flink.table.typeutils.TypeConverter
+import org.apache.flink.table.typeutils.TypeCheckUtils._
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable
+
+/**
+  * A code generator for generating Flink [[org.apache.flink.api.common.functions.Function]]s.
+  *
+  * @param config configuration that determines runtime behavior
+  * @param nullableInput input(s) can be null.
+  * @param input1 type information about the first input of the Function
+  * @param input2 type information about the second input if the Function is binary
+  * @param input1PojoFieldMapping additional mapping information if input1 is a POJO (POJO types
+  *                              have no deterministic field order).
+  * @param input2PojoFieldMapping additional mapping information if input2 is a POJO (POJO types
+  *                              have no deterministic field order).
+  *
+  */
+class CodeGenerator(
+   config: TableConfig,
+   nullableInput: Boolean,
+   input1: TypeInformation[Any],
+   input2: Option[TypeInformation[Any]] = None,
+   input1PojoFieldMapping: Option[Array[Int]] = None,
+   input2PojoFieldMapping: Option[Array[Int]] = None)
+  extends RexVisitor[GeneratedExpression] {
+
+  // check if nullCheck is enabled when inputs can be null
+  if (nullableInput && !config.getNullCheck) {
+    throw new CodeGenException("Null check must be enabled if entire rows can be null.")
+  }
+
+  // check for POJO input1 mapping
+  input1 match {
+    case pt: PojoTypeInfo[_] =>
+      input1PojoFieldMapping.getOrElse(
+        throw new CodeGenException("No input mapping is specified for input1 of type POJO."))
+    case _ => // ok
+  }
+
+  // check for POJO input2 mapping
+  input2 match {
+    case Some(pt: PojoTypeInfo[_]) =>
+      input2PojoFieldMapping.getOrElse(
+        throw new CodeGenException("No input mapping is specified for input2 of type POJO."))
+    case _ => // ok
+  }
+
+  /**
+    * A code generator for generating unary Flink
+    * [[org.apache.flink.api.common.functions.Function]]s with one input.
+    *
+    * @param config configuration that determines runtime behavior
+    * @param nullableInput input(s) can be null.
+    * @param input type information about the input of the Function
+    * @param inputPojoFieldMapping additional mapping information necessary if input is a
+    *                              POJO (POJO types have no deterministic field order).
+    */
+  def this(
+      config: TableConfig,
+      nullableInput: Boolean,
+      input: TypeInformation[Any],
+      inputPojoFieldMapping: Array[Int]) =
+    this(config, nullableInput, input, None, Some(inputPojoFieldMapping))
+
+  /**
+    * A code generator for generating Flink input formats.
+    *
+    * @param config configuration that determines runtime behavior
+    */
+  def this(config: TableConfig) =
+    this(config, false, TypeConverter.DEFAULT_ROW_TYPE, None, None)
+
+  // set of member statements that will be added only once
+  // we use a LinkedHashSet to keep the insertion order
+  private val reusableMemberStatements = mutable.LinkedHashSet[String]()
+
+  // set of constructor statements that will be added only once
+  // we use a LinkedHashSet to keep the insertion order
+  private val reusableInitStatements = mutable.LinkedHashSet[String]()
+
+  // set of statements that will be added only once per record
+  // we use a LinkedHashSet to keep the insertion order
+  private val reusablePerRecordStatements = mutable.LinkedHashSet[String]()
+
+  // map of initial input unboxing expressions that will be added only once
+  // (inputTerm, index) -> expr
+  private val reusableInputUnboxingExprs = mutable.Map[(String, Int), GeneratedExpression]()
+
+  /**
+    * @return code block of statements that need to be placed in the member area of the Function
+    *         (e.g. member variables and their initialization)
+    */
+  def reuseMemberCode(): String = {
+    reusableMemberStatements.mkString("", "\n", "\n")
+  }
+
+  /**
+    * @return code block of statements that need to be placed in the constructor of the Function
+    */
+  def reuseInitCode(): String = {
+    reusableInitStatements.mkString("", "\n", "\n")
+  }
+
+  /**
+    * @return code block of statements that need to be placed in the SAM of the Function
+    */
+  def reusePerRecordCode(): String = {
+    reusablePerRecordStatements.mkString("", "\n", "\n")
+  }
+
+  /**
+    * @return code block of statements that unbox input variables to a primitive variable
+    *         and a corresponding null flag variable
+    */
+  def reuseInputUnboxingCode(): String = {
+    reusableInputUnboxingExprs.values.map(_.code).mkString("", "\n", "\n")
+  }
+
+  /**
+    * @return term of the (casted and possibly boxed) first input
+    */
+  var input1Term = "in1"
+
+  /**
+    * @return term of the (casted and possibly boxed) second input
+    */
+  var input2Term = "in2"
+
+  /**
+    * @return term of the (casted) output collector
+    */
+  var collectorTerm = "c"
+
+  /**
+    * @return term of the output record (possibly defined in the member area e.g. Row, Tuple)
+    */
+  var outRecordTerm = "out"
+
+  /**
+    * @return returns if null checking is enabled
+    */
+  def nullCheck: Boolean = config.getNullCheck
+
+  /**
+    * Generates an expression from a RexNode. If objects or variables can be reused, they will be
+    * added to reusable code sections internally.
+    *
+    * @param rex Calcite row expression
+    * @return instance of GeneratedExpression
+    */
+  def generateExpression(rex: RexNode): GeneratedExpression = {
+    rex.accept(this)
+  }
+
+  /**
+    * Generates a [[org.apache.flink.api.common.functions.Function]] that can be passed to Java
+    * compiler.
+    *
+    * @param name Class name of the Function. Must not be unique but has to be a valid Java class
+    *             identifier.
+    * @param clazz Flink Function to be generated.
+    * @param bodyCode code contents of the SAM (Single Abstract Method). Inputs, collector, or
+    *                 output record can be accessed via the given term methods.
+    * @param returnType expected return type
+    * @tparam T Flink Function to be generated.
+    * @return instance of GeneratedFunction
+    */
+  def generateFunction[T <: Function](
+      name: String,
+      clazz: Class[T],
+      bodyCode: String,
+      returnType: TypeInformation[Any])
+    : GeneratedFunction[T] = {
+    val funcName = newName(name)
+
+    // Janino does not support generics, that's why we need
+    // manual casting here
+    val samHeader =
+      // FlatMapFunction
+      if (clazz == classOf[FlatMapFunction[_,_]]) {
+        val inputTypeTerm = boxedTypeTermForTypeInfo(input1)
+        (s"void flatMap(Object _in1, org.apache.flink.util.Collector $collectorTerm)",
+          List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;"))
+      }
+
+      // MapFunction
+      else if (clazz == classOf[MapFunction[_,_]]) {
+        val inputTypeTerm = boxedTypeTermForTypeInfo(input1)
+        ("Object map(Object _in1)",
+          List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;"))
+      }
+
+      // FlatJoinFunction
+      else if (clazz == classOf[FlatJoinFunction[_,_,_]]) {
+        val inputTypeTerm1 = boxedTypeTermForTypeInfo(input1)
+        val inputTypeTerm2 = boxedTypeTermForTypeInfo(input2.getOrElse(
+            throw new CodeGenException("Input 2 for FlatJoinFunction should not be null")))
+        (s"void join(Object _in1, Object _in2, org.apache.flink.util.Collector $collectorTerm)",
+          List(s"$inputTypeTerm1 $input1Term = ($inputTypeTerm1) _in1;",
+          s"$inputTypeTerm2 $input2Term = ($inputTypeTerm2) _in2;"))
+      }
+      else {
+        // TODO more functions
+        throw new CodeGenException("Unsupported Function.")
+      }
+
+    val funcCode = j"""
+      public class $funcName
+          implements ${clazz.getCanonicalName} {
+
+        ${reuseMemberCode()}
+
+        public $funcName() throws Exception {
+          ${reuseInitCode()}
+        }
+
+        @Override
+        public ${samHeader._1} throws Exception {
+          ${samHeader._2.mkString("\n")}
+          ${reusePerRecordCode()}
+          ${reuseInputUnboxingCode()}
+          $bodyCode
+        }
+      }
+    """.stripMargin
+
+    GeneratedFunction(funcName, returnType, funcCode)
+  }
+
+  /**
+    * Generates a values input format that can be passed to Java compiler.
+    *
+    * @param name Class name of the input format. Must not be unique but has to be a
+    *             valid Java class identifier.
+    * @param records code for creating records
+    * @param returnType expected return type
+    * @tparam T Flink Function to be generated.
+    * @return instance of GeneratedFunction
+    */
+  def generateValuesInputFormat[T](
+      name: String,
+      records: Seq[String],
+      returnType: TypeInformation[Any])
+    : GeneratedFunction[GenericInputFormat[T]] = {
+    val funcName = newName(name)
+
+    addReusableOutRecord(returnType)
+
+    val funcCode = j"""
+      public class $funcName extends ${classOf[GenericInputFormat[_]].getCanonicalName} {
+
+        private int nextIdx = 0;
+
+        ${reuseMemberCode()}
+
+        public $funcName() throws Exception {
+          ${reuseInitCode()}
+        }
+
+        @Override
+        public boolean reachedEnd() throws java.io.IOException {
+          return nextIdx >= ${records.length};
+        }
+
+        @Override
+        public Object nextRecord(Object reuse) {
+          switch (nextIdx) {
+            ${records.zipWithIndex.map { case (r, i) =>
+              s"""
+                 |case $i:
+                 |  $r
+                 |break;
+               """.stripMargin
+            }.mkString("\n")}
+          }
+          nextIdx++;
+          return $outRecordTerm;
+        }
+      }
+    """.stripMargin
+
+    GeneratedFunction[GenericInputFormat[T]](funcName, returnType, funcCode)
+  }
+
+  /**
+    * Generates an expression that converts the first input (and second input) into the given type.
+    * If two inputs are converted, the second input is appended. If objects or variables can
+    * be reused, they will be added to reusable code sections internally. The evaluation result
+    * may be stored in the global result variable (see [[outRecordTerm]]).
+    *
+    * @param returnType conversion target type. Inputs and output must have the same arity.
+    * @param resultFieldNames result field names necessary for a mapping to POJO fields.
+    * @return instance of GeneratedExpression
+    */
+  def generateConverterResultExpression(
+      returnType: TypeInformation[_ <: Any],
+      resultFieldNames: Seq[String])
+    : GeneratedExpression = {
+    val input1AccessExprs = for (i <- 0 until input1.getArity)
+      yield generateInputAccess(input1, input1Term, i, input1PojoFieldMapping)
+
+    val input2AccessExprs = input2 match {
+      case Some(ti) => for (i <- 0 until ti.getArity)
+        yield generateInputAccess(ti, input2Term, i, input2PojoFieldMapping)
+      case None => Seq() // add nothing
+    }
+
+    generateResultExpression(input1AccessExprs ++ input2AccessExprs, returnType, resultFieldNames)
+  }
+
+  /**
+    * Generates an expression from the left input and the right table function.
+    */
+  def generateCorrelateAccessExprs: (Seq[GeneratedExpression], Seq[GeneratedExpression]) = {
+    val input1AccessExprs = for (i <- 0 until input1.getArity)
+      yield generateInputAccess(input1, input1Term, i, input1PojoFieldMapping)
+
+    val input2AccessExprs = input2 match {
+      case Some(ti) => for (i <- 0 until ti.getArity)
+        // use generateFieldAccess instead of generateInputAccess to avoid the generated table
+        // function's field access code is put on the top of function body rather than
+        // the while loop
+        yield generateFieldAccess(ti, input2Term, i, input2PojoFieldMapping)
+      case None => throw new CodeGenException("Type information of input2 must not be null.")
+    }
+    (input1AccessExprs, input2AccessExprs)
+  }
+
+  /**
+    * Generates an expression from a sequence of RexNode. If objects or variables can be reused,
+    * they will be added to reusable code sections internally. The evaluation result
+    * may be stored in the global result variable (see [[outRecordTerm]]).
+    *
+    * @param returnType conversion target type. Type must have the same arity than rexNodes.
+    * @param resultFieldNames result field names necessary for a mapping to POJO fields.
+    * @param rexNodes sequence of RexNode to be converted
+    * @return instance of GeneratedExpression
+    */
+  def generateResultExpression(
+      returnType: TypeInformation[_ <: Any],
+      resultFieldNames: Seq[String],
+      rexNodes: Seq[RexNode])
+    : GeneratedExpression = {
+    val fieldExprs = rexNodes.map(generateExpression)
+    generateResultExpression(fieldExprs, returnType, resultFieldNames)
+  }
+
+  /**
+    * Generates an expression from a sequence of other expressions. If objects or variables can
+    * be reused, they will be added to reusable code sections internally. The evaluation result
+    * may be stored in the global result variable (see [[outRecordTerm]]).
+    *
+    * @param fieldExprs field expressions to be converted
+    * @param returnType conversion target type. Type must have the same arity than fieldExprs.
+    * @param resultFieldNames result field names necessary for a mapping to POJO fields.
+    * @return instance of GeneratedExpression
+    */
+  def generateResultExpression(
+      fieldExprs: Seq[GeneratedExpression],
+      returnType: TypeInformation[_ <: Any],
+      resultFieldNames: Seq[String])
+    : GeneratedExpression = {
+    // initial type check
+    if (returnType.getArity != fieldExprs.length) {
+      throw new CodeGenException("Arity of result type does not match number of expressions.")
+    }
+    if (resultFieldNames.length != fieldExprs.length) {
+      throw new CodeGenException("Arity of result field names does not match number of " +
+        "expressions.")
+    }
+    // type check
+    returnType match {
+      case pt: PojoTypeInfo[_] =>
+        fieldExprs.zipWithIndex foreach {
+          case (fieldExpr, i) if fieldExpr.resultType != pt.getTypeAt(resultFieldNames(i)) =>
+            throw new CodeGenException("Incompatible types of expression and result type.")
+
+          case _ => // ok
+        }
+
+      case ct: CompositeType[_] =>
+        fieldExprs.zipWithIndex foreach {
+          case (fieldExpr, i) if fieldExpr.resultType != ct.getTypeAt(i) =>
+            throw new CodeGenException("Incompatible types of expression and result type.")
+          case _ => // ok
+        }
+
+      case at: AtomicType[_] if at != fieldExprs.head.resultType =>
+        throw new CodeGenException("Incompatible types of expression and result type.")
+
+      case _ => // ok
+    }
+
+    val returnTypeTerm = boxedTypeTermForTypeInfo(returnType)
+    val boxedFieldExprs = fieldExprs.map(generateOutputFieldBoxing)
+
+    // generate result expression
+    returnType match {
+      case ri: RowTypeInfo =>
+        addReusableOutRecord(ri)
+        val resultSetters: String = boxedFieldExprs.zipWithIndex map {
+          case (fieldExpr, i) =>
+            if (nullCheck) {
+              s"""
+              |${fieldExpr.code}
+              |if (${fieldExpr.nullTerm}) {
+              |  $outRecordTerm.setField($i, null);
+              |}
+              |else {
+              |  $outRecordTerm.setField($i, ${fieldExpr.resultTerm});
+              |}
+              |""".stripMargin
+            }
+            else {
+              s"""
+              |${fieldExpr.code}
+              |$outRecordTerm.setField($i, ${fieldExpr.resultTerm});
+              |""".stripMargin
+            }
+        } mkString "\n"
+
+        GeneratedExpression(outRecordTerm, "false", resultSetters, returnType)
+
+      case pt: PojoTypeInfo[_] =>
+        addReusableOutRecord(pt)
+        val resultSetters: String = boxedFieldExprs.zip(resultFieldNames) map {
+          case (fieldExpr, fieldName) =>
+            val accessor = getFieldAccessor(pt.getTypeClass, fieldName)
+
+            accessor match {
+              // Reflective access of primitives/Objects
+              case ObjectPrivateFieldAccessor(field) =>
+                val fieldTerm = addReusablePrivateFieldAccess(pt.getTypeClass, fieldName)
+
+                val defaultIfNull = if (isFieldPrimitive(field)) {
+                  primitiveDefaultValue(fieldExpr.resultType)
+                } else {
+                  "null"
+                }
+
+                if (nullCheck) {
+                  s"""
+                    |${fieldExpr.code}
+                    |if (${fieldExpr.nullTerm}) {
+                    |  ${reflectiveFieldWriteAccess(
+                          fieldTerm,
+                          field,
+                          outRecordTerm,
+                          defaultIfNull)};
+                    |}
+                    |else {
+                    |  ${reflectiveFieldWriteAccess(
+                          fieldTerm,
+                          field,
+                          outRecordTerm,
+                          fieldExpr.resultTerm)};
+                    |}
+                    |""".stripMargin
+                }
+                else {
+                  s"""
+                    |${fieldExpr.code}
+                    |${reflectiveFieldWriteAccess(
+                          fieldTerm,
+                          field,
+                          outRecordTerm,
+                          fieldExpr.resultTerm)};
+                    |""".stripMargin
+                }
+
+              // primitive or Object field access (implicit boxing)
+              case _ =>
+                if (nullCheck) {
+                  s"""
+                    |${fieldExpr.code}
+                    |if (${fieldExpr.nullTerm}) {
+                    |  $outRecordTerm.$fieldName = null;
+                    |}
+                    |else {
+                    |  $outRecordTerm.$fieldName = ${fieldExpr.resultTerm};
+                    |}
+                    |""".stripMargin
+                }
+                else {
+                  s"""
+                    |${fieldExpr.code}
+                    |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm};
+                    |""".stripMargin
+                }
+              }
+          } mkString "\n"
+
+        GeneratedExpression(outRecordTerm, "false", resultSetters, returnType)
+
+      case tup: TupleTypeInfo[_] =>
+        addReusableOutRecord(tup)
+        val resultSetters: String = boxedFieldExprs.zipWithIndex map {
+          case (fieldExpr, i) =>
+            val fieldName = "f" + i
+            if (nullCheck) {
+              s"""
+                |${fieldExpr.code}
+                |if (${fieldExpr.nullTerm}) {
+                |  throw new NullPointerException("Null result cannot be stored in a Tuple.");
+                |}
+                |else {
+                |  $outRecordTerm.$fieldName = ${fieldExpr.resultTerm};
+                |}
+                |""".stripMargin
+            }
+            else {
+              s"""
+                |${fieldExpr.code}
+                |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm};
+                |""".stripMargin
+            }
+        } mkString "\n"
+
+        GeneratedExpression(outRecordTerm, "false", resultSetters, returnType)
+
+      case cc: CaseClassTypeInfo[_] =>
+        val fieldCodes: String = boxedFieldExprs.map(_.code).mkString("\n")
+        val constructorParams: String = boxedFieldExprs.map(_.resultTerm).mkString(", ")
+        val resultTerm = newName(outRecordTerm)
+
+        val nullCheckCode = if (nullCheck) {
+        boxedFieldExprs map { (fieldExpr) =>
+          s"""
+              |if (${fieldExpr.nullTerm}) {
+              |  throw new NullPointerException("Null result cannot be stored in a Case Class.");
+              |}
+              |""".stripMargin
+          } mkString "\n"
+        } else {
+          ""
+        }
+
+        val resultCode =
+          s"""
+            |$fieldCodes
+            |$nullCheckCode
+            |$returnTypeTerm $resultTerm = new $returnTypeTerm($constructorParams);
+            |""".stripMargin
+
+        GeneratedExpression(resultTerm, "false", resultCode, returnType)
+
+      case a: AtomicType[_] =>
+        val fieldExpr = boxedFieldExprs.head
+        val nullCheckCode = if (nullCheck) {
+          s"""
+          |if (${fieldExpr.nullTerm}) {
+          |  throw new NullPointerException("Null result cannot be used for atomic types.");
+          |}
+          |""".stripMargin
+        } else {
+          ""
+        }
+        val resultCode =
+          s"""
+            |${fieldExpr.code}
+            |$nullCheckCode
+            |""".stripMargin
+
+        GeneratedExpression(fieldExpr.resultTerm, "false", resultCode, returnType)
+
+      case _ =>
+        throw new CodeGenException(s"Unsupported result type: $returnType")
+    }
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // RexVisitor methods
+  // ----------------------------------------------------------------------------------------------
+
+  override def visitInputRef(inputRef: RexInputRef): GeneratedExpression = {
+    // if inputRef index is within size of input1 we work with input1, input2 otherwise
+    val input = if (inputRef.getIndex < input1.getArity) {
+      (input1, input1Term, input1PojoFieldMapping)
+    } else {
+      (input2.getOrElse(throw new CodeGenException("Invalid input access.")),
+        input2Term,
+        input2PojoFieldMapping)
+    }
+
+    val index = if (input._2 == input1Term) {
+      inputRef.getIndex
+    } else {
+      inputRef.getIndex - input1.getArity
+    }
+
+    generateInputAccess(input._1, input._2, index, input._3)
+  }
+
+  override def visitFieldAccess(rexFieldAccess: RexFieldAccess): GeneratedExpression = {
+    val refExpr = rexFieldAccess.getReferenceExpr.accept(this)
+    val index = rexFieldAccess.getField.getIndex
+    val fieldAccessExpr = generateFieldAccess(
+      refExpr.resultType,
+      refExpr.resultTerm,
+      index,
+      input1PojoFieldMapping)
+
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldAccessExpr.resultType)
+    val defaultValue = primitiveDefaultValue(fieldAccessExpr.resultType)
+    val resultCode = if (nullCheck) {
+      s"""
+        |${refExpr.code}
+        |$resultTypeTerm $resultTerm;
+        |boolean $nullTerm;
+        |if (${refExpr.nullTerm}) {
+        |  $resultTerm = $defaultValue;
+        |  $nullTerm = true;
+        |}
+        |else {
+        |  ${fieldAccessExpr.code}
+        |  $resultTerm = ${fieldAccessExpr.resultTerm};
+        |  $nullTerm = ${fieldAccessExpr.nullTerm};
+        |}
+        |""".stripMargin
+    } else {
+      s"""
+        |${refExpr.code}
+        |${fieldAccessExpr.code}
+        |$resultTypeTerm $resultTerm = ${fieldAccessExpr.resultTerm};
+        |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, resultCode, fieldAccessExpr.resultType)
+  }
+
+  override def visitLiteral(literal: RexLiteral): GeneratedExpression = {
+    val resultType = FlinkTypeFactory.toTypeInfo(literal.getType)
+    val value = literal.getValue3
+    // null value with type
+    if (value == null) {
+      return generateNullLiteral(resultType)
+    }
+    // non-null values
+    literal.getType.getSqlTypeName match {
+
+      case BOOLEAN =>
+        generateNonNullLiteral(resultType, literal.getValue3.toString)
+
+      case TINYINT =>
+        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
+        if (decimal.isValidByte) {
+          generateNonNullLiteral(resultType, decimal.byteValue().toString)
+        }
+        else {
+          throw new CodeGenException("Decimal can not be converted to byte.")
+        }
+
+      case SMALLINT =>
+        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
+        if (decimal.isValidShort) {
+          generateNonNullLiteral(resultType, decimal.shortValue().toString)
+        }
+        else {
+          throw new CodeGenException("Decimal can not be converted to short.")
+        }
+
+      case INTEGER =>
+        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
+        if (decimal.isValidInt) {
+          generateNonNullLiteral(resultType, decimal.intValue().toString)
+        }
+        else {
+          throw new CodeGenException("Decimal can not be converted to integer.")
+        }
+
+      case BIGINT =>
+        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
+        if (decimal.isValidLong) {
+          generateNonNullLiteral(resultType, decimal.longValue().toString + "L")
+        }
+        else {
+          throw new CodeGenException("Decimal can not be converted to long.")
+        }
+
+      case FLOAT =>
+        val floatValue = value.asInstanceOf[JBigDecimal].floatValue()
+        floatValue match {
+          case Float.NaN => generateNonNullLiteral(resultType, "java.lang.Float.NaN")
+          case Float.NegativeInfinity =>
+            generateNonNullLiteral(resultType, "java.lang.Float.NEGATIVE_INFINITY")
+          case Float.PositiveInfinity =>
+            generateNonNullLiteral(resultType, "java.lang.Float.POSITIVE_INFINITY")
+          case _ => generateNonNullLiteral(resultType, floatValue.toString + "f")
+        }
+
+      case DOUBLE =>
+        val doubleValue = value.asInstanceOf[JBigDecimal].doubleValue()
+        doubleValue match {
+          case Double.NaN => generateNonNullLiteral(resultType, "java.lang.Double.NaN")
+          case Double.NegativeInfinity =>
+            generateNonNullLiteral(resultType, "java.lang.Double.NEGATIVE_INFINITY")
+          case Double.PositiveInfinity =>
+            generateNonNullLiteral(resultType, "java.lang.Double.POSITIVE_INFINITY")
+          case _ => generateNonNullLiteral(resultType, doubleValue.toString + "d")
+        }
+      case DECIMAL =>
+        val decimalField = addReusableDecimal(value.asInstanceOf[JBigDecimal])
+        generateNonNullLiteral(resultType, decimalField)
+
+      case VARCHAR | CHAR =>
+        generateNonNullLiteral(resultType, "\"" + value.toString + "\"")
+
+      case SYMBOL =>
+        generateSymbol(value.asInstanceOf[Enum[_]])
+
+      case DATE =>
+        generateNonNullLiteral(resultType, value.toString)
+
+      case TIME =>
+        generateNonNullLiteral(resultType, value.toString)
+
+      case TIMESTAMP =>
+        generateNonNullLiteral(resultType, value.toString + "L")
+
+      case typeName if YEAR_INTERVAL_TYPES.contains(typeName) =>
+        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
+        if (decimal.isValidInt) {
+          generateNonNullLiteral(resultType, decimal.intValue().toString)
+        } else {
+          throw new CodeGenException("Decimal can not be converted to interval of months.")
+        }
+
+      case typeName if DAY_INTERVAL_TYPES.contains(typeName) =>
+        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
+        if (decimal.isValidLong) {
+          generateNonNullLiteral(resultType, decimal.longValue().toString + "L")
+        } else {
+          throw new CodeGenException("Decimal can not be converted to interval of milliseconds.")
+        }
+
+      case t@_ =>
+        throw new CodeGenException(s"Type not supported: $t")
+    }
+  }
+
+  override def visitCorrelVariable(correlVariable: RexCorrelVariable): GeneratedExpression = {
+    GeneratedExpression(input1Term, NEVER_NULL, NO_CODE, input1)
+  }
+
+  override def visitLocalRef(localRef: RexLocalRef): GeneratedExpression =
+    throw new CodeGenException("Local variables are not supported yet.")
+
+  override def visitRangeRef(rangeRef: RexRangeRef): GeneratedExpression =
+    throw new CodeGenException("Range references are not supported yet.")
+
+  override def visitDynamicParam(dynamicParam: RexDynamicParam): GeneratedExpression =
+    throw new CodeGenException("Dynamic parameter references are not supported yet.")
+
+  override def visitCall(call: RexCall): GeneratedExpression = {
+    val operands = call.getOperands.map(_.accept(this))
+    val resultType = FlinkTypeFactory.toTypeInfo(call.getType)
+
+    call.getOperator match {
+      // arithmetic
+      case PLUS if isNumeric(resultType) =>
+        val left = operands.head
+        val right = operands(1)
+        requireNumeric(left)
+        requireNumeric(right)
+        generateArithmeticOperator("+", nullCheck, resultType, left, right)
+
+      case PLUS | DATETIME_PLUS if isTemporal(resultType) =>
+        val left = operands.head
+        val right = operands(1)
+        requireTemporal(left)
+        requireTemporal(right)
+        generateTemporalPlusMinus(plus = true, nullCheck, left, right)
+
+      case MINUS if isNumeric(resultType) =>
+        val left = operands.head
+        val right = operands(1)
+        requireNumeric(left)
+        requireNumeric(right)
+        generateArithmeticOperator("-", nullCheck, resultType, left, right)
+
+      case MINUS | MINUS_DATE if isTemporal(resultType) =>
+        val left = operands.head
+        val right = operands(1)
+        requireTemporal(left)
+        requireTemporal(right)
+        generateTemporalPlusMinus(plus = false, nullCheck, left, right)
+
+      case MULTIPLY if isNumeric(resultType) =>
+        val left = operands.head
+        val right = operands(1)
+        requireNumeric(left)
+        requireNumeric(right)
+        generateArithmeticOperator("*", nullCheck, resultType, left, right)
+
+      case DIVIDE | DIVIDE_INTEGER if isNumeric(resultType) =>
+        val left = operands.head
+        val right = operands(1)
+        requireNumeric(left)
+        requireNumeric(right)
+        generateArithmeticOperator("/", nullCheck, resultType, left, right)
+
+      case MOD if isNumeric(resultType) =>
+        val left = operands.head
+        val right = operands(1)
+        requireNumeric(left)
+        requireNumeric(right)
+        generateArithmeticOperator("%", nullCheck, resultType, left, right)
+
+      case UNARY_MINUS if isNumeric(resultType) =>
+        val operand = operands.head
+        requireNumeric(operand)
+        generateUnaryArithmeticOperator("-", nullCheck, resultType, operand)
+
+      case UNARY_MINUS if isTimeInterval(resultType) =>
+        val operand = operands.head
+        requireTimeInterval(operand)
+        generateUnaryIntervalPlusMinus(plus = false, nullCheck, operand)
+
+      case UNARY_PLUS if isNumeric(resultType) =>
+        val operand = operands.head
+        requireNumeric(operand)
+        generateUnaryArithmeticOperator("+", nullCheck, resultType, operand)
+
+      case UNARY_PLUS if isTimeInterval(resultType) =>
+        val operand = operands.head
+        requireTimeInterval(operand)
+        generateUnaryIntervalPlusMinus(plus = true, nullCheck, operand)
+
+      // comparison
+      case EQUALS =>
+        val left = operands.head
+        val right = operands(1)
+        generateEquals(nullCheck, left, right)
+
+      case NOT_EQUALS =>
+        val left = operands.head
+        val right = operands(1)
+        generateNotEquals(nullCheck, left, right)
+
+      case GREATER_THAN =>
+        val left = operands.head
+        val right = operands(1)
+        requireComparable(left)
+        requireComparable(right)
+        generateComparison(">", nullCheck, left, right)
+
+      case GREATER_THAN_OR_EQUAL =>
+        val left = operands.head
+        val right = operands(1)
+        requireComparable(left)
+        requireComparable(right)
+        generateComparison(">=", nullCheck, left, right)
+
+      case LESS_THAN =>
+        val left = operands.head
+        val right = operands(1)
+        requireComparable(left)
+        requireComparable(right)
+        generateComparison("<", nullCheck, left, right)
+
+      case LESS_THAN_OR_EQUAL =>
+        val left = operands.head
+        val right = operands(1)
+        requireComparable(left)
+        requireComparable(right)
+        generateComparison("<=", nullCheck, left, right)
+
+      case IS_NULL =>
+        val operand = operands.head
+        generateIsNull(nullCheck, operand)
+
+      case IS_NOT_NULL =>
+        val operand = operands.head
+        generateIsNotNull(nullCheck, operand)
+
+      // logic
+      case AND =>
+        operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) =>
+          requireBoolean(left)
+          requireBoolean(right)
+          generateAnd(nullCheck, left, right)
+        }
+
+      case OR =>
+        operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) =>
+          requireBoolean(left)
+          requireBoolean(right)
+          generateOr(nullCheck, left, right)
+        }
+
+      case NOT =>
+        val operand = operands.head
+        requireBoolean(operand)
+        generateNot(nullCheck, operand)
+
+      case CASE =>
+        generateIfElse(nullCheck, operands, resultType)
+
+      case IS_TRUE =>
+        val operand = operands.head
+        requireBoolean(operand)
+        generateIsTrue(operand)
+
+      case IS_NOT_TRUE =>
+        val operand = operands.head
+        requireBoolean(operand)
+        generateIsNotTrue(operand)
+
+      case IS_FALSE =>
+        val operand = operands.head
+        requireBoolean(operand)
+        generateIsFalse(operand)
+
+      case IS_NOT_FALSE =>
+        val operand = operands.head
+        requireBoolean(operand)
+        generateIsNotFalse(operand)
+
+      // casting
+      case CAST | REINTERPRET =>
+        val operand = operands.head
+        generateCast(nullCheck, operand, resultType)
+
+      // as / renaming
+      case AS =>
+        operands.head
+
+      // string arithmetic
+      case CONCAT =>
+        val left = operands.head
+        val right = operands(1)
+        requireString(left)
+        generateArithmeticOperator("+", nullCheck, resultType, left, right)
+
+      // arrays
+      case ARRAY_VALUE_CONSTRUCTOR =>
+        generateArray(this, resultType, operands)
+
+      case ITEM =>
+        val array = operands.head
+        val index = operands(1)
+        requireArray(array)
+        requireInteger(index)
+        generateArrayElementAt(this, array, index)
+
+      case CARDINALITY =>
+        val array = operands.head
+        requireArray(array)
+        generateArrayCardinality(nullCheck, array)
+
+      case ELEMENT =>
+        val array = operands.head
+        requireArray(array)
+        generateArrayElement(this, array)
+
+      // advanced scalar functions
+      case sqlOperator: SqlOperator =>
+        val callGen = FunctionGenerator.getCallGenerator(
+          sqlOperator,
+          operands.map(_.resultType),
+          resultType)
+        callGen
+          .getOrElse(throw new CodeGenException(s"Unsupported call: $sqlOperator \n" +
+            s"If you think this function should be supported, " +
+            s"you can create an issue and start a discussion for it."))
+          .generate(this, operands)
+
+      // unknown or invalid
+      case call@_ =>
+        throw new CodeGenException(s"Unsupported call: $call")
+    }
+  }
+
+  override def visitOver(over: RexOver): GeneratedExpression =
+    throw new CodeGenException("Aggregate functions over windows are not supported yet.")
+
+  override def visitSubQuery(subQuery: RexSubQuery): GeneratedExpression =
+    throw new CodeGenException("Subqueries are not supported yet.")
+
+  // ----------------------------------------------------------------------------------------------
+  // generator helping methods
+  // ----------------------------------------------------------------------------------------------
+
+  private def generateInputAccess(
+      inputType: TypeInformation[Any],
+      inputTerm: String,
+      index: Int,
+      pojoFieldMapping: Option[Array[Int]])
+    : GeneratedExpression = {
+    // if input has been used before, we can reuse the code that
+    // has already been generated
+    val inputExpr = reusableInputUnboxingExprs.get((inputTerm, index)) match {
+      // input access and unboxing has already been generated
+      case Some(expr) =>
+        expr
+
+      // generate input access and unboxing if necessary
+      case None =>
+        val expr = if (nullableInput) {
+          generateNullableInputFieldAccess(inputType, inputTerm, index, pojoFieldMapping)
+        } else {
+          generateFieldAccess(inputType, inputTerm, index, pojoFieldMapping)
+        }
+
+        reusableInputUnboxingExprs((inputTerm, index)) = expr
+        expr
+    }
+    // hide the generated code as it will be executed only once
+    GeneratedExpression(inputExpr.resultTerm, inputExpr.nullTerm, "", inputExpr.resultType)
+  }
+
+  private def generateNullableInputFieldAccess(
+      inputType: TypeInformation[Any],
+      inputTerm: String,
+      index: Int,
+      pojoFieldMapping: Option[Array[Int]])
+    : GeneratedExpression = {
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+
+    val fieldType = inputType match {
+      case ct: CompositeType[_] =>
+        val fieldIndex = if (ct.isInstanceOf[PojoTypeInfo[_]]) {
+          pojoFieldMapping.get(index)
+        }
+        else {
+          index
+        }
+        ct.getTypeAt(fieldIndex)
+      case at: AtomicType[_] => at
+      case _ => throw new CodeGenException("Unsupported type for input field access.")
+    }
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType)
+    val defaultValue = primitiveDefaultValue(fieldType)
+    val fieldAccessExpr = generateFieldAccess(inputType, inputTerm, index, pojoFieldMapping)
+
+    val inputCheckCode =
+      s"""
+        |$resultTypeTerm $resultTerm;
+        |boolean $nullTerm;
+        |if ($inputTerm == null) {
+        |  $resultTerm = $defaultValue;
+        |  $nullTerm = true;
+        |}
+        |else {
+        |  ${fieldAccessExpr.code}
+        |  $resultTerm = ${fieldAccessExpr.resultTerm};
+        |  $nullTerm = ${fieldAccessExpr.nullTerm};
+        |}
+        |""".stripMargin
+
+    GeneratedExpression(resultTerm, nullTerm, inputCheckCode, fieldType)
+  }
+
+  private def generateFieldAccess(
+      inputType: TypeInformation[_],
+      inputTerm: String,
+      index: Int,
+      pojoFieldMapping: Option[Array[Int]])
+    : GeneratedExpression = {
+    inputType match {
+      case ct: CompositeType[_] =>
+        val fieldIndex = if (ct.isInstanceOf[PojoTypeInfo[_]] && pojoFieldMapping.nonEmpty) {
+          pojoFieldMapping.get(index)
+        }
+        else {
+          index
+        }
+        val accessor = fieldAccessorFor(ct, fieldIndex)
+        val fieldType: TypeInformation[Any] = ct.getTypeAt(fieldIndex)
+        val fieldTypeTerm = boxedTypeTermForTypeInfo(fieldType)
+
+        accessor match {
+          case ObjectFieldAccessor(field) =>
+            // primitive
+            if (isFieldPrimitive(field)) {
+              generateNonNullLiteral(fieldType, s"$inputTerm.${field.getName}")
+            }
+            // Object
+            else {
+              generateInputFieldUnboxing(
+                fieldType,
+                s"($fieldTypeTerm) $inputTerm.${field.getName}")
+            }
+
+          case ObjectGenericFieldAccessor(fieldName) =>
+            // Object
+            val inputCode = s"($fieldTypeTerm) $inputTerm.$fieldName"
+            generateInputFieldUnboxing(fieldType, inputCode)
+
+          case ObjectMethodAccessor(methodName) =>
+            // Object
+            val inputCode = s"($fieldTypeTerm) $inputTerm.$methodName()"
+            generateInputFieldUnboxing(fieldType, inputCode)
+
+          case ProductAccessor(i) =>
+            // Object
+            val inputCode = s"($fieldTypeTerm) $inputTerm.getField($i)"
+            generateInputFieldUnboxing(fieldType, inputCode)
+
+          case ObjectPrivateFieldAccessor(field) =>
+            val fieldTerm = addReusablePrivateFieldAccess(ct.getTypeClass, field.getName)
+            val reflectiveAccessCode = reflectiveFieldReadAccess(fieldTerm, field, inputTerm)
+            // primitive
+            if (isFieldPrimitive(field)) {
+              generateNonNullLiteral(fieldType, reflectiveAccessCode)
+            }
+            // Object
+            else {
+              generateInputFieldUnboxing(fieldType, reflectiveAccessCode)
+            }
+        }
+
+      case at: AtomicType[_] =>
+        val fieldTypeTerm = boxedTypeTermForTypeInfo(at)
+        val inputCode = s"($fieldTypeTerm) $inputTerm"
+        generateInputFieldUnboxing(at, inputCode)
+
+      case _ =>
+        throw new CodeGenException("Unsupported type for input field access.")
+    }
+  }
+
+  private def generateNullLiteral(resultType: TypeInformation[_]): GeneratedExpression = {
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
+    val defaultValue = primitiveDefaultValue(resultType)
+
+    if (nullCheck) {
+      val wrappedCode = s"""
+        |$resultTypeTerm $resultTerm = $defaultValue;
+        |boolean $nullTerm = true;
+        |""".stripMargin
+      GeneratedExpression(resultTerm, nullTerm, wrappedCode, resultType)
+    } else {
+      throw new CodeGenException("Null literals are not allowed if nullCheck is disabled.")
+    }
+  }
+
+  private[flink] def generateNonNullLiteral(
+      literalType: TypeInformation[_],
+      literalCode: String)
+    : GeneratedExpression = {
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(literalType)
+
+    val resultCode = if (nullCheck) {
+      s"""
+        |$resultTypeTerm $resultTerm = $literalCode;
+        |boolean $nullTerm = false;
+        |""".stripMargin
+    } else {
+      s"""
+        |$resultTypeTerm $resultTerm = $literalCode;
+        |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, resultCode, literalType)
+  }
+
+  private[flink] def generateSymbol(enum: Enum[_]): GeneratedExpression = {
+    GeneratedExpression(
+      qualifyEnum(enum),
+      "false",
+      "",
+      new GenericTypeInfo(enum.getDeclaringClass))
+  }
+
+  /**
+    * Converts the external boxed format to an internal mostly primitive field representation.
+    * Wrapper types can autoboxed to their corresponding primitive type (Integer -> int). External
+    * objects are converted to their internal representation (Timestamp -> internal timestamp
+    * in long).
+    *
+    * @param fieldType type of field
+    * @param fieldTerm expression term of field to be unboxed
+    * @return internal unboxed field representation
+    */
+  private[flink] def generateInputFieldUnboxing(
+      fieldType: TypeInformation[_],
+      fieldTerm: String)
+    : GeneratedExpression = {
+    val tmpTerm = newName("tmp")
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+    val tmpTypeTerm = boxedTypeTermForTypeInfo(fieldType)
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType)
+    val defaultValue = primitiveDefaultValue(fieldType)
+
+    // explicit unboxing
+    val unboxedFieldCode = if (isTimePoint(fieldType)) {
+      timePointToInternalCode(fieldType, fieldTerm)
+    } else {
+      fieldTerm
+    }
+
+    val wrappedCode = if (nullCheck && !isReference(fieldType)) {
+      s"""
+        |$tmpTypeTerm $tmpTerm = $unboxedFieldCode;
+        |boolean $nullTerm = $tmpTerm == null;
+        |$resultTypeTerm $resultTerm;
+        |if ($nullTerm) {
+        |  $resultTerm = $defaultValue;
+        |}
+        |else {
+        |  $resultTerm = $tmpTerm;
+        |}
+        |""".stripMargin
+    } else if (nullCheck) {
+      s"""
+        |$resultTypeTerm $resultTerm = $unboxedFieldCode;
+        |boolean $nullTerm = $fieldTerm == null;
+        |""".stripMargin
+    } else {
+      s"""
+        |$resultTypeTerm $resultTerm = $unboxedFieldCode;
+        |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, wrappedCode, fieldType)
+  }
+
+  /**
+    * Converts the internal mostly primitive field representation to an external boxed format.
+    * Primitive types can autoboxed to their corresponding object type (int -> Integer). Internal
+    * representations are converted to their external objects (internal timestamp
+    * in long -> Timestamp).
+    *
+    * @param expr expression to be boxed
+    * @return external boxed field representation
+    */
+  private[flink] def generateOutputFieldBoxing(expr: GeneratedExpression): GeneratedExpression = {
+    expr.resultType match {
+      // convert internal date/time/timestamp to java.sql.* objects
+      case SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIME | SqlTimeTypeInfo.TIMESTAMP =>
+        val resultTerm = newName("result")
+        val resultTypeTerm = boxedTypeTermForTypeInfo(expr.resultType)
+        val convMethod = internalToTimePointCode(expr.resultType, expr.resultTerm)
+
+        val resultCode = if (nullCheck) {
+          s"""
+            |${expr.code}
+            |$resultTypeTerm $resultTerm;
+            |if (${expr.nullTerm}) {
+            |  $resultTerm = null;
+            |}
+            |else {
+            |  $resultTerm = $convMethod;
+            |}
+            |""".stripMargin
+        } else {
+          s"""
+            |${expr.code}
+            |$resultTypeTerm $resultTerm = $convMethod;
+            |""".stripMargin
+        }
+
+        GeneratedExpression(resultTerm, expr.nullTerm, resultCode, expr.resultType)
+
+      // other types are autoboxed or need no boxing
+      case _ => expr
+    }
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // Reusable code snippets
+  // ----------------------------------------------------------------------------------------------
+
+  /**
+    * Adds a reusable output record to the member area of the generated [[Function]].
+    * The passed [[TypeInformation]] defines the type class to be instantiated.
+    *
+    * @param ti type information of type class to be instantiated during runtime
+    * @return member variable term
+    */
+  def addReusableOutRecord(ti: TypeInformation[_]): Unit = {
+    val statement = ti match {
+      case rt: RowTypeInfo =>
+        s"""
+          |transient ${ti.getTypeClass.getCanonicalName} $outRecordTerm =
+          |    new ${ti.getTypeClass.getCanonicalName}(${rt.getArity});
+          |""".stripMargin
+      case _ =>
+        s"""
+          |${ti.getTypeClass.getCanonicalName} $outRecordTerm =
+          |    new ${ti.getTypeClass.getCanonicalName}();
+          |""".stripMargin
+    }
+    reusableMemberStatements.add(statement)
+  }
+
+  /**
+    * Adds a reusable [[java.lang.reflect.Field]] to the member area of the generated [[Function]].
+    * The field can be used for accessing POJO fields more efficiently during runtime, however,
+    * the field does not have to be public.
+    *
+    * @param clazz class of containing field
+    * @param fieldName name of field to be extracted and instantiated during runtime
+    * @return member variable term
+    */
+  def addReusablePrivateFieldAccess(clazz: Class[_], fieldName: String): String = {
+    val fieldTerm = s"field_${clazz.getCanonicalName.replace('.', '$')}_$fieldName"
+    val fieldExtraction =
+      s"""
+        |transient java.lang.reflect.Field $fieldTerm =
+        |    org.apache.flink.api.java.typeutils.TypeExtractor.getDeclaredField(
+        |      ${clazz.getCanonicalName}.class, "$fieldName");
+        |""".stripMargin
+    reusableMemberStatements.add(fieldExtraction)
+
+    val fieldAccessibility =
+      s"""
+        |$fieldTerm.setAccessible(true);
+        |""".stripMargin
+    reusableInitStatements.add(fieldAccessibility)
+
+    fieldTerm
+  }
+
+  /**
+    * Adds a reusable [[java.math.BigDecimal]] to the member area of the generated [[Function]].
+    *
+    * @param decimal decimal object to be instantiated during runtime
+    * @return member variable term
+    */
+  def addReusableDecimal(decimal: JBigDecimal): String = decimal match {
+    case JBigDecimal.ZERO => "java.math.BigDecimal.ZERO"
+    case JBigDecimal.ONE => "java.math.BigDecimal.ONE"
+    case JBigDecimal.TEN => "java.math.BigDecimal.TEN"
+    case _ =>
+      val fieldTerm = newName("decimal")
+      val fieldDecimal =
+        s"""
+          |transient java.math.BigDecimal $fieldTerm =
+          |    new java.math.BigDecimal("${decimal.toString}");
+          |""".stripMargin
+      reusableMemberStatements.add(fieldDecimal)
+      fieldTerm
+  }
+
+  /**
+    * Adds a reusable [[UserDefinedFunction]] to the member area of the generated [[Function]].
+    * The [[UserDefinedFunction]] must have a default constructor, however, it does not have
+    * to be public.
+    *
+    * @param function [[UserDefinedFunction]] object to be instantiated during runtime
+    * @return member variable term
+    */
+  def addReusableFunction(function: UserDefinedFunction): String = {
+    val classQualifier = function.getClass.getCanonicalName
+    val fieldTerm = s"function_${classQualifier.replace('.', '$')}"
+
+    val fieldFunction =
+      s"""
+        |transient $classQualifier $fieldTerm = null;
+        |""".stripMargin
+    reusableMemberStatements.add(fieldFunction)
+
+    val constructorTerm = s"constructor_${classQualifier.replace('.', '$')}"
+    val constructorAccessibility =
+      s"""
+        |java.lang.reflect.Constructor $constructorTerm =
+        |  $classQualifier.class.getDeclaredConstructor();
+        |$constructorTerm.setAccessible(true);
+        |$fieldTerm = ($classQualifier) $constructorTerm.newInstance();
+       """.stripMargin
+    reusableInitStatements.add(constructorAccessibility)
+    fieldTerm
+  }
+
+  /**
+    * Adds a reusable array to the member area of the generated [[Function]].
+    */
+  def addReusableArray(clazz: Class[_], size: Int): String = {
+    val fieldTerm = newName("array")
+    val classQualifier = clazz.getCanonicalName // works also for int[] etc.
+    val initArray = classQualifier.replaceFirst("\\[", s"[$size")
+    val fieldArray =
+      s"""
+        |transient $classQualifier $fieldTerm =
+        |    new $initArray;
+        |""".stripMargin
+    reusableMemberStatements.add(fieldArray)
+    fieldTerm
+  }
+
+  /**
+    * Adds a reusable timestamp to the beginning of the SAM of the generated [[Function]].
+    */
+  def addReusableTimestamp(): String = {
+    val fieldTerm = s"timestamp"
+
+    val field =
+      s"""
+        |final long $fieldTerm = java.lang.System.currentTimeMillis();
+        |""".stripMargin
+    reusablePerRecordStatements.add(field)
+    fieldTerm
+  }
+
+    /**
+    * Adds a reusable local timestamp to the beginning of the SAM of the generated [[Function]].
+    */
+  def addReusableLocalTimestamp(): String = {
+    val fieldTerm = s"localtimestamp"
+
+    val timestamp = addReusableTimestamp()
+
+    val field =
+      s"""
+        |final long $fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset(timestamp);
+        |""".stripMargin
+    reusablePerRecordStatements.add(field)
+    fieldTerm
+  }
+
+  /**
+    * Adds a reusable time to the beginning of the SAM of the generated [[Function]].
+    */
+  def addReusableTime(): String = {
+    val fieldTerm = s"time"
+
+    val timestamp = addReusableTimestamp()
+
+    // adopted from org.apache.calcite.runtime.SqlFunctions.currentTime()
+    val field =
+      s"""
+        |final int $fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY});
+        |if (time < 0) {
+        |  time += ${DateTimeUtils.MILLIS_PER_DAY};
+        |}
+        |""".stripMargin
+    reusablePerRecordStatements.add(field)
+    fieldTerm
+  }
+
+  /**
+    * Adds a reusable local time to the beginning of the SAM of the generated [[Function]].
+    */
+  def addReusableLocalTime(): String = {
+    val fieldTerm = s"localtime"
+
+    val localtimestamp = addReusableLocalTimestamp()
+
+    // adopted from org.apache.calcite.runtime.SqlFunctions.localTime()
+    val field =
+      s"""
+        |final int $fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY});
+        |""".stripMargin
+    reusablePerRecordStatements.add(field)
+    fieldTerm
+  }
+
+
+  /**
+    * Adds a reusable date to the beginning of the SAM of the generated [[Function]].
+    */
+  def addReusableDate(): String = {
+    val fieldTerm = s"date"
+
+    val timestamp = addReusableTimestamp()
+    val time = addReusableTime()
+
+    // adopted from org.apache.calcite.runtime.SqlFunctions.currentDate()
+    val field =
+      s"""
+        |final int $fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY});
+        |if ($time < 0) {
+        |  $fieldTerm -= 1;
+        |}
+        |""".stripMargin
+    reusablePerRecordStatements.add(field)
+    fieldTerm
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/Compiler.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/Compiler.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/Compiler.scala
new file mode 100644
index 0000000..4c12003
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/Compiler.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.codegen
+
+import org.apache.flink.api.common.InvalidProgramException
+import org.codehaus.commons.compiler.CompileException
+import org.codehaus.janino.SimpleCompiler
+
+trait Compiler[T] {
+
+  @throws(classOf[CompileException])
+  def compile(cl: ClassLoader, name: String, code: String): Class[T] = {
+    require(cl != null, "Classloader must not be null.")
+    val compiler = new SimpleCompiler()
+    compiler.setParentClassLoader(cl)
+    try {
+      compiler.cook(code)
+    } catch {
+      case e: CompileException =>
+        throw new InvalidProgramException("Table program cannot be compiled. " +
+          "This is a bug. Please file an issue.", e)
+    }
+    compiler.getClassLoader.loadClass(name).asInstanceOf[Class[T]]
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala
new file mode 100644
index 0000000..94007de
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala
@@ -0,0 +1,120 @@
+/*
+ * 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.codegen
+
+import java.util
+
+import org.apache.calcite.plan.RelOptPlanner
+import org.apache.calcite.rex.{RexBuilder, RexNode}
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.typeutils.TypeConverter
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConverters._
+
+/**
+  * Evaluates constant expressions using Flink's [[CodeGenerator]].
+  */
+class ExpressionReducer(config: TableConfig)
+  extends RelOptPlanner.Executor with Compiler[MapFunction[Row, Row]] {
+
+  private val EMPTY_ROW_INFO = TypeConverter.DEFAULT_ROW_TYPE
+  private val EMPTY_ROW = new Row(0)
+
+  override def reduce(
+    rexBuilder: RexBuilder,
+    constExprs: util.List[RexNode],
+    reducedValues: util.List[RexNode]): Unit = {
+
+    val typeFactory = rexBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+
+    val literals = constExprs.asScala.map(e => (e.getType.getSqlTypeName, e)).flatMap {
+
+      // we need to cast here for RexBuilder.makeLiteral
+      case (SqlTypeName.DATE, e) =>
+        Some(
+          rexBuilder.makeCast(typeFactory.createTypeFromTypeInfo(BasicTypeInfo.INT_TYPE_INFO), e)
+        )
+      case (SqlTypeName.TIME, e) =>
+        Some(
+          rexBuilder.makeCast(typeFactory.createTypeFromTypeInfo(BasicTypeInfo.INT_TYPE_INFO), e)
+        )
+      case (SqlTypeName.TIMESTAMP, e) =>
+        Some(
+          rexBuilder.makeCast(typeFactory.createTypeFromTypeInfo(BasicTypeInfo.LONG_TYPE_INFO), e)
+        )
+
+      // we don't support object literals yet, we skip those constant expressions
+      case (SqlTypeName.ANY, _) | (SqlTypeName.ROW, _) | (SqlTypeName.ARRAY, _) => None
+
+      case (_, e) => Some(e)
+    }
+
+    val literalTypes = literals.map(e => FlinkTypeFactory.toTypeInfo(e.getType))
+    val resultType = new RowTypeInfo(literalTypes: _*)
+
+    // generate MapFunction
+    val generator = new CodeGenerator(config, false, EMPTY_ROW_INFO)
+
+    val result = generator.generateResultExpression(
+      resultType,
+      resultType.getFieldNames,
+      literals)
+
+    val generatedFunction = generator.generateFunction[MapFunction[Row, Row]](
+      "ExpressionReducer",
+      classOf[MapFunction[Row, Row]],
+      s"""
+        |${result.code}
+        |return ${result.resultTerm};
+        |""".stripMargin,
+      resultType.asInstanceOf[TypeInformation[Any]])
+
+    val clazz = compile(getClass.getClassLoader, generatedFunction.name, generatedFunction.code)
+    val function = clazz.newInstance()
+
+    // execute
+    val reduced = function.map(EMPTY_ROW)
+
+    // add the reduced results or keep them unreduced
+    var i = 0
+    var reducedIdx = 0
+    while (i < constExprs.size()) {
+      val unreduced = constExprs.get(i)
+      unreduced.getType.getSqlTypeName match {
+        // we insert the original expression for object literals
+        case SqlTypeName.ANY | SqlTypeName.ROW | SqlTypeName.ARRAY =>
+          reducedValues.add(unreduced)
+        case _ =>
+          val literal = rexBuilder.makeLiteral(
+            reduced.getField(reducedIdx),
+            unreduced.getType,
+            true)
+          reducedValues.add(literal)
+          reducedIdx += 1
+      }
+      i += 1
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/Indenter.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/Indenter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/Indenter.scala
new file mode 100644
index 0000000..187e730
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/Indenter.scala
@@ -0,0 +1,54 @@
+/*
+ * 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.codegen
+
+class IndentStringContext(sc: StringContext) {
+  def j(args: Any*): String = {
+    val sb = new StringBuilder()
+    for ((s, a) <- sc.parts zip args) {
+      sb append s
+
+      val ind = getindent(s)
+      if (ind.nonEmpty) {
+        sb append a.toString.replaceAll("\n", "\n" + ind)
+      } else {
+        sb append a.toString
+      }
+    }
+    if (sc.parts.size > args.size) {
+      sb append sc.parts.last
+    }
+
+    sb.toString()
+  }
+
+  // get white indent after the last new line, if any
+  def getindent(str: String): String = {
+    val lastnl = str.lastIndexOf("\n")
+    if (lastnl == -1) ""
+    else {
+      val ind = str.substring(lastnl + 1)
+      if (ind.trim.isEmpty) ind  // ind is all whitespace. Use this
+      else ""
+    }
+  }
+}
+
+object Indenter {
+  implicit def toISC(sc: StringContext): IndentStringContext = new IndentStringContext(sc)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
new file mode 100644
index 0000000..649d3b2
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
@@ -0,0 +1,39 @@
+/*
+ * 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.codegen.calls
+
+import java.math.{BigDecimal => JBigDecimal}
+
+import org.apache.calcite.linq4j.tree.Types
+import org.apache.calcite.runtime.SqlFunctions
+import org.apache.flink.table.functions.utils.MathFunctions
+
+object BuiltInMethods {
+  val LOG10 = Types.lookupMethod(classOf[Math], "log10", classOf[Double])
+  val EXP = Types.lookupMethod(classOf[Math], "exp", classOf[Double])
+  val POWER = Types.lookupMethod(classOf[Math], "pow", classOf[Double], classOf[Double])
+  val POWER_DEC = Types.lookupMethod(
+    classOf[MathFunctions], "power", classOf[Double], classOf[JBigDecimal])
+  val LN = Types.lookupMethod(classOf[Math], "log", classOf[Double])
+  val ABS = Types.lookupMethod(classOf[SqlFunctions], "abs", classOf[Double])
+  val ABS_DEC = Types.lookupMethod(classOf[SqlFunctions], "abs", classOf[JBigDecimal])
+  val LIKE_WITH_ESCAPE = Types.lookupMethod(classOf[SqlFunctions], "like",
+    classOf[String], classOf[String], classOf[String])
+  val SIMILAR_WITH_ESCAPE = Types.lookupMethod(classOf[SqlFunctions], "similar",
+    classOf[String], classOf[String], classOf[String])
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/CallGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/CallGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/CallGenerator.scala
new file mode 100644
index 0000000..1bc9fbb
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/CallGenerator.scala
@@ -0,0 +1,74 @@
+/*
+ * 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.codegen.calls
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.codegen.CodeGenUtils._
+import org.apache.flink.table.codegen.{CodeGenerator, GeneratedExpression}
+
+trait CallGenerator {
+
+  def generate(
+      codeGenerator: CodeGenerator,
+      operands: Seq[GeneratedExpression])
+    : GeneratedExpression
+
+}
+
+object CallGenerator {
+
+  def generateCallIfArgsNotNull(
+      nullCheck: Boolean,
+      returnType: TypeInformation[_],
+      operands: Seq[GeneratedExpression])
+      (call: (Seq[String]) => String)
+    : GeneratedExpression = {
+    val resultTerm = newName("result")
+    val nullTerm = newName("isNull")
+    val resultTypeTerm = primitiveTypeTermForTypeInfo(returnType)
+    val defaultValue = primitiveDefaultValue(returnType)
+
+    val resultCode = if (nullCheck && operands.nonEmpty) {
+      s"""
+        |${operands.map(_.code).mkString("\n")}
+        |boolean $nullTerm = ${operands.map(_.nullTerm).mkString(" || ")};
+        |$resultTypeTerm $resultTerm;
+        |if ($nullTerm) {
+        |  $resultTerm = $defaultValue;
+        |}
+        |else {
+        |  $resultTerm = ${call(operands.map(_.resultTerm))};
+        |}
+        |""".stripMargin
+    } else if (nullCheck && operands.isEmpty) {
+      s"""
+        |${operands.map(_.code).mkString("\n")}
+        |boolean $nullTerm = false;
+        |$resultTypeTerm $resultTerm = ${call(operands.map(_.resultTerm))};
+        |""".stripMargin
+    } else{
+      s"""
+        |${operands.map(_.code).mkString("\n")}
+        |$resultTypeTerm $resultTerm = ${call(operands.map(_.resultTerm))};
+        |""".stripMargin
+    }
+
+    GeneratedExpression(resultTerm, nullTerm, resultCode, returnType)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/CurrentTimePointCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/CurrentTimePointCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/CurrentTimePointCallGen.scala
new file mode 100644
index 0000000..d644847
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/CurrentTimePointCallGen.scala
@@ -0,0 +1,58 @@
+/*
+ * 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.codegen.calls
+
+import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.table.codegen.{CodeGenerator, GeneratedExpression}
+
+/**
+  * Generates function call to determine current time point (as date/time/timestamp) in
+  * local timezone or not.
+  */
+class CurrentTimePointCallGen(
+    targetType: TypeInformation[_],
+    local: Boolean)
+  extends CallGenerator {
+
+  override def generate(
+      codeGenerator: CodeGenerator,
+      operands: Seq[GeneratedExpression])
+    : GeneratedExpression = targetType match {
+    case SqlTimeTypeInfo.TIME if local =>
+      val time = codeGenerator.addReusableLocalTime()
+      codeGenerator.generateNonNullLiteral(targetType, time)
+
+    case SqlTimeTypeInfo.TIMESTAMP if local =>
+      val timestamp = codeGenerator.addReusableLocalTimestamp()
+      codeGenerator.generateNonNullLiteral(targetType, timestamp)
+
+    case SqlTimeTypeInfo.DATE =>
+      val date = codeGenerator.addReusableDate()
+      codeGenerator.generateNonNullLiteral(targetType, date)
+
+    case SqlTimeTypeInfo.TIME =>
+      val time = codeGenerator.addReusableTime()
+      codeGenerator.generateNonNullLiteral(targetType, time)
+
+    case SqlTimeTypeInfo.TIMESTAMP =>
+      val timestamp = codeGenerator.addReusableTimestamp()
+      codeGenerator.generateNonNullLiteral(targetType, timestamp)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FloorCeilCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FloorCeilCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FloorCeilCallGen.scala
new file mode 100644
index 0000000..dfbb436
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FloorCeilCallGen.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.codegen.calls
+
+import java.lang.reflect.Method
+
+import org.apache.calcite.avatica.util.TimeUnitRange
+import org.apache.calcite.avatica.util.TimeUnitRange.{MONTH, YEAR}
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{BIG_DEC_TYPE_INFO, DOUBLE_TYPE_INFO, FLOAT_TYPE_INFO}
+import org.apache.flink.table.codegen.CodeGenUtils.{getEnum, primitiveTypeTermForTypeInfo, qualifyMethod}
+import org.apache.flink.table.codegen.calls.CallGenerator.generateCallIfArgsNotNull
+import org.apache.flink.table.codegen.{CodeGenerator, GeneratedExpression}
+
+/**
+  * Generates floor/ceil function calls.
+  */
+class FloorCeilCallGen(
+    arithmeticMethod: Method,
+    temporalMethod: Option[Method] = None)
+  extends MultiTypeMethodCallGen(arithmeticMethod) {
+
+  override def generate(
+      codeGenerator: CodeGenerator,
+      operands: Seq[GeneratedExpression])
+    : GeneratedExpression = operands.size match {
+    // arithmetic
+    case 1 =>
+      operands.head.resultType match {
+        case FLOAT_TYPE_INFO | DOUBLE_TYPE_INFO | BIG_DEC_TYPE_INFO =>
+          super.generate(codeGenerator, operands)
+        case _ =>
+          operands.head // no floor/ceil necessary
+      }
+
+    // temporal
+    case 2 =>
+      val operand = operands.head
+      val unit = getEnum(operands(1)).asInstanceOf[TimeUnitRange]
+      val internalType = primitiveTypeTermForTypeInfo(operand.resultType)
+
+      generateCallIfArgsNotNull(codeGenerator.nullCheck, operand.resultType, operands) {
+        (terms) =>
+          unit match {
+            case YEAR | MONTH =>
+              s"""
+                |($internalType) ${qualifyMethod(temporalMethod.get)}(${terms(1)}, ${terms.head})
+                |""".stripMargin
+            case _ =>
+              s"""
+                |${qualifyMethod(arithmeticMethod)}(
+                |  ($internalType) ${terms.head},
+                |  ($internalType) ${unit.startUnit.multiplier.intValue()})
+                |""".stripMargin
+          }
+      }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
new file mode 100644
index 0000000..dfc9055
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
@@ -0,0 +1,369 @@
+/*
+ * 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.codegen.calls
+
+import java.lang.reflect.Method
+
+import org.apache.calcite.avatica.util.TimeUnitRange
+import org.apache.calcite.sql.SqlOperator
+import org.apache.calcite.sql.fun.SqlStdOperatorTable._
+import org.apache.calcite.sql.fun.SqlTrimFunction
+import org.apache.calcite.util.BuiltInMethod
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.GenericTypeInfo
+import org.apache.flink.table.functions.utils.{TableSqlFunction, ScalarSqlFunction}
+
+import scala.collection.mutable
+
+/**
+  * Global hub for user-defined and built-in advanced SQL functions.
+  */
+object FunctionGenerator {
+
+  private val sqlFunctions: mutable.Map[(SqlOperator, Seq[TypeInformation[_]]), CallGenerator] =
+    mutable.Map()
+
+  // ----------------------------------------------------------------------------------------------
+  // String functions
+  // ----------------------------------------------------------------------------------------------
+
+  addSqlFunctionMethod(
+    SUBSTRING,
+    Seq(STRING_TYPE_INFO, INT_TYPE_INFO, INT_TYPE_INFO),
+    STRING_TYPE_INFO,
+    BuiltInMethod.SUBSTRING.method)
+
+  addSqlFunctionMethod(
+    SUBSTRING,
+    Seq(STRING_TYPE_INFO, INT_TYPE_INFO),
+    STRING_TYPE_INFO,
+    BuiltInMethod.SUBSTRING.method)
+
+  addSqlFunction(
+    TRIM,
+    Seq(new GenericTypeInfo(classOf[SqlTrimFunction.Flag]), STRING_TYPE_INFO, STRING_TYPE_INFO),
+    new TrimCallGen())
+
+  addSqlFunctionMethod(
+    CHAR_LENGTH,
+    Seq(STRING_TYPE_INFO),
+    INT_TYPE_INFO,
+    BuiltInMethod.CHAR_LENGTH.method)
+
+  addSqlFunctionMethod(
+    CHARACTER_LENGTH,
+    Seq(STRING_TYPE_INFO),
+    INT_TYPE_INFO,
+    BuiltInMethod.CHAR_LENGTH.method)
+
+  addSqlFunctionMethod(
+    UPPER,
+    Seq(STRING_TYPE_INFO),
+    STRING_TYPE_INFO,
+    BuiltInMethod.UPPER.method)
+
+  addSqlFunctionMethod(
+    LOWER,
+    Seq(STRING_TYPE_INFO),
+    STRING_TYPE_INFO,
+    BuiltInMethod.LOWER.method)
+
+  addSqlFunctionMethod(
+    INITCAP,
+    Seq(STRING_TYPE_INFO),
+    STRING_TYPE_INFO,
+    BuiltInMethod.INITCAP.method)
+
+  addSqlFunctionMethod(
+    LIKE,
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO),
+    BOOLEAN_TYPE_INFO,
+    BuiltInMethod.LIKE.method)
+
+  addSqlFunctionMethod(
+    LIKE,
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO, STRING_TYPE_INFO),
+    BOOLEAN_TYPE_INFO,
+    BuiltInMethods.LIKE_WITH_ESCAPE)
+
+  addSqlFunctionNotMethod(
+    NOT_LIKE,
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO),
+    BuiltInMethod.LIKE.method)
+
+  addSqlFunctionMethod(
+    SIMILAR_TO,
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO),
+    BOOLEAN_TYPE_INFO,
+    BuiltInMethod.SIMILAR.method)
+
+  addSqlFunctionMethod(
+    SIMILAR_TO,
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO, STRING_TYPE_INFO),
+    BOOLEAN_TYPE_INFO,
+    BuiltInMethods.SIMILAR_WITH_ESCAPE)
+
+  addSqlFunctionNotMethod(
+    NOT_SIMILAR_TO,
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO),
+    BuiltInMethod.SIMILAR.method)
+
+  addSqlFunctionMethod(
+    POSITION,
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO),
+    INT_TYPE_INFO,
+    BuiltInMethod.POSITION.method)
+
+  addSqlFunctionMethod(
+    OVERLAY,
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO, INT_TYPE_INFO),
+    STRING_TYPE_INFO,
+    BuiltInMethod.OVERLAY.method)
+
+  addSqlFunctionMethod(
+    OVERLAY,
+    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO, INT_TYPE_INFO, INT_TYPE_INFO),
+    STRING_TYPE_INFO,
+    BuiltInMethod.OVERLAY.method)
+
+  // ----------------------------------------------------------------------------------------------
+  // Arithmetic functions
+  // ----------------------------------------------------------------------------------------------
+
+  addSqlFunctionMethod(
+    LOG10,
+    Seq(DOUBLE_TYPE_INFO),
+    DOUBLE_TYPE_INFO,
+    BuiltInMethods.LOG10)
+
+  addSqlFunctionMethod(
+    LN,
+    Seq(DOUBLE_TYPE_INFO),
+    DOUBLE_TYPE_INFO,
+    BuiltInMethods.LN)
+
+  addSqlFunctionMethod(
+    EXP,
+    Seq(DOUBLE_TYPE_INFO),
+    DOUBLE_TYPE_INFO,
+    BuiltInMethods.EXP)
+
+  addSqlFunctionMethod(
+    POWER,
+    Seq(DOUBLE_TYPE_INFO, DOUBLE_TYPE_INFO),
+    DOUBLE_TYPE_INFO,
+    BuiltInMethods.POWER)
+
+  addSqlFunctionMethod(
+    POWER,
+    Seq(DOUBLE_TYPE_INFO, BIG_DEC_TYPE_INFO),
+    DOUBLE_TYPE_INFO,
+    BuiltInMethods.POWER_DEC)
+
+  addSqlFunction(
+    ABS,
+    Seq(DOUBLE_TYPE_INFO),
+    new MultiTypeMethodCallGen(BuiltInMethods.ABS))
+
+  addSqlFunction(
+    ABS,
+    Seq(BIG_DEC_TYPE_INFO),
+    new MultiTypeMethodCallGen(BuiltInMethods.ABS_DEC))
+
+  addSqlFunction(
+    FLOOR,
+    Seq(DOUBLE_TYPE_INFO),
+    new FloorCeilCallGen(BuiltInMethod.FLOOR.method))
+
+  addSqlFunction(
+    FLOOR,
+    Seq(BIG_DEC_TYPE_INFO),
+    new FloorCeilCallGen(BuiltInMethod.FLOOR.method))
+
+  addSqlFunction(
+    CEIL,
+    Seq(DOUBLE_TYPE_INFO),
+    new FloorCeilCallGen(BuiltInMethod.CEIL.method))
+
+  addSqlFunction(
+    CEIL,
+    Seq(BIG_DEC_TYPE_INFO),
+    new FloorCeilCallGen(BuiltInMethod.CEIL.method))
+
+  // ----------------------------------------------------------------------------------------------
+  // Temporal functions
+  // ----------------------------------------------------------------------------------------------
+
+  addSqlFunctionMethod(
+    EXTRACT_DATE,
+    Seq(new GenericTypeInfo(classOf[TimeUnitRange]), LONG_TYPE_INFO),
+    LONG_TYPE_INFO,
+    BuiltInMethod.UNIX_DATE_EXTRACT.method)
+
+  addSqlFunctionMethod(
+    EXTRACT_DATE,
+    Seq(new GenericTypeInfo(classOf[TimeUnitRange]), SqlTimeTypeInfo.DATE),
+    LONG_TYPE_INFO,
+    BuiltInMethod.UNIX_DATE_EXTRACT.method)
+
+  addSqlFunction(
+    FLOOR,
+    Seq(SqlTimeTypeInfo.DATE, new GenericTypeInfo(classOf[TimeUnitRange])),
+    new FloorCeilCallGen(
+      BuiltInMethod.FLOOR.method,
+      Some(BuiltInMethod.UNIX_DATE_FLOOR.method)))
+
+  addSqlFunction(
+    FLOOR,
+    Seq(SqlTimeTypeInfo.TIME, new GenericTypeInfo(classOf[TimeUnitRange])),
+    new FloorCeilCallGen(
+      BuiltInMethod.FLOOR.method,
+      Some(BuiltInMethod.UNIX_DATE_FLOOR.method)))
+
+  addSqlFunction(
+    FLOOR,
+    Seq(SqlTimeTypeInfo.TIMESTAMP, new GenericTypeInfo(classOf[TimeUnitRange])),
+    new FloorCeilCallGen(
+      BuiltInMethod.FLOOR.method,
+      Some(BuiltInMethod.UNIX_TIMESTAMP_FLOOR.method)))
+
+  addSqlFunction(
+    CEIL,
+    Seq(SqlTimeTypeInfo.DATE, new GenericTypeInfo(classOf[TimeUnitRange])),
+    new FloorCeilCallGen(
+      BuiltInMethod.CEIL.method,
+      Some(BuiltInMethod.UNIX_DATE_CEIL.method)))
+
+  addSqlFunction(
+    CEIL,
+    Seq(SqlTimeTypeInfo.TIME, new GenericTypeInfo(classOf[TimeUnitRange])),
+    new FloorCeilCallGen(
+      BuiltInMethod.CEIL.method,
+      Some(BuiltInMethod.UNIX_DATE_CEIL.method)))
+
+  addSqlFunction(
+    CEIL,
+    Seq(SqlTimeTypeInfo.TIMESTAMP, new GenericTypeInfo(classOf[TimeUnitRange])),
+    new FloorCeilCallGen(
+      BuiltInMethod.CEIL.method,
+      Some(BuiltInMethod.UNIX_TIMESTAMP_CEIL.method)))
+
+  addSqlFunction(
+    CURRENT_DATE,
+    Seq(),
+    new CurrentTimePointCallGen(SqlTimeTypeInfo.DATE, local = false))
+
+  addSqlFunction(
+    CURRENT_TIME,
+    Seq(),
+    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIME, local = false))
+
+  addSqlFunction(
+    CURRENT_TIMESTAMP,
+    Seq(),
+    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIMESTAMP, local = false))
+
+  addSqlFunction(
+    LOCALTIME,
+    Seq(),
+    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIME, local = true))
+
+  addSqlFunction(
+    LOCALTIMESTAMP,
+    Seq(),
+    new CurrentTimePointCallGen(SqlTimeTypeInfo.TIMESTAMP, local = true))
+
+  // ----------------------------------------------------------------------------------------------
+
+  /**
+    * Returns a [[CallGenerator]] that generates all required code for calling the given
+    * [[SqlOperator]].
+    *
+    * @param sqlOperator SQL operator (might be overloaded)
+    * @param operandTypes actual operand types
+    * @param resultType expected return type
+    * @return [[CallGenerator]]
+    */
+  def getCallGenerator(
+      sqlOperator: SqlOperator,
+      operandTypes: Seq[TypeInformation[_]],
+      resultType: TypeInformation[_])
+    : Option[CallGenerator] = sqlOperator match {
+
+    // user-defined scalar function
+    case ssf: ScalarSqlFunction =>
+      Some(
+        new ScalarFunctionCallGen(
+          ssf.getScalarFunction,
+          operandTypes,
+          resultType
+        )
+      )
+
+    // user-defined table function
+    case tsf: TableSqlFunction =>
+      Some(
+        new TableFunctionCallGen(
+          tsf.getTableFunction,
+          operandTypes,
+          resultType
+        )
+      )
+
+    // built-in scalar function
+    case _ =>
+      sqlFunctions.get((sqlOperator, operandTypes))
+        .orElse(sqlFunctions.find(entry => entry._1._1 == sqlOperator
+          && entry._1._2.length == operandTypes.length
+          && entry._1._2.zip(operandTypes).forall {
+          case (x: BasicTypeInfo[_], y: BasicTypeInfo[_]) => y.shouldAutocastTo(x) || x == y
+          case _ => false
+        }).map(_._2))
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  private def addSqlFunctionMethod(
+      sqlOperator: SqlOperator,
+      operandTypes: Seq[TypeInformation[_]],
+      returnType: TypeInformation[_],
+      method: Method)
+    : Unit = {
+    sqlFunctions((sqlOperator, operandTypes)) = new MethodCallGen(returnType, method)
+  }
+
+  private def addSqlFunctionNotMethod(
+      sqlOperator: SqlOperator,
+      operandTypes: Seq[TypeInformation[_]],
+      method: Method)
+    : Unit = {
+    sqlFunctions((sqlOperator, operandTypes)) =
+      new NotCallGenerator(new MethodCallGen(BOOLEAN_TYPE_INFO, method))
+  }
+
+  private def addSqlFunction(
+      sqlOperator: SqlOperator,
+      operandTypes: Seq[TypeInformation[_]],
+      callGenerator: CallGenerator)
+    : Unit = {
+    sqlFunctions((sqlOperator, operandTypes)) = callGenerator
+  }
+
+}


[41/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/java/org/apache/flink/table/explain/PlanJsonParser.java
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/explain/PlanJsonParser.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/explain/PlanJsonParser.java
new file mode 100644
index 0000000..f13c042
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/explain/PlanJsonParser.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.explain;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+public class PlanJsonParser {
+
+	public static String getSqlExecutionPlan(String t, Boolean extended) throws Exception {
+		ObjectMapper objectMapper = new ObjectMapper();
+
+		//not every node is same, ignore the unknown field
+		objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+		PlanTree tree = objectMapper.readValue(t, PlanTree.class);
+		LinkedHashMap<String, Integer> map = new LinkedHashMap<>();
+		StringWriter sw = new StringWriter();
+		PrintWriter pw = new PrintWriter(sw);
+		int tabCount = 0;
+
+		for (int index = 0; index < tree.getNodes().size(); index++) {
+			Node tempNode = tree.getNodes().get(index);
+
+			//input with operation such as join or union is coordinate, keep the same indent 
+			if ((tempNode.getPact().equals("Data Source")) && (map.containsKey(tempNode.getPact()))) {
+				tabCount = map.get(tempNode.getPact());
+			}
+			else {
+				map.put(tempNode.getPact(), tabCount);
+			}
+
+			printTab(tabCount, pw);
+			pw.print("Stage " + tempNode.getId() + " : " + tempNode.getPact() + "\n");
+
+			printTab(tabCount + 1, pw);
+			String content = tempNode.getContents();
+
+			//drop the hashcode of object instance
+			int dele = tempNode.getContents().indexOf("@");
+			if (dele > -1) {
+				content = tempNode.getContents().substring(0, dele);
+			}
+
+			//replace with certain content if node is dataSource to pass
+			//unit tests, because java and scala use different api to
+			//get input element
+			if (tempNode.getPact().equals("Data Source")) {
+				content = "collect elements with CollectionInputFormat";
+			}
+			pw.print("content : " + content + "\n");
+
+			List<Predecessors> predecessors = tempNode.getPredecessors();
+			if (predecessors != null) {
+				printTab(tabCount + 1, pw);
+				pw.print("ship_strategy : " + predecessors.get(0).getShip_strategy() + "\n");
+
+				String mode = predecessors.get(0).getExchange_mode();
+				if (mode != null) {
+					printTab(tabCount + 1, pw);
+					pw.print("exchange_mode : " + mode + "\n");
+				}
+			}
+
+			if (tempNode.getDriver_strategy() != null) {
+				printTab(tabCount + 1, pw);
+				pw.print("driver_strategy : " + tempNode.getDriver_strategy() + "\n");
+			}
+
+			if (tempNode.getGlobal_properties() != null) {
+				printTab(tabCount + 1, pw);
+				pw.print(tempNode.getGlobal_properties().get(0).getName() + " : "
+					+ tempNode.getGlobal_properties().get(0).getValue() + "\n");
+			}
+
+			if (extended) {
+				List<Global_properties> globalProperties = tempNode.getGlobal_properties();
+				for (int i = 1; i < globalProperties.size(); i++) {
+					printTab(tabCount + 1, pw);
+					pw.print(globalProperties.get(i).getName() + " : "
+					+ globalProperties.get(i).getValue() + "\n");
+				}
+
+				List<LocalProperty> localProperties = tempNode.getLocal_properties();
+				for (int i = 0; i < localProperties.size(); i++) {
+					printTab(tabCount + 1, pw);
+					pw.print(localProperties.get(i).getName() + " : "
+					+ localProperties.get(i).getValue() + "\n");
+				}
+
+				List<Estimates> estimates = tempNode.getEstimates();
+				for (int i = 0; i < estimates.size(); i++) {
+					printTab(tabCount + 1, pw);
+					pw.print(estimates.get(i).getName() + " : "
+					+ estimates.get(i).getValue() + "\n");
+				}
+
+				List<Costs> costs = tempNode.getCosts();
+				for (int i = 0; i < costs.size(); i++) {
+					printTab(tabCount + 1, pw);
+					pw.print(costs.get(i).getName() + " : "
+					+ costs.get(i).getValue() + "\n");
+				}
+
+				List<Compiler_hints> compilerHintses = tempNode.getCompiler_hints();
+				for (int i = 0; i < compilerHintses.size(); i++) {
+					printTab(tabCount + 1, pw);
+					pw.print(compilerHintses.get(i).getName() + " : "
+					+ compilerHintses.get(i).getValue() + "\n");
+				}
+			}
+			tabCount++;
+			pw.print("\n");
+		}
+		pw.close();
+		return sw.toString();
+	}
+
+	private static void printTab(int tabCount, PrintWriter pw) {
+		for (int i = 0; i < tabCount; i++)
+			pw.print("\t");
+	}
+}
+
+class PlanTree {
+	private List<Node> nodes;
+
+	public List<Node> getNodes() {
+		return nodes;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/BatchTableEnvironment.scala
deleted file mode 100644
index 0f748c5..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/BatchTableEnvironment.scala
+++ /dev/null
@@ -1,181 +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.api.java.table
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.TypeExtractor
-import org.apache.flink.api.java.{DataSet, ExecutionEnvironment}
-import org.apache.flink.api.table.expressions.ExpressionParser
-import org.apache.flink.api.table.functions.TableFunction
-import org.apache.flink.api.table.{Table, TableConfig}
-
-/**
-  * The [[org.apache.flink.api.table.TableEnvironment]] for a Java batch [[DataSet]]
-  * [[ExecutionEnvironment]].
-  *
-  * A TableEnvironment can be used to:
-  * - convert a [[DataSet]] to a [[Table]]
-  * - register a [[DataSet]] in the [[org.apache.flink.api.table.TableEnvironment]]'s catalog
-  * - register a [[Table]] in the [[org.apache.flink.api.table.TableEnvironment]]'s catalog
-  * - scan a registered table to obtain a [[Table]]
-  * - specify a SQL query on registered tables to obtain a [[Table]]
-  * - convert a [[Table]] into a [[DataSet]]
-  * - explain the AST and execution plan of a [[Table]]
-  *
-  * @param execEnv The Java batch [[ExecutionEnvironment]] of the TableEnvironment.
-  * @param config The configuration of the TableEnvironment.
-  */
-class BatchTableEnvironment(
-    execEnv: ExecutionEnvironment,
-    config: TableConfig)
-  extends org.apache.flink.api.table.BatchTableEnvironment(execEnv, config) {
-
-  /**
-    * Converts the given [[DataSet]] into a [[Table]].
-    *
-    * The field names of the [[Table]] are automatically derived from the type of the [[DataSet]].
-    *
-    * @param dataSet The [[DataSet]] to be converted.
-    * @tparam T The type of the [[DataSet]].
-    * @return The converted [[Table]].
-    */
-  def fromDataSet[T](dataSet: DataSet[T]): Table = {
-
-    val name = createUniqueTableName()
-    registerDataSetInternal(name, dataSet)
-    scan(name)
-  }
-
-  /**
-    * Converts the given [[DataSet]] into a [[Table]] with specified field names.
-    *
-    * Example:
-    *
-    * {{{
-    *   DataSet<Tuple2<String, Long>> set = ...
-    *   Table tab = tableEnv.fromDataSet(set, "a, b")
-    * }}}
-    *
-    * @param dataSet The [[DataSet]] to be converted.
-    * @param fields The field names of the resulting [[Table]].
-    * @tparam T The type of the [[DataSet]].
-    * @return The converted [[Table]].
-    */
-  def fromDataSet[T](dataSet: DataSet[T], fields: String): Table = {
-    val exprs = ExpressionParser
-      .parseExpressionList(fields)
-      .toArray
-
-    val name = createUniqueTableName()
-    registerDataSetInternal(name, dataSet, exprs)
-    scan(name)
-  }
-
-  /**
-    * Registers the given [[DataSet]] as table in the
-    * [[org.apache.flink.api.table.TableEnvironment]]'s catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * The field names of the [[Table]] are automatically derived from the type of the [[DataSet]].
-    *
-    * @param name The name under which the [[DataSet]] is registered in the catalog.
-    * @param dataSet The [[DataSet]] to register.
-    * @tparam T The type of the [[DataSet]] to register.
-    */
-  def registerDataSet[T](name: String, dataSet: DataSet[T]): Unit = {
-
-    checkValidTableName(name)
-    registerDataSetInternal(name, dataSet)
-  }
-
-  /**
-    * Registers the given [[DataSet]] as table with specified field names in the
-    * [[org.apache.flink.api.table.TableEnvironment]]'s catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * Example:
-    *
-    * {{{
-    *   DataSet<Tuple2<String, Long>> set = ...
-    *   tableEnv.registerDataSet("myTable", set, "a, b")
-    * }}}
-    *
-    * @param name The name under which the [[DataSet]] is registered in the catalog.
-    * @param dataSet The [[DataSet]] to register.
-    * @param fields The field names of the registered table.
-    * @tparam T The type of the [[DataSet]] to register.
-    */
-  def registerDataSet[T](name: String, dataSet: DataSet[T], fields: String): Unit = {
-    val exprs = ExpressionParser
-      .parseExpressionList(fields)
-      .toArray
-
-    checkValidTableName(name)
-    registerDataSetInternal(name, dataSet, exprs)
-  }
-
-  /**
-    * Converts the given [[Table]] into a [[DataSet]] of a specified type.
-    *
-    * The fields of the [[Table]] are mapped to [[DataSet]] 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 [[DataSet]] 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 [[DataSet]].
-    * @tparam T The type of the resulting [[DataSet]].
-    * @return The converted [[DataSet]].
-    */
-  def toDataSet[T](table: Table, clazz: Class[T]): DataSet[T] = {
-    translate[T](table)(TypeExtractor.createTypeInfo(clazz))
-  }
-
-  /**
-    * Converts the given [[Table]] into a [[DataSet]] of a specified type.
-    *
-    * The fields of the [[Table]] are mapped to [[DataSet]] 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 [[DataSet]] 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 resulting [[DataSet]].
-    * @tparam T The type of the resulting [[DataSet]].
-    * @return The converted [[DataSet]].
-    */
-  def toDataSet[T](table: Table, typeInfo: TypeInformation[T]): DataSet[T] = {
-    translate[T](table)(typeInfo)
-  }
-
-  /**
-    * Registers a [[TableFunction]] under a unique name in the TableEnvironment's catalog.
-    * Registered functions can be referenced in Table API and SQL queries.
-    *
-    * @param name The name under which the function is registered.
-    * @param tf The TableFunction to register.
-    * @tparam T The type of the output row.
-    */
-  def registerFunction[T](name: String, tf: TableFunction[T]): Unit = {
-    implicit val typeInfo: TypeInformation[T] = TypeExtractor
-      .createTypeInfo(tf, classOf[TableFunction[_]], tf.getClass, 0)
-      .asInstanceOf[TypeInformation[T]]
-
-    registerTableFunctionInternal[T](name, tf)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/StreamTableEnvironment.scala
deleted file mode 100644
index 3218ced..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/StreamTableEnvironment.scala
+++ /dev/null
@@ -1,183 +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.api.java.table
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.TypeExtractor
-import org.apache.flink.api.table.functions.TableFunction
-import org.apache.flink.api.table.{TableConfig, Table}
-import org.apache.flink.api.table.expressions.ExpressionParser
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
-
-/**
-  * The [[org.apache.flink.api.table.TableEnvironment]] for a Java [[StreamExecutionEnvironment]].
-  *
-  * A TableEnvironment can be used to:
-  * - convert a [[DataStream]] to a [[Table]]
-  * - register a [[DataStream]] in the [[org.apache.flink.api.table.TableEnvironment]]'s catalog
-  * - register a [[Table]] in the [[org.apache.flink.api.table.TableEnvironment]]'s catalog
-  * - scan a registered table to obtain a [[Table]]
-  * - specify a SQL query on registered tables to obtain a [[Table]]
-  * - convert a [[Table]] into a [[DataStream]]
-  * - explain the AST and execution plan of a [[Table]]
-  *
-  * @param execEnv The Java [[StreamExecutionEnvironment]] of the TableEnvironment.
-  * @param config The configuration of the TableEnvironment.
-  */
-class StreamTableEnvironment(
-    execEnv: StreamExecutionEnvironment,
-    config: TableConfig)
-  extends org.apache.flink.api.table.StreamTableEnvironment(execEnv, config) {
-
-  /**
-    * Converts the given [[DataStream]] into a [[Table]].
-    *
-    * The field names of the [[Table]] are automatically derived from the type of the
-    * [[DataStream]].
-    *
-    * @param dataStream The [[DataStream]] to be converted.
-    * @tparam T The type of the [[DataStream]].
-    * @return The converted [[Table]].
-    */
-  def fromDataStream[T](dataStream: DataStream[T]): Table = {
-
-    val name = createUniqueTableName()
-    registerDataStreamInternal(name, dataStream)
-    ingest(name)
-  }
-
-  /**
-    * Converts the given [[DataStream]] into a [[Table]] with specified field names.
-    *
-    * Example:
-    *
-    * {{{
-    *   DataStream<Tuple2<String, Long>> stream = ...
-    *   Table tab = tableEnv.fromDataStream(stream, "a, b")
-    * }}}
-    *
-    * @param dataStream The [[DataStream]] to be converted.
-    * @param fields The field names of the resulting [[Table]].
-    * @tparam T The type of the [[DataStream]].
-    * @return The converted [[Table]].
-    */
-  def fromDataStream[T](dataStream: DataStream[T], fields: String): Table = {
-    val exprs = ExpressionParser
-      .parseExpressionList(fields)
-      .toArray
-
-    val name = createUniqueTableName()
-    registerDataStreamInternal(name, dataStream, exprs)
-    ingest(name)
-  }
-
-  /**
-    * Registers the given [[DataStream]] as table in the
-    * [[org.apache.flink.api.table.TableEnvironment]]'s catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * The field names of the [[Table]] are automatically derived
-    * from the type of the [[DataStream]].
-    *
-    * @param name The name under which the [[DataStream]] is registered in the catalog.
-    * @param dataStream The [[DataStream]] to register.
-    * @tparam T The type of the [[DataStream]] to register.
-    */
-  def registerDataStream[T](name: String, dataStream: DataStream[T]): Unit = {
-
-    checkValidTableName(name)
-    registerDataStreamInternal(name, dataStream)
-  }
-
-  /**
-    * Registers the given [[DataStream]] as table with specified field names in the
-    * [[org.apache.flink.api.table.TableEnvironment]]'s catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * Example:
-    *
-    * {{{
-    *   DataStream<Tuple2<String, Long>> set = ...
-    *   tableEnv.registerDataStream("myTable", set, "a, b")
-    * }}}
-    *
-    * @param name The name under which the [[DataStream]] is registered in the catalog.
-    * @param dataStream The [[DataStream]] to register.
-    * @param fields The field names of the registered table.
-    * @tparam T The type of the [[DataStream]] to register.
-    */
-  def registerDataStream[T](name: String, dataStream: DataStream[T], fields: String): Unit = {
-    val exprs = ExpressionParser
-      .parseExpressionList(fields)
-      .toArray
-
-    checkValidTableName(name)
-    registerDataStreamInternal(name, dataStream, exprs)
-  }
-
-  /**
-    * Converts the given [[Table]] into a [[DataStream]] of a specified type.
-    *
-    * 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]].
-    * @tparam T The type of the resulting [[DataStream]].
-    * @return The converted [[DataStream]].
-    */
-  def toDataStream[T](table: Table, clazz: Class[T]): DataStream[T] = {
-    translate[T](table)(TypeExtractor.createTypeInfo(clazz))
-  }
-
-  /**
-    * Converts the given [[Table]] into a [[DataStream]] of a specified type.
-    *
-    * 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] = {
-    translate[T](table)(typeInfo)
-  }
-
-  /**
-    * Registers a [[TableFunction]] under a unique name in the TableEnvironment's catalog.
-    * Registered functions can be referenced in Table API and SQL queries.
-    *
-    * @param name The name under which the function is registered.
-    * @param tf The TableFunction to register.
-    * @tparam T The type of the output row.
-    */
-  def registerFunction[T](name: String, tf: TableFunction[T]): Unit = {
-    implicit val typeInfo: TypeInformation[T] = TypeExtractor
-      .createTypeInfo(tf, classOf[TableFunction[_]], tf.getClass, 0)
-      .asInstanceOf[TypeInformation[T]]
-
-    registerTableFunctionInternal[T](name, tf)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/groupWindows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/groupWindows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/groupWindows.scala
deleted file mode 100644
index 3bbe753..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/java/table/groupWindows.scala
+++ /dev/null
@@ -1,84 +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.api.java.table
-
-import org.apache.flink.api.table.{SessionWindow, SlideWithSize, TumblingWindow}
-
-/**
-  * Helper class for creating a tumbling window. Tumbling windows are consecutive, non-overlapping
-  * windows of a specified fixed length. For example, a tumbling window of 5 minutes size groups
-  * elements in 5 minutes intervals.
-  */
-object Tumble {
-
-  /**
-    * Creates a tumbling window. Tumbling windows are consecutive, non-overlapping
-    * windows of a specified fixed length. For example, a tumbling window of 5 minutes size groups
-    * elements in 5 minutes intervals.
-    *
-    * @param size the size of the window as time or row-count interval.
-    * @return a tumbling window
-    */
-  def over(size: String): TumblingWindow = new TumblingWindow(size)
-}
-
-/**
-  * Helper class for creating a sliding window. Sliding windows have a fixed size and slide by
-  * a specified slide interval. If the slide interval is smaller than the window size, sliding
-  * windows are overlapping. Thus, an element can be assigned to multiple windows.
-  *
-  * For example, a sliding window of size 15 minutes with 5 minutes sliding interval groups elements
-  * of 15 minutes and evaluates every five minutes. Each element is contained in three consecutive
-  * window evaluations.
-  */
-object Slide {
-
-  /**
-    * Creates a sliding window. Sliding windows have a fixed size and slide by
-    * a specified slide interval. If the slide interval is smaller than the window size, sliding
-    * windows are overlapping. Thus, an element can be assigned to multiple windows.
-    *
-    * For example, a sliding window of size 15 minutes with 5 minutes sliding interval groups
-    * elements of 15 minutes and evaluates every five minutes. Each element is contained in three
-    * consecutive window evaluations.
-    *
-    * @param size the size of the window as time or row-count interval
-    * @return a partially specified sliding window
-    */
-  def over(size: String): SlideWithSize = new SlideWithSize(size)
-}
-
-/**
-  * Helper class for creating a session window. The boundary of session windows are defined by
-  * intervals of inactivity, i.e., a session window is closes if no event appears for a defined
-  * gap period.
-  */
-object Session {
-
-  /**
-    * Creates a session window. The boundary of session windows are defined by
-    * intervals of inactivity, i.e., a session window is closes if no event appears for a defined
-    * gap period.
-    *
-    * @param gap specifies how long (as interval of milliseconds) to wait for new data before
-    *            closing the session window.
-    * @return a session window
-    */
-  def withGap(gap: String): SessionWindow = new SessionWindow(gap)
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/BatchTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/BatchTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/BatchTableEnvironment.scala
deleted file mode 100644
index 26fe51e..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/BatchTableEnvironment.scala
+++ /dev/null
@@ -1,154 +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.api.scala.table
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala._
-import org.apache.flink.api.table.expressions.Expression
-import org.apache.flink.api.table.functions.TableFunction
-import org.apache.flink.api.table.{TableConfig, Table}
-
-import scala.reflect.ClassTag
-
-/**
-  * The [[org.apache.flink.api.table.TableEnvironment]] for a Scala batch [[DataSet]]
-  * [[ExecutionEnvironment]].
-  *
-  * A TableEnvironment can be used to:
-  * - convert a [[DataSet]] to a [[Table]]
-  * - register a [[DataSet]] in the [[org.apache.flink.api.table.TableEnvironment]]'s catalog
-  * - register a [[Table]] in the [[org.apache.flink.api.table.TableEnvironment]]'s catalog
-  * - scan a registered table to obtain a [[Table]]
-  * - specify a SQL query on registered tables to obtain a [[Table]]
-  * - convert a [[Table]] into a [[DataSet]]
-  * - explain the AST and execution plan of a [[Table]]
-  *
-  * @param execEnv The Scala batch [[ExecutionEnvironment]] of the TableEnvironment.
-  * @param config The configuration of the TableEnvironment.
-  */
-class BatchTableEnvironment(
-    execEnv: ExecutionEnvironment,
-    config: TableConfig)
-  extends org.apache.flink.api.table.BatchTableEnvironment(execEnv.getJavaEnv, config) {
-
-  /**
-    * Converts the given [[DataSet]] into a [[Table]].
-    *
-    * The field names of the [[Table]] are automatically derived from the type of the [[DataSet]].
-    *
-    * @param dataSet The [[DataSet]] to be converted.
-    * @tparam T The type of the [[DataSet]].
-    * @return The converted [[Table]].
-    */
-  def fromDataSet[T](dataSet: DataSet[T]): Table = {
-
-    val name = createUniqueTableName()
-    registerDataSetInternal(name, dataSet.javaSet)
-    scan(name)
-  }
-
-  /**
-    * Converts the given [[DataSet]] into a [[Table]] with specified field names.
-    *
-    * Example:
-    *
-    * {{{
-    *   val set: DataSet[(String, Long)] = ...
-    *   val tab: Table = tableEnv.fromDataSet(set, 'a, 'b)
-    * }}}
-    *
-    * @param dataSet The [[DataSet]] to be converted.
-    * @param fields The field names of the resulting [[Table]].
-    * @tparam T The type of the [[DataSet]].
-    * @return The converted [[Table]].
-    */
-  def fromDataSet[T](dataSet: DataSet[T], fields: Expression*): Table = {
-
-    val name = createUniqueTableName()
-    registerDataSetInternal(name, dataSet.javaSet, fields.toArray)
-    scan(name)
-  }
-
-  /**
-    * Registers the given [[DataSet]] as table in the
-    * [[org.apache.flink.api.table.TableEnvironment]]'s catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * The field names of the [[Table]] are automatically derived from the type of the [[DataSet]].
-    *
-    * @param name The name under which the [[DataSet]] is registered in the catalog.
-    * @param dataSet The [[DataSet]] to register.
-    * @tparam T The type of the [[DataSet]] to register.
-    */
-  def registerDataSet[T](name: String, dataSet: DataSet[T]): Unit = {
-
-    checkValidTableName(name)
-    registerDataSetInternal(name, dataSet.javaSet)
-  }
-
-  /**
-    * Registers the given [[DataSet]] as table with specified field names in the
-    * [[org.apache.flink.api.table.TableEnvironment]]'s catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * Example:
-    *
-    * {{{
-    *   val set: DataSet[(String, Long)] = ...
-    *   tableEnv.registerDataSet("myTable", set, 'a, 'b)
-    * }}}
-    *
-    * @param name The name under which the [[DataSet]] is registered in the catalog.
-    * @param dataSet The [[DataSet]] to register.
-    * @param fields The field names of the registered table.
-    * @tparam T The type of the [[DataSet]] to register.
-    */
-  def registerDataSet[T](name: String, dataSet: DataSet[T], fields: Expression*): Unit = {
-
-    checkValidTableName(name)
-    registerDataSetInternal(name, dataSet.javaSet, fields.toArray)
-  }
-
-  /**
-    * Converts the given [[Table]] into a [[DataSet]] of a specified type.
-    *
-    * The fields of the [[Table]] are mapped to [[DataSet]] 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 [[DataSet]] types: Fields are mapped by field name, field types must match.
-    *
-    * @param table The [[Table]] to convert.
-    * @tparam T The type of the resulting [[DataSet]].
-    * @return The converted [[DataSet]].
-    */
-  def toDataSet[T: TypeInformation](table: Table): DataSet[T] = {
-    wrap[T](translate(table))(ClassTag.AnyRef.asInstanceOf[ClassTag[T]])
-  }
-
-  /**
-    * Registers a [[TableFunction]] under a unique name in the TableEnvironment's catalog.
-    * Registered functions can be referenced in Table API and SQL queries.
-    *
-    * @param name The name under which the function is registered.
-    * @param tf The TableFunction to register.
-    * @tparam T The type of the output row.
-    */
-  def registerFunction[T: TypeInformation](name: String, tf: TableFunction[T]): Unit = {
-    registerTableFunctionInternal(name, tf)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/DataSetConversions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/DataSetConversions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/DataSetConversions.scala
deleted file mode 100644
index ce437c3..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/DataSetConversions.scala
+++ /dev/null
@@ -1,63 +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.api.scala.table
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala._
-import org.apache.flink.api.table._
-import org.apache.flink.api.table.expressions.Expression
-
-/**
-  * Holds methods to convert a [[DataSet]] into a [[Table]].
-  *
-  * @param dataSet The [[DataSet]] to convert.
-  * @param inputType The [[TypeInformation]] for the type of the [[DataSet]].
-  * @tparam T The type of the [[DataSet]].
-  */
-class DataSetConversions[T](dataSet: DataSet[T], inputType: TypeInformation[T]) {
-
-  /**
-    * Converts the [[DataSet]] into a [[Table]].
-    *
-    * The field name of the new [[Table]] can be specified like this:
-    *
-    * {{{
-    *   val env = ExecutionEnvironment.getExecutionEnvironment
-    *   val tEnv = TableEnvironment.getTableEnvironment(env)
-    *
-    *   val set: DataSet[(String, Int)] = ...
-    *   val table = set.toTable(tEnv, 'name, 'amount)
-    * }}}
-    *
-    * If not explicitly specified, field names are automatically extracted from the type of
-    * the [[DataSet]].
-    *
-    * @param tableEnv The [[BatchTableEnvironment]] in which the new [[Table]] is created.
-    * @param fields The field names of the new [[Table]] (optional).
-    * @return The resulting [[Table]].
-    */
-  def toTable(tableEnv: BatchTableEnvironment, fields: Expression*): Table = {
-    if (fields.isEmpty) {
-      tableEnv.fromDataSet(dataSet)
-    } else {
-      tableEnv.fromDataSet(dataSet, fields: _*)
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/DataStreamConversions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/DataStreamConversions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/DataStreamConversions.scala
deleted file mode 100644
index 3b724cf..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/DataStreamConversions.scala
+++ /dev/null
@@ -1,63 +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.api.scala.table
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table._
-import org.apache.flink.api.table.expressions.Expression
-import org.apache.flink.streaming.api.scala.DataStream
-
-/**
-  * Holds methods to convert a [[DataStream]] into a [[Table]].
-  *
-  * @param dataStream The [[DataStream]] to convert.
-  * @param inputType The [[TypeInformation]] for the type of the [[DataStream]].
-  * @tparam T The type of the [[DataStream]].
-  */
-class DataStreamConversions[T](dataStream: DataStream[T], inputType: TypeInformation[T]) {
-
-  /**
-    * Converts the [[DataStream]] into a [[Table]].
-    *
-    * The field name of the new [[Table]] can be specified like this:
-    *
-    * {{{
-    *   val env = StreamExecutionEnvironment.getExecutionEnvironment
-    *   val tEnv = TableEnvironment.getTableEnvironment(env)
-    *
-    *   val stream: DataStream[(String, Int)] = ...
-    *   val table = stream.toTable(tEnv, 'name, 'amount)
-    * }}}
-    *
-    * If not explicitly specified, field names are automatically extracted from the type of
-    * the [[DataStream]].
-    *
-    * @param tableEnv The [[StreamTableEnvironment]] in which the new [[Table]] is created.
-    * @param fields The field names of the new [[Table]] (optional).
-    * @return The resulting [[Table]].
-    */
-  def toTable(tableEnv: StreamTableEnvironment, fields: Expression*): Table = {
-    if (fields.isEmpty) {
-      tableEnv.fromDataStream(dataStream)
-    } else {
-      tableEnv.fromDataStream(dataStream, fields:_*)
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/StreamTableEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/StreamTableEnvironment.scala
deleted file mode 100644
index 044ace8..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/StreamTableEnvironment.scala
+++ /dev/null
@@ -1,156 +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.api.scala.table
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.functions.TableFunction
-import org.apache.flink.api.table.{TableConfig, Table}
-import org.apache.flink.api.table.expressions.Expression
-import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment, DataStream}
-
-import org.apache.flink.streaming.api.scala.asScalaStream
-
-/**
-  * The [[org.apache.flink.api.table.TableEnvironment]] for a Scala [[StreamExecutionEnvironment]].
-  *
-  * A TableEnvironment can be used to:
-  * - convert a [[DataStream]] to a [[Table]]
-  * - register a [[DataStream]] in the [[org.apache.flink.api.table.TableEnvironment]]'s catalog
-  * - register a [[Table]] in the [[org.apache.flink.api.table.TableEnvironment]]'s catalog
-  * - scan a registered table to obtain a [[Table]]
-  * - specify a SQL query on registered tables to obtain a [[Table]]
-  * - convert a [[Table]] into a [[DataStream]]
-  * - explain the AST and execution plan of a [[Table]]
-  *
-  * @param execEnv The Scala [[StreamExecutionEnvironment]] of the TableEnvironment.
-  * @param config The configuration of the TableEnvironment.
-  */
-class StreamTableEnvironment(
-    execEnv: StreamExecutionEnvironment,
-    config: TableConfig)
-  extends org.apache.flink.api.table.StreamTableEnvironment(
-    execEnv.getWrappedStreamExecutionEnvironment,
-    config) {
-
-  /**
-    * Converts the given [[DataStream]] into a [[Table]].
-    *
-    * The field names of the [[Table]] are automatically derived from the type of the
-    * [[DataStream]].
-    *
-    * @param dataStream The [[DataStream]] to be converted.
-    * @tparam T The type of the [[DataStream]].
-    * @return The converted [[Table]].
-    */
-  def fromDataStream[T](dataStream: DataStream[T]): Table = {
-
-    val name = createUniqueTableName()
-    registerDataStreamInternal(name, dataStream.javaStream)
-    ingest(name)
-  }
-
-  /**
-    * Converts the given [[DataStream]] into a [[Table]] with specified field names.
-    *
-    * Example:
-    *
-    * {{{
-    *   val stream: DataStream[(String, Long)] = ...
-    *   val tab: Table = tableEnv.fromDataStream(stream, 'a, 'b)
-    * }}}
-    *
-    * @param dataStream The [[DataStream]] to be converted.
-    * @param fields The field names of the resulting [[Table]].
-    * @tparam T The type of the [[DataStream]].
-    * @return The converted [[Table]].
-    */
-  def fromDataStream[T](dataStream: DataStream[T], fields: Expression*): Table = {
-
-    val name = createUniqueTableName()
-    registerDataStreamInternal(name, dataStream.javaStream, fields.toArray)
-    ingest(name)
-  }
-
-  /**
-    * Registers the given [[DataStream]] as table in the
-    * [[org.apache.flink.api.table.TableEnvironment]]'s catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * The field names of the [[Table]] are automatically derived
-    * from the type of the [[DataStream]].
-    *
-    * @param name The name under which the [[DataStream]] is registered in the catalog.
-    * @param dataStream The [[DataStream]] to register.
-    * @tparam T The type of the [[DataStream]] to register.
-    */
-  def registerDataStream[T](name: String, dataStream: DataStream[T]): Unit = {
-
-    checkValidTableName(name)
-    registerDataStreamInternal(name, dataStream.javaStream)
-  }
-
-  /**
-    * Registers the given [[DataStream]] as table with specified field names in the
-    * [[org.apache.flink.api.table.TableEnvironment]]'s catalog.
-    * Registered tables can be referenced in SQL queries.
-    *
-    * Example:
-    *
-    * {{{
-    *   val set: DataStream[(String, Long)] = ...
-    *   tableEnv.registerDataStream("myTable", set, 'a, 'b)
-    * }}}
-    *
-    * @param name The name under which the [[DataStream]] is registered in the catalog.
-    * @param dataStream The [[DataStream]] to register.
-    * @param fields The field names of the registered table.
-    * @tparam T The type of the [[DataStream]] to register.
-    */
-  def registerDataStream[T](name: String, dataStream: DataStream[T], fields: Expression*): Unit = {
-
-    checkValidTableName(name)
-    registerDataStreamInternal(name, dataStream.javaStream, fields.toArray)
-  }
-
-  /**
-    * Converts the given [[Table]] into a [[DataStream]] of a specified type.
-    *
-    * 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.
-    * @tparam T The type of the resulting [[DataStream]].
-    * @return The converted [[DataStream]].
-    */
-  def toDataStream[T: TypeInformation](table: Table): DataStream[T] = {
-    asScalaStream(translate(table))
-  }
-
-  /**
-    * Registers a [[TableFunction]] under a unique name in the TableEnvironment's catalog.
-    * Registered functions can be referenced in SQL queries.
-    *
-    * @param name The name under which the function is registered.
-    * @param tf The TableFunction to register
-    */
-  def registerFunction[T: TypeInformation](name: String, tf: TableFunction[T]): Unit = {
-    registerTableFunctionInternal(name, tf)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/TableConversions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/TableConversions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/TableConversions.scala
deleted file mode 100644
index 720dac0..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/TableConversions.scala
+++ /dev/null
@@ -1,62 +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.api.scala.table
-
-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.api.table.{Table, TableException}
-import org.apache.flink.api.scala.table.{BatchTableEnvironment => ScalaBatchTableEnv}
-import org.apache.flink.api.scala.table.{StreamTableEnvironment => ScalaStreamTableEnv}
-
-/**
-  * Holds methods to convert a [[Table]] into a [[DataSet]] or a [[DataStream]].
-  *
-  * @param table The table to convert.
-  */
-class TableConversions(table: Table) {
-
-  /** Converts the [[Table]] to a [[DataSet]] of the specified type. */
-  def toDataSet[T: TypeInformation]: DataSet[T] = {
-
-    table.tableEnv match {
-      case tEnv: ScalaBatchTableEnv =>
-        tEnv.toDataSet(table)
-      case _ =>
-        throw new TableException(
-          "Only tables that originate from Scala DataSets can be converted to Scala DataSets.")
-    }
-  }
-
-  /** Converts the [[Table]] to a [[DataStream]] of the specified type. */
-  def toDataStream[T: TypeInformation]: DataStream[T] = {
-
-    table.tableEnv match {
-      case tEnv: ScalaStreamTableEnv =>
-        tEnv.toDataStream(table)
-      case _ =>
-        throw new TableException(
-          "Only tables that originate from Scala DataStreams " +
-            "can be converted to Scala DataStreams.")
-    }
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala
deleted file mode 100644
index 823458a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/expressionDsl.scala
+++ /dev/null
@@ -1,691 +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.api.scala.table
-
-import java.sql.{Date, Time, Timestamp}
-
-import org.apache.calcite.avatica.util.DateTimeUtils._
-import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation}
-import org.apache.flink.api.table.expressions.ExpressionUtils.{convertArray, toMilliInterval, toMonthInterval, toRowInterval}
-import org.apache.flink.api.table.expressions.TimeIntervalUnit.TimeIntervalUnit
-import org.apache.flink.api.table.expressions._
-import java.math.{BigDecimal => JBigDecimal}
-
-import scala.language.implicitConversions
-
-/**
- * These are all the operations that can be used to construct an [[Expression]] AST for expression
- * operations.
- *
- * These operations must be kept in sync with the parser in
- * [[org.apache.flink.api.table.expressions.ExpressionParser]].
- */
-trait ImplicitExpressionOperations {
-  private[flink] def expr: Expression
-
-  /**
-    * Enables literals on left side of binary expressions.
-    *
-    * e.g. 12.toExpr % 'a
-    *
-    * @return expression
-    */
-  def toExpr: Expression = expr
-
-  def && (other: Expression) = And(expr, other)
-  def || (other: Expression) = Or(expr, other)
-
-  def > (other: Expression) = GreaterThan(expr, other)
-  def >= (other: Expression) = GreaterThanOrEqual(expr, other)
-  def < (other: Expression) = LessThan(expr, other)
-  def <= (other: Expression) = LessThanOrEqual(expr, other)
-
-  def === (other: Expression) = EqualTo(expr, other)
-  def !== (other: Expression) = NotEqualTo(expr, other)
-
-  def unary_! = Not(expr)
-  def unary_- = UnaryMinus(expr)
-
-  def isNull = IsNull(expr)
-  def isNotNull = IsNotNull(expr)
-
-  /**
-    * Returns true if given boolean expression is true. False otherwise (for null and false).
-    */
-  def isTrue = IsTrue(expr)
-
-  /**
-    * Returns true if given boolean expression is false. False otherwise (for null and true).
-    */
-  def isFalse = IsFalse(expr)
-
-  /**
-    * Returns true if given boolean expression is not true (for null and false). False otherwise.
-    */
-  def isNotTrue = IsNotTrue(expr)
-
-  /**
-    * Returns true if given boolean expression is not false (for null and true). False otherwise.
-    */
-  def isNotFalse = IsNotFalse(expr)
-
-  def + (other: Expression) = Plus(expr, other)
-  def - (other: Expression) = Minus(expr, other)
-  def / (other: Expression) = Div(expr, other)
-  def * (other: Expression) = Mul(expr, other)
-  def % (other: Expression) = mod(other)
-
-  def sum = Sum(expr)
-  def min = Min(expr)
-  def max = Max(expr)
-  def count = Count(expr)
-  def avg = Avg(expr)
-
-  def cast(toType: TypeInformation[_]) = Cast(expr, toType)
-
-  /**
-    * Specifies a name for an expression i.e. a field.
-    *
-    * @param name name for one field
-    * @param extraNames additional names if the expression expands to multiple fields
-    * @return field with an alias
-    */
-  def as(name: Symbol, extraNames: Symbol*) = Alias(expr, name.name, extraNames.map(_.name))
-
-  def asc = Asc(expr)
-  def desc = Desc(expr)
-
-  /**
-    * Returns the start time of a window when applied on a window reference.
-    */
-  def start = WindowStart(expr)
-
-  /**
-    * Returns the end time of a window when applied on a window reference.
-    */
-  def end = WindowEnd(expr)
-
-  /**
-    * Ternary conditional operator that decides which of two other expressions should be evaluated
-    * based on a evaluated boolean condition.
-    *
-    * e.g. (42 > 5).?("A", "B") leads to "A"
-    *
-    * @param ifTrue expression to be evaluated if condition holds
-    * @param ifFalse expression to be evaluated if condition does not hold
-    */
-  def ?(ifTrue: Expression, ifFalse: Expression) = {
-    If(expr, ifTrue, ifFalse)
-  }
-
-  // scalar functions
-
-  /**
-    * Calculates the remainder of division the given number by another one.
-    */
-  def mod(other: Expression) = Mod(expr, other)
-
-  /**
-    * Calculates the Euler's number raised to the given power.
-    */
-  def exp() = Exp(expr)
-
-  /**
-    * Calculates the base 10 logarithm of given value.
-    */
-  def log10() = Log10(expr)
-
-  /**
-    * Calculates the natural logarithm of given value.
-    */
-  def ln() = Ln(expr)
-
-  /**
-    * Calculates the given number raised to the power of the other value.
-    */
-  def power(other: Expression) = Power(expr, other)
-
-  /**
-    * Calculates the square root of a given value.
-    */
-  def sqrt() = Sqrt(expr)
-
-  /**
-    * Calculates the absolute value of given value.
-    */
-  def abs() = Abs(expr)
-
-  /**
-    * Calculates the largest integer less than or equal to a given number.
-    */
-  def floor() = Floor(expr)
-
-  /**
-    * Calculates the smallest integer greater than or equal to a given number.
-    */
-  def ceil() = Ceil(expr)
-
-  // String operations
-
-  /**
-    * Creates a substring of the given string at given index for a given length.
-    *
-    * @param beginIndex first character of the substring (starting at 1, inclusive)
-    * @param length number of characters of the substring
-    * @return substring
-    */
-  def substring(beginIndex: Expression, length: Expression) =
-    Substring(expr, beginIndex, length)
-
-  /**
-    * Creates a substring of the given string beginning at the given index to the end.
-    *
-    * @param beginIndex first character of the substring (starting at 1, inclusive)
-    * @return substring
-    */
-  def substring(beginIndex: Expression) =
-    new Substring(expr, beginIndex)
-
-  /**
-    * Removes leading and/or trailing characters from the given string.
-    *
-    * @param removeLeading if true, remove leading characters (default: true)
-    * @param removeTrailing if true, remove trailing characters (default: true)
-    * @param character string containing the character (default: " ")
-    * @return trimmed string
-    */
-  def trim(
-      removeLeading: Boolean = true,
-      removeTrailing: Boolean = true,
-      character: Expression = TrimConstants.TRIM_DEFAULT_CHAR) = {
-    if (removeLeading && removeTrailing) {
-      Trim(TrimMode.BOTH, character, expr)
-    } else if (removeLeading) {
-      Trim(TrimMode.LEADING, character, expr)
-    } else if (removeTrailing) {
-      Trim(TrimMode.TRAILING, character, expr)
-    } else {
-      expr
-    }
-  }
-
-  /**
-    * Returns the length of a string.
-    */
-  def charLength() = CharLength(expr)
-
-  /**
-    * Returns all of the characters in a string in upper case using the rules of
-    * the default locale.
-    */
-  def upperCase() = Upper(expr)
-
-  /**
-    * Returns all of the characters in a string in lower case using the rules of
-    * the default locale.
-    */
-  def lowerCase() = Lower(expr)
-
-  /**
-    * Converts the initial letter of each word in a string to uppercase.
-    * Assumes a string containing only [A-Za-z0-9], everything else is treated as whitespace.
-    */
-  def initCap() = InitCap(expr)
-
-  /**
-    * Returns true, if a string matches the specified LIKE pattern.
-    *
-    * e.g. "Jo_n%" matches all strings that start with "Jo(arbitrary letter)n"
-    */
-  def like(pattern: Expression) = Like(expr, pattern)
-
-  /**
-    * Returns true, if a string matches the specified SQL regex pattern.
-    *
-    * e.g. "A+" matches all strings that consist of at least one A
-    */
-  def similar(pattern: Expression) = Similar(expr, pattern)
-
-  /**
-    * Returns the position of string in an other string starting at 1.
-    * Returns 0 if string could not be found.
-    *
-    * e.g. "a".position("bbbbba") leads to 6
-    */
-  def position(haystack: Expression) = Position(expr, haystack)
-
-  /**
-    * Replaces a substring of string with a string starting at a position (starting at 1).
-    *
-    * e.g. "xxxxxtest".overlay("xxxx", 6) leads to "xxxxxxxxx"
-    */
-  def overlay(newString: Expression, starting: Expression) = new Overlay(expr, newString, starting)
-
-  /**
-    * Replaces a substring of string with a string starting at a position (starting at 1).
-    * The length specifies how many characters should be removed.
-    *
-    * e.g. "xxxxxtest".overlay("xxxx", 6, 2) leads to "xxxxxxxxxst"
-    */
-  def overlay(newString: Expression, starting: Expression, length: Expression) =
-    Overlay(expr, newString, starting, length)
-
-  // Temporal operations
-
-  /**
-    * Parses a date string in the form "yy-mm-dd" to a SQL Date.
-    */
-  def toDate = Cast(expr, SqlTimeTypeInfo.DATE)
-
-  /**
-    * Parses a time string in the form "hh:mm:ss" to a SQL Time.
-    */
-  def toTime = Cast(expr, SqlTimeTypeInfo.TIME)
-
-  /**
-    * Parses a timestamp string in the form "yy-mm-dd hh:mm:ss.fff" to a SQL Timestamp.
-    */
-  def toTimestamp = Cast(expr, SqlTimeTypeInfo.TIMESTAMP)
-
-  /**
-    * Extracts parts of a time point or time interval. Returns the part as a long value.
-    *
-    * e.g. "2006-06-05".toDate.extract(DAY) leads to 5
-    */
-  def extract(timeIntervalUnit: TimeIntervalUnit) = Extract(timeIntervalUnit, expr)
-
-  /**
-    * Returns the quarter of a year from a SQL date.
-    *
-    * e.g. "1994-09-27".toDate.quarter() leads to 3
-    */
-  def quarter() = Quarter(expr)
-
-  /**
-    * Rounds down a time point to the given unit.
-    *
-    * e.g. "12:44:31".toDate.floor(MINUTE) leads to 12:44:00
-    */
-  def floor(timeIntervalUnit: TimeIntervalUnit) = TemporalFloor(timeIntervalUnit, expr)
-
-  /**
-    * Rounds up a time point to the given unit.
-    *
-    * e.g. "12:44:31".toDate.ceil(MINUTE) leads to 12:45:00
-    */
-  def ceil(timeIntervalUnit: TimeIntervalUnit) = TemporalCeil(timeIntervalUnit, expr)
-
-  // Interval types
-
-  /**
-    * Creates an interval of the given number of years.
-    *
-    * @return interval of months
-    */
-  def year = toMonthInterval(expr, 12)
-
-  /**
-    * Creates an interval of the given number of years.
-    *
-    * @return interval of months
-    */
-  def years = year
-
-  /**
-    * Creates an interval of the given number of months.
-    *
-    * @return interval of months
-    */
-  def month = toMonthInterval(expr, 1)
-
-  /**
-    * Creates an interval of the given number of months.
-    *
-    * @return interval of months
-    */
-  def months = month
-
-  /**
-    * Creates an interval of the given number of days.
-    *
-    * @return interval of milliseconds
-    */
-  def day = toMilliInterval(expr, MILLIS_PER_DAY)
-
-  /**
-    * Creates an interval of the given number of days.
-    *
-    * @return interval of milliseconds
-    */
-  def days = day
-
-  /**
-    * Creates an interval of the given number of hours.
-    *
-    * @return interval of milliseconds
-    */
-  def hour = toMilliInterval(expr, MILLIS_PER_HOUR)
-
-  /**
-    * Creates an interval of the given number of hours.
-    *
-    * @return interval of milliseconds
-    */
-  def hours = hour
-
-  /**
-    * Creates an interval of the given number of minutes.
-    *
-    * @return interval of milliseconds
-    */
-  def minute = toMilliInterval(expr, MILLIS_PER_MINUTE)
-
-  /**
-    * Creates an interval of the given number of minutes.
-    *
-    * @return interval of milliseconds
-    */
-  def minutes = minute
-
-  /**
-    * Creates an interval of the given number of seconds.
-    *
-    * @return interval of milliseconds
-    */
-  def second = toMilliInterval(expr, MILLIS_PER_SECOND)
-
-  /**
-    * Creates an interval of the given number of seconds.
-    *
-    * @return interval of milliseconds
-    */
-  def seconds = second
-
-  /**
-    * Creates an interval of the given number of milliseconds.
-    *
-    * @return interval of milliseconds
-    */
-  def milli = toMilliInterval(expr, 1)
-
-  /**
-    * Creates an interval of the given number of milliseconds.
-    *
-    * @return interval of milliseconds
-    */
-  def millis = milli
-
-  // row interval type
-
-  /**
-    * Creates an interval of rows.
-    *
-    * @return interval of rows
-    */
-  def rows = toRowInterval(expr)
-
-  /**
-    * Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by name and
-    * returns it's value.
-    *
-    * @param name name of the field (similar to Flink's field expressions)
-    * @return value of the field
-    */
-  def get(name: String) = GetCompositeField(expr, name)
-
-  /**
-    * Accesses the field of a Flink composite type (such as Tuple, POJO, etc.) by index and
-    * returns it's value.
-    *
-    * @param index position of the field
-    * @return value of the field
-    */
-  def get(index: Int) = GetCompositeField(expr, index)
-
-  /**
-    * Converts a Flink composite type (such as Tuple, POJO, etc.) and all of its direct subtypes
-    * into a flat representation where every subtype is a separate field.
-    */
-  def flatten() = Flattening(expr)
-
-  /**
-    * Accesses the element of an array based on an index (starting at 1).
-    *
-    * @param index position of the element (starting at 1)
-    * @return value of the element
-    */
-  def at(index: Expression) = ArrayElementAt(expr, index)
-
-  /**
-    * Returns the number of elements of an array.
-    *
-    * @return number of elements
-    */
-  def cardinality() = ArrayCardinality(expr)
-
-  /**
-    * Returns the sole element of an array with a single element. Returns null if the array is
-    * empty. Throws an exception if the array has more than one element.
-    *
-    * @return the first and only element of an array with a single element
-    */
-  def element() = ArrayElement(expr)
-}
-
-/**
- * Implicit conversions from Scala Literals to Expression [[Literal]] and from [[Expression]]
- * to [[ImplicitExpressionOperations]].
- */
-trait ImplicitExpressionConversions {
-  implicit class WithOperations(e: Expression) extends ImplicitExpressionOperations {
-    def expr = e
-  }
-
-  implicit class UnresolvedFieldExpression(s: Symbol) extends ImplicitExpressionOperations {
-    def expr = UnresolvedFieldReference(s.name)
-  }
-
-  implicit class LiteralLongExpression(l: Long) extends ImplicitExpressionOperations {
-    def expr = Literal(l)
-  }
-
-  implicit class LiteralByteExpression(b: Byte) extends ImplicitExpressionOperations {
-    def expr = Literal(b)
-  }
-
-  implicit class LiteralShortExpression(s: Short) extends ImplicitExpressionOperations {
-    def expr = Literal(s)
-  }
-
-  implicit class LiteralIntExpression(i: Int) extends ImplicitExpressionOperations {
-    def expr = Literal(i)
-  }
-
-  implicit class LiteralFloatExpression(f: Float) extends ImplicitExpressionOperations {
-    def expr = Literal(f)
-  }
-
-  implicit class LiteralDoubleExpression(d: Double) extends ImplicitExpressionOperations {
-    def expr = Literal(d)
-  }
-
-  implicit class LiteralStringExpression(str: String) extends ImplicitExpressionOperations {
-    def expr = Literal(str)
-  }
-
-  implicit class LiteralBooleanExpression(bool: Boolean) extends ImplicitExpressionOperations {
-    def expr = Literal(bool)
-  }
-
-  implicit class LiteralJavaDecimalExpression(javaDecimal: java.math.BigDecimal)
-      extends ImplicitExpressionOperations {
-    def expr = Literal(javaDecimal)
-  }
-
-  implicit class LiteralScalaDecimalExpression(scalaDecimal: scala.math.BigDecimal)
-      extends ImplicitExpressionOperations {
-    def expr = Literal(scalaDecimal.bigDecimal)
-  }
-
-  implicit class LiteralSqlDateExpression(sqlDate: Date) extends ImplicitExpressionOperations {
-    def expr = Literal(sqlDate)
-  }
-
-  implicit class LiteralSqlTimeExpression(sqlTime: Time) extends ImplicitExpressionOperations {
-    def expr = Literal(sqlTime)
-  }
-
-  implicit class LiteralSqlTimestampExpression(sqlTimestamp: Timestamp)
-      extends ImplicitExpressionOperations {
-    def expr = Literal(sqlTimestamp)
-  }
-
-  implicit def symbol2FieldExpression(sym: Symbol): Expression = UnresolvedFieldReference(sym.name)
-  implicit def byte2Literal(b: Byte): Expression = Literal(b)
-  implicit def short2Literal(s: Short): Expression = Literal(s)
-  implicit def int2Literal(i: Int): Expression = Literal(i)
-  implicit def long2Literal(l: Long): Expression = Literal(l)
-  implicit def double2Literal(d: Double): Expression = Literal(d)
-  implicit def float2Literal(d: Float): Expression = Literal(d)
-  implicit def string2Literal(str: String): Expression = Literal(str)
-  implicit def boolean2Literal(bool: Boolean): Expression = Literal(bool)
-  implicit def javaDec2Literal(javaDec: JBigDecimal): Expression = Literal(javaDec)
-  implicit def scalaDec2Literal(scalaDec: BigDecimal): Expression =
-    Literal(scalaDec.bigDecimal)
-  implicit def sqlDate2Literal(sqlDate: Date): Expression = Literal(sqlDate)
-  implicit def sqlTime2Literal(sqlTime: Time): Expression = Literal(sqlTime)
-  implicit def sqlTimestamp2Literal(sqlTimestamp: Timestamp): Expression =
-    Literal(sqlTimestamp)
-  implicit def array2ArrayConstructor(array: Array[_]): Expression = convertArray(array)
-}
-
-// ------------------------------------------------------------------------------------------------
-// Expressions with no parameters
-// ------------------------------------------------------------------------------------------------
-
-// we disable the object checker here as it checks for capital letters of objects
-// but we want that objects look like functions in certain cases e.g. array(1, 2, 3)
-// scalastyle:off object.name
-
-/**
-  * Returns the current SQL date in UTC time zone.
-  */
-object currentDate {
-
-  /**
-    * Returns the current SQL date in UTC time zone.
-    */
-  def apply(): Expression = {
-    CurrentDate()
-  }
-}
-
-/**
-  * Returns the current SQL time in UTC time zone.
-  */
-object currentTime {
-
-  /**
-    * Returns the current SQL time in UTC time zone.
-    */
-  def apply(): Expression = {
-    CurrentTime()
-  }
-}
-
-/**
-  * Returns the current SQL timestamp in UTC time zone.
-  */
-object currentTimestamp {
-
-  /**
-    * Returns the current SQL timestamp in UTC time zone.
-    */
-  def apply(): Expression = {
-    CurrentTimestamp()
-  }
-}
-
-/**
-  * Returns the current SQL time in local time zone.
-  */
-object localTime {
-
-  /**
-    * Returns the current SQL time in local time zone.
-    */
-  def apply(): Expression = {
-    LocalTime()
-  }
-}
-
-/**
-  * Returns the current SQL timestamp in local time zone.
-  */
-object localTimestamp {
-
-  /**
-    * Returns the current SQL timestamp in local time zone.
-    */
-  def apply(): Expression = {
-    LocalTimestamp()
-  }
-}
-
-/**
-  * Determines whether two anchored time intervals overlap. Time point and temporal are
-  * transformed into a range defined by two time points (start, end). The function
-  * evaluates <code>leftEnd >= rightStart && rightEnd >= leftStart</code>.
-  *
-  * It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
-  *
-  * e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
-  */
-object temporalOverlaps {
-
-  /**
-    * Determines whether two anchored time intervals overlap. Time point and temporal are
-    * transformed into a range defined by two time points (start, end).
-    *
-    * It evaluates: leftEnd >= rightStart && rightEnd >= leftStart
-    *
-    * e.g. temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hour) leads to true
-    */
-  def apply(
-      leftTimePoint: Expression,
-      leftTemporal: Expression,
-      rightTimePoint: Expression,
-      rightTemporal: Expression): Expression = {
-    TemporalOverlaps(leftTimePoint, leftTemporal, rightTimePoint, rightTemporal)
-  }
-}
-
-/**
-  * Creates an array of literals. The array will be an array of objects (not primitives).
-  */
-object array {
-
-  /**
-    * Creates an array of literals. The array will be an array of objects (not primitives).
-    */
-  def apply(head: Expression, tail: Expression*): Expression = {
-    ArrayConstructor(head +: tail.toSeq)
-  }
-}
-
-// scalastyle:on object.name

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/groupWindows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/groupWindows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/groupWindows.scala
deleted file mode 100644
index 16fda5b..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/groupWindows.scala
+++ /dev/null
@@ -1,85 +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.api.scala.table
-
-import org.apache.flink.api.table.expressions.Expression
-import org.apache.flink.api.table.{SessionWindow, SlideWithSize, TumblingWindow}
-
-/**
-  * Helper object for creating a tumbling window. Tumbling windows are consecutive, non-overlapping
-  * windows of a specified fixed length. For example, a tumbling window of 5 minutes size groups
-  * elements in 5 minutes intervals.
-  */
-object Tumble {
-
-  /**
-    * Creates a tumbling window. Tumbling windows are fixed-size, consecutive, non-overlapping
-    * windows. For example, a tumbling window of 5 minutes size groups
-    * elements in 5 minutes intervals.
-    *
-    * @param size the size of the window as time or row-count interval.
-    * @return a tumbling window
-    */
-  def over(size: Expression): TumblingWindow = new TumblingWindow(size)
-}
-
-/**
-  * Helper object for creating a sliding window. Sliding windows have a fixed size and slide by
-  * a specified slide interval. If the slide interval is smaller than the window size, sliding
-  * windows are overlapping. Thus, an element can be assigned to multiple windows.
-  *
-  * For example, a sliding window of size 15 minutes with 5 minutes sliding interval groups elements
-  * of 15 minutes and evaluates every five minutes. Each element is contained in three consecutive
-  * window evaluations.
-  */
-object Slide {
-
-  /**
-    * Creates a sliding window. Sliding windows have a fixed size and slide by
-    * a specified slide interval. If the slide interval is smaller than the window size, sliding
-    * windows are overlapping. Thus, an element can be assigned to multiple windows.
-    *
-    * For example, a sliding window of size 15 minutes with 5 minutes sliding interval groups
-    * elements of 15 minutes and evaluates every five minutes. Each element is contained in three
-    * consecutive
-    *
-    * @param size the size of the window as time or row-count interval
-    * @return a partially specified sliding window
-    */
-  def over(size: Expression): SlideWithSize = new SlideWithSize(size)
-}
-
-/**
-  * Helper object for creating a session window. The boundary of session windows are defined by
-  * intervals of inactivity, i.e., a session window is closes if no event appears for a defined
-  * gap period.
-  */
-object Session {
-
-  /**
-    * Creates a session window. The boundary of session windows are defined by
-    * intervals of inactivity, i.e., a session window is closes if no event appears for a defined
-    * gap period.
-    *
-    * @param gap specifies how long (as interval of milliseconds) to wait for new data before
-    *            closing the session window.
-    * @return a session window
-    */
-  def withGap(gap: Expression): SessionWindow = new SessionWindow(gap)
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/package.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/package.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/package.scala
deleted file mode 100644
index 1e8bf39..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/scala/table/package.scala
+++ /dev/null
@@ -1,90 +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.api.scala
-
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.table.Table
-import org.apache.flink.types.Row
-import scala.language.implicitConversions
-import org.apache.flink.streaming.api.scala._
-
-import scala.reflect.ClassTag
-
-/**
-  * == Table API (Scala) ==
-  *
-  * Importing this package with:
-  *
-  * {{{
-  *   import org.apache.flink.api.scala.table._
-  * }}}
-  *
-  * imports implicit conversions for converting a [[DataSet]] and a [[DataStream]] to a
-  * [[Table]]. This can be used to perform SQL-like queries on data. Please have
-  * a look at [[Table]] to see which operations are supported and
-  * [[org.apache.flink.api.scala.table.ImplicitExpressionOperations]] to see how an
-  * expression can be specified.
-  *
-  * When writing a query you can use Scala Symbols to refer to field names. One would
-  * refer to field `a` by writing `'a`. Sometimes it is necessary to manually convert a
-  * Scala literal to an Expression literal, in those cases use `Literal`, as in `Literal(3)`.
-  *
-  * Example:
-  *
-  * {{{
-  *   import org.apache.flink.api.scala._
-  *   import org.apache.flink.api.scala.table._
-  *
-  *   val env = ExecutionEnvironment.getExecutionEnvironment
-  *   val tEnv = TableEnvironment.getTableEnvironment(env)
-  *
-  *   val input: DataSet[(String, Int)] = env.fromElements(("Hello", 2), ("Hello", 5), ("Ciao", 3))
-  *   val result = input
-  *         .toTable(tEnv, 'word, 'count)
-  *         .groupBy('word)
-  *         .select('word, 'count.avg)
-  *
-  *   result.print()
-  * }}}
-  *
-  */
-package object table extends ImplicitExpressionConversions {
-
-  implicit def table2TableConversions(table: Table): TableConversions = {
-    new TableConversions(table)
-  }
-
-  implicit def dataSet2DataSetConversions[T](set: DataSet[T]): DataSetConversions[T] = {
-    new DataSetConversions[T](set, set.getType())
-  }
-
-  implicit def table2RowDataSet(table: Table): DataSet[Row] = {
-    val tableEnv = table.tableEnv.asInstanceOf[BatchTableEnvironment]
-    tableEnv.toDataSet[Row](table)
-  }
-
-  implicit def dataStream2DataStreamConversions[T](set: DataStream[T]): DataStreamConversions[T] = {
-    new DataStreamConversions[T](set, set.dataType.asInstanceOf[CompositeType[T]])
-  }
-
-  implicit def table2RowDataStream(table: Table): DataStream[Row] = {
-    val tableEnv = table.tableEnv.asInstanceOf[StreamTableEnvironment]
-    tableEnv.toDataStream[Row](table)
-  }
-
-}


[17/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/Aggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/Aggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/Aggregate.scala
new file mode 100644
index 0000000..a614783
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/Aggregate.scala
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.aggregate
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.types.Row
+
+/**
+ * The interface for all Flink aggregate functions, which expressed in terms of initiate(),
+ * prepare(), merge() and evaluate(). The aggregate functions would be executed in 2 phases:
+ * -- In Map phase, use prepare() to transform aggregate field value into intermediate
+ * aggregate value.
+ * -- In GroupReduce phase, use merge() to merge grouped intermediate aggregate values
+ * into aggregate buffer. Then use evaluate() to calculate the final aggregated value.
+ * For associative decomposable aggregate functions, they support partial aggregate. To optimize
+ * the performance, a Combine phase would be added between Map phase and GroupReduce phase,
+ * -- In Combine phase, use merge() to merge sub-grouped intermediate aggregate values
+ * into aggregate buffer.
+ *
+ * The intermediate aggregate value is stored inside Row, aggOffsetInRow is used as the start
+ * field index in Row, so different aggregate functions could share the same Row as intermediate
+ * aggregate value/aggregate buffer, as their aggregate values could be stored in distinct fields
+ * of Row with no conflict. The intermediate aggregate value is required to be a sequence of JVM
+ * primitives, and Flink use intermediateDataType() to get its data types in SQL side.
+ *
+ * @tparam T Aggregated value type.
+ */
+trait Aggregate[T] extends Serializable {
+
+  /**
+    * Transform the aggregate field value into intermediate aggregate data.
+    *
+    * @param value The value to insert into the intermediate aggregate row.
+    * @param intermediate The intermediate aggregate row into which the value is inserted.
+    */
+  def prepare(value: Any, intermediate: Row): Unit
+
+  /**
+    * Initiate the intermediate aggregate value in Row.
+    *
+    * @param intermediate The intermediate aggregate row to initiate.
+    */
+  def initiate(intermediate: Row): Unit
+
+  /**
+    * Merge intermediate aggregate data into aggregate buffer.
+    *
+    * @param intermediate The intermediate aggregate row to merge.
+    * @param buffer The aggregate buffer into which the intermedidate is merged.
+    */
+  def merge(intermediate: Row, buffer: Row): Unit
+
+  /**
+    * Calculate the final aggregated result based on aggregate buffer.
+    *
+    * @param buffer The aggregate buffer from which the final aggregate is computed.
+    * @return The final result of the aggregate.
+    */
+  def evaluate(buffer: Row): T
+
+  /**
+    * Intermediate aggregate value types.
+    *
+    * @return The types of the intermediate fields of this aggregate.
+    */
+  def intermediateDataType: Array[TypeInformation[_]]
+
+  /**
+    * Set the aggregate data offset in Row.
+    *
+    * @param aggOffset The offset of this aggregate in the intermediate aggregate rows.
+    */
+  def setAggOffsetInRow(aggOffset: Int)
+
+  /**
+    * Whether aggregate function support partial aggregate.
+    *
+    * @return True if the aggregate supports partial aggregation, False otherwise.
+    */
+  def supportPartial: Boolean = false
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala
new file mode 100644
index 0000000..234ecfb
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.Iterable
+
+import org.apache.flink.api.common.functions.RichGroupReduceFunction
+import org.apache.flink.types.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow
+import org.apache.flink.util.Collector
+
+class AggregateAllTimeWindowFunction(
+    groupReduceFunction: RichGroupReduceFunction[Row, Row],
+    windowStartPos: Option[Int],
+    windowEndPos: Option[Int])
+  extends AggregateAllWindowFunction[TimeWindow](groupReduceFunction) {
+
+  private var collector: TimeWindowPropertyCollector = _
+
+  override def open(parameters: Configuration): Unit = {
+    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
+    super.open(parameters)
+  }
+
+  override def apply(window: TimeWindow, input: Iterable[Row], out: Collector[Row]): Unit = {
+
+    // set collector and window
+    collector.wrappedCollector = out
+    collector.timeWindow = window
+
+    // call wrapped reduce function with property collector
+    super.apply(window, input, collector)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAllWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAllWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAllWindowFunction.scala
new file mode 100644
index 0000000..10a06da
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateAllWindowFunction.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime.aggregate
+
+import java.lang.Iterable
+
+import org.apache.flink.api.common.functions.RichGroupReduceFunction
+import org.apache.flink.types.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
+import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.util.Collector
+
+class AggregateAllWindowFunction[W <: Window](
+    groupReduceFunction: RichGroupReduceFunction[Row, Row])
+  extends RichAllWindowFunction[Row, Row, W] {
+
+  override def open(parameters: Configuration): Unit = {
+    groupReduceFunction.open(parameters)
+  }
+
+  override def apply(window: W, input: Iterable[Row], out: Collector[Row]): Unit = {
+    groupReduceFunction.reduce(input, out)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateMapFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateMapFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateMapFunction.scala
new file mode 100644
index 0000000..21a96e0
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateMapFunction.scala
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.aggregate
+
+import org.apache.flink.api.common.functions.RichMapFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.types.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Preconditions
+
+class AggregateMapFunction[IN, OUT](
+    private val aggregates: Array[Aggregate[_]],
+    private val aggFields: Array[Int],
+    private val groupingKeys: Array[Int],
+    @transient private val returnType: TypeInformation[OUT])
+  extends RichMapFunction[IN, OUT]
+  with ResultTypeQueryable[OUT] {
+  
+  private var output: Row = _
+  
+  override def open(config: Configuration) {
+    Preconditions.checkNotNull(aggregates)
+    Preconditions.checkNotNull(aggFields)
+    Preconditions.checkArgument(aggregates.size == aggFields.size)
+    val partialRowLength = groupingKeys.length +
+        aggregates.map(_.intermediateDataType.length).sum
+    output = new Row(partialRowLength)
+  }
+
+  override def map(value: IN): OUT = {
+    
+    val input = value.asInstanceOf[Row]
+    for (i <- 0 until aggregates.length) {
+      val fieldValue = input.getField(aggFields(i))
+      aggregates(i).prepare(fieldValue, output)
+    }
+    for (i <- 0 until groupingKeys.length) {
+      output.setField(i, input.getField(groupingKeys(i)))
+    }
+    output.asInstanceOf[OUT]
+  }
+
+  override def getProducedType: TypeInformation[OUT] = {
+    returnType
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceCombineFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceCombineFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceCombineFunction.scala
new file mode 100644
index 0000000..31b85cd
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceCombineFunction.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.aggregate
+
+import java.lang.Iterable
+
+import org.apache.flink.api.common.functions.CombineFunction
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+
+/**
+ * It wraps the aggregate logic inside of
+ * [[org.apache.flink.api.java.operators.GroupReduceOperator]] and
+ * [[org.apache.flink.api.java.operators.GroupCombineOperator]]
+ *
+ * @param aggregates   The aggregate functions.
+ * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
+ *                         and output Row.
+ * @param aggregateMapping The index mapping between aggregate function list and aggregated value
+ *                         index in output Row.
+ */
+class AggregateReduceCombineFunction(
+    private val aggregates: Array[Aggregate[_ <: Any]],
+    private val groupKeysMapping: Array[(Int, Int)],
+    private val aggregateMapping: Array[(Int, Int)],
+    private val intermediateRowArity: Int,
+    private val finalRowArity: Int)
+  extends AggregateReduceGroupFunction(
+    aggregates,
+    groupKeysMapping,
+    aggregateMapping,
+    intermediateRowArity,
+    finalRowArity)
+  with CombineFunction[Row, Row] {
+
+  /**
+   * For sub-grouped intermediate aggregate Rows, merge all of them into aggregate buffer,
+   *
+   * @param records  Sub-grouped intermediate aggregate Rows iterator.
+   * @return Combined intermediate aggregate Row.
+   *
+   */
+  override def combine(records: Iterable[Row]): Row = {
+
+    // Initiate intermediate aggregate value.
+    aggregates.foreach(_.initiate(aggregateBuffer))
+
+    // Merge intermediate aggregate value to buffer.
+    var last: Row = null
+    records.foreach((record) => {
+      aggregates.foreach(_.merge(record, aggregateBuffer))
+      last = record
+    })
+
+    // Set group keys to aggregateBuffer.
+    for (i <- groupKeysMapping.indices) {
+      aggregateBuffer.setField(i, last.getField(i))
+    }
+
+    aggregateBuffer
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceGroupFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceGroupFunction.scala
new file mode 100644
index 0000000..c1efebb
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateReduceGroupFunction.scala
@@ -0,0 +1,92 @@
+/*
+ * 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.Iterable
+
+import org.apache.flink.api.common.functions.RichGroupReduceFunction
+import org.apache.flink.types.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.{Collector, Preconditions}
+
+import scala.collection.JavaConversions._
+
+/**
+ * It wraps the aggregate logic inside of 
+ * [[org.apache.flink.api.java.operators.GroupReduceOperator]].
+ *
+ * @param aggregates   The aggregate functions.
+ * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row 
+ *                         and output Row.
+ * @param aggregateMapping The index mapping between aggregate function list and aggregated value
+ *                         index in output Row.
+ */
+class AggregateReduceGroupFunction(
+    private val aggregates: Array[Aggregate[_ <: Any]],
+    private val groupKeysMapping: Array[(Int, Int)],
+    private val aggregateMapping: Array[(Int, Int)],
+    private val intermediateRowArity: Int,
+    private val finalRowArity: Int)
+  extends RichGroupReduceFunction[Row, Row] {
+
+  protected var aggregateBuffer: Row = _
+  private var output: Row = _
+
+  override def open(config: Configuration) {
+    Preconditions.checkNotNull(aggregates)
+    Preconditions.checkNotNull(groupKeysMapping)
+    aggregateBuffer = new Row(intermediateRowArity)
+    output = new Row(finalRowArity)
+  }
+
+  /**
+   * For grouped intermediate aggregate Rows, merge all of them into aggregate buffer,
+   * calculate aggregated values output by aggregate buffer, and set them into output 
+   * Row based on the mapping relation between intermediate aggregate data and output data.
+   *
+   * @param records  Grouped intermediate aggregate Rows iterator.
+   * @param out The collector to hand results to.
+   *
+   */
+  override def reduce(records: Iterable[Row], out: Collector[Row]): Unit = {
+
+    // Initiate intermediate aggregate value.
+    aggregates.foreach(_.initiate(aggregateBuffer))
+
+    // Merge intermediate aggregate value to buffer.
+    var last: Row = null
+    records.foreach((record) => {
+      aggregates.foreach(_.merge(record, aggregateBuffer))
+      last = record
+    })
+
+    // Set group keys value to final output.
+    groupKeysMapping.foreach {
+      case (after, previous) =>
+        output.setField(after, last.getField(previous))
+    }
+
+    // Evaluate final aggregate value and set to output.
+    aggregateMapping.foreach {
+      case (after, previous) =>
+        output.setField(after, aggregates(previous).evaluate(aggregateBuffer))
+    }
+
+    out.collect(output)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateTimeWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateTimeWindowFunction.scala
new file mode 100644
index 0000000..b7419dd
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateTimeWindowFunction.scala
@@ -0,0 +1,56 @@
+/*
+ * 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.Iterable
+
+import org.apache.flink.api.common.functions.RichGroupReduceFunction
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.types.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow
+import org.apache.flink.util.Collector
+
+class AggregateTimeWindowFunction(
+    groupReduceFunction: RichGroupReduceFunction[Row, Row],
+    windowStartPos: Option[Int],
+    windowEndPos: Option[Int])
+  extends AggregateWindowFunction[TimeWindow](groupReduceFunction) {
+
+  private var collector: TimeWindowPropertyCollector = _
+
+  override def open(parameters: Configuration): Unit = {
+    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
+    super.open(parameters)
+  }
+
+  override def apply(
+    key: Tuple,
+    window: TimeWindow,
+    input: Iterable[Row],
+    out: Collector[Row]): Unit = {
+
+    // set collector and window
+    collector.wrappedCollector = out
+    collector.timeWindow = window
+
+    // call wrapped reduce function with property collector
+    super.apply(key, window, input, collector)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..282e6c0
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
@@ -0,0 +1,595 @@
+/*
+ * 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
+
+import org.apache.calcite.rel.`type`._
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.sql.{SqlAggFunction, SqlKind}
+import org.apache.calcite.sql.`type`.SqlTypeName._
+import org.apache.calcite.sql.`type`.{SqlTypeFactoryImpl, SqlTypeName}
+import org.apache.calcite.sql.fun._
+import org.apache.flink.api.common.functions.{MapFunction, RichGroupReduceFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.calcite.{FlinkRelBuilder, FlinkTypeFactory}
+import FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.expressions.{WindowEnd, WindowStart}
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.typeutils.TypeCheckUtils._
+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.types.Row
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.ArrayBuffer
+
+object AggregateUtil {
+
+  type CalcitePair[T, R] = org.apache.calcite.util.Pair[T, R]
+  type JavaList[T] = java.util.List[T]
+
+  /**
+    * Create a [[org.apache.flink.api.common.functions.MapFunction]] that prepares for aggregates.
+    * The function returns intermediate aggregate values of all aggregate function which are
+    * organized by the following format:
+    *
+    * {{{
+    *                   avg(x) aggOffsetInRow = 2          count(z) aggOffsetInRow = 5
+    *                             |                          |
+    *                             v                          v
+    *        +---------+---------+--------+--------+--------+--------+
+    *        |groupKey1|groupKey2|  sum1  | count1 |  sum2  | count2 |
+    *        +---------+---------+--------+--------+--------+--------+
+    *                                              ^
+    *                                              |
+    *                               sum(y) aggOffsetInRow = 4
+    * }}}
+    *
+    */
+  private[flink] def createPrepareMapFunction(
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    groupings: Array[Int],
+    inputType: RelDataType): MapFunction[Any, Row] = {
+
+    val (aggFieldIndexes,aggregates) = transformToAggregateFunctions(
+      namedAggregates.map(_.getKey),
+      inputType,
+      groupings.length)
+
+    val mapReturnType: RowTypeInfo =
+      createAggregateBufferDataType(groupings, aggregates, inputType)
+
+    val mapFunction = new AggregateMapFunction[Row, Row](
+      aggregates,
+      aggFieldIndexes,
+      groupings,
+      mapReturnType.asInstanceOf[RowTypeInfo]).asInstanceOf[MapFunction[Any, Row]]
+
+    mapFunction
+  }
+
+  /**
+    * Create a [[org.apache.flink.api.common.functions.GroupReduceFunction]] to compute aggregates.
+    * If all aggregates support partial aggregation, the
+    * [[org.apache.flink.api.common.functions.GroupReduceFunction]] implements
+    * [[org.apache.flink.api.common.functions.CombineFunction]] as well.
+    *
+    */
+  private[flink] def createAggregateGroupReduceFunction(
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    inputType: RelDataType,
+    outputType: RelDataType,
+    groupings: Array[Int]): RichGroupReduceFunction[Row, Row] = {
+
+    val aggregates = transformToAggregateFunctions(
+      namedAggregates.map(_.getKey),
+      inputType,
+      groupings.length)._2
+
+    val (groupingOffsetMapping, aggOffsetMapping) =
+      getGroupingOffsetAndAggOffsetMapping(
+        namedAggregates,
+        inputType,
+        outputType,
+        groupings)
+
+    val allPartialAggregate: Boolean = aggregates.forall(_.supportPartial)
+
+    val intermediateRowArity = groupings.length +
+      aggregates.map(_.intermediateDataType.length).sum
+
+    val groupReduceFunction =
+      if (allPartialAggregate) {
+        new AggregateReduceCombineFunction(
+          aggregates,
+          groupingOffsetMapping,
+          aggOffsetMapping,
+          intermediateRowArity,
+          outputType.getFieldCount)
+      }
+      else {
+        new AggregateReduceGroupFunction(
+          aggregates,
+          groupingOffsetMapping,
+          aggOffsetMapping,
+          intermediateRowArity,
+          outputType.getFieldCount)
+      }
+    groupReduceFunction
+  }
+
+  /**
+    * Create a [[org.apache.flink.api.common.functions.ReduceFunction]] for incremental window
+    * aggregation.
+    *
+    */
+  private[flink] def createIncrementalAggregateReduceFunction(
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    inputType: RelDataType,
+    outputType: RelDataType,
+    groupings: Array[Int]): IncrementalAggregateReduceFunction = {
+
+    val aggregates = transformToAggregateFunctions(
+      namedAggregates.map(_.getKey),inputType,groupings.length)._2
+
+    val groupingOffsetMapping =
+      getGroupingOffsetAndAggOffsetMapping(
+        namedAggregates,
+        inputType,
+        outputType,
+        groupings)._1
+
+    val intermediateRowArity = groupings.length + aggregates.map(_.intermediateDataType.length).sum
+    val reduceFunction = new IncrementalAggregateReduceFunction(
+      aggregates,
+      groupingOffsetMapping,
+      intermediateRowArity)
+    reduceFunction
+  }
+
+  /**
+    * Create an [[AllWindowFunction]] to compute non-partitioned group window aggregates.
+    */
+  private[flink] def createAllWindowAggregationFunction(
+    window: LogicalWindow,
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    inputType: RelDataType,
+    outputType: RelDataType,
+    groupings: Array[Int],
+    properties: Seq[NamedWindowProperty])
+  : AllWindowFunction[Row, Row, DataStreamWindow] = {
+
+    val aggFunction =
+      createAggregateGroupReduceFunction(
+        namedAggregates,
+        inputType,
+        outputType,
+        groupings)
+
+    if (isTimeWindow(window)) {
+      val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
+      new AggregateAllTimeWindowFunction(aggFunction, startPos, endPos)
+      .asInstanceOf[AllWindowFunction[Row, Row, DataStreamWindow]]
+    } else {
+      new AggregateAllWindowFunction(aggFunction)
+    }
+  }
+
+  /**
+    * Create a [[WindowFunction]] to compute partitioned group window aggregates.
+    *
+    */
+  private[flink] def createWindowAggregationFunction(
+    window: LogicalWindow,
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    inputType: RelDataType,
+    outputType: RelDataType,
+    groupings: Array[Int],
+    properties: Seq[NamedWindowProperty])
+  : WindowFunction[Row, Row, Tuple, DataStreamWindow] = {
+
+    val aggFunction =
+      createAggregateGroupReduceFunction(
+        namedAggregates,
+        inputType,
+        outputType,
+        groupings)
+
+    if (isTimeWindow(window)) {
+      val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
+      new AggregateTimeWindowFunction(aggFunction, startPos, endPos)
+      .asInstanceOf[WindowFunction[Row, Row, Tuple, DataStreamWindow]]
+    } else {
+      new AggregateWindowFunction(aggFunction)
+    }
+  }
+
+  /**
+    * Create an [[AllWindowFunction]] to finalize incrementally pre-computed non-partitioned
+    * window aggreagtes.
+    */
+  private[flink] def createAllWindowIncrementalAggregationFunction(
+    window: LogicalWindow,
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    inputType: RelDataType,
+    outputType: RelDataType,
+    groupings: Array[Int],
+    properties: Seq[NamedWindowProperty]): AllWindowFunction[Row, Row, DataStreamWindow] = {
+
+    val aggregates = transformToAggregateFunctions(
+      namedAggregates.map(_.getKey),inputType,groupings.length)._2
+
+    val (groupingOffsetMapping, aggOffsetMapping) =
+      getGroupingOffsetAndAggOffsetMapping(
+      namedAggregates,
+      inputType,
+      outputType,
+      groupings)
+
+    val finalRowArity = outputType.getFieldCount
+
+    if (isTimeWindow(window)) {
+      val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
+      new IncrementalAggregateAllTimeWindowFunction(
+        aggregates,
+        groupingOffsetMapping,
+        aggOffsetMapping,
+        finalRowArity,
+        startPos,
+        endPos)
+      .asInstanceOf[AllWindowFunction[Row, Row, DataStreamWindow]]
+    } else {
+      new IncrementalAggregateAllWindowFunction(
+        aggregates,
+        groupingOffsetMapping,
+        aggOffsetMapping,
+        finalRowArity)
+    }
+  }
+
+  /**
+    * Create a [[WindowFunction]] to finalize incrementally pre-computed window aggregates.
+    */
+  private[flink] def createWindowIncrementalAggregationFunction(
+    window: LogicalWindow,
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    inputType: RelDataType,
+    outputType: RelDataType,
+    groupings: Array[Int],
+    properties: Seq[NamedWindowProperty]): WindowFunction[Row, Row, Tuple, DataStreamWindow] = {
+
+    val aggregates = transformToAggregateFunctions(
+      namedAggregates.map(_.getKey),inputType,groupings.length)._2
+
+    val (groupingOffsetMapping, aggOffsetMapping) =
+      getGroupingOffsetAndAggOffsetMapping(
+        namedAggregates,
+        inputType,
+        outputType,
+        groupings)
+
+    val finalRowArity = outputType.getFieldCount
+
+    if (isTimeWindow(window)) {
+      val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
+      new IncrementalAggregateTimeWindowFunction(
+        aggregates,
+        groupingOffsetMapping,
+        aggOffsetMapping,
+        finalRowArity,
+        startPos,
+        endPos)
+      .asInstanceOf[WindowFunction[Row, Row, Tuple, DataStreamWindow]]
+    } else {
+      new IncrementalAggregateWindowFunction(
+        aggregates,
+        groupingOffsetMapping,
+        aggOffsetMapping,
+        finalRowArity)
+    }
+  }
+
+  /**
+    * Return true if all aggregates can be partially computed. False otherwise.
+    */
+  private[flink] def doAllSupportPartialAggregation(
+    aggregateCalls: Seq[AggregateCall],
+    inputType: RelDataType,
+    groupKeysCount: Int): Boolean = {
+    transformToAggregateFunctions(
+      aggregateCalls,
+      inputType,
+      groupKeysCount)._2.forall(_.supportPartial)
+  }
+
+  /**
+    * @return groupingOffsetMapping (mapping relation between field index of intermediate
+    *         aggregate Row and output Row.)
+    *         and aggOffsetMapping (the mapping relation between aggregate function index in list
+    *         and its corresponding field index in output Row.)
+    */
+  private def getGroupingOffsetAndAggOffsetMapping(
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    inputType: RelDataType,
+    outputType: RelDataType,
+    groupings: Array[Int]): (Array[(Int, Int)], Array[(Int, Int)]) = {
+
+    // the mapping relation between field index of intermediate aggregate Row and output Row.
+    val groupingOffsetMapping = getGroupKeysMapping(inputType, outputType, groupings)
+
+    // the mapping relation between aggregate function index in list and its corresponding
+    // field index in output Row.
+    val aggOffsetMapping = getAggregateMapping(namedAggregates, outputType)
+
+    if (groupingOffsetMapping.length != groupings.length ||
+      aggOffsetMapping.length != namedAggregates.length) {
+      throw new TableException(
+        "Could not find output field in input data type " +
+          "or aggregate functions.")
+    }
+    (groupingOffsetMapping, aggOffsetMapping)
+  }
+
+  private def isTimeWindow(window: LogicalWindow) = {
+    window match {
+      case ProcessingTimeTumblingGroupWindow(_, size) => isTimeInterval(size.resultType)
+      case ProcessingTimeSlidingGroupWindow(_, size, _) => isTimeInterval(size.resultType)
+      case ProcessingTimeSessionGroupWindow(_, _) => true
+      case EventTimeTumblingGroupWindow(_, _, size) => isTimeInterval(size.resultType)
+      case EventTimeSlidingGroupWindow(_, _, size, _) => isTimeInterval(size.resultType)
+      case EventTimeSessionGroupWindow(_, _, _) => true
+    }
+  }
+
+  private def computeWindowStartEndPropertyPos(
+    properties: Seq[NamedWindowProperty]): (Option[Int], Option[Int]) = {
+
+    val propPos = properties.foldRight((None: Option[Int], None: Option[Int], 0)) {
+      (p, x) => p match {
+        case NamedWindowProperty(name, prop) =>
+          prop match {
+            case WindowStart(_) if x._1.isDefined =>
+              throw new TableException("Duplicate WindowStart property encountered. This is a bug.")
+            case WindowStart(_) =>
+              (Some(x._3), x._2, x._3 - 1)
+            case WindowEnd(_) if x._2.isDefined =>
+              throw new TableException("Duplicate WindowEnd property encountered. This is a bug.")
+            case WindowEnd(_) =>
+              (x._1, Some(x._3), x._3 - 1)
+          }
+      }
+    }
+    (propPos._1, propPos._2)
+  }
+
+  private def transformToAggregateFunctions(
+    aggregateCalls: Seq[AggregateCall],
+    inputType: RelDataType,
+    groupKeysCount: Int): (Array[Int], Array[Aggregate[_ <: Any]]) = {
+
+    // store the aggregate fields of each aggregate function, by the same order of aggregates.
+    val aggFieldIndexes = new Array[Int](aggregateCalls.size)
+    val aggregates = new Array[Aggregate[_ <: Any]](aggregateCalls.size)
+
+    // set the start offset of aggregate buffer value to group keys' length, 
+    // as all the group keys would be moved to the start fields of intermediate
+    // aggregate data.
+    var aggOffset = groupKeysCount
+
+    // create aggregate function instances by function type and aggregate field data type.
+    aggregateCalls.zipWithIndex.foreach { case (aggregateCall, index) =>
+      val argList: util.List[Integer] = aggregateCall.getArgList
+      if (argList.isEmpty) {
+        if (aggregateCall.getAggregation.isInstanceOf[SqlCountAggFunction]) {
+          aggFieldIndexes(index) = 0
+        } else {
+          throw new TableException("Aggregate fields should not be empty.")
+        }
+      } else {
+        if (argList.size() > 1) {
+          throw new TableException("Currently, do not support aggregate on multi fields.")
+        }
+        aggFieldIndexes(index) = argList.get(0)
+      }
+      val sqlTypeName = inputType.getFieldList.get(aggFieldIndexes(index)).getType.getSqlTypeName
+      aggregateCall.getAggregation match {
+        case _: SqlSumAggFunction | _: SqlSumEmptyIsZeroAggFunction => {
+          aggregates(index) = sqlTypeName match {
+            case TINYINT =>
+              new ByteSumAggregate
+            case SMALLINT =>
+              new ShortSumAggregate
+            case INTEGER =>
+              new IntSumAggregate
+            case BIGINT =>
+              new LongSumAggregate
+            case FLOAT =>
+              new FloatSumAggregate
+            case DOUBLE =>
+              new DoubleSumAggregate
+            case DECIMAL =>
+              new DecimalSumAggregate
+            case sqlType: SqlTypeName =>
+              throw new TableException("Sum aggregate does no support type:" + sqlType)
+          }
+        }
+        case _: SqlAvgAggFunction => {
+          aggregates(index) = sqlTypeName match {
+            case TINYINT =>
+               new ByteAvgAggregate
+            case SMALLINT =>
+              new ShortAvgAggregate
+            case INTEGER =>
+              new IntAvgAggregate
+            case BIGINT =>
+              new LongAvgAggregate
+            case FLOAT =>
+              new FloatAvgAggregate
+            case DOUBLE =>
+              new DoubleAvgAggregate
+            case DECIMAL =>
+              new DecimalAvgAggregate
+            case sqlType: SqlTypeName =>
+              throw new TableException("Avg aggregate does no support type:" + sqlType)
+          }
+        }
+        case sqlMinMaxFunction: SqlMinMaxAggFunction => {
+          aggregates(index) = if (sqlMinMaxFunction.getKind == SqlKind.MIN) {
+            sqlTypeName match {
+              case TINYINT =>
+                new ByteMinAggregate
+              case SMALLINT =>
+                new ShortMinAggregate
+              case INTEGER =>
+                new IntMinAggregate
+              case BIGINT =>
+                new LongMinAggregate
+              case FLOAT =>
+                new FloatMinAggregate
+              case DOUBLE =>
+                new DoubleMinAggregate
+              case DECIMAL =>
+                new DecimalMinAggregate
+              case BOOLEAN =>
+                new BooleanMinAggregate
+              case sqlType: SqlTypeName =>
+                throw new TableException("Min aggregate does no support type:" + sqlType)
+            }
+          } else {
+            sqlTypeName match {
+              case TINYINT =>
+                new ByteMaxAggregate
+              case SMALLINT =>
+                new ShortMaxAggregate
+              case INTEGER =>
+                new IntMaxAggregate
+              case BIGINT =>
+                new LongMaxAggregate
+              case FLOAT =>
+                new FloatMaxAggregate
+              case DOUBLE =>
+                new DoubleMaxAggregate
+              case DECIMAL =>
+                new DecimalMaxAggregate
+              case BOOLEAN =>
+                new BooleanMaxAggregate
+              case sqlType: SqlTypeName =>
+                throw new TableException("Max aggregate does no support type:" + sqlType)
+            }
+          }
+        }
+        case _: SqlCountAggFunction =>
+          aggregates(index) = new CountAggregate
+        case unSupported: SqlAggFunction =>
+          throw new TableException("unsupported Function: " + unSupported.getName)
+      }
+      setAggregateDataOffset(index)
+    }
+
+    // set the aggregate intermediate data start index in Row, and update current value.
+    def setAggregateDataOffset(index: Int): Unit = {
+      aggregates(index).setAggOffsetInRow(aggOffset)
+      aggOffset += aggregates(index).intermediateDataType.length
+    }
+
+    (aggFieldIndexes, aggregates)
+  }
+
+  private def createAggregateBufferDataType(
+    groupings: Array[Int],
+    aggregates: Array[Aggregate[_]],
+    inputType: RelDataType): RowTypeInfo = {
+
+    // get the field data types of group keys.
+    val groupingTypes: Seq[TypeInformation[_]] = groupings
+      .map(inputType.getFieldList.get(_).getType)
+      .map(FlinkTypeFactory.toTypeInfo)
+
+    val aggPartialNameSuffix = "agg_buffer_"
+    val factory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT)
+
+    // get all field data types of all intermediate aggregates
+    val aggTypes: Seq[TypeInformation[_]] = aggregates.flatMap(_.intermediateDataType)
+
+    // concat group key types and aggregation types
+    val allFieldTypes = groupingTypes ++: aggTypes
+    val partialType = new RowTypeInfo(allFieldTypes: _*)
+    partialType
+  }
+
+  // Find the mapping between the index of aggregate list and aggregated value index in output Row.
+  private def getAggregateMapping(
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    outputType: RelDataType): Array[(Int, Int)] = {
+
+    // the mapping relation between aggregate function index in list and its corresponding
+    // field index in output Row.
+    var aggOffsetMapping = ArrayBuffer[(Int, Int)]()
+
+    outputType.getFieldList.zipWithIndex.foreach{
+      case (outputFieldType, outputIndex) =>
+        namedAggregates.zipWithIndex.foreach {
+          case (namedAggCall, aggregateIndex) =>
+            if (namedAggCall.getValue.equals(outputFieldType.getName) &&
+                namedAggCall.getKey.getType.equals(outputFieldType.getType)) {
+              aggOffsetMapping += ((outputIndex, aggregateIndex))
+            }
+        }
+    }
+   
+    aggOffsetMapping.toArray
+  }
+
+  // Find the mapping between the index of group key in intermediate aggregate Row and its index
+  // in output Row.
+  private def getGroupKeysMapping(
+    inputDatType: RelDataType,
+    outputType: RelDataType,
+    groupKeys: Array[Int]): Array[(Int, Int)] = {
+
+    // the mapping relation between field index of intermediate aggregate Row and output Row.
+    var groupingOffsetMapping = ArrayBuffer[(Int, Int)]()
+
+    outputType.getFieldList.zipWithIndex.foreach {
+      case (outputFieldType, outputIndex) =>
+        inputDatType.getFieldList.zipWithIndex.foreach {
+          // find the field index in input data type.
+          case (inputFieldType, inputIndex) =>
+            if (outputFieldType.getName.equals(inputFieldType.getName) &&
+                outputFieldType.getType.equals(inputFieldType.getType)) {
+              // as aggregated field in output data type would not have a matched field in
+              // input data, so if inputIndex is not -1, it must be a group key. Then we can
+              // find the field index in buffer data by the group keys index mapping between
+              // input data and buffer data.
+              for (i <- groupKeys.indices) {
+                if (inputIndex == groupKeys(i)) {
+                  groupingOffsetMapping += ((outputIndex, i))
+                }
+              }
+            }
+        }
+    }
+
+    groupingOffsetMapping.toArray
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateWindowFunction.scala
new file mode 100644
index 0000000..5491b1d
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateWindowFunction.scala
@@ -0,0 +1,46 @@
+/*
+ * 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.Iterable
+
+import org.apache.flink.api.common.functions.RichGroupReduceFunction
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.types.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
+import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.util.Collector
+
+class AggregateWindowFunction[W <: Window](groupReduceFunction: RichGroupReduceFunction[Row, Row])
+  extends RichWindowFunction[Row, Row, Tuple, W] {
+
+  override def open(parameters: Configuration): Unit = {
+    groupReduceFunction.open(parameters)
+  }
+
+  override def apply(
+    key: Tuple,
+    window: W,
+    input: Iterable[Row],
+    out: Collector[Row]): Unit = {
+
+    groupReduceFunction.reduce(input, out)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AvgAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AvgAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AvgAggregate.scala
new file mode 100644
index 0000000..cb94ca1
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AvgAggregate.scala
@@ -0,0 +1,296 @@
+/*
+ * 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 com.google.common.math.LongMath
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.types.Row
+import java.math.BigDecimal
+import java.math.BigInteger
+
+abstract class AvgAggregate[T] extends Aggregate[T] {
+  protected var partialSumIndex: Int = _
+  protected var partialCountIndex: Int = _
+
+  override def supportPartial: Boolean = true
+
+  override def setAggOffsetInRow(aggOffset: Int): Unit = {
+    partialSumIndex = aggOffset
+    partialCountIndex = aggOffset + 1
+  }
+}
+
+abstract class IntegralAvgAggregate[T] extends AvgAggregate[T] {
+
+  override def initiate(partial: Row): Unit = {
+    partial.setField(partialSumIndex, 0L)
+    partial.setField(partialCountIndex, 0L)
+  }
+
+  override def prepare(value: Any, partial: Row): Unit = {
+    if (value == null) {
+      partial.setField(partialSumIndex, 0L)
+      partial.setField(partialCountIndex, 0L)
+    } else {
+      doPrepare(value, partial)
+    }
+  }
+
+  override def merge(partial: Row, buffer: Row): Unit = {
+    val partialSum = partial.getField(partialSumIndex).asInstanceOf[Long]
+    val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long]
+    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long]
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+    buffer.setField(partialSumIndex, LongMath.checkedAdd(partialSum, bufferSum))
+    buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount))
+  }
+
+  override def evaluate(buffer : Row): T = {
+    doEvaluate(buffer).asInstanceOf[T]
+  }
+
+  override def intermediateDataType = Array(
+    BasicTypeInfo.LONG_TYPE_INFO,
+    BasicTypeInfo.LONG_TYPE_INFO)
+
+  def doPrepare(value: Any, partial: Row): Unit
+
+  def doEvaluate(buffer: Row): Any
+}
+
+class ByteAvgAggregate extends IntegralAvgAggregate[Byte] {
+  override def doPrepare(value: Any, partial: Row): Unit = {
+    val input = value.asInstanceOf[Byte]
+    partial.setField(partialSumIndex, input.toLong)
+    partial.setField(partialCountIndex, 1L)
+  }
+
+  override def doEvaluate(buffer: Row): Any = {
+    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long]
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+    if (bufferCount == 0L) {
+      null
+    } else {
+      (bufferSum / bufferCount).toByte
+    }
+  }
+}
+
+class ShortAvgAggregate extends IntegralAvgAggregate[Short] {
+
+  override def doPrepare(value: Any, partial: Row): Unit = {
+    val input = value.asInstanceOf[Short]
+    partial.setField(partialSumIndex, input.toLong)
+    partial.setField(partialCountIndex, 1L)
+  }
+
+  override def doEvaluate(buffer: Row): Any = {
+    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long]
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+    if (bufferCount == 0L) {
+      null
+    } else {
+      (bufferSum / bufferCount).toShort
+    }
+  }
+}
+
+class IntAvgAggregate extends IntegralAvgAggregate[Int] {
+
+  override def doPrepare(value: Any, partial: Row): Unit = {
+    val input = value.asInstanceOf[Int]
+    partial.setField(partialSumIndex, input.toLong)
+    partial.setField(partialCountIndex, 1L)
+  }
+
+  override def doEvaluate(buffer: Row): Any = {
+    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long]
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+    if (bufferCount == 0L) {
+      null
+    } else {
+      (bufferSum / bufferCount).toInt
+    }
+  }
+}
+
+class LongAvgAggregate extends IntegralAvgAggregate[Long] {
+
+  override def intermediateDataType = Array(
+    BasicTypeInfo.BIG_INT_TYPE_INFO,
+    BasicTypeInfo.LONG_TYPE_INFO)
+
+  override def initiate(partial: Row): Unit = {
+    partial.setField(partialSumIndex, BigInteger.ZERO)
+    partial.setField(partialCountIndex, 0L)
+  }
+
+  override def prepare(value: Any, partial: Row): Unit = {
+    if (value == null) {
+      partial.setField(partialSumIndex, BigInteger.ZERO)
+      partial.setField(partialCountIndex, 0L)
+    } else {
+      doPrepare(value, partial)
+    }
+  }
+
+  override def doPrepare(value: Any, partial: Row): Unit = {
+    val input = value.asInstanceOf[Long]
+    partial.setField(partialSumIndex, BigInteger.valueOf(input))
+    partial.setField(partialCountIndex, 1L)
+  }
+
+  override def merge(partial: Row, buffer: Row): Unit = {
+    val partialSum = partial.getField(partialSumIndex).asInstanceOf[BigInteger]
+    val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long]
+    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigInteger]
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+    buffer.setField(partialSumIndex, partialSum.add(bufferSum))
+    buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount))
+  }
+
+  override def doEvaluate(buffer: Row): Any = {
+    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigInteger]
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+    if (bufferCount == 0L) {
+      null
+    } else {
+      bufferSum.divide(BigInteger.valueOf(bufferCount)).longValue()
+    }
+  }
+}
+
+abstract class FloatingAvgAggregate[T: Numeric] extends AvgAggregate[T] {
+
+  override def initiate(partial: Row): Unit = {
+    partial.setField(partialSumIndex, 0D)
+    partial.setField(partialCountIndex, 0L)
+  }
+
+  override def prepare(value: Any, partial: Row): Unit = {
+    if (value == null) {
+      partial.setField(partialSumIndex, 0D)
+      partial.setField(partialCountIndex, 0L)
+    } else {
+      doPrepare(value, partial)
+    }
+  }
+
+  override def merge(partial: Row, buffer: Row): Unit = {
+    val partialSum = partial.getField(partialSumIndex).asInstanceOf[Double]
+    val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long]
+    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Double]
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+
+    buffer.setField(partialSumIndex, partialSum + bufferSum)
+    buffer.setField(partialCountIndex, partialCount + bufferCount)
+  }
+
+  override def evaluate(buffer : Row): T = {
+    doEvaluate(buffer).asInstanceOf[T]
+  }
+
+  override def intermediateDataType = Array(
+    BasicTypeInfo.DOUBLE_TYPE_INFO,
+    BasicTypeInfo.LONG_TYPE_INFO)
+
+  def doPrepare(value: Any, partial: Row): Unit
+
+  def doEvaluate(buffer: Row): Any
+}
+
+class FloatAvgAggregate extends FloatingAvgAggregate[Float] {
+
+  override def doPrepare(value: Any, partial: Row): Unit = {
+    val input = value.asInstanceOf[Float]
+    partial.setField(partialSumIndex, input.toDouble)
+    partial.setField(partialCountIndex, 1L)
+  }
+
+
+  override def doEvaluate(buffer: Row): Any = {
+    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Double]
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+    if (bufferCount == 0L) {
+      null
+    } else {
+      (bufferSum / bufferCount).toFloat
+    }
+  }
+}
+
+class DoubleAvgAggregate extends FloatingAvgAggregate[Double] {
+
+  override def doPrepare(value: Any, partial: Row): Unit = {
+    val input = value.asInstanceOf[Double]
+    partial.setField(partialSumIndex, input)
+    partial.setField(partialCountIndex, 1L)
+  }
+
+  override def doEvaluate(buffer: Row): Any = {
+    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Double]
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+    if (bufferCount == 0L) {
+      null
+    } else {
+      (bufferSum / bufferCount)
+    }
+  }
+}
+
+class DecimalAvgAggregate extends AvgAggregate[BigDecimal] {
+
+  override def intermediateDataType = Array(
+    BasicTypeInfo.BIG_DEC_TYPE_INFO,
+    BasicTypeInfo.LONG_TYPE_INFO)
+
+  override def initiate(partial: Row): Unit = {
+    partial.setField(partialSumIndex, BigDecimal.ZERO)
+    partial.setField(partialCountIndex, 0L)
+  }
+
+  override def prepare(value: Any, partial: Row): Unit = {
+    if (value == null) {
+      initiate(partial)
+    } else {
+      val input = value.asInstanceOf[BigDecimal]
+      partial.setField(partialSumIndex, input)
+      partial.setField(partialCountIndex, 1L)
+    }
+  }
+
+  override def merge(partial: Row, buffer: Row): Unit = {
+    val partialSum = partial.getField(partialSumIndex).asInstanceOf[BigDecimal]
+    val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long]
+    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigDecimal]
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+    buffer.setField(partialSumIndex, partialSum.add(bufferSum))
+    buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount))
+  }
+
+  override def evaluate(buffer: Row): BigDecimal = {
+    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
+    if (bufferCount != 0) {
+      val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigDecimal]
+      bufferSum.divide(BigDecimal.valueOf(bufferCount))
+    } else {
+      null.asInstanceOf[BigDecimal]
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CountAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CountAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CountAggregate.scala
new file mode 100644
index 0000000..ea8e1d8
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/CountAggregate.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.aggregate
+
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.types.Row
+
+class CountAggregate extends Aggregate[Long] {
+  private var countIndex: Int = _
+
+  override def initiate(intermediate: Row): Unit = {
+    intermediate.setField(countIndex, 0L)
+  }
+
+  override def merge(intermediate: Row, buffer: Row): Unit = {
+    val partialCount = intermediate.getField(countIndex).asInstanceOf[Long]
+    val bufferCount = buffer.getField(countIndex).asInstanceOf[Long]
+    buffer.setField(countIndex, partialCount + bufferCount)
+  }
+
+  override def evaluate(buffer: Row): Long = {
+    buffer.getField(countIndex).asInstanceOf[Long]
+  }
+
+  override def prepare(value: Any, intermediate: Row): Unit = {
+    if (value == null) {
+      intermediate.setField(countIndex, 0L)
+    } else {
+      intermediate.setField(countIndex, 1L)
+    }
+  }
+
+  override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO)
+
+  override def supportPartial: Boolean = true
+
+  override def setAggOffsetInRow(aggIndex: Int): Unit = {
+    countIndex = aggIndex
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala
new file mode 100644
index 0000000..5d7a94b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.Iterable
+
+import org.apache.flink.types.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow
+import org.apache.flink.util.Collector
+/**
+  *
+  * Computes the final aggregate value from incrementally computed aggreagtes.
+  *
+  * @param aggregates   The aggregate functions.
+  * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
+  *                         and output Row.
+  * @param aggregateMapping The index mapping between aggregate function list and aggregated value
+  *                         index in output Row.
+  * @param finalRowArity  The arity of the final output row.
+  */
+class IncrementalAggregateAllTimeWindowFunction(
+    private val aggregates: Array[Aggregate[_ <: Any]],
+    private val groupKeysMapping: Array[(Int, Int)],
+    private val aggregateMapping: Array[(Int, Int)],
+    private val finalRowArity: Int,
+    private val windowStartPos: Option[Int],
+    private val windowEndPos: Option[Int])
+  extends IncrementalAggregateAllWindowFunction[TimeWindow](
+    aggregates,
+    groupKeysMapping,
+    aggregateMapping,
+    finalRowArity) {
+
+  private var collector: TimeWindowPropertyCollector = _
+
+  override def open(parameters: Configuration): Unit = {
+    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
+    super.open(parameters)
+  }
+
+  override def apply(
+    window: TimeWindow,
+    records: Iterable[Row],
+    out: Collector[Row]): Unit = {
+
+    // set collector and window
+    collector.wrappedCollector = out
+    collector.timeWindow = window
+
+    super.apply(window, records, collector)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala
new file mode 100644
index 0000000..3c41a62
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.aggregate
+
+import java.lang.Iterable
+
+import org.apache.flink.types.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
+import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.util.{Collector, Preconditions}
+
+/**
+  * Computes the final aggregate value from incrementally computed aggreagtes.
+  *
+  * @param aggregates   The aggregate functions.
+  * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
+  *                         and output Row.
+  * @param aggregateMapping The index mapping between aggregate function list and aggregated value
+  *                         index in output Row.
+  * @param finalRowArity  The arity of the final output row.
+  */
+class IncrementalAggregateAllWindowFunction[W <: Window](
+    private val aggregates: Array[Aggregate[_ <: Any]],
+    private val groupKeysMapping: Array[(Int, Int)],
+    private val aggregateMapping: Array[(Int, Int)],
+    private val finalRowArity: Int)
+  extends RichAllWindowFunction[Row, Row, W] {
+
+  private var output: Row = _
+
+  override def open(parameters: Configuration): Unit = {
+    Preconditions.checkNotNull(aggregates)
+    Preconditions.checkNotNull(groupKeysMapping)
+    output = new Row(finalRowArity)
+  }
+
+  /**
+    * Calculate aggregated values output by aggregate buffer, and set them into output
+    * Row based on the mapping relation between intermediate aggregate data and output data.
+    */
+  override def apply(
+    window: W,
+    records: Iterable[Row],
+    out: Collector[Row]): Unit = {
+
+    val iterator = records.iterator
+
+    if (iterator.hasNext) {
+      val record = iterator.next()
+      // Set group keys value to final output.
+      groupKeysMapping.foreach {
+        case (after, previous) =>
+          output.setField(after, record.getField(previous))
+      }
+      // Evaluate final aggregate value and set to output.
+      aggregateMapping.foreach {
+        case (after, previous) =>
+          output.setField(after, aggregates(previous).evaluate(record))
+      }
+      out.collect(output)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala
new file mode 100644
index 0000000..14b44e8
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.aggregate
+
+import org.apache.flink.api.common.functions.ReduceFunction
+import org.apache.flink.types.Row
+import org.apache.flink.util.Preconditions
+
+/**
+  * Incrementally computes group window aggregates.
+  *
+  * @param aggregates   The aggregate functions.
+  * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
+  *                         and output Row.
+  */
+class IncrementalAggregateReduceFunction(
+    private val aggregates: Array[Aggregate[_]],
+    private val groupKeysMapping: Array[(Int, Int)],
+    private val intermediateRowArity: Int)
+  extends ReduceFunction[Row] {
+
+  Preconditions.checkNotNull(aggregates)
+  Preconditions.checkNotNull(groupKeysMapping)
+
+  /**
+    * For Incremental intermediate aggregate Rows, merge value1 and value2
+    * into aggregate buffer, return aggregate buffer.
+    *
+    * @param value1 The first value to combined.
+    * @param value2 The second value to combined.
+    * @return  accumulatorRow A resulting row that combines two input values.
+    *
+    */
+  override def reduce(value1: Row, value2: Row): Row = {
+
+    // TODO: once FLINK-5105 is solved, we can avoid creating a new row for each invocation
+    //   and directly merge value1 and value2.
+    val accumulatorRow = new Row(intermediateRowArity)
+
+    // copy all fields of value1 into accumulatorRow
+    (0 until intermediateRowArity)
+    .foreach(i => accumulatorRow.setField(i, value1.getField(i)))
+    // merge value2 to accumulatorRow
+    aggregates.foreach(_.merge(value2, accumulatorRow))
+
+    accumulatorRow
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala
new file mode 100644
index 0000000..a96ce7a
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala
@@ -0,0 +1,69 @@
+/*
+ * 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.Iterable
+
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.types.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow
+import org.apache.flink.util.Collector
+
+/**
+  * Computes the final aggregate value from incrementally computed aggreagtes.
+  *
+  * @param aggregates   The aggregate functions.
+  * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
+  *                         and output Row.
+  * @param aggregateMapping The index mapping between aggregate function list and aggregated value
+  *                         index in output Row.
+  * @param finalRowArity  The arity of the final output row.
+  */
+class IncrementalAggregateTimeWindowFunction(
+    private val aggregates: Array[Aggregate[_ <: Any]],
+    private val groupKeysMapping: Array[(Int, Int)],
+    private val aggregateMapping: Array[(Int, Int)],
+    private val finalRowArity: Int,
+    private val windowStartPos: Option[Int],
+    private val windowEndPos: Option[Int])
+  extends IncrementalAggregateWindowFunction[TimeWindow](
+    aggregates,
+    groupKeysMapping,
+    aggregateMapping, finalRowArity) {
+
+  private var collector: TimeWindowPropertyCollector = _
+
+  override def open(parameters: Configuration): Unit = {
+    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
+    super.open(parameters)
+  }
+
+  override def apply(
+    key: Tuple,
+    window: TimeWindow,
+    records: Iterable[Row],
+    out: Collector[Row]): Unit = {
+
+    // set collector and window
+    collector.wrappedCollector = out
+    collector.timeWindow = window
+
+    super.apply(key, window, records, collector)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala
new file mode 100644
index 0000000..30f7a7b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime.aggregate
+
+import java.lang.Iterable
+
+import org.apache.flink.api.java.tuple.Tuple
+import org.apache.flink.types.Row
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
+import org.apache.flink.streaming.api.windowing.windows.Window
+import org.apache.flink.util.{Collector, Preconditions}
+
+/**
+  * Computes the final aggregate value from incrementally computed aggreagtes.
+  *
+  * @param aggregates   The aggregate functions.
+  * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
+  *                         and output Row.
+  * @param aggregateMapping The index mapping between aggregate function list and aggregated value
+  *                         index in output Row.
+  * @param finalRowArity  The arity of the final output row.
+  */
+class IncrementalAggregateWindowFunction[W <: Window](
+    private val aggregates: Array[Aggregate[_ <: Any]],
+    private val groupKeysMapping: Array[(Int, Int)],
+    private val aggregateMapping: Array[(Int, Int)],
+    private val finalRowArity: Int)
+  extends RichWindowFunction[Row, Row, Tuple, W] {
+
+  private var output: Row = _
+
+  override def open(parameters: Configuration): Unit = {
+    Preconditions.checkNotNull(aggregates)
+    Preconditions.checkNotNull(groupKeysMapping)
+    output = new Row(finalRowArity)
+  }
+
+  /**
+    * Calculate aggregated values output by aggregate buffer, and set them into output
+    * Row based on the mapping relation between intermediate aggregate data and output data.
+    */
+  override def apply(
+    key: Tuple,
+    window: W,
+    records: Iterable[Row],
+    out: Collector[Row]): Unit = {
+
+    val iterator = records.iterator
+
+    if (iterator.hasNext) {
+      val record = iterator.next()
+      // Set group keys value to final output.
+      groupKeysMapping.foreach {
+        case (after, previous) =>
+          output.setField(after, record.getField(previous))
+      }
+      // Evaluate final aggregate value and set to output.
+      aggregateMapping.foreach {
+        case (after, previous) =>
+          output.setField(after, aggregates(previous).evaluate(record))
+      }
+      out.collect(output)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/MaxAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/MaxAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/MaxAggregate.scala
new file mode 100644
index 0000000..34b25e0
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/MaxAggregate.scala
@@ -0,0 +1,171 @@
+/*
+ * 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.math.BigDecimal
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.types.Row
+
+abstract class MaxAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] {
+
+  protected var maxIndex = -1
+
+  /**
+   * Initiate the intermediate aggregate value in Row.
+   *
+   * @param intermediate The intermediate aggregate row to initiate.
+   */
+  override def initiate(intermediate: Row): Unit = {
+    intermediate.setField(maxIndex, null)
+  }
+
+  /**
+   * Accessed in MapFunction, prepare the input of partial aggregate.
+   *
+   * @param value
+   * @param intermediate
+   */
+  override def prepare(value: Any, intermediate: Row): Unit = {
+    if (value == null) {
+      initiate(intermediate)
+    } else {
+      intermediate.setField(maxIndex, value)
+    }
+  }
+
+  /**
+   * Accessed in CombineFunction and GroupReduceFunction, merge partial
+   * aggregate result into aggregate buffer.
+   *
+   * @param intermediate
+   * @param buffer
+   */
+  override def merge(intermediate: Row, buffer: Row): Unit = {
+    val partialValue = intermediate.getField(maxIndex).asInstanceOf[T]
+    if (partialValue != null) {
+      val bufferValue = buffer.getField(maxIndex).asInstanceOf[T]
+      if (bufferValue != null) {
+        val max : T = if (ord.compare(partialValue, bufferValue) > 0) partialValue else bufferValue
+        buffer.setField(maxIndex, max)
+      } else {
+        buffer.setField(maxIndex, partialValue)
+      }
+    }
+  }
+
+  /**
+   * Return the final aggregated result based on aggregate buffer.
+   *
+   * @param buffer
+   * @return
+   */
+  override def evaluate(buffer: Row): T = {
+    buffer.getField(maxIndex).asInstanceOf[T]
+  }
+
+  override def supportPartial: Boolean = true
+
+  override def setAggOffsetInRow(aggOffset: Int): Unit = {
+    maxIndex = aggOffset
+  }
+}
+
+class ByteMaxAggregate extends MaxAggregate[Byte] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.BYTE_TYPE_INFO)
+
+}
+
+class ShortMaxAggregate extends MaxAggregate[Short] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.SHORT_TYPE_INFO)
+
+}
+
+class IntMaxAggregate extends MaxAggregate[Int] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.INT_TYPE_INFO)
+
+}
+
+class LongMaxAggregate extends MaxAggregate[Long] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO)
+
+}
+
+class FloatMaxAggregate extends MaxAggregate[Float] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.FLOAT_TYPE_INFO)
+
+}
+
+class DoubleMaxAggregate extends MaxAggregate[Double] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.DOUBLE_TYPE_INFO)
+
+}
+
+class BooleanMaxAggregate extends MaxAggregate[Boolean] {
+
+  override def intermediateDataType = Array(BasicTypeInfo.BOOLEAN_TYPE_INFO)
+
+}
+
+class DecimalMaxAggregate extends Aggregate[BigDecimal] {
+
+  protected var minIndex: Int = _
+
+  override def intermediateDataType = Array(BasicTypeInfo.BIG_DEC_TYPE_INFO)
+
+  override def initiate(intermediate: Row): Unit = {
+    intermediate.setField(minIndex, null)
+  }
+
+  override def prepare(value: Any, partial: Row): Unit = {
+    if (value == null) {
+      initiate(partial)
+    } else {
+      partial.setField(minIndex, value)
+    }
+  }
+
+  override def merge(partial: Row, buffer: Row): Unit = {
+    val partialValue = partial.getField(minIndex).asInstanceOf[BigDecimal]
+    if (partialValue != null) {
+      val bufferValue = buffer.getField(minIndex).asInstanceOf[BigDecimal]
+      if (bufferValue != null) {
+        val min = if (partialValue.compareTo(bufferValue) > 0) partialValue else bufferValue
+        buffer.setField(minIndex, min)
+      } else {
+        buffer.setField(minIndex, partialValue)
+      }
+    }
+  }
+
+  override def evaluate(buffer: Row): BigDecimal = {
+    buffer.getField(minIndex).asInstanceOf[BigDecimal]
+  }
+
+  override def supportPartial: Boolean = true
+
+  override def setAggOffsetInRow(aggOffset: Int): Unit = {
+    minIndex = aggOffset
+  }
+}


[31/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala
deleted file mode 100644
index 273aa60..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/Aggregate.scala
+++ /dev/null
@@ -1,96 +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.api.table.runtime.aggregate
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.types.Row
-
-/**
- * The interface for all Flink aggregate functions, which expressed in terms of initiate(),
- * prepare(), merge() and evaluate(). The aggregate functions would be executed in 2 phases:
- * -- In Map phase, use prepare() to transform aggregate field value into intermediate
- * aggregate value.
- * -- In GroupReduce phase, use merge() to merge grouped intermediate aggregate values
- * into aggregate buffer. Then use evaluate() to calculate the final aggregated value.
- * For associative decomposable aggregate functions, they support partial aggregate. To optimize
- * the performance, a Combine phase would be added between Map phase and GroupReduce phase,
- * -- In Combine phase, use merge() to merge sub-grouped intermediate aggregate values
- * into aggregate buffer.
- *
- * The intermediate aggregate value is stored inside Row, aggOffsetInRow is used as the start
- * field index in Row, so different aggregate functions could share the same Row as intermediate
- * aggregate value/aggregate buffer, as their aggregate values could be stored in distinct fields
- * of Row with no conflict. The intermediate aggregate value is required to be a sequence of JVM
- * primitives, and Flink use intermediateDataType() to get its data types in SQL side.
- *
- * @tparam T Aggregated value type.
- */
-trait Aggregate[T] extends Serializable {
-
-  /**
-    * Transform the aggregate field value into intermediate aggregate data.
-    *
-    * @param value The value to insert into the intermediate aggregate row.
-    * @param intermediate The intermediate aggregate row into which the value is inserted.
-    */
-  def prepare(value: Any, intermediate: Row): Unit
-
-  /**
-    * Initiate the intermediate aggregate value in Row.
-    *
-    * @param intermediate The intermediate aggregate row to initiate.
-    */
-  def initiate(intermediate: Row): Unit
-
-  /**
-    * Merge intermediate aggregate data into aggregate buffer.
-    *
-    * @param intermediate The intermediate aggregate row to merge.
-    * @param buffer The aggregate buffer into which the intermedidate is merged.
-    */
-  def merge(intermediate: Row, buffer: Row): Unit
-
-  /**
-    * Calculate the final aggregated result based on aggregate buffer.
-    *
-    * @param buffer The aggregate buffer from which the final aggregate is computed.
-    * @return The final result of the aggregate.
-    */
-  def evaluate(buffer: Row): T
-
-  /**
-    * Intermediate aggregate value types.
-    *
-    * @return The types of the intermediate fields of this aggregate.
-    */
-  def intermediateDataType: Array[TypeInformation[_]]
-
-  /**
-    * Set the aggregate data offset in Row.
-    *
-    * @param aggOffset The offset of this aggregate in the intermediate aggregate rows.
-    */
-  def setAggOffsetInRow(aggOffset: Int)
-
-  /**
-    * Whether aggregate function support partial aggregate.
-    *
-    * @return True if the aggregate supports partial aggregation, False otherwise.
-    */
-  def supportPartial: Boolean = false
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala
deleted file mode 100644
index 4c473d4..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllTimeWindowFunction.scala
+++ /dev/null
@@ -1,52 +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.api.table.runtime.aggregate
-
-import java.lang.Iterable
-
-import org.apache.flink.api.common.functions.RichGroupReduceFunction
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow
-import org.apache.flink.util.Collector
-
-class AggregateAllTimeWindowFunction(
-    groupReduceFunction: RichGroupReduceFunction[Row, Row],
-    windowStartPos: Option[Int],
-    windowEndPos: Option[Int])
-  extends AggregateAllWindowFunction[TimeWindow](groupReduceFunction) {
-
-  private var collector: TimeWindowPropertyCollector = _
-
-  override def open(parameters: Configuration): Unit = {
-    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
-    super.open(parameters)
-  }
-
-  override def apply(window: TimeWindow, input: Iterable[Row], out: Collector[Row]): Unit = {
-
-    // set collector and window
-    collector.wrappedCollector = out
-    collector.timeWindow = window
-
-    // call wrapped reduce function with property collector
-    super.apply(window, input, collector)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllWindowFunction.scala
deleted file mode 100644
index db5f477..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateAllWindowFunction.scala
+++ /dev/null
@@ -1,41 +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.api.table.runtime.aggregate
-
-import java.lang.Iterable
-
-import org.apache.flink.api.common.functions.RichGroupReduceFunction
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
-import org.apache.flink.streaming.api.windowing.windows.Window
-import org.apache.flink.util.Collector
-
-class AggregateAllWindowFunction[W <: Window](
-    groupReduceFunction: RichGroupReduceFunction[Row, Row])
-  extends RichAllWindowFunction[Row, Row, W] {
-
-  override def open(parameters: Configuration): Unit = {
-    groupReduceFunction.open(parameters)
-  }
-
-  override def apply(window: W, input: Iterable[Row], out: Collector[Row]): Unit = {
-    groupReduceFunction.reduce(input, out)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateMapFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateMapFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateMapFunction.scala
deleted file mode 100644
index 0699bfa..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateMapFunction.scala
+++ /dev/null
@@ -1,62 +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.api.table.runtime.aggregate
-
-import org.apache.flink.api.common.functions.RichMapFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.util.Preconditions
-
-class AggregateMapFunction[IN, OUT](
-    private val aggregates: Array[Aggregate[_]],
-    private val aggFields: Array[Int],
-    private val groupingKeys: Array[Int],
-    @transient private val returnType: TypeInformation[OUT])
-  extends RichMapFunction[IN, OUT]
-  with ResultTypeQueryable[OUT] {
-  
-  private var output: Row = _
-  
-  override def open(config: Configuration) {
-    Preconditions.checkNotNull(aggregates)
-    Preconditions.checkNotNull(aggFields)
-    Preconditions.checkArgument(aggregates.size == aggFields.size)
-    val partialRowLength = groupingKeys.length +
-        aggregates.map(_.intermediateDataType.length).sum
-    output = new Row(partialRowLength)
-  }
-
-  override def map(value: IN): OUT = {
-    
-    val input = value.asInstanceOf[Row]
-    for (i <- 0 until aggregates.length) {
-      val fieldValue = input.getField(aggFields(i))
-      aggregates(i).prepare(fieldValue, output)
-    }
-    for (i <- 0 until groupingKeys.length) {
-      output.setField(i, input.getField(groupingKeys(i)))
-    }
-    output.asInstanceOf[OUT]
-  }
-
-  override def getProducedType: TypeInformation[OUT] = {
-    returnType
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceCombineFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceCombineFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceCombineFunction.scala
deleted file mode 100644
index b2cf07e..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceCombineFunction.scala
+++ /dev/null
@@ -1,82 +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.api.table.runtime.aggregate
-
-import java.lang.Iterable
-
-import org.apache.flink.api.common.functions.{CombineFunction, RichGroupReduceFunction}
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.util.{Collector, Preconditions}
-
-import scala.collection.JavaConversions._
-
-
-/**
- * It wraps the aggregate logic inside of
- * [[org.apache.flink.api.java.operators.GroupReduceOperator]] and
- * [[org.apache.flink.api.java.operators.GroupCombineOperator]]
- *
- * @param aggregates   The aggregate functions.
- * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
- *                         and output Row.
- * @param aggregateMapping The index mapping between aggregate function list and aggregated value
- *                         index in output Row.
- */
-class AggregateReduceCombineFunction(
-    private val aggregates: Array[Aggregate[_ <: Any]],
-    private val groupKeysMapping: Array[(Int, Int)],
-    private val aggregateMapping: Array[(Int, Int)],
-    private val intermediateRowArity: Int,
-    private val finalRowArity: Int)
-  extends AggregateReduceGroupFunction(
-    aggregates,
-    groupKeysMapping,
-    aggregateMapping,
-    intermediateRowArity,
-    finalRowArity)
-  with CombineFunction[Row, Row] {
-
-  /**
-   * For sub-grouped intermediate aggregate Rows, merge all of them into aggregate buffer,
-   *
-   * @param records  Sub-grouped intermediate aggregate Rows iterator.
-   * @return Combined intermediate aggregate Row.
-   *
-   */
-  override def combine(records: Iterable[Row]): Row = {
-
-    // Initiate intermediate aggregate value.
-    aggregates.foreach(_.initiate(aggregateBuffer))
-
-    // Merge intermediate aggregate value to buffer.
-    var last: Row = null
-    records.foreach((record) => {
-      aggregates.foreach(_.merge(record, aggregateBuffer))
-      last = record
-    })
-
-    // Set group keys to aggregateBuffer.
-    for (i <- groupKeysMapping.indices) {
-      aggregateBuffer.setField(i, last.getField(i))
-    }
-
-    aggregateBuffer
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceGroupFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceGroupFunction.scala
deleted file mode 100644
index 6fe712b..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateReduceGroupFunction.scala
+++ /dev/null
@@ -1,92 +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.api.table.runtime.aggregate
-
-import java.lang.Iterable
-
-import org.apache.flink.api.common.functions.RichGroupReduceFunction
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.util.{Collector, Preconditions}
-
-import scala.collection.JavaConversions._
-
-/**
- * It wraps the aggregate logic inside of 
- * [[org.apache.flink.api.java.operators.GroupReduceOperator]].
- *
- * @param aggregates   The aggregate functions.
- * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row 
- *                         and output Row.
- * @param aggregateMapping The index mapping between aggregate function list and aggregated value
- *                         index in output Row.
- */
-class AggregateReduceGroupFunction(
-    private val aggregates: Array[Aggregate[_ <: Any]],
-    private val groupKeysMapping: Array[(Int, Int)],
-    private val aggregateMapping: Array[(Int, Int)],
-    private val intermediateRowArity: Int,
-    private val finalRowArity: Int)
-  extends RichGroupReduceFunction[Row, Row] {
-
-  protected var aggregateBuffer: Row = _
-  private var output: Row = _
-
-  override def open(config: Configuration) {
-    Preconditions.checkNotNull(aggregates)
-    Preconditions.checkNotNull(groupKeysMapping)
-    aggregateBuffer = new Row(intermediateRowArity)
-    output = new Row(finalRowArity)
-  }
-
-  /**
-   * For grouped intermediate aggregate Rows, merge all of them into aggregate buffer,
-   * calculate aggregated values output by aggregate buffer, and set them into output 
-   * Row based on the mapping relation between intermediate aggregate data and output data.
-   *
-   * @param records  Grouped intermediate aggregate Rows iterator.
-   * @param out The collector to hand results to.
-   *
-   */
-  override def reduce(records: Iterable[Row], out: Collector[Row]): Unit = {
-
-    // Initiate intermediate aggregate value.
-    aggregates.foreach(_.initiate(aggregateBuffer))
-
-    // Merge intermediate aggregate value to buffer.
-    var last: Row = null
-    records.foreach((record) => {
-      aggregates.foreach(_.merge(record, aggregateBuffer))
-      last = record
-    })
-
-    // Set group keys value to final output.
-    groupKeysMapping.foreach {
-      case (after, previous) =>
-        output.setField(after, last.getField(previous))
-    }
-
-    // Evaluate final aggregate value and set to output.
-    aggregateMapping.foreach {
-      case (after, previous) =>
-        output.setField(after, aggregates(previous).evaluate(aggregateBuffer))
-    }
-
-    out.collect(output)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTimeWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTimeWindowFunction.scala
deleted file mode 100644
index ff8f6fb..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTimeWindowFunction.scala
+++ /dev/null
@@ -1,57 +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.api.table.runtime.aggregate
-
-import java.lang.Iterable
-
-import org.apache.flink.api.common.functions.RichGroupReduceFunction
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow
-import org.apache.flink.util.Collector
-
-class AggregateTimeWindowFunction(
-    groupReduceFunction: RichGroupReduceFunction[Row, Row],
-    windowStartPos: Option[Int],
-    windowEndPos: Option[Int])
-  extends AggregateWindowFunction[TimeWindow](groupReduceFunction) {
-
-  private var collector: TimeWindowPropertyCollector = _
-
-  override def open(parameters: Configuration): Unit = {
-    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
-    super.open(parameters)
-  }
-
-  override def apply(
-    key: Tuple,
-    window: TimeWindow,
-    input: Iterable[Row],
-    out: Collector[Row]): Unit = {
-
-    // set collector and window
-    collector.wrappedCollector = out
-    collector.timeWindow = window
-
-    // call wrapped reduce function with property collector
-    super.apply(key, window, input, collector)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala
deleted file mode 100644
index a181068..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateUtil.scala
+++ /dev/null
@@ -1,593 +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.api.table.runtime.aggregate
-
-import java.util
-
-import org.apache.calcite.rel.`type`._
-import org.apache.calcite.rel.core.AggregateCall
-import org.apache.calcite.sql.{SqlAggFunction, SqlKind}
-import org.apache.calcite.sql.`type`.SqlTypeName._
-import org.apache.calcite.sql.`type`.{SqlTypeFactoryImpl, SqlTypeName}
-import org.apache.calcite.sql.fun._
-import org.apache.flink.api.common.functions.{MapFunction, RichGroupReduceFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.api.table.FlinkRelBuilder.NamedWindowProperty
-import org.apache.flink.api.table.expressions.{WindowEnd, WindowStart}
-import org.apache.flink.api.table.plan.logical._
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.typeutils.TypeCheckUtils._
-import org.apache.flink.api.table.{FlinkTypeFactory,  TableException}
-import org.apache.flink.types.Row
-import org.apache.flink.streaming.api.functions.windowing.{AllWindowFunction, WindowFunction}
-import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow}
-import scala.collection.JavaConversions._
-import scala.collection.mutable.ArrayBuffer
-
-object AggregateUtil {
-
-  type CalcitePair[T, R] = org.apache.calcite.util.Pair[T, R]
-  type JavaList[T] = java.util.List[T]
-
-  /**
-    * Create a [[org.apache.flink.api.common.functions.MapFunction]] that prepares for aggregates.
-    * The function returns intermediate aggregate values of all aggregate function which are
-    * organized by the following format:
-    *
-    * {{{
-    *                   avg(x) aggOffsetInRow = 2          count(z) aggOffsetInRow = 5
-    *                             |                          |
-    *                             v                          v
-    *        +---------+---------+--------+--------+--------+--------+
-    *        |groupKey1|groupKey2|  sum1  | count1 |  sum2  | count2 |
-    *        +---------+---------+--------+--------+--------+--------+
-    *                                              ^
-    *                                              |
-    *                               sum(y) aggOffsetInRow = 4
-    * }}}
-    *
-    */
-  private[flink] def createPrepareMapFunction(
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    groupings: Array[Int],
-    inputType: RelDataType): MapFunction[Any, Row] = {
-
-    val (aggFieldIndexes,aggregates) = transformToAggregateFunctions(
-      namedAggregates.map(_.getKey),
-      inputType,
-      groupings.length)
-
-    val mapReturnType: RowTypeInfo =
-      createAggregateBufferDataType(groupings, aggregates, inputType)
-
-    val mapFunction = new AggregateMapFunction[Row, Row](
-      aggregates,
-      aggFieldIndexes,
-      groupings,
-      mapReturnType.asInstanceOf[RowTypeInfo]).asInstanceOf[MapFunction[Any, Row]]
-
-    mapFunction
-  }
-
-  /**
-    * Create a [[org.apache.flink.api.common.functions.GroupReduceFunction]] to compute aggregates.
-    * If all aggregates support partial aggregation, the
-    * [[org.apache.flink.api.common.functions.GroupReduceFunction]] implements
-    * [[org.apache.flink.api.common.functions.CombineFunction]] as well.
-    *
-    */
-  private[flink] def createAggregateGroupReduceFunction(
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    inputType: RelDataType,
-    outputType: RelDataType,
-    groupings: Array[Int]): RichGroupReduceFunction[Row, Row] = {
-
-    val aggregates = transformToAggregateFunctions(
-      namedAggregates.map(_.getKey),
-      inputType,
-      groupings.length)._2
-
-    val (groupingOffsetMapping, aggOffsetMapping) =
-      getGroupingOffsetAndAggOffsetMapping(
-        namedAggregates,
-        inputType,
-        outputType,
-        groupings)
-
-    val allPartialAggregate: Boolean = aggregates.forall(_.supportPartial)
-
-    val intermediateRowArity = groupings.length +
-      aggregates.map(_.intermediateDataType.length).sum
-
-    val groupReduceFunction =
-      if (allPartialAggregate) {
-        new AggregateReduceCombineFunction(
-          aggregates,
-          groupingOffsetMapping,
-          aggOffsetMapping,
-          intermediateRowArity,
-          outputType.getFieldCount)
-      }
-      else {
-        new AggregateReduceGroupFunction(
-          aggregates,
-          groupingOffsetMapping,
-          aggOffsetMapping,
-          intermediateRowArity,
-          outputType.getFieldCount)
-      }
-    groupReduceFunction
-  }
-
-  /**
-    * Create a [[org.apache.flink.api.common.functions.ReduceFunction]] for incremental window
-    * aggregation.
-    *
-    */
-  private[flink] def createIncrementalAggregateReduceFunction(
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    inputType: RelDataType,
-    outputType: RelDataType,
-    groupings: Array[Int]): IncrementalAggregateReduceFunction = {
-
-    val aggregates = transformToAggregateFunctions(
-      namedAggregates.map(_.getKey),inputType,groupings.length)._2
-
-    val groupingOffsetMapping =
-      getGroupingOffsetAndAggOffsetMapping(
-        namedAggregates,
-        inputType,
-        outputType,
-        groupings)._1
-
-    val intermediateRowArity = groupings.length + aggregates.map(_.intermediateDataType.length).sum
-    val reduceFunction = new IncrementalAggregateReduceFunction(
-      aggregates,
-      groupingOffsetMapping,
-      intermediateRowArity)
-    reduceFunction
-  }
-
-  /**
-    * Create an [[AllWindowFunction]] to compute non-partitioned group window aggregates.
-    */
-  private[flink] def createAllWindowAggregationFunction(
-    window: LogicalWindow,
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    inputType: RelDataType,
-    outputType: RelDataType,
-    groupings: Array[Int],
-    properties: Seq[NamedWindowProperty])
-  : AllWindowFunction[Row, Row, DataStreamWindow] = {
-
-    val aggFunction =
-      createAggregateGroupReduceFunction(
-        namedAggregates,
-        inputType,
-        outputType,
-        groupings)
-
-    if (isTimeWindow(window)) {
-      val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
-      new AggregateAllTimeWindowFunction(aggFunction, startPos, endPos)
-      .asInstanceOf[AllWindowFunction[Row, Row, DataStreamWindow]]
-    } else {
-      new AggregateAllWindowFunction(aggFunction)
-    }
-  }
-
-  /**
-    * Create a [[WindowFunction]] to compute partitioned group window aggregates.
-    *
-    */
-  private[flink] def createWindowAggregationFunction(
-    window: LogicalWindow,
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    inputType: RelDataType,
-    outputType: RelDataType,
-    groupings: Array[Int],
-    properties: Seq[NamedWindowProperty])
-  : WindowFunction[Row, Row, Tuple, DataStreamWindow] = {
-
-    val aggFunction =
-      createAggregateGroupReduceFunction(
-        namedAggregates,
-        inputType,
-        outputType,
-        groupings)
-
-    if (isTimeWindow(window)) {
-      val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
-      new AggregateTimeWindowFunction(aggFunction, startPos, endPos)
-      .asInstanceOf[WindowFunction[Row, Row, Tuple, DataStreamWindow]]
-    } else {
-      new AggregateWindowFunction(aggFunction)
-    }
-  }
-
-  /**
-    * Create an [[AllWindowFunction]] to finalize incrementally pre-computed non-partitioned
-    * window aggreagtes.
-    */
-  private[flink] def createAllWindowIncrementalAggregationFunction(
-    window: LogicalWindow,
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    inputType: RelDataType,
-    outputType: RelDataType,
-    groupings: Array[Int],
-    properties: Seq[NamedWindowProperty]): AllWindowFunction[Row, Row, DataStreamWindow] = {
-
-    val aggregates = transformToAggregateFunctions(
-      namedAggregates.map(_.getKey),inputType,groupings.length)._2
-
-    val (groupingOffsetMapping, aggOffsetMapping) =
-      getGroupingOffsetAndAggOffsetMapping(
-      namedAggregates,
-      inputType,
-      outputType,
-      groupings)
-
-    val finalRowArity = outputType.getFieldCount
-
-    if (isTimeWindow(window)) {
-      val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
-      new IncrementalAggregateAllTimeWindowFunction(
-        aggregates,
-        groupingOffsetMapping,
-        aggOffsetMapping,
-        finalRowArity,
-        startPos,
-        endPos)
-      .asInstanceOf[AllWindowFunction[Row, Row, DataStreamWindow]]
-    } else {
-      new IncrementalAggregateAllWindowFunction(
-        aggregates,
-        groupingOffsetMapping,
-        aggOffsetMapping,
-        finalRowArity)
-    }
-  }
-
-  /**
-    * Create a [[WindowFunction]] to finalize incrementally pre-computed window aggregates.
-    */
-  private[flink] def createWindowIncrementalAggregationFunction(
-    window: LogicalWindow,
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    inputType: RelDataType,
-    outputType: RelDataType,
-    groupings: Array[Int],
-    properties: Seq[NamedWindowProperty]): WindowFunction[Row, Row, Tuple, DataStreamWindow] = {
-
-    val aggregates = transformToAggregateFunctions(
-      namedAggregates.map(_.getKey),inputType,groupings.length)._2
-
-    val (groupingOffsetMapping, aggOffsetMapping) =
-      getGroupingOffsetAndAggOffsetMapping(
-        namedAggregates,
-        inputType,
-        outputType,
-        groupings)
-
-    val finalRowArity = outputType.getFieldCount
-
-    if (isTimeWindow(window)) {
-      val (startPos, endPos) = computeWindowStartEndPropertyPos(properties)
-      new IncrementalAggregateTimeWindowFunction(
-        aggregates,
-        groupingOffsetMapping,
-        aggOffsetMapping,
-        finalRowArity,
-        startPos,
-        endPos)
-      .asInstanceOf[WindowFunction[Row, Row, Tuple, DataStreamWindow]]
-    } else {
-      new IncrementalAggregateWindowFunction(
-        aggregates,
-        groupingOffsetMapping,
-        aggOffsetMapping,
-        finalRowArity)
-    }
-  }
-
-  /**
-    * Return true if all aggregates can be partially computed. False otherwise.
-    */
-  private[flink] def doAllSupportPartialAggregation(
-    aggregateCalls: Seq[AggregateCall],
-    inputType: RelDataType,
-    groupKeysCount: Int): Boolean = {
-    transformToAggregateFunctions(
-      aggregateCalls,
-      inputType,
-      groupKeysCount)._2.forall(_.supportPartial)
-  }
-
-  /**
-    * @return groupingOffsetMapping (mapping relation between field index of intermediate
-    *         aggregate Row and output Row.)
-    *         and aggOffsetMapping (the mapping relation between aggregate function index in list
-    *         and its corresponding field index in output Row.)
-    */
-  private def getGroupingOffsetAndAggOffsetMapping(
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    inputType: RelDataType,
-    outputType: RelDataType,
-    groupings: Array[Int]): (Array[(Int, Int)], Array[(Int, Int)]) = {
-
-    // the mapping relation between field index of intermediate aggregate Row and output Row.
-    val groupingOffsetMapping = getGroupKeysMapping(inputType, outputType, groupings)
-
-    // the mapping relation between aggregate function index in list and its corresponding
-    // field index in output Row.
-    val aggOffsetMapping = getAggregateMapping(namedAggregates, outputType)
-
-    if (groupingOffsetMapping.length != groupings.length ||
-      aggOffsetMapping.length != namedAggregates.length) {
-      throw new TableException(
-        "Could not find output field in input data type " +
-          "or aggregate functions.")
-    }
-    (groupingOffsetMapping, aggOffsetMapping)
-  }
-
-  private def isTimeWindow(window: LogicalWindow) = {
-    window match {
-      case ProcessingTimeTumblingGroupWindow(_, size) => isTimeInterval(size.resultType)
-      case ProcessingTimeSlidingGroupWindow(_, size, _) => isTimeInterval(size.resultType)
-      case ProcessingTimeSessionGroupWindow(_, _) => true
-      case EventTimeTumblingGroupWindow(_, _, size) => isTimeInterval(size.resultType)
-      case EventTimeSlidingGroupWindow(_, _, size, _) => isTimeInterval(size.resultType)
-      case EventTimeSessionGroupWindow(_, _, _) => true
-    }
-  }
-
-  private def computeWindowStartEndPropertyPos(
-    properties: Seq[NamedWindowProperty]): (Option[Int], Option[Int]) = {
-
-    val propPos = properties.foldRight((None: Option[Int], None: Option[Int], 0)) {
-      (p, x) => p match {
-        case NamedWindowProperty(name, prop) =>
-          prop match {
-            case WindowStart(_) if x._1.isDefined =>
-              throw new TableException("Duplicate WindowStart property encountered. This is a bug.")
-            case WindowStart(_) =>
-              (Some(x._3), x._2, x._3 - 1)
-            case WindowEnd(_) if x._2.isDefined =>
-              throw new TableException("Duplicate WindowEnd property encountered. This is a bug.")
-            case WindowEnd(_) =>
-              (x._1, Some(x._3), x._3 - 1)
-          }
-      }
-    }
-    (propPos._1, propPos._2)
-  }
-
-  private def transformToAggregateFunctions(
-    aggregateCalls: Seq[AggregateCall],
-    inputType: RelDataType,
-    groupKeysCount: Int): (Array[Int], Array[Aggregate[_ <: Any]]) = {
-
-    // store the aggregate fields of each aggregate function, by the same order of aggregates.
-    val aggFieldIndexes = new Array[Int](aggregateCalls.size)
-    val aggregates = new Array[Aggregate[_ <: Any]](aggregateCalls.size)
-
-    // set the start offset of aggregate buffer value to group keys' length, 
-    // as all the group keys would be moved to the start fields of intermediate
-    // aggregate data.
-    var aggOffset = groupKeysCount
-
-    // create aggregate function instances by function type and aggregate field data type.
-    aggregateCalls.zipWithIndex.foreach { case (aggregateCall, index) =>
-      val argList: util.List[Integer] = aggregateCall.getArgList
-      if (argList.isEmpty) {
-        if (aggregateCall.getAggregation.isInstanceOf[SqlCountAggFunction]) {
-          aggFieldIndexes(index) = 0
-        } else {
-          throw new TableException("Aggregate fields should not be empty.")
-        }
-      } else {
-        if (argList.size() > 1) {
-          throw new TableException("Currently, do not support aggregate on multi fields.")
-        }
-        aggFieldIndexes(index) = argList.get(0)
-      }
-      val sqlTypeName = inputType.getFieldList.get(aggFieldIndexes(index)).getType.getSqlTypeName
-      aggregateCall.getAggregation match {
-        case _: SqlSumAggFunction | _: SqlSumEmptyIsZeroAggFunction => {
-          aggregates(index) = sqlTypeName match {
-            case TINYINT =>
-              new ByteSumAggregate
-            case SMALLINT =>
-              new ShortSumAggregate
-            case INTEGER =>
-              new IntSumAggregate
-            case BIGINT =>
-              new LongSumAggregate
-            case FLOAT =>
-              new FloatSumAggregate
-            case DOUBLE =>
-              new DoubleSumAggregate
-            case DECIMAL =>
-              new DecimalSumAggregate
-            case sqlType: SqlTypeName =>
-              throw new TableException("Sum aggregate does no support type:" + sqlType)
-          }
-        }
-        case _: SqlAvgAggFunction => {
-          aggregates(index) = sqlTypeName match {
-            case TINYINT =>
-               new ByteAvgAggregate
-            case SMALLINT =>
-              new ShortAvgAggregate
-            case INTEGER =>
-              new IntAvgAggregate
-            case BIGINT =>
-              new LongAvgAggregate
-            case FLOAT =>
-              new FloatAvgAggregate
-            case DOUBLE =>
-              new DoubleAvgAggregate
-            case DECIMAL =>
-              new DecimalAvgAggregate
-            case sqlType: SqlTypeName =>
-              throw new TableException("Avg aggregate does no support type:" + sqlType)
-          }
-        }
-        case sqlMinMaxFunction: SqlMinMaxAggFunction => {
-          aggregates(index) = if (sqlMinMaxFunction.getKind == SqlKind.MIN) {
-            sqlTypeName match {
-              case TINYINT =>
-                new ByteMinAggregate
-              case SMALLINT =>
-                new ShortMinAggregate
-              case INTEGER =>
-                new IntMinAggregate
-              case BIGINT =>
-                new LongMinAggregate
-              case FLOAT =>
-                new FloatMinAggregate
-              case DOUBLE =>
-                new DoubleMinAggregate
-              case DECIMAL =>
-                new DecimalMinAggregate
-              case BOOLEAN =>
-                new BooleanMinAggregate
-              case sqlType: SqlTypeName =>
-                throw new TableException("Min aggregate does no support type:" + sqlType)
-            }
-          } else {
-            sqlTypeName match {
-              case TINYINT =>
-                new ByteMaxAggregate
-              case SMALLINT =>
-                new ShortMaxAggregate
-              case INTEGER =>
-                new IntMaxAggregate
-              case BIGINT =>
-                new LongMaxAggregate
-              case FLOAT =>
-                new FloatMaxAggregate
-              case DOUBLE =>
-                new DoubleMaxAggregate
-              case DECIMAL =>
-                new DecimalMaxAggregate
-              case BOOLEAN =>
-                new BooleanMaxAggregate
-              case sqlType: SqlTypeName =>
-                throw new TableException("Max aggregate does no support type:" + sqlType)
-            }
-          }
-        }
-        case _: SqlCountAggFunction =>
-          aggregates(index) = new CountAggregate
-        case unSupported: SqlAggFunction =>
-          throw new TableException("unsupported Function: " + unSupported.getName)
-      }
-      setAggregateDataOffset(index)
-    }
-
-    // set the aggregate intermediate data start index in Row, and update current value.
-    def setAggregateDataOffset(index: Int): Unit = {
-      aggregates(index).setAggOffsetInRow(aggOffset)
-      aggOffset += aggregates(index).intermediateDataType.length
-    }
-
-    (aggFieldIndexes, aggregates)
-  }
-
-  private def createAggregateBufferDataType(
-    groupings: Array[Int],
-    aggregates: Array[Aggregate[_]],
-    inputType: RelDataType): RowTypeInfo = {
-
-    // get the field data types of group keys.
-    val groupingTypes: Seq[TypeInformation[_]] = groupings
-      .map(inputType.getFieldList.get(_).getType)
-      .map(FlinkTypeFactory.toTypeInfo)
-
-    val aggPartialNameSuffix = "agg_buffer_"
-    val factory = new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT)
-
-    // get all field data types of all intermediate aggregates
-    val aggTypes: Seq[TypeInformation[_]] = aggregates.flatMap(_.intermediateDataType)
-
-    // concat group key types and aggregation types
-    val allFieldTypes = groupingTypes ++: aggTypes
-    val partialType = new RowTypeInfo(allFieldTypes: _*)
-    partialType
-  }
-
-  // Find the mapping between the index of aggregate list and aggregated value index in output Row.
-  private def getAggregateMapping(
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    outputType: RelDataType): Array[(Int, Int)] = {
-
-    // the mapping relation between aggregate function index in list and its corresponding
-    // field index in output Row.
-    var aggOffsetMapping = ArrayBuffer[(Int, Int)]()
-
-    outputType.getFieldList.zipWithIndex.foreach{
-      case (outputFieldType, outputIndex) =>
-        namedAggregates.zipWithIndex.foreach {
-          case (namedAggCall, aggregateIndex) =>
-            if (namedAggCall.getValue.equals(outputFieldType.getName) &&
-                namedAggCall.getKey.getType.equals(outputFieldType.getType)) {
-              aggOffsetMapping += ((outputIndex, aggregateIndex))
-            }
-        }
-    }
-   
-    aggOffsetMapping.toArray
-  }
-
-  // Find the mapping between the index of group key in intermediate aggregate Row and its index
-  // in output Row.
-  private def getGroupKeysMapping(
-    inputDatType: RelDataType,
-    outputType: RelDataType,
-    groupKeys: Array[Int]): Array[(Int, Int)] = {
-
-    // the mapping relation between field index of intermediate aggregate Row and output Row.
-    var groupingOffsetMapping = ArrayBuffer[(Int, Int)]()
-
-    outputType.getFieldList.zipWithIndex.foreach {
-      case (outputFieldType, outputIndex) =>
-        inputDatType.getFieldList.zipWithIndex.foreach {
-          // find the field index in input data type.
-          case (inputFieldType, inputIndex) =>
-            if (outputFieldType.getName.equals(inputFieldType.getName) &&
-                outputFieldType.getType.equals(inputFieldType.getType)) {
-              // as aggregated field in output data type would not have a matched field in
-              // input data, so if inputIndex is not -1, it must be a group key. Then we can
-              // find the field index in buffer data by the group keys index mapping between
-              // input data and buffer data.
-              for (i <- groupKeys.indices) {
-                if (inputIndex == groupKeys(i)) {
-                  groupingOffsetMapping += ((outputIndex, i))
-                }
-              }
-            }
-        }
-    }
-
-    groupingOffsetMapping.toArray
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateWindowFunction.scala
deleted file mode 100644
index 4e77549..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AggregateWindowFunction.scala
+++ /dev/null
@@ -1,46 +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.api.table.runtime.aggregate
-
-import java.lang.Iterable
-
-import org.apache.flink.api.common.functions.RichGroupReduceFunction
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
-import org.apache.flink.streaming.api.windowing.windows.Window
-import org.apache.flink.util.Collector
-
-class AggregateWindowFunction[W <: Window](groupReduceFunction: RichGroupReduceFunction[Row, Row])
-  extends RichWindowFunction[Row, Row, Tuple, W] {
-
-  override def open(parameters: Configuration): Unit = {
-    groupReduceFunction.open(parameters)
-  }
-
-  override def apply(
-    key: Tuple,
-    window: W,
-    input: Iterable[Row],
-    out: Collector[Row]): Unit = {
-
-    groupReduceFunction.reduce(input, out)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala
deleted file mode 100644
index 998ae62..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregate.scala
+++ /dev/null
@@ -1,296 +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.api.table.runtime.aggregate
-
-import com.google.common.math.LongMath
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.types.Row
-import java.math.BigDecimal
-import java.math.BigInteger
-
-abstract class AvgAggregate[T] extends Aggregate[T] {
-  protected var partialSumIndex: Int = _
-  protected var partialCountIndex: Int = _
-
-  override def supportPartial: Boolean = true
-
-  override def setAggOffsetInRow(aggOffset: Int): Unit = {
-    partialSumIndex = aggOffset
-    partialCountIndex = aggOffset + 1
-  }
-}
-
-abstract class IntegralAvgAggregate[T] extends AvgAggregate[T] {
-
-  override def initiate(partial: Row): Unit = {
-    partial.setField(partialSumIndex, 0L)
-    partial.setField(partialCountIndex, 0L)
-  }
-
-  override def prepare(value: Any, partial: Row): Unit = {
-    if (value == null) {
-      partial.setField(partialSumIndex, 0L)
-      partial.setField(partialCountIndex, 0L)
-    } else {
-      doPrepare(value, partial)
-    }
-  }
-
-  override def merge(partial: Row, buffer: Row): Unit = {
-    val partialSum = partial.getField(partialSumIndex).asInstanceOf[Long]
-    val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long]
-    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long]
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-    buffer.setField(partialSumIndex, LongMath.checkedAdd(partialSum, bufferSum))
-    buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount))
-  }
-
-  override def evaluate(buffer : Row): T = {
-    doEvaluate(buffer).asInstanceOf[T]
-  }
-
-  override def intermediateDataType = Array(
-    BasicTypeInfo.LONG_TYPE_INFO,
-    BasicTypeInfo.LONG_TYPE_INFO)
-
-  def doPrepare(value: Any, partial: Row): Unit
-
-  def doEvaluate(buffer: Row): Any
-}
-
-class ByteAvgAggregate extends IntegralAvgAggregate[Byte] {
-  override def doPrepare(value: Any, partial: Row): Unit = {
-    val input = value.asInstanceOf[Byte]
-    partial.setField(partialSumIndex, input.toLong)
-    partial.setField(partialCountIndex, 1L)
-  }
-
-  override def doEvaluate(buffer: Row): Any = {
-    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long]
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-    if (bufferCount == 0L) {
-      null
-    } else {
-      (bufferSum / bufferCount).toByte
-    }
-  }
-}
-
-class ShortAvgAggregate extends IntegralAvgAggregate[Short] {
-
-  override def doPrepare(value: Any, partial: Row): Unit = {
-    val input = value.asInstanceOf[Short]
-    partial.setField(partialSumIndex, input.toLong)
-    partial.setField(partialCountIndex, 1L)
-  }
-
-  override def doEvaluate(buffer: Row): Any = {
-    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long]
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-    if (bufferCount == 0L) {
-      null
-    } else {
-      (bufferSum / bufferCount).toShort
-    }
-  }
-}
-
-class IntAvgAggregate extends IntegralAvgAggregate[Int] {
-
-  override def doPrepare(value: Any, partial: Row): Unit = {
-    val input = value.asInstanceOf[Int]
-    partial.setField(partialSumIndex, input.toLong)
-    partial.setField(partialCountIndex, 1L)
-  }
-
-  override def doEvaluate(buffer: Row): Any = {
-    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Long]
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-    if (bufferCount == 0L) {
-      null
-    } else {
-      (bufferSum / bufferCount).toInt
-    }
-  }
-}
-
-class LongAvgAggregate extends IntegralAvgAggregate[Long] {
-
-  override def intermediateDataType = Array(
-    BasicTypeInfo.BIG_INT_TYPE_INFO,
-    BasicTypeInfo.LONG_TYPE_INFO)
-
-  override def initiate(partial: Row): Unit = {
-    partial.setField(partialSumIndex, BigInteger.ZERO)
-    partial.setField(partialCountIndex, 0L)
-  }
-
-  override def prepare(value: Any, partial: Row): Unit = {
-    if (value == null) {
-      partial.setField(partialSumIndex, BigInteger.ZERO)
-      partial.setField(partialCountIndex, 0L)
-    } else {
-      doPrepare(value, partial)
-    }
-  }
-
-  override def doPrepare(value: Any, partial: Row): Unit = {
-    val input = value.asInstanceOf[Long]
-    partial.setField(partialSumIndex, BigInteger.valueOf(input))
-    partial.setField(partialCountIndex, 1L)
-  }
-
-  override def merge(partial: Row, buffer: Row): Unit = {
-    val partialSum = partial.getField(partialSumIndex).asInstanceOf[BigInteger]
-    val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long]
-    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigInteger]
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-    buffer.setField(partialSumIndex, partialSum.add(bufferSum))
-    buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount))
-  }
-
-  override def doEvaluate(buffer: Row): Any = {
-    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigInteger]
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-    if (bufferCount == 0L) {
-      null
-    } else {
-      bufferSum.divide(BigInteger.valueOf(bufferCount)).longValue()
-    }
-  }
-}
-
-abstract class FloatingAvgAggregate[T: Numeric] extends AvgAggregate[T] {
-
-  override def initiate(partial: Row): Unit = {
-    partial.setField(partialSumIndex, 0D)
-    partial.setField(partialCountIndex, 0L)
-  }
-
-  override def prepare(value: Any, partial: Row): Unit = {
-    if (value == null) {
-      partial.setField(partialSumIndex, 0D)
-      partial.setField(partialCountIndex, 0L)
-    } else {
-      doPrepare(value, partial)
-    }
-  }
-
-  override def merge(partial: Row, buffer: Row): Unit = {
-    val partialSum = partial.getField(partialSumIndex).asInstanceOf[Double]
-    val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long]
-    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Double]
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-
-    buffer.setField(partialSumIndex, partialSum + bufferSum)
-    buffer.setField(partialCountIndex, partialCount + bufferCount)
-  }
-
-  override def evaluate(buffer : Row): T = {
-    doEvaluate(buffer).asInstanceOf[T]
-  }
-
-  override def intermediateDataType = Array(
-    BasicTypeInfo.DOUBLE_TYPE_INFO,
-    BasicTypeInfo.LONG_TYPE_INFO)
-
-  def doPrepare(value: Any, partial: Row): Unit
-
-  def doEvaluate(buffer: Row): Any
-}
-
-class FloatAvgAggregate extends FloatingAvgAggregate[Float] {
-
-  override def doPrepare(value: Any, partial: Row): Unit = {
-    val input = value.asInstanceOf[Float]
-    partial.setField(partialSumIndex, input.toDouble)
-    partial.setField(partialCountIndex, 1L)
-  }
-
-
-  override def doEvaluate(buffer: Row): Any = {
-    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Double]
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-    if (bufferCount == 0L) {
-      null
-    } else {
-      (bufferSum / bufferCount).toFloat
-    }
-  }
-}
-
-class DoubleAvgAggregate extends FloatingAvgAggregate[Double] {
-
-  override def doPrepare(value: Any, partial: Row): Unit = {
-    val input = value.asInstanceOf[Double]
-    partial.setField(partialSumIndex, input)
-    partial.setField(partialCountIndex, 1L)
-  }
-
-  override def doEvaluate(buffer: Row): Any = {
-    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[Double]
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-    if (bufferCount == 0L) {
-      null
-    } else {
-      (bufferSum / bufferCount)
-    }
-  }
-}
-
-class DecimalAvgAggregate extends AvgAggregate[BigDecimal] {
-
-  override def intermediateDataType = Array(
-    BasicTypeInfo.BIG_DEC_TYPE_INFO,
-    BasicTypeInfo.LONG_TYPE_INFO)
-
-  override def initiate(partial: Row): Unit = {
-    partial.setField(partialSumIndex, BigDecimal.ZERO)
-    partial.setField(partialCountIndex, 0L)
-  }
-
-  override def prepare(value: Any, partial: Row): Unit = {
-    if (value == null) {
-      initiate(partial)
-    } else {
-      val input = value.asInstanceOf[BigDecimal]
-      partial.setField(partialSumIndex, input)
-      partial.setField(partialCountIndex, 1L)
-    }
-  }
-
-  override def merge(partial: Row, buffer: Row): Unit = {
-    val partialSum = partial.getField(partialSumIndex).asInstanceOf[BigDecimal]
-    val partialCount = partial.getField(partialCountIndex).asInstanceOf[Long]
-    val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigDecimal]
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-    buffer.setField(partialSumIndex, partialSum.add(bufferSum))
-    buffer.setField(partialCountIndex, LongMath.checkedAdd(partialCount, bufferCount))
-  }
-
-  override def evaluate(buffer: Row): BigDecimal = {
-    val bufferCount = buffer.getField(partialCountIndex).asInstanceOf[Long]
-    if (bufferCount != 0) {
-      val bufferSum = buffer.getField(partialSumIndex).asInstanceOf[BigDecimal]
-      bufferSum.divide(BigDecimal.valueOf(bufferCount))
-    } else {
-      null.asInstanceOf[BigDecimal]
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala
deleted file mode 100644
index 4d6d20b..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregate.scala
+++ /dev/null
@@ -1,55 +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.api.table.runtime.aggregate
-
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.types.Row
-
-class CountAggregate extends Aggregate[Long] {
-  private var countIndex: Int = _
-
-  override def initiate(intermediate: Row): Unit = {
-    intermediate.setField(countIndex, 0L)
-  }
-
-  override def merge(intermediate: Row, buffer: Row): Unit = {
-    val partialCount = intermediate.getField(countIndex).asInstanceOf[Long]
-    val bufferCount = buffer.getField(countIndex).asInstanceOf[Long]
-    buffer.setField(countIndex, partialCount + bufferCount)
-  }
-
-  override def evaluate(buffer: Row): Long = {
-    buffer.getField(countIndex).asInstanceOf[Long]
-  }
-
-  override def prepare(value: Any, intermediate: Row): Unit = {
-    if (value == null) {
-      intermediate.setField(countIndex, 0L)
-    } else {
-      intermediate.setField(countIndex, 1L)
-    }
-  }
-
-  override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO)
-
-  override def supportPartial: Boolean = true
-
-  override def setAggOffsetInRow(aggIndex: Int): Unit = {
-    countIndex = aggIndex
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala
deleted file mode 100644
index 48e2313..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllTimeWindowFunction.scala
+++ /dev/null
@@ -1,68 +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.api.table.runtime.aggregate
-
-import java.lang.Iterable
-
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.streaming.api.windowing.windows.{TimeWindow, Window}
-import org.apache.flink.util.Collector
-/**
-  *
-  * Computes the final aggregate value from incrementally computed aggreagtes.
-  *
-  * @param aggregates   The aggregate functions.
-  * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
-  *                         and output Row.
-  * @param aggregateMapping The index mapping between aggregate function list and aggregated value
-  *                         index in output Row.
-  * @param finalRowArity  The arity of the final output row.
-  */
-class IncrementalAggregateAllTimeWindowFunction(
-    private val aggregates: Array[Aggregate[_ <: Any]],
-    private val groupKeysMapping: Array[(Int, Int)],
-    private val aggregateMapping: Array[(Int, Int)],
-    private val finalRowArity: Int,
-    private val windowStartPos: Option[Int],
-    private val windowEndPos: Option[Int])
-  extends IncrementalAggregateAllWindowFunction[TimeWindow](
-    aggregates,
-    groupKeysMapping,
-    aggregateMapping,
-    finalRowArity) {
-
-  private var collector: TimeWindowPropertyCollector = _
-
-  override def open(parameters: Configuration): Unit = {
-    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
-    super.open(parameters)
-  }
-
-  override def apply(
-    window: TimeWindow,
-    records: Iterable[Row],
-    out: Collector[Row]): Unit = {
-
-    // set collector and window
-    collector.wrappedCollector = out
-    collector.timeWindow = window
-
-    super.apply(window, records, collector)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala
deleted file mode 100644
index 1a85dca..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateAllWindowFunction.scala
+++ /dev/null
@@ -1,79 +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.api.table.runtime.aggregate
-
-import java.lang.Iterable
-
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction
-import org.apache.flink.streaming.api.windowing.windows.Window
-import org.apache.flink.util.{Collector, Preconditions}
-
-/**
-  * Computes the final aggregate value from incrementally computed aggreagtes.
-  *
-  * @param aggregates   The aggregate functions.
-  * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
-  *                         and output Row.
-  * @param aggregateMapping The index mapping between aggregate function list and aggregated value
-  *                         index in output Row.
-  * @param finalRowArity  The arity of the final output row.
-  */
-class IncrementalAggregateAllWindowFunction[W <: Window](
-    private val aggregates: Array[Aggregate[_ <: Any]],
-    private val groupKeysMapping: Array[(Int, Int)],
-    private val aggregateMapping: Array[(Int, Int)],
-    private val finalRowArity: Int)
-  extends RichAllWindowFunction[Row, Row, W] {
-
-  private var output: Row = _
-
-  override def open(parameters: Configuration): Unit = {
-    Preconditions.checkNotNull(aggregates)
-    Preconditions.checkNotNull(groupKeysMapping)
-    output = new Row(finalRowArity)
-  }
-
-  /**
-    * Calculate aggregated values output by aggregate buffer, and set them into output
-    * Row based on the mapping relation between intermediate aggregate data and output data.
-    */
-  override def apply(
-    window: W,
-    records: Iterable[Row],
-    out: Collector[Row]): Unit = {
-
-    val iterator = records.iterator
-
-    if (iterator.hasNext) {
-      val record = iterator.next()
-      // Set group keys value to final output.
-      groupKeysMapping.foreach {
-        case (after, previous) =>
-          output.setField(after, record.getField(previous))
-      }
-      // Evaluate final aggregate value and set to output.
-      aggregateMapping.foreach {
-        case (after, previous) =>
-          output.setField(after, aggregates(previous).evaluate(record))
-      }
-      out.collect(output)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala
deleted file mode 100644
index 5c36821..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateReduceFunction.scala
+++ /dev/null
@@ -1,63 +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.api.table.runtime.aggregate
-
-import org.apache.flink.api.common.functions.ReduceFunction
-import org.apache.flink.types.Row
-import org.apache.flink.util.Preconditions
-
-/**
-  * Incrementally computes group window aggregates.
-  *
-  * @param aggregates   The aggregate functions.
-  * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
-  *                         and output Row.
-  */
-class IncrementalAggregateReduceFunction(
-    private val aggregates: Array[Aggregate[_]],
-    private val groupKeysMapping: Array[(Int, Int)],
-    private val intermediateRowArity: Int)
-  extends ReduceFunction[Row] {
-
-  Preconditions.checkNotNull(aggregates)
-  Preconditions.checkNotNull(groupKeysMapping)
-
-  /**
-    * For Incremental intermediate aggregate Rows, merge value1 and value2
-    * into aggregate buffer, return aggregate buffer.
-    *
-    * @param value1 The first value to combined.
-    * @param value2 The second value to combined.
-    * @return  accumulatorRow A resulting row that combines two input values.
-    *
-    */
-  override def reduce(value1: Row, value2: Row): Row = {
-
-    // TODO: once FLINK-5105 is solved, we can avoid creating a new row for each invocation
-    //   and directly merge value1 and value2.
-    val accumulatorRow = new Row(intermediateRowArity)
-
-    // copy all fields of value1 into accumulatorRow
-    (0 until intermediateRowArity)
-    .foreach(i => accumulatorRow.setField(i, value1.getField(i)))
-    // merge value2 to accumulatorRow
-    aggregates.foreach(_.merge(value2, accumulatorRow))
-
-    accumulatorRow
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala
deleted file mode 100644
index 2513383..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateTimeWindowFunction.scala
+++ /dev/null
@@ -1,69 +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.api.table.runtime.aggregate
-
-import java.lang.Iterable
-
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow
-import org.apache.flink.util.Collector
-
-/**
-  * Computes the final aggregate value from incrementally computed aggreagtes.
-  *
-  * @param aggregates   The aggregate functions.
-  * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
-  *                         and output Row.
-  * @param aggregateMapping The index mapping between aggregate function list and aggregated value
-  *                         index in output Row.
-  * @param finalRowArity  The arity of the final output row.
-  */
-class IncrementalAggregateTimeWindowFunction(
-    private val aggregates: Array[Aggregate[_ <: Any]],
-    private val groupKeysMapping: Array[(Int, Int)],
-    private val aggregateMapping: Array[(Int, Int)],
-    private val finalRowArity: Int,
-    private val windowStartPos: Option[Int],
-    private val windowEndPos: Option[Int])
-  extends IncrementalAggregateWindowFunction[TimeWindow](
-    aggregates,
-    groupKeysMapping,
-    aggregateMapping, finalRowArity) {
-
-  private var collector: TimeWindowPropertyCollector = _
-
-  override def open(parameters: Configuration): Unit = {
-    collector = new TimeWindowPropertyCollector(windowStartPos, windowEndPos)
-    super.open(parameters)
-  }
-
-  override def apply(
-    key: Tuple,
-    window: TimeWindow,
-    records: Iterable[Row],
-    out: Collector[Row]): Unit = {
-
-    // set collector and window
-    collector.wrappedCollector = out
-    collector.timeWindow = window
-
-    super.apply(key, window, records, collector)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala
deleted file mode 100644
index d0d71ee..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/IncrementalAggregateWindowFunction.scala
+++ /dev/null
@@ -1,81 +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.api.table.runtime.aggregate
-
-import java.lang.Iterable
-
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.types.Row
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction
-import org.apache.flink.streaming.api.windowing.windows.Window
-import org.apache.flink.util.{Collector, Preconditions}
-
-/**
-  * Computes the final aggregate value from incrementally computed aggreagtes.
-  *
-  * @param aggregates   The aggregate functions.
-  * @param groupKeysMapping The index mapping of group keys between intermediate aggregate Row
-  *                         and output Row.
-  * @param aggregateMapping The index mapping between aggregate function list and aggregated value
-  *                         index in output Row.
-  * @param finalRowArity  The arity of the final output row.
-  */
-class IncrementalAggregateWindowFunction[W <: Window](
-    private val aggregates: Array[Aggregate[_ <: Any]],
-    private val groupKeysMapping: Array[(Int, Int)],
-    private val aggregateMapping: Array[(Int, Int)],
-    private val finalRowArity: Int)
-  extends RichWindowFunction[Row, Row, Tuple, W] {
-
-  private var output: Row = _
-
-  override def open(parameters: Configuration): Unit = {
-    Preconditions.checkNotNull(aggregates)
-    Preconditions.checkNotNull(groupKeysMapping)
-    output = new Row(finalRowArity)
-  }
-
-  /**
-    * Calculate aggregated values output by aggregate buffer, and set them into output
-    * Row based on the mapping relation between intermediate aggregate data and output data.
-    */
-  override def apply(
-    key: Tuple,
-    window: W,
-    records: Iterable[Row],
-    out: Collector[Row]): Unit = {
-
-    val iterator = records.iterator
-
-    if (iterator.hasNext) {
-      val record = iterator.next()
-      // Set group keys value to final output.
-      groupKeysMapping.foreach {
-        case (after, previous) =>
-          output.setField(after, record.getField(previous))
-      }
-      // Evaluate final aggregate value and set to output.
-      aggregateMapping.foreach {
-        case (after, previous) =>
-          output.setField(after, aggregates(previous).evaluate(record))
-      }
-      out.collect(output)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala
deleted file mode 100644
index 2cb3dc7..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregate.scala
+++ /dev/null
@@ -1,171 +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.api.table.runtime.aggregate
-
-import java.math.BigDecimal
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.types.Row
-
-abstract class MaxAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] {
-
-  protected var maxIndex = -1
-
-  /**
-   * Initiate the intermediate aggregate value in Row.
-   *
-   * @param intermediate The intermediate aggregate row to initiate.
-   */
-  override def initiate(intermediate: Row): Unit = {
-    intermediate.setField(maxIndex, null)
-  }
-
-  /**
-   * Accessed in MapFunction, prepare the input of partial aggregate.
-   *
-   * @param value
-   * @param intermediate
-   */
-  override def prepare(value: Any, intermediate: Row): Unit = {
-    if (value == null) {
-      initiate(intermediate)
-    } else {
-      intermediate.setField(maxIndex, value)
-    }
-  }
-
-  /**
-   * Accessed in CombineFunction and GroupReduceFunction, merge partial
-   * aggregate result into aggregate buffer.
-   *
-   * @param intermediate
-   * @param buffer
-   */
-  override def merge(intermediate: Row, buffer: Row): Unit = {
-    val partialValue = intermediate.getField(maxIndex).asInstanceOf[T]
-    if (partialValue != null) {
-      val bufferValue = buffer.getField(maxIndex).asInstanceOf[T]
-      if (bufferValue != null) {
-        val max : T = if (ord.compare(partialValue, bufferValue) > 0) partialValue else bufferValue
-        buffer.setField(maxIndex, max)
-      } else {
-        buffer.setField(maxIndex, partialValue)
-      }
-    }
-  }
-
-  /**
-   * Return the final aggregated result based on aggregate buffer.
-   *
-   * @param buffer
-   * @return
-   */
-  override def evaluate(buffer: Row): T = {
-    buffer.getField(maxIndex).asInstanceOf[T]
-  }
-
-  override def supportPartial: Boolean = true
-
-  override def setAggOffsetInRow(aggOffset: Int): Unit = {
-    maxIndex = aggOffset
-  }
-}
-
-class ByteMaxAggregate extends MaxAggregate[Byte] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.BYTE_TYPE_INFO)
-
-}
-
-class ShortMaxAggregate extends MaxAggregate[Short] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.SHORT_TYPE_INFO)
-
-}
-
-class IntMaxAggregate extends MaxAggregate[Int] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.INT_TYPE_INFO)
-
-}
-
-class LongMaxAggregate extends MaxAggregate[Long] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO)
-
-}
-
-class FloatMaxAggregate extends MaxAggregate[Float] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.FLOAT_TYPE_INFO)
-
-}
-
-class DoubleMaxAggregate extends MaxAggregate[Double] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.DOUBLE_TYPE_INFO)
-
-}
-
-class BooleanMaxAggregate extends MaxAggregate[Boolean] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.BOOLEAN_TYPE_INFO)
-
-}
-
-class DecimalMaxAggregate extends Aggregate[BigDecimal] {
-
-  protected var minIndex: Int = _
-
-  override def intermediateDataType = Array(BasicTypeInfo.BIG_DEC_TYPE_INFO)
-
-  override def initiate(intermediate: Row): Unit = {
-    intermediate.setField(minIndex, null)
-  }
-
-  override def prepare(value: Any, partial: Row): Unit = {
-    if (value == null) {
-      initiate(partial)
-    } else {
-      partial.setField(minIndex, value)
-    }
-  }
-
-  override def merge(partial: Row, buffer: Row): Unit = {
-    val partialValue = partial.getField(minIndex).asInstanceOf[BigDecimal]
-    if (partialValue != null) {
-      val bufferValue = buffer.getField(minIndex).asInstanceOf[BigDecimal]
-      if (bufferValue != null) {
-        val min = if (partialValue.compareTo(bufferValue) > 0) partialValue else bufferValue
-        buffer.setField(minIndex, min)
-      } else {
-        buffer.setField(minIndex, partialValue)
-      }
-    }
-  }
-
-  override def evaluate(buffer: Row): BigDecimal = {
-    buffer.getField(minIndex).asInstanceOf[BigDecimal]
-  }
-
-  override def supportPartial: Boolean = true
-
-  override def setAggOffsetInRow(aggOffset: Int): Unit = {
-    minIndex = aggOffset
-  }
-}


[32/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSortRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSortRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSortRule.scala
deleted file mode 100644
index 5c1fb53..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSortRule.scala
+++ /dev/null
@@ -1,54 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.{LogicalJoin, LogicalSort}
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetSort}
-
-class DataSetSortRule
-  extends ConverterRule(
-    classOf[LogicalSort],
-    Convention.NONE,
-    DataSetConvention.INSTANCE,
-    "DataSetSortRule") {
-
-  override def convert(rel: RelNode): RelNode = {
-
-    val sort: LogicalSort = rel.asInstanceOf[LogicalSort]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-    val convInput: RelNode = RelOptRule.convert(sort.getInput, DataSetConvention.INSTANCE)
-
-    new DataSetSort(
-      rel.getCluster,
-      traitSet,
-      convInput,
-      sort.getCollation,
-      rel.getRowType,
-      sort.offset,
-      sort.fetch
-    )
-  }
-}
-
-object DataSetSortRule {
-  val INSTANCE: RelOptRule = new DataSetSortRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetUnionRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetUnionRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetUnionRule.scala
deleted file mode 100644
index ea35637..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetUnionRule.scala
+++ /dev/null
@@ -1,64 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{RelOptRuleCall, Convention, RelOptRule, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.LogicalUnion
-import org.apache.calcite.rel.rules.UnionToDistinctRule
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetUnion}
-
-class DataSetUnionRule
-  extends ConverterRule(
-      classOf[LogicalUnion],
-      Convention.NONE,
-      DataSetConvention.INSTANCE,
-      "DataSetUnionRule")
-  {
-
-  /**
-   * Only translate UNION ALL.
-   * Note: A distinct Union are translated into
-   * an Aggregate on top of a UNION ALL by [[UnionToDistinctRule]]
-   */
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val union: LogicalUnion = call.rel(0).asInstanceOf[LogicalUnion]
-    union.all
-  }
-
-  def convert(rel: RelNode): RelNode = {
-
-    val union: LogicalUnion = rel.asInstanceOf[LogicalUnion]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-    val convLeft: RelNode = RelOptRule.convert(union.getInput(0), DataSetConvention.INSTANCE)
-    val convRight: RelNode = RelOptRule.convert(union.getInput(1), DataSetConvention.INSTANCE)
-
-    new DataSetUnion(
-      rel.getCluster,
-      traitSet,
-      convLeft,
-      convRight,
-      rel.getRowType)
-  }
-}
-
-object DataSetUnionRule {
-  val INSTANCE: RelOptRule = new DataSetUnionRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetValuesRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetValuesRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetValuesRule.scala
deleted file mode 100644
index 3d6c0de..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetValuesRule.scala
+++ /dev/null
@@ -1,51 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{RelOptRule, RelTraitSet, Convention}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.LogicalValues
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetValues, DataSetConvention}
-
-class DataSetValuesRule
-  extends ConverterRule(
-    classOf[LogicalValues],
-    Convention.NONE,
-    DataSetConvention.INSTANCE,
-    "DataSetValuesRule")
-{
-
-  def convert(rel: RelNode): RelNode = {
-
-    val values: LogicalValues = rel.asInstanceOf[LogicalValues]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-
-    new DataSetValues(
-      rel.getCluster,
-      traitSet,
-      rel.getRowType,
-      values.getTuples,
-      description)
-  }
-}
-
-object DataSetValuesRule {
-  val INSTANCE: RelOptRule = new DataSetValuesRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala
deleted file mode 100644
index 301a45b..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala
+++ /dev/null
@@ -1,84 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
-import org.apache.calcite.plan.RelOptRule.{none, operand}
-import org.apache.flink.api.table.plan.nodes.dataset.{BatchTableSourceScan, DataSetCalc}
-import org.apache.flink.api.table.plan.rules.util.RexProgramProjectExtractor._
-import org.apache.flink.api.table.sources.{BatchTableSource, ProjectableTableSource}
-
-/**
-  * This rule tries to push projections into a BatchTableSourceScan.
-  */
-class PushProjectIntoBatchTableSourceScanRule extends RelOptRule(
-  operand(classOf[DataSetCalc],
-          operand(classOf[BatchTableSourceScan], none)),
-  "PushProjectIntoBatchTableSourceScanRule") {
-
-  override def matches(call: RelOptRuleCall) = {
-    val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
-    scan.tableSource match {
-      case _: ProjectableTableSource[_] => true
-      case _ => false
-    }
-  }
-
-  override def onMatch(call: RelOptRuleCall) {
-    val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
-    val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
-
-    val usedFields: Array[Int] = extractRefInputFields(calc.calcProgram)
-
-    // if no fields can be projected, there is no need to transform subtree
-    if (scan.tableSource.getNumberOfFields != usedFields.length) {
-      val originTableSource = scan.tableSource.asInstanceOf[ProjectableTableSource[_]]
-      val newTableSource = originTableSource.projectFields(usedFields)
-      val newScan = new BatchTableSourceScan(
-        scan.getCluster,
-        scan.getTraitSet,
-        scan.getTable,
-        newTableSource.asInstanceOf[BatchTableSource[_]])
-
-      val newCalcProgram = rewriteRexProgram(
-        calc.calcProgram,
-        newScan.getRowType,
-        usedFields,
-        calc.getCluster.getRexBuilder)
-
-      // if project merely returns its input and doesn't exist filter, remove datasetCalc nodes
-      if (newCalcProgram.isTrivial) {
-        call.transformTo(newScan)
-      } else {
-        val newCalc = new DataSetCalc(
-          calc.getCluster,
-          calc.getTraitSet,
-          newScan,
-          calc.getRowType,
-          newCalcProgram,
-          description)
-        call.transformTo(newCalc)
-      }
-    }
-  }
-}
-
-object PushProjectIntoBatchTableSourceScanRule {
-  val INSTANCE: RelOptRule = new PushProjectIntoBatchTableSourceScanRule
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamCalcRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamCalcRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamCalcRule.scala
deleted file mode 100644
index b62967a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamCalcRule.scala
+++ /dev/null
@@ -1,53 +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.api.table.plan.rules.datastream
-
-import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.LogicalCalc
-import org.apache.flink.api.table.plan.nodes.datastream.DataStreamCalc
-import org.apache.flink.api.table.plan.nodes.datastream.DataStreamConvention
-
-class DataStreamCalcRule
-  extends ConverterRule(
-    classOf[LogicalCalc],
-    Convention.NONE,
-    DataStreamConvention.INSTANCE,
-    "DataStreamCalcRule")
-{
-
-  def convert(rel: RelNode): RelNode = {
-    val calc: LogicalCalc = rel.asInstanceOf[LogicalCalc]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
-    val convInput: RelNode = RelOptRule.convert(calc.getInput, DataStreamConvention.INSTANCE)
-
-    new DataStreamCalc(
-      rel.getCluster,
-      traitSet,
-      convInput,
-      rel.getRowType,
-      calc.getProgram,
-      description)
-  }
-}
-
-object DataStreamCalcRule {
-  val INSTANCE: RelOptRule = new DataStreamCalcRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamCorrelateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamCorrelateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamCorrelateRule.scala
deleted file mode 100644
index 554c6c1..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamCorrelateRule.scala
+++ /dev/null
@@ -1,91 +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.api.table.plan.rules.datastream
-
-import org.apache.calcite.plan.volcano.RelSubset
-import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.{LogicalFilter, LogicalCorrelate, LogicalTableFunctionScan}
-import org.apache.calcite.rex.RexNode
-import org.apache.flink.api.table.plan.nodes.datastream.{DataStreamCorrelate, DataStreamConvention}
-
-/**
-  * Rule to convert a LogicalCorrelate into a DataStreamCorrelate.
-  */
-class DataStreamCorrelateRule
-  extends ConverterRule(
-    classOf[LogicalCorrelate],
-    Convention.NONE,
-    DataStreamConvention.INSTANCE,
-    "DataStreamCorrelateRule") {
-
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val join: LogicalCorrelate = call.rel(0).asInstanceOf[LogicalCorrelate]
-    val right = join.getRight.asInstanceOf[RelSubset].getOriginal
-
-    right match {
-      // right node is a table function
-      case scan: LogicalTableFunctionScan => true
-      // a filter is pushed above the table function
-      case filter: LogicalFilter =>
-        filter
-          .getInput.asInstanceOf[RelSubset]
-          .getOriginal
-          .isInstanceOf[LogicalTableFunctionScan]
-      case _ => false
-    }
-  }
-
-  override def convert(rel: RelNode): RelNode = {
-    val join: LogicalCorrelate = rel.asInstanceOf[LogicalCorrelate]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
-    val convInput: RelNode = RelOptRule.convert(join.getInput(0), DataStreamConvention.INSTANCE)
-    val right: RelNode = join.getInput(1)
-
-    def convertToCorrelate(relNode: RelNode, condition: Option[RexNode]): DataStreamCorrelate = {
-      relNode match {
-        case rel: RelSubset =>
-          convertToCorrelate(rel.getRelList.get(0), condition)
-
-        case filter: LogicalFilter =>
-          convertToCorrelate(
-            filter.getInput.asInstanceOf[RelSubset].getOriginal,
-            Some(filter.getCondition))
-
-        case scan: LogicalTableFunctionScan =>
-          new DataStreamCorrelate(
-            rel.getCluster,
-            traitSet,
-            convInput,
-            scan,
-            condition,
-            rel.getRowType,
-            join.getRowType,
-            join.getJoinType,
-            description)
-      }
-    }
-    convertToCorrelate(right, None)
-  }
-
-}
-
-object DataStreamCorrelateRule {
-  val INSTANCE: RelOptRule = new DataStreamCorrelateRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamScanRule.scala
deleted file mode 100644
index 62638bc..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamScanRule.scala
+++ /dev/null
@@ -1,64 +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.api.table.plan.rules.datastream
-
-import org.apache.calcite.plan.{RelOptRuleCall, Convention, RelOptRule, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.core.TableScan
-import org.apache.calcite.rel.logical.LogicalTableScan
-import org.apache.flink.api.table.plan.nodes.datastream.DataStreamConvention
-import org.apache.flink.api.table.plan.nodes.datastream.DataStreamScan
-import org.apache.flink.api.table.plan.schema.DataStreamTable
-
-class DataStreamScanRule
-  extends ConverterRule(
-    classOf[LogicalTableScan],
-    Convention.NONE,
-    DataStreamConvention.INSTANCE,
-    "DataStreamScanRule")
-{
-
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val scan: TableScan = call.rel(0).asInstanceOf[TableScan]
-    val dataSetTable = scan.getTable.unwrap(classOf[DataStreamTable[Any]])
-    dataSetTable match {
-      case _: DataStreamTable[Any] =>
-        true
-      case _ =>
-        false
-    }
-  }
-
-  def convert(rel: RelNode): RelNode = {
-    val scan: LogicalTableScan = rel.asInstanceOf[LogicalTableScan]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
-
-    new DataStreamScan(
-      rel.getCluster,
-      traitSet,
-      scan.getTable,
-      rel.getRowType
-    )
-  }
-}
-
-object DataStreamScanRule {
-  val INSTANCE: RelOptRule = new DataStreamScanRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamUnionRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamUnionRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamUnionRule.scala
deleted file mode 100644
index 78a5486..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamUnionRule.scala
+++ /dev/null
@@ -1,53 +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.api.table.plan.rules.datastream
-
-import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.LogicalUnion
-import org.apache.flink.api.table.plan.nodes.datastream.DataStreamConvention
-import org.apache.flink.api.table.plan.nodes.datastream.DataStreamUnion
-
-class DataStreamUnionRule
-  extends ConverterRule(
-    classOf[LogicalUnion],
-    Convention.NONE,
-    DataStreamConvention.INSTANCE,
-    "DataStreamUnionRule")
-{
-
-  def convert(rel: RelNode): RelNode = {
-    val union: LogicalUnion = rel.asInstanceOf[LogicalUnion]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
-    val convLeft: RelNode = RelOptRule.convert(union.getInput(0), DataStreamConvention.INSTANCE)
-    val convRight: RelNode = RelOptRule.convert(union.getInput(1), DataStreamConvention.INSTANCE)
-
-    new DataStreamUnion(
-      rel.getCluster,
-      traitSet,
-      convLeft,
-      convRight,
-      rel.getRowType)
-  }
-}
-
-object DataStreamUnionRule {
-  val INSTANCE: RelOptRule = new DataStreamUnionRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamValuesRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamValuesRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamValuesRule.scala
deleted file mode 100644
index 738642d..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/DataStreamValuesRule.scala
+++ /dev/null
@@ -1,51 +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.api.table.plan.rules.datastream
-
-import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.LogicalValues
-import org.apache.flink.api.table.plan.nodes.datastream.{DataStreamValues, DataStreamConvention}
-
-class DataStreamValuesRule
-  extends ConverterRule(
-    classOf[LogicalValues],
-    Convention.NONE,
-    DataStreamConvention.INSTANCE,
-    "DataStreamValuesRule")
-{
-
-  def convert(rel: RelNode): RelNode = {
-
-    val values: LogicalValues = rel.asInstanceOf[LogicalValues]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
-
-    new DataStreamValues(
-      rel.getCluster,
-      traitSet,
-      rel.getRowType,
-      values.getTuples,
-      description)
-  }
-}
-
-object DataStreamValuesRule {
-  val INSTANCE: RelOptRule = new DataStreamValuesRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala
deleted file mode 100644
index 91dd255..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/datastream/StreamTableSourceScanRule.scala
+++ /dev/null
@@ -1,76 +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.api.table.plan.rules.datastream
-
-import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.core.TableScan
-import org.apache.calcite.rel.logical.LogicalTableScan
-import org.apache.flink.api.table.plan.nodes.datastream.
-  {StreamTableSourceScan, DataStreamConvention}
-import org.apache.flink.api.table.plan.schema.TableSourceTable
-import org.apache.flink.api.table.sources.StreamTableSource
-
-/** Rule to convert a [[LogicalTableScan]] into a [[StreamTableSourceScan]]. */
-class StreamTableSourceScanRule
-  extends ConverterRule(
-    classOf[LogicalTableScan],
-    Convention.NONE,
-    DataStreamConvention.INSTANCE,
-    "StreamTableSourceScanRule")
-{
-
-  /** Rule must only match if TableScan targets a [[StreamTableSource]] */
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val scan: TableScan = call.rel(0).asInstanceOf[TableScan]
-    val dataSetTable = scan.getTable.unwrap(classOf[TableSourceTable])
-    dataSetTable match {
-      case tst: TableSourceTable =>
-        tst.tableSource match {
-          case _: StreamTableSource[_] =>
-            true
-          case _ =>
-            false
-        }
-      case _ =>
-        false
-    }
-  }
-
-  def convert(rel: RelNode): RelNode = {
-    val scan: LogicalTableScan = rel.asInstanceOf[LogicalTableScan]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
-
-    // The original registered table source
-    val table: TableSourceTable = scan.getTable.unwrap(classOf[TableSourceTable])
-    val tableSource: StreamTableSource[_] = table.tableSource.asInstanceOf[StreamTableSource[_]]
-
-    new StreamTableSourceScan(
-      rel.getCluster,
-      traitSet,
-      scan.getTable,
-      tableSource
-    )
-  }
-}
-
-object StreamTableSourceScanRule {
-  val INSTANCE: RelOptRule = new StreamTableSourceScanRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/util/RexProgramProjectExtractor.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/util/RexProgramProjectExtractor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/util/RexProgramProjectExtractor.scala
deleted file mode 100644
index d78e07f..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/util/RexProgramProjectExtractor.scala
+++ /dev/null
@@ -1,120 +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.api.table.plan.rules.util
-
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rex._
-
-import scala.collection.JavaConversions._
-import scala.collection.mutable
-import scala.collection.JavaConverters._
-
-object RexProgramProjectExtractor {
-
-  /**
-    * Extracts the indexes of input fields accessed by the RexProgram.
-    *
-    * @param rexProgram The RexProgram to analyze
-    * @return The indexes of accessed input fields
-    */
-  def extractRefInputFields(rexProgram: RexProgram): Array[Int] = {
-    val visitor = new RefFieldsVisitor
-    // extract input fields from project expressions
-    rexProgram.getProjectList.foreach(exp => rexProgram.expandLocalRef(exp).accept(visitor))
-    val condition = rexProgram.getCondition
-    // extract input fields from condition expression
-    if (condition != null) {
-      rexProgram.expandLocalRef(condition).accept(visitor)
-    }
-    visitor.getFields
-  }
-
-  /**
-    * Generates a new RexProgram based on mapped input fields.
-    *
-    * @param rexProgram      original RexProgram
-    * @param inputRowType    input row type
-    * @param usedInputFields indexes of used input fields
-    * @param rexBuilder      builder for Rex expressions
-    *
-    * @return A RexProgram with mapped input field expressions.
-    */
-  def rewriteRexProgram(
-      rexProgram: RexProgram,
-      inputRowType: RelDataType,
-      usedInputFields: Array[Int],
-      rexBuilder: RexBuilder): RexProgram = {
-
-    val inputRewriter = new InputRewriter(usedInputFields)
-    val newProjectExpressions = rexProgram.getProjectList.map(
-      exp => rexProgram.expandLocalRef(exp).accept(inputRewriter)
-    ).toList.asJava
-
-    val oldCondition = rexProgram.getCondition
-    val newConditionExpression = {
-      oldCondition match {
-        case ref: RexLocalRef => rexProgram.expandLocalRef(ref).accept(inputRewriter)
-        case _ => null // null does not match any type
-      }
-    }
-    RexProgram.create(
-      inputRowType,
-      newProjectExpressions,
-      newConditionExpression,
-      rexProgram.getOutputRowType,
-      rexBuilder
-    )
-  }
-}
-
-/**
-  * A RexVisitor to extract used input fields
-  */
-class RefFieldsVisitor extends RexVisitorImpl[Unit](true) {
-  private var fields = mutable.LinkedHashSet[Int]()
-
-  def getFields: Array[Int] = fields.toArray
-
-  override def visitInputRef(inputRef: RexInputRef): Unit = fields += inputRef.getIndex
-
-  override def visitCall(call: RexCall): Unit =
-    call.operands.foreach(operand => operand.accept(this))
-}
-
-/**
-  * A RexShuttle to rewrite field accesses of a RexProgram.
-  *
-  * @param fields fields mapping
-  */
-class InputRewriter(fields: Array[Int]) extends RexShuttle {
-
-  /** old input fields ref index -> new input fields ref index mappings */
-  private val fieldMap: Map[Int, Int] =
-    fields.zipWithIndex.toMap
-
-  override def visitInputRef(inputRef: RexInputRef): RexNode =
-    new RexInputRef(relNodeIndex(inputRef), inputRef.getType)
-
-  override def visitLocalRef(localRef: RexLocalRef): RexNode =
-    new RexInputRef(relNodeIndex(localRef), localRef.getType)
-
-  private def relNodeIndex(ref: RexSlot): Int =
-    fieldMap.getOrElse(ref.getIndex,
-      throw new IllegalArgumentException("input field contains invalid index"))
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/ArrayRelDataType.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/ArrayRelDataType.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/ArrayRelDataType.scala
deleted file mode 100644
index 92fcb83..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/ArrayRelDataType.scala
+++ /dev/null
@@ -1,53 +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.api.table.plan.schema
-
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.sql.`type`.ArraySqlType
-import org.apache.flink.api.common.typeinfo.TypeInformation
-
-/**
-  * Flink distinguishes between primitive arrays (int[], double[], ...) and
-  * object arrays (Integer[], MyPojo[], ...). This custom type supports both cases.
-  */
-class ArrayRelDataType(
-    val typeInfo: TypeInformation[_],
-    elementType: RelDataType,
-    isNullable: Boolean)
-  extends ArraySqlType(
-    elementType,
-    isNullable) {
-
-  override def toString = s"ARRAY($typeInfo)"
-
-  def canEqual(other: Any): Boolean = other.isInstanceOf[ArrayRelDataType]
-
-  override def equals(other: Any): Boolean = other match {
-    case that: ArrayRelDataType =>
-      super.equals(that) &&
-        (that canEqual this) &&
-        typeInfo == that.typeInfo
-    case _ => false
-  }
-
-  override def hashCode(): Int = {
-    typeInfo.hashCode()
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/CompositeRelDataType.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/CompositeRelDataType.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/CompositeRelDataType.scala
deleted file mode 100644
index b9ceff0..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/CompositeRelDataType.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.api.table.plan.schema
-
-import java.util
-
-import org.apache.calcite.rel.`type`.{RelDataTypeField, RelDataTypeFieldImpl, RelRecordType}
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.table.FlinkTypeFactory
-import org.apache.flink.api.table.plan.schema.CompositeRelDataType.createFieldList
-
-import scala.collection.JavaConverters._
-
-/**
-  * Composite type for encapsulating Flink's [[CompositeType]].
-  *
-  * @param compositeType CompositeType to encapsulate
-  * @param typeFactory Flink's type factory
-  */
-class CompositeRelDataType(
-    val compositeType: CompositeType[_],
-    typeFactory: FlinkTypeFactory)
-  extends RelRecordType(createFieldList(compositeType, typeFactory)) {
-
-  override def toString = s"COMPOSITE($compositeType)"
-
-  def canEqual(other: Any): Boolean = other.isInstanceOf[CompositeRelDataType]
-
-  override def equals(other: Any): Boolean = other match {
-    case that: CompositeRelDataType =>
-      super.equals(that) &&
-        (that canEqual this) &&
-        compositeType == that.compositeType
-    case _ => false
-  }
-
-  override def hashCode(): Int = {
-    compositeType.hashCode()
-  }
-
-}
-
-object CompositeRelDataType {
-
-  /**
-    * Converts the fields of a composite type to list of [[RelDataTypeField]].
-    */
-  private def createFieldList(
-      compositeType: CompositeType[_],
-      typeFactory: FlinkTypeFactory)
-    : util.List[RelDataTypeField] = {
-
-    compositeType
-      .getFieldNames
-      .zipWithIndex
-      .map { case (name, index) =>
-        new RelDataTypeFieldImpl(
-          name,
-          index,
-          typeFactory.createTypeFromTypeInfo(compositeType.getTypeAt(index)))
-            .asInstanceOf[RelDataTypeField]
-      }
-      .toList
-      .asJava
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/DataSetTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/DataSetTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/DataSetTable.scala
deleted file mode 100644
index bbcba13..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/DataSetTable.scala
+++ /dev/null
@@ -1,51 +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.api.table.plan.schema
-
-import java.lang.Double
-import java.util
-import java.util.Collections
-
-import org.apache.calcite.rel.{RelCollation, RelDistribution}
-import org.apache.calcite.schema.Statistic
-import org.apache.calcite.util.ImmutableBitSet
-import org.apache.flink.api.java.DataSet
-
-class DataSetTable[T](
-    val dataSet: DataSet[T],
-    override val fieldIndexes: Array[Int],
-    override val fieldNames: Array[String])
-  extends FlinkTable[T](dataSet.getType, fieldIndexes, fieldNames) {
-
-  override def getStatistic: Statistic = {
-    new DefaultDataSetStatistic
-  }
-
-}
-
-class DefaultDataSetStatistic extends Statistic {
-
-  override def getRowCount: Double = 1000d
-
-  override def getCollations: util.List[RelCollation] = Collections.emptyList()
-
-  override def isKey(columns: ImmutableBitSet): Boolean = false
-
-  override def getDistribution: RelDistribution = null
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/DataStreamTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/DataStreamTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/DataStreamTable.scala
deleted file mode 100644
index 570d723..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/DataStreamTable.scala
+++ /dev/null
@@ -1,28 +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.api.table.plan.schema
-
-import org.apache.flink.streaming.api.datastream.DataStream
-
-class DataStreamTable[T](
-    val dataStream: DataStream[T],
-    override val fieldIndexes: Array[Int],
-    override val fieldNames: Array[String])
-  extends FlinkTable[T](dataStream.getType, fieldIndexes, fieldNames) {
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTable.scala
deleted file mode 100644
index 84d6d7e..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTable.scala
+++ /dev/null
@@ -1,66 +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.api.table.plan.schema
-
-import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
-import org.apache.calcite.schema.impl.AbstractTable
-import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.table.{FlinkTypeFactory, TableException}
-
-abstract class FlinkTable[T](
-    val typeInfo: TypeInformation[T],
-    val fieldIndexes: Array[Int],
-    val fieldNames: Array[String])
-  extends AbstractTable {
-
-  if (fieldIndexes.length != fieldNames.length) {
-    throw new TableException(
-      "Number of field indexes and field names must be equal.")
-  }
-
-  // check uniqueness of field names
-  if (fieldNames.length != fieldNames.toSet.size) {
-    throw new TableException(
-      "Table field names must be unique.")
-  }
-
-  val fieldTypes: Array[TypeInformation[_]] =
-    typeInfo match {
-      case cType: CompositeType[T] =>
-        if (fieldNames.length != cType.getArity) {
-          throw new TableException(
-          s"Arity of type (" + cType.getFieldNames.deep + ") " +
-            "not equal to number of field names " + fieldNames.deep + ".")
-        }
-        fieldIndexes.map(cType.getTypeAt(_).asInstanceOf[TypeInformation[_]])
-      case aType: AtomicType[T] =>
-        if (fieldIndexes.length != 1 || fieldIndexes(0) != 0) {
-          throw new TableException(
-            "Non-composite input type may have only a single field and its index must be 0.")
-        }
-        Array(aType)
-    }
-
-  override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = {
-    val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory]
-    flinkTypeFactory.buildRowDataType(fieldNames, fieldTypes)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTableFunctionImpl.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTableFunctionImpl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTableFunctionImpl.scala
deleted file mode 100644
index 540a5c8..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/FlinkTableFunctionImpl.scala
+++ /dev/null
@@ -1,84 +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.api.table.plan.schema
-
-import java.lang.reflect.{Method, Type}
-import java.util
-
-import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
-import org.apache.calcite.schema.TableFunction
-import org.apache.calcite.schema.impl.ReflectiveFunctionBase
-import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.table.{FlinkTypeFactory, TableException}
-
-/**
-  * This is heavily inspired by Calcite's [[org.apache.calcite.schema.impl.TableFunctionImpl]].
-  * We need it in order to create a [[org.apache.flink.api.table.functions.utils.TableSqlFunction]].
-  * The main difference is that we override the [[getRowType()]] and [[getElementType()]].
-  */
-class FlinkTableFunctionImpl[T](
-    val typeInfo: TypeInformation[T],
-    val fieldIndexes: Array[Int],
-    val fieldNames: Array[String],
-    val evalMethod: Method)
-  extends ReflectiveFunctionBase(evalMethod)
-  with TableFunction {
-
-  if (fieldIndexes.length != fieldNames.length) {
-    throw new TableException(
-      "Number of field indexes and field names must be equal.")
-  }
-
-  // check uniqueness of field names
-  if (fieldNames.length != fieldNames.toSet.size) {
-    throw new TableException(
-      "Table field names must be unique.")
-  }
-
-  val fieldTypes: Array[TypeInformation[_]] =
-    typeInfo match {
-      case cType: CompositeType[T] =>
-        if (fieldNames.length != cType.getArity) {
-          throw new TableException(
-            s"Arity of type (" + cType.getFieldNames.deep + ") " +
-              "not equal to number of field names " + fieldNames.deep + ".")
-        }
-        fieldIndexes.map(cType.getTypeAt(_).asInstanceOf[TypeInformation[_]])
-      case aType: AtomicType[T] =>
-        if (fieldIndexes.length != 1 || fieldIndexes(0) != 0) {
-          throw new TableException(
-            "Non-composite input type may have only a single field and its index must be 0.")
-        }
-        Array(aType)
-    }
-
-  override def getElementType(arguments: util.List[AnyRef]): Type = classOf[Array[Object]]
-
-  override def getRowType(typeFactory: RelDataTypeFactory,
-                          arguments: util.List[AnyRef]): RelDataType = {
-    val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory]
-    val builder = flinkTypeFactory.builder
-    fieldNames
-      .zip(fieldTypes)
-      .foreach { f =>
-        builder.add(f._1, flinkTypeFactory.createTypeFromTypeInfo(f._2)).nullable(true)
-      }
-    builder.build
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/GenericRelDataType.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/GenericRelDataType.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/GenericRelDataType.scala
deleted file mode 100644
index a3012d1..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/GenericRelDataType.scala
+++ /dev/null
@@ -1,53 +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.api.table.plan.schema
-
-import org.apache.calcite.sql.`type`.{BasicSqlType, SqlTypeName}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.FlinkTypeSystem
-
-/**
-  * Generic type for encapsulating Flink's [[TypeInformation]].
-  *
-  * @param typeInfo TypeInformation to encapsulate
-  * @param typeSystem Flink's type system
-  */
-class GenericRelDataType(
-    val typeInfo: TypeInformation[_],
-    typeSystem: FlinkTypeSystem)
-  extends BasicSqlType(
-    typeSystem,
-    SqlTypeName.ANY) {
-
-  override def toString = s"ANY($typeInfo)"
-
-  def canEqual(other: Any): Boolean = other.isInstanceOf[GenericRelDataType]
-
-  override def equals(other: Any): Boolean = other match {
-    case that: GenericRelDataType =>
-      super.equals(that) &&
-        (that canEqual this) &&
-        typeInfo == that.typeInfo
-    case _ => false
-  }
-
-  override def hashCode(): Int = {
-    typeInfo.hashCode()
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/RelTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/RelTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/RelTable.scala
deleted file mode 100644
index f952d83..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/RelTable.scala
+++ /dev/null
@@ -1,46 +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.api.table.plan.schema
-
-import org.apache.calcite.plan.RelOptTable
-import org.apache.calcite.plan.RelOptTable.ToRelContext
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
-import org.apache.calcite.schema.Schema.TableType
-import org.apache.calcite.schema.impl.AbstractTable
-import org.apache.calcite.schema.TranslatableTable
-
-/**
- * A [[org.apache.calcite.schema.Table]] implementation for registering
- * Table API Tables in the Calcite schema to be used by Flink SQL.
- * It implements [[TranslatableTable]] so that its logical scan
- * can be converted to a relational expression.
- *
- * @see [[DataSetTable]]
- */
-class RelTable(relNode: RelNode) extends AbstractTable with TranslatableTable {
-
-  override def getJdbcTableType: TableType = ???
-
-  override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = relNode.getRowType
-
-  override def toRel(context: ToRelContext, relOptTable: RelOptTable): RelNode = {
-    relNode
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala
deleted file mode 100644
index 72be00c..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/schema/TableSourceTable.scala
+++ /dev/null
@@ -1,30 +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.api.table.plan.schema
-
-import org.apache.flink.types.Row
-import org.apache.flink.api.table.sources.TableSource
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-
-/** Table which defines an external table via a [[TableSource]] */
-class TableSourceTable(val tableSource: TableSource[_])
-  extends FlinkTable[Row](
-    typeInfo = new RowTypeInfo(tableSource.getFieldTypes: _*),
-    fieldIndexes = 0.until(tableSource.getNumberOfFields).toArray,
-    fieldNames = tableSource.getFieldsNames)

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/CountPartitionFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/CountPartitionFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/CountPartitionFunction.scala
deleted file mode 100644
index 5896f4c..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/CountPartitionFunction.scala
+++ /dev/null
@@ -1,40 +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.api.table.runtime
-
-import java.lang.Iterable
-
-import org.apache.flink.api.common.functions.RichMapPartitionFunction
-import org.apache.flink.util.Collector
-
-class CountPartitionFunction[IN] extends RichMapPartitionFunction[IN, (Int, Long)] {
-
-  override def mapPartition(value: Iterable[IN], out: Collector[(Int, Long)]): Unit = {
-    val partitionIndex = getRuntimeContext.getIndexOfThisSubtask
-    var elementCount = 0L
-    val iterator = value.iterator()
-    while (iterator.hasNext) {
-      if (elementCount != Long.MaxValue) { // prevent overflow
-        elementCount += 1L
-      }
-      iterator.next()
-    }
-    out.collect(partitionIndex, elementCount)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/FlatJoinRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/FlatJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/FlatJoinRunner.scala
deleted file mode 100644
index 2e57a0f..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/FlatJoinRunner.scala
+++ /dev/null
@@ -1,52 +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.api.table.runtime
-
-import org.apache.flink.api.common.functions.{FlatJoinFunction, RichFlatJoinFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable
-import org.apache.flink.api.table.codegen.Compiler
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.util.Collector
-import org.slf4j.LoggerFactory
-
-class FlatJoinRunner[IN1, IN2, OUT](
-    name: String,
-    code: String,
-    @transient returnType: TypeInformation[OUT])
-  extends RichFlatJoinFunction[IN1, IN2, OUT]
-  with ResultTypeQueryable[OUT]
-  with Compiler[FlatJoinFunction[IN1, IN2, OUT]] {
-
-  val LOG = LoggerFactory.getLogger(this.getClass)
-
-  private var function: FlatJoinFunction[IN1, IN2, OUT] = null
-
-  override def open(parameters: Configuration): Unit = {
-    LOG.debug(s"Compiling FlatJoinFunction: $name \n\n Code:\n$code")
-    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
-    LOG.debug("Instantiating FlatJoinFunction.")
-    function = clazz.newInstance()
-  }
-
-  override def join(first: IN1, second: IN2, out: Collector[OUT]): Unit =
-    function.join(first, second, out)
-
-  override def getProducedType: TypeInformation[OUT] = returnType
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/FlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/FlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/FlatMapRunner.scala
deleted file mode 100644
index e228e2b..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/FlatMapRunner.scala
+++ /dev/null
@@ -1,52 +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.api.table.runtime
-
-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.api.table.codegen.Compiler
-import org.apache.flink.configuration.Configuration
-import org.apache.flink.util.Collector
-import org.slf4j.LoggerFactory
-
-class FlatMapRunner[IN, OUT](
-    name: String,
-    code: String,
-    @transient returnType: TypeInformation[OUT])
-  extends RichFlatMapFunction[IN, OUT]
-  with ResultTypeQueryable[OUT]
-  with Compiler[FlatMapFunction[IN, OUT]] {
-
-  val LOG = LoggerFactory.getLogger(this.getClass)
-
-  private var function: FlatMapFunction[IN, OUT] = _
-
-  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()
-  }
-
-  override def flatMap(in: IN, out: Collector[OUT]): Unit =
-    function.flatMap(in, out)
-
-  override def getProducedType: TypeInformation[OUT] = returnType
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/IntersectCoGroupFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/IntersectCoGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/IntersectCoGroupFunction.scala
deleted file mode 100644
index 9930811..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/IntersectCoGroupFunction.scala
+++ /dev/null
@@ -1,42 +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.api.table.runtime
-
-import java.lang.{Iterable => JIterable}
-
-import org.apache.flink.api.common.functions.CoGroupFunction
-import org.apache.flink.util.Collector
-
-class IntersectCoGroupFunction[T](all: Boolean) extends CoGroupFunction[T, T, T]{
-  override def coGroup(first: JIterable[T], second: JIterable[T], out: Collector[T]): Unit = {
-    if (first == null || second == null) return
-    val leftIter = first.iterator()
-    val rightIter = second.iterator()
-    if (all) {
-      while (leftIter.hasNext && rightIter.hasNext) {
-        out.collect(leftIter.next)
-        rightIter.next
-      }
-    } else {
-      if (leftIter.hasNext && rightIter.hasNext) {
-        out.collect(leftIter.next)
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/LimitFilterFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/LimitFilterFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/LimitFilterFunction.scala
deleted file mode 100644
index 5ec9035..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/LimitFilterFunction.scala
+++ /dev/null
@@ -1,64 +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.api.table.runtime
-
-import org.apache.flink.api.common.functions.RichFilterFunction
-import org.apache.flink.configuration.Configuration
-
-import scala.collection.JavaConverters._
-
-
-class LimitFilterFunction[T](
-    limitStart: Long,
-    limitEnd: Long,
-    broadcastName: String)
-  extends RichFilterFunction[T] {
-
-  var partitionIndex: Int = _
-  var elementCount: Long = _
-  var countList: Array[Long] = _
-
-  override def open(config: Configuration) {
-    partitionIndex = getRuntimeContext.getIndexOfThisSubtask
-
-    val countPartitionResult = getRuntimeContext
-      .getBroadcastVariable[(Int, Long)](broadcastName)
-      .asScala
-
-    // sort by partition index, extract number per partition, sum with intermediate results
-    countList = countPartitionResult.sortWith(_._1 < _._1).map(_._2).scanLeft(0L) { case (a, b) =>
-        val sum = a + b
-        if (sum < 0L) { // prevent overflow
-          Long.MaxValue
-        }
-        sum
-    }.toArray
-
-    elementCount = 0
-  }
-
-  override def filter(value: T): Boolean = {
-    if (elementCount != Long.MaxValue) { // prevent overflow
-      elementCount += 1L
-    }
-    // we filter out records that are not within the limit (Long.MaxValue is unlimited)
-    limitStart - countList(partitionIndex) < elementCount &&
-      (limitEnd == Long.MaxValue || limitEnd - countList(partitionIndex) >= elementCount)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapJoinLeftRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapJoinLeftRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapJoinLeftRunner.scala
deleted file mode 100644
index 76650c2..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapJoinLeftRunner.scala
+++ /dev/null
@@ -1,33 +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.api.table.runtime
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.util.Collector
-
-class MapJoinLeftRunner[IN1, IN2, OUT](
-    name: String,
-    code: String,
-    @transient returnType: TypeInformation[OUT],
-    broadcastSetName: String)
-  extends MapSideJoinRunner[IN1, IN2, IN2, IN1, OUT](name, code, returnType, broadcastSetName) {
-
-  override def flatMap(multiInput: IN1, out: Collector[OUT]): Unit =
-    function.join(multiInput, singleInput, out)
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapJoinRightRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapJoinRightRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapJoinRightRunner.scala
deleted file mode 100644
index 52b01cf..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapJoinRightRunner.scala
+++ /dev/null
@@ -1,33 +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.api.table.runtime
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.util.Collector
-
-class MapJoinRightRunner[IN1, IN2, OUT](
-    name: String,
-    code: String,
-    @transient returnType: TypeInformation[OUT],
-    broadcastSetName: String)
-  extends MapSideJoinRunner[IN1, IN2, IN1, IN2, OUT](name, code, returnType, broadcastSetName) {
-
-  override def flatMap(multiInput: IN2, out: Collector[OUT]): Unit =
-    function.join(singleInput, multiInput, out)
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapSideJoinRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapSideJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapSideJoinRunner.scala
deleted file mode 100644
index b355d49..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MapSideJoinRunner.scala
+++ /dev/null
@@ -1,51 +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.api.table.runtime
-
-import org.apache.flink.api.common.functions.{FlatJoinFunction, RichFlatMapFunction}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable
-import org.apache.flink.api.table.codegen.Compiler
-import org.apache.flink.configuration.Configuration
-import org.slf4j.LoggerFactory
-
-abstract class MapSideJoinRunner[IN1, IN2, SINGLE_IN, MULTI_IN, OUT](
-    name: String,
-    code: String,
-    @transient returnType: TypeInformation[OUT],
-    broadcastSetName: String)
-  extends RichFlatMapFunction[MULTI_IN, OUT]
-    with ResultTypeQueryable[OUT]
-    with Compiler[FlatJoinFunction[IN1, IN2, OUT]] {
-
-  val LOG = LoggerFactory.getLogger(this.getClass)
-
-  protected var function: FlatJoinFunction[IN1, IN2, OUT] = _
-  protected var singleInput: SINGLE_IN = _
-
-  override def open(parameters: Configuration): Unit = {
-    LOG.debug(s"Compiling FlatJoinFunction: $name \n\n Code:\n$code")
-    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
-    LOG.debug("Instantiating FlatJoinFunction.")
-    function = clazz.newInstance()
-    singleInput = getRuntimeContext.getBroadcastVariable(broadcastSetName).get(0)
-  }
-
-  override def getProducedType: TypeInformation[OUT] = returnType
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MinusCoGroupFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MinusCoGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MinusCoGroupFunction.scala
deleted file mode 100644
index cac4fe6..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/MinusCoGroupFunction.scala
+++ /dev/null
@@ -1,47 +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.api.table.runtime
-
-import java.lang.Iterable
-
-import org.apache.flink.api.common.functions.CoGroupFunction
-import org.apache.flink.util.Collector
-
-class MinusCoGroupFunction[T](all: Boolean) extends CoGroupFunction[T, T, T] {
-  override def coGroup(first: Iterable[T], second: Iterable[T], out: Collector[T]): Unit = {
-    if (first == null || second == null) return
-    val leftIter = first.iterator
-    val rightIter = second.iterator
-
-    if (all) {
-      while (rightIter.hasNext && leftIter.hasNext) {
-        leftIter.next()
-        rightIter.next()
-      }
-
-      while (leftIter.hasNext) {
-        out.collect(leftIter.next())
-      }
-    } else {
-      if (!rightIter.hasNext && leftIter.hasNext) {
-        out.collect(leftIter.next())
-      }
-    }
-  }
-}
-


[39/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenUtils.scala
deleted file mode 100644
index a706309..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenUtils.scala
+++ /dev/null
@@ -1,293 +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.api.table.codegen
-
-import java.lang.reflect.{Field, Method}
-import java.util.concurrent.atomic.AtomicInteger
-
-import org.apache.calcite.util.BuiltInMethod
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo._
-import org.apache.flink.api.common.typeinfo.{FractionalTypeInfo, SqlTimeTypeInfo, TypeInformation}
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.java.typeutils.{PojoTypeInfo, TupleTypeInfo, TypeExtractor}
-import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.typeutils.{TimeIntervalTypeInfo, TypeCheckUtils}
-
-object CodeGenUtils {
-
-  private val nameCounter = new AtomicInteger
-
-  def newName(name: String): String = {
-    s"$name$$${nameCounter.getAndIncrement}"
-  }
-
-  // when casting we first need to unbox Primitives, for example,
-  // float a = 1.0f;
-  // byte b = (byte) a;
-  // works, but for boxed types we need this:
-  // Float a = 1.0f;
-  // Byte b = (byte)(float) a;
-  def primitiveTypeTermForTypeInfo(tpe: TypeInformation[_]): String = tpe match {
-    case INT_TYPE_INFO => "int"
-    case LONG_TYPE_INFO => "long"
-    case SHORT_TYPE_INFO => "short"
-    case BYTE_TYPE_INFO => "byte"
-    case FLOAT_TYPE_INFO => "float"
-    case DOUBLE_TYPE_INFO => "double"
-    case BOOLEAN_TYPE_INFO => "boolean"
-    case CHAR_TYPE_INFO => "char"
-
-    // From PrimitiveArrayTypeInfo we would get class "int[]", scala reflections
-    // does not seem to like this, so we manually give the correct type here.
-    case INT_PRIMITIVE_ARRAY_TYPE_INFO => "int[]"
-    case LONG_PRIMITIVE_ARRAY_TYPE_INFO => "long[]"
-    case SHORT_PRIMITIVE_ARRAY_TYPE_INFO => "short[]"
-    case BYTE_PRIMITIVE_ARRAY_TYPE_INFO => "byte[]"
-    case FLOAT_PRIMITIVE_ARRAY_TYPE_INFO => "float[]"
-    case DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO => "double[]"
-    case BOOLEAN_PRIMITIVE_ARRAY_TYPE_INFO => "boolean[]"
-    case CHAR_PRIMITIVE_ARRAY_TYPE_INFO => "char[]"
-
-    // internal primitive representation of time points
-    case SqlTimeTypeInfo.DATE => "int"
-    case SqlTimeTypeInfo.TIME => "int"
-    case SqlTimeTypeInfo.TIMESTAMP => "long"
-
-    // internal primitive representation of time intervals
-    case TimeIntervalTypeInfo.INTERVAL_MONTHS => "int"
-    case TimeIntervalTypeInfo.INTERVAL_MILLIS => "long"
-
-    case _ =>
-      tpe.getTypeClass.getCanonicalName
-  }
-
-  def boxedTypeTermForTypeInfo(tpe: TypeInformation[_]): String = tpe match {
-    // From PrimitiveArrayTypeInfo we would get class "int[]", scala reflections
-    // does not seem to like this, so we manually give the correct type here.
-    case INT_PRIMITIVE_ARRAY_TYPE_INFO => "int[]"
-    case LONG_PRIMITIVE_ARRAY_TYPE_INFO => "long[]"
-    case SHORT_PRIMITIVE_ARRAY_TYPE_INFO => "short[]"
-    case BYTE_PRIMITIVE_ARRAY_TYPE_INFO => "byte[]"
-    case FLOAT_PRIMITIVE_ARRAY_TYPE_INFO => "float[]"
-    case DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO => "double[]"
-    case BOOLEAN_PRIMITIVE_ARRAY_TYPE_INFO => "boolean[]"
-    case CHAR_PRIMITIVE_ARRAY_TYPE_INFO => "char[]"
-
-    case _ =>
-      tpe.getTypeClass.getCanonicalName
-  }
-
-  def primitiveDefaultValue(tpe: TypeInformation[_]): String = tpe match {
-    case INT_TYPE_INFO => "-1"
-    case LONG_TYPE_INFO => "-1L"
-    case SHORT_TYPE_INFO => "-1"
-    case BYTE_TYPE_INFO => "-1"
-    case FLOAT_TYPE_INFO => "-1.0f"
-    case DOUBLE_TYPE_INFO => "-1.0d"
-    case BOOLEAN_TYPE_INFO => "false"
-    case STRING_TYPE_INFO => "\"\""
-    case CHAR_TYPE_INFO => "'\\0'"
-    case SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIME => "-1"
-    case SqlTimeTypeInfo.TIMESTAMP => "-1L"
-    case TimeIntervalTypeInfo.INTERVAL_MONTHS => "-1"
-    case TimeIntervalTypeInfo.INTERVAL_MILLIS => "-1L"
-
-    case _ => "null"
-  }
-
-  def superPrimitive(typeInfo: TypeInformation[_]): String = typeInfo match {
-    case _: FractionalTypeInfo[_] => "double"
-    case _ => "long"
-  }
-
-  def qualifyMethod(method: Method): String =
-    method.getDeclaringClass.getCanonicalName + "." + method.getName
-
-  def qualifyEnum(enum: Enum[_]): String =
-    enum.getClass.getCanonicalName + "." + enum.name()
-
-  def internalToTimePointCode(resultType: TypeInformation[_], resultTerm: String) =
-    resultType match {
-      case SqlTimeTypeInfo.DATE =>
-        s"${qualifyMethod(BuiltInMethod.INTERNAL_TO_DATE.method)}($resultTerm)"
-      case SqlTimeTypeInfo.TIME =>
-        s"${qualifyMethod(BuiltInMethod.INTERNAL_TO_TIME.method)}($resultTerm)"
-      case SqlTimeTypeInfo.TIMESTAMP =>
-        s"${qualifyMethod(BuiltInMethod.INTERNAL_TO_TIMESTAMP.method)}($resultTerm)"
-    }
-
-  def timePointToInternalCode(resultType: TypeInformation[_], resultTerm: String) =
-    resultType match {
-      case SqlTimeTypeInfo.DATE =>
-        s"${qualifyMethod(BuiltInMethod.DATE_TO_INT.method)}($resultTerm)"
-      case SqlTimeTypeInfo.TIME =>
-        s"${qualifyMethod(BuiltInMethod.TIME_TO_INT.method)}($resultTerm)"
-      case SqlTimeTypeInfo.TIMESTAMP =>
-        s"${qualifyMethod(BuiltInMethod.TIMESTAMP_TO_LONG.method)}($resultTerm)"
-    }
-
-  def compareEnum(term: String, enum: Enum[_]): Boolean = term == qualifyEnum(enum)
-
-  def getEnum(genExpr: GeneratedExpression): Enum[_] = {
-    val split = genExpr.resultTerm.split('.')
-    val value = split.last
-    val clazz = genExpr.resultType.getTypeClass
-    enumValueOf(clazz, value)
-  }
-
-  def enumValueOf[T <: Enum[T]](cls: Class[_], stringValue: String): Enum[_] =
-    Enum.valueOf(cls.asInstanceOf[Class[T]], stringValue).asInstanceOf[Enum[_]]
-
-  // ----------------------------------------------------------------------------------------------
-
-  def requireNumeric(genExpr: GeneratedExpression) =
-    if (!TypeCheckUtils.isNumeric(genExpr.resultType)) {
-      throw new CodeGenException("Numeric expression type expected, but was " +
-        s"'${genExpr.resultType}'.")
-    }
-
-  def requireComparable(genExpr: GeneratedExpression) =
-    if (!TypeCheckUtils.isComparable(genExpr.resultType)) {
-      throw new CodeGenException(s"Comparable type expected, but was '${genExpr.resultType}'.")
-    }
-
-  def requireString(genExpr: GeneratedExpression) =
-    if (!TypeCheckUtils.isString(genExpr.resultType)) {
-      throw new CodeGenException("String expression type expected.")
-    }
-
-  def requireBoolean(genExpr: GeneratedExpression) =
-    if (!TypeCheckUtils.isBoolean(genExpr.resultType)) {
-      throw new CodeGenException("Boolean expression type expected.")
-    }
-
-  def requireTemporal(genExpr: GeneratedExpression) =
-    if (!TypeCheckUtils.isTemporal(genExpr.resultType)) {
-      throw new CodeGenException("Temporal expression type expected.")
-    }
-
-  def requireTimeInterval(genExpr: GeneratedExpression) =
-    if (!TypeCheckUtils.isTimeInterval(genExpr.resultType)) {
-      throw new CodeGenException("Interval expression type expected.")
-    }
-
-  def requireArray(genExpr: GeneratedExpression) =
-    if (!TypeCheckUtils.isArray(genExpr.resultType)) {
-      throw new CodeGenException("Array expression type expected.")
-    }
-
-  def requireInteger(genExpr: GeneratedExpression) =
-    if (!TypeCheckUtils.isInteger(genExpr.resultType)) {
-      throw new CodeGenException("Integer expression type expected.")
-    }
-
-  // ----------------------------------------------------------------------------------------------
-
-  def isReference(genExpr: GeneratedExpression): Boolean = isReference(genExpr.resultType)
-
-  def isReference(typeInfo: TypeInformation[_]): Boolean = typeInfo match {
-    case INT_TYPE_INFO
-         | LONG_TYPE_INFO
-         | SHORT_TYPE_INFO
-         | BYTE_TYPE_INFO
-         | FLOAT_TYPE_INFO
-         | DOUBLE_TYPE_INFO
-         | BOOLEAN_TYPE_INFO
-         | CHAR_TYPE_INFO => false
-    case _ => true
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  sealed abstract class FieldAccessor
-
-  case class ObjectFieldAccessor(field: Field) extends FieldAccessor
-
-  case class ObjectGenericFieldAccessor(name: String) extends FieldAccessor
-
-  case class ObjectPrivateFieldAccessor(field: Field) extends FieldAccessor
-
-  case class ObjectMethodAccessor(methodName: String) extends FieldAccessor
-
-  case class ProductAccessor(i: Int) extends FieldAccessor
-
-  def fieldAccessorFor(compType: CompositeType[_], index: Int): FieldAccessor = {
-    compType match {
-      case ri: RowTypeInfo =>
-        ProductAccessor(index)
-
-      case cc: CaseClassTypeInfo[_] =>
-        ObjectMethodAccessor(cc.getFieldNames()(index))
-
-      case javaTup: TupleTypeInfo[_] =>
-        ObjectGenericFieldAccessor("f" + index)
-
-      case pt: PojoTypeInfo[_] =>
-        val fieldName = pt.getFieldNames()(index)
-        getFieldAccessor(pt.getTypeClass, fieldName)
-
-      case _ => throw new CodeGenException("Unsupported composite type.")
-    }
-  }
-
-  def getFieldAccessor(clazz: Class[_], fieldName: String): FieldAccessor = {
-    val field = TypeExtractor.getDeclaredField(clazz, fieldName)
-    if (field.isAccessible) {
-      ObjectFieldAccessor(field)
-    }
-    else {
-      ObjectPrivateFieldAccessor(field)
-    }
-  }
-
-  def isFieldPrimitive(field: Field): Boolean = field.getType.isPrimitive
-
-  def reflectiveFieldReadAccess(fieldTerm: String, field: Field, objectTerm: String): String =
-    field.getType match {
-      case java.lang.Integer.TYPE => s"$fieldTerm.getInt($objectTerm)"
-      case java.lang.Long.TYPE => s"$fieldTerm.getLong($objectTerm)"
-      case java.lang.Short.TYPE => s"$fieldTerm.getShort($objectTerm)"
-      case java.lang.Byte.TYPE => s"$fieldTerm.getByte($objectTerm)"
-      case java.lang.Float.TYPE => s"$fieldTerm.getFloat($objectTerm)"
-      case java.lang.Double.TYPE => s"$fieldTerm.getDouble($objectTerm)"
-      case java.lang.Boolean.TYPE => s"$fieldTerm.getBoolean($objectTerm)"
-      case java.lang.Character.TYPE => s"$fieldTerm.getChar($objectTerm)"
-      case _ => s"(${field.getType.getCanonicalName}) $fieldTerm.get($objectTerm)"
-    }
-
-  def reflectiveFieldWriteAccess(
-      fieldTerm: String,
-      field: Field,
-      objectTerm: String,
-      valueTerm: String)
-    : String =
-    field.getType match {
-      case java.lang.Integer.TYPE => s"$fieldTerm.setInt($objectTerm, $valueTerm)"
-      case java.lang.Long.TYPE => s"$fieldTerm.setLong($objectTerm, $valueTerm)"
-      case java.lang.Short.TYPE => s"$fieldTerm.setShort($objectTerm, $valueTerm)"
-      case java.lang.Byte.TYPE => s"$fieldTerm.setByte($objectTerm, $valueTerm)"
-      case java.lang.Float.TYPE => s"$fieldTerm.setFloat($objectTerm, $valueTerm)"
-      case java.lang.Double.TYPE => s"$fieldTerm.setDouble($objectTerm, $valueTerm)"
-      case java.lang.Boolean.TYPE => s"$fieldTerm.setBoolean($objectTerm, $valueTerm)"
-      case java.lang.Character.TYPE => s"$fieldTerm.setChar($objectTerm, $valueTerm)"
-      case _ => s"$fieldTerm.set($objectTerm, $valueTerm)"
-    }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala
deleted file mode 100644
index cdb3753..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/CodeGenerator.scala
+++ /dev/null
@@ -1,1522 +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.api.table.codegen
-
-import java.math.{BigDecimal => JBigDecimal}
-
-import org.apache.calcite.avatica.util.DateTimeUtils
-import org.apache.calcite.rex._
-import org.apache.calcite.sql.SqlOperator
-import org.apache.calcite.sql.`type`.SqlTypeName._
-import org.apache.calcite.sql.fun.SqlStdOperatorTable._
-import org.apache.flink.api.common.functions.{FlatJoinFunction, FlatMapFunction, Function, MapFunction}
-import org.apache.flink.api.common.io.GenericInputFormat
-import org.apache.flink.api.common.typeinfo.{AtomicType, SqlTimeTypeInfo, TypeInformation}
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.java.typeutils.{GenericTypeInfo, PojoTypeInfo, TupleTypeInfo}
-import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
-import org.apache.flink.api.table.codegen.CodeGenUtils._
-import org.apache.flink.api.table.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE}
-import org.apache.flink.api.table.codegen.Indenter.toISC
-import org.apache.flink.api.table.codegen.calls.FunctionGenerator
-import org.apache.flink.api.table.codegen.calls.ScalarOperators._
-import org.apache.flink.api.table.functions.UserDefinedFunction
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.typeutils.TypeConverter
-import org.apache.flink.api.table.typeutils.TypeCheckUtils._
-import org.apache.flink.api.table.{FlinkTypeFactory, TableConfig}
-
-import scala.collection.JavaConversions._
-import scala.collection.mutable
-
-/**
-  * A code generator for generating Flink [[org.apache.flink.api.common.functions.Function]]s.
-  *
-  * @param config configuration that determines runtime behavior
-  * @param nullableInput input(s) can be null.
-  * @param input1 type information about the first input of the Function
-  * @param input2 type information about the second input if the Function is binary
-  * @param input1PojoFieldMapping additional mapping information if input1 is a POJO (POJO types
-  *                              have no deterministic field order).
-  * @param input2PojoFieldMapping additional mapping information if input2 is a POJO (POJO types
-  *                              have no deterministic field order).
-  *
-  */
-class CodeGenerator(
-   config: TableConfig,
-   nullableInput: Boolean,
-   input1: TypeInformation[Any],
-   input2: Option[TypeInformation[Any]] = None,
-   input1PojoFieldMapping: Option[Array[Int]] = None,
-   input2PojoFieldMapping: Option[Array[Int]] = None)
-  extends RexVisitor[GeneratedExpression] {
-
-  // check if nullCheck is enabled when inputs can be null
-  if (nullableInput && !config.getNullCheck) {
-    throw new CodeGenException("Null check must be enabled if entire rows can be null.")
-  }
-
-  // check for POJO input1 mapping
-  input1 match {
-    case pt: PojoTypeInfo[_] =>
-      input1PojoFieldMapping.getOrElse(
-        throw new CodeGenException("No input mapping is specified for input1 of type POJO."))
-    case _ => // ok
-  }
-
-  // check for POJO input2 mapping
-  input2 match {
-    case Some(pt: PojoTypeInfo[_]) =>
-      input2PojoFieldMapping.getOrElse(
-        throw new CodeGenException("No input mapping is specified for input2 of type POJO."))
-    case _ => // ok
-  }
-
-  /**
-    * A code generator for generating unary Flink
-    * [[org.apache.flink.api.common.functions.Function]]s with one input.
-    *
-    * @param config configuration that determines runtime behavior
-    * @param nullableInput input(s) can be null.
-    * @param input type information about the input of the Function
-    * @param inputPojoFieldMapping additional mapping information necessary if input is a
-    *                              POJO (POJO types have no deterministic field order).
-    */
-  def this(
-      config: TableConfig,
-      nullableInput: Boolean,
-      input: TypeInformation[Any],
-      inputPojoFieldMapping: Array[Int]) =
-    this(config, nullableInput, input, None, Some(inputPojoFieldMapping))
-
-  /**
-    * A code generator for generating Flink input formats.
-    *
-    * @param config configuration that determines runtime behavior
-    */
-  def this(config: TableConfig) =
-    this(config, false, TypeConverter.DEFAULT_ROW_TYPE, None, None)
-
-  // set of member statements that will be added only once
-  // we use a LinkedHashSet to keep the insertion order
-  private val reusableMemberStatements = mutable.LinkedHashSet[String]()
-
-  // set of constructor statements that will be added only once
-  // we use a LinkedHashSet to keep the insertion order
-  private val reusableInitStatements = mutable.LinkedHashSet[String]()
-
-  // set of statements that will be added only once per record
-  // we use a LinkedHashSet to keep the insertion order
-  private val reusablePerRecordStatements = mutable.LinkedHashSet[String]()
-
-  // map of initial input unboxing expressions that will be added only once
-  // (inputTerm, index) -> expr
-  private val reusableInputUnboxingExprs = mutable.Map[(String, Int), GeneratedExpression]()
-
-  /**
-    * @return code block of statements that need to be placed in the member area of the Function
-    *         (e.g. member variables and their initialization)
-    */
-  def reuseMemberCode(): String = {
-    reusableMemberStatements.mkString("", "\n", "\n")
-  }
-
-  /**
-    * @return code block of statements that need to be placed in the constructor of the Function
-    */
-  def reuseInitCode(): String = {
-    reusableInitStatements.mkString("", "\n", "\n")
-  }
-
-  /**
-    * @return code block of statements that need to be placed in the SAM of the Function
-    */
-  def reusePerRecordCode(): String = {
-    reusablePerRecordStatements.mkString("", "\n", "\n")
-  }
-
-  /**
-    * @return code block of statements that unbox input variables to a primitive variable
-    *         and a corresponding null flag variable
-    */
-  def reuseInputUnboxingCode(): String = {
-    reusableInputUnboxingExprs.values.map(_.code).mkString("", "\n", "\n")
-  }
-
-  /**
-    * @return term of the (casted and possibly boxed) first input
-    */
-  var input1Term = "in1"
-
-  /**
-    * @return term of the (casted and possibly boxed) second input
-    */
-  var input2Term = "in2"
-
-  /**
-    * @return term of the (casted) output collector
-    */
-  var collectorTerm = "c"
-
-  /**
-    * @return term of the output record (possibly defined in the member area e.g. Row, Tuple)
-    */
-  var outRecordTerm = "out"
-
-  /**
-    * @return returns if null checking is enabled
-    */
-  def nullCheck: Boolean = config.getNullCheck
-
-  /**
-    * Generates an expression from a RexNode. If objects or variables can be reused, they will be
-    * added to reusable code sections internally.
-    *
-    * @param rex Calcite row expression
-    * @return instance of GeneratedExpression
-    */
-  def generateExpression(rex: RexNode): GeneratedExpression = {
-    rex.accept(this)
-  }
-
-  /**
-    * Generates a [[org.apache.flink.api.common.functions.Function]] that can be passed to Java
-    * compiler.
-    *
-    * @param name Class name of the Function. Must not be unique but has to be a valid Java class
-    *             identifier.
-    * @param clazz Flink Function to be generated.
-    * @param bodyCode code contents of the SAM (Single Abstract Method). Inputs, collector, or
-    *                 output record can be accessed via the given term methods.
-    * @param returnType expected return type
-    * @tparam T Flink Function to be generated.
-    * @return instance of GeneratedFunction
-    */
-  def generateFunction[T <: Function](
-      name: String,
-      clazz: Class[T],
-      bodyCode: String,
-      returnType: TypeInformation[Any])
-    : GeneratedFunction[T] = {
-    val funcName = newName(name)
-
-    // Janino does not support generics, that's why we need
-    // manual casting here
-    val samHeader =
-      // FlatMapFunction
-      if (clazz == classOf[FlatMapFunction[_,_]]) {
-        val inputTypeTerm = boxedTypeTermForTypeInfo(input1)
-        (s"void flatMap(Object _in1, org.apache.flink.util.Collector $collectorTerm)",
-          List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;"))
-      }
-
-      // MapFunction
-      else if (clazz == classOf[MapFunction[_,_]]) {
-        val inputTypeTerm = boxedTypeTermForTypeInfo(input1)
-        ("Object map(Object _in1)",
-          List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;"))
-      }
-
-      // FlatJoinFunction
-      else if (clazz == classOf[FlatJoinFunction[_,_,_]]) {
-        val inputTypeTerm1 = boxedTypeTermForTypeInfo(input1)
-        val inputTypeTerm2 = boxedTypeTermForTypeInfo(input2.getOrElse(
-            throw new CodeGenException("Input 2 for FlatJoinFunction should not be null")))
-        (s"void join(Object _in1, Object _in2, org.apache.flink.util.Collector $collectorTerm)",
-          List(s"$inputTypeTerm1 $input1Term = ($inputTypeTerm1) _in1;",
-          s"$inputTypeTerm2 $input2Term = ($inputTypeTerm2) _in2;"))
-      }
-      else {
-        // TODO more functions
-        throw new CodeGenException("Unsupported Function.")
-      }
-
-    val funcCode = j"""
-      public class $funcName
-          implements ${clazz.getCanonicalName} {
-
-        ${reuseMemberCode()}
-
-        public $funcName() throws Exception {
-          ${reuseInitCode()}
-        }
-
-        @Override
-        public ${samHeader._1} throws Exception {
-          ${samHeader._2.mkString("\n")}
-          ${reusePerRecordCode()}
-          ${reuseInputUnboxingCode()}
-          $bodyCode
-        }
-      }
-    """.stripMargin
-
-    GeneratedFunction(funcName, returnType, funcCode)
-  }
-
-  /**
-    * Generates a values input format that can be passed to Java compiler.
-    *
-    * @param name Class name of the input format. Must not be unique but has to be a
-    *             valid Java class identifier.
-    * @param records code for creating records
-    * @param returnType expected return type
-    * @tparam T Flink Function to be generated.
-    * @return instance of GeneratedFunction
-    */
-  def generateValuesInputFormat[T](
-      name: String,
-      records: Seq[String],
-      returnType: TypeInformation[Any])
-    : GeneratedFunction[GenericInputFormat[T]] = {
-    val funcName = newName(name)
-
-    addReusableOutRecord(returnType)
-
-    val funcCode = j"""
-      public class $funcName extends ${classOf[GenericInputFormat[_]].getCanonicalName} {
-
-        private int nextIdx = 0;
-
-        ${reuseMemberCode()}
-
-        public $funcName() throws Exception {
-          ${reuseInitCode()}
-        }
-
-        @Override
-        public boolean reachedEnd() throws java.io.IOException {
-          return nextIdx >= ${records.length};
-        }
-
-        @Override
-        public Object nextRecord(Object reuse) {
-          switch (nextIdx) {
-            ${records.zipWithIndex.map { case (r, i) =>
-              s"""
-                 |case $i:
-                 |  $r
-                 |break;
-               """.stripMargin
-            }.mkString("\n")}
-          }
-          nextIdx++;
-          return $outRecordTerm;
-        }
-      }
-    """.stripMargin
-
-    GeneratedFunction[GenericInputFormat[T]](funcName, returnType, funcCode)
-  }
-
-  /**
-    * Generates an expression that converts the first input (and second input) into the given type.
-    * If two inputs are converted, the second input is appended. If objects or variables can
-    * be reused, they will be added to reusable code sections internally. The evaluation result
-    * may be stored in the global result variable (see [[outRecordTerm]]).
-    *
-    * @param returnType conversion target type. Inputs and output must have the same arity.
-    * @param resultFieldNames result field names necessary for a mapping to POJO fields.
-    * @return instance of GeneratedExpression
-    */
-  def generateConverterResultExpression(
-      returnType: TypeInformation[_ <: Any],
-      resultFieldNames: Seq[String])
-    : GeneratedExpression = {
-    val input1AccessExprs = for (i <- 0 until input1.getArity)
-      yield generateInputAccess(input1, input1Term, i, input1PojoFieldMapping)
-
-    val input2AccessExprs = input2 match {
-      case Some(ti) => for (i <- 0 until ti.getArity)
-        yield generateInputAccess(ti, input2Term, i, input2PojoFieldMapping)
-      case None => Seq() // add nothing
-    }
-
-    generateResultExpression(input1AccessExprs ++ input2AccessExprs, returnType, resultFieldNames)
-  }
-
-  /**
-    * Generates an expression from the left input and the right table function.
-    */
-  def generateCorrelateAccessExprs: (Seq[GeneratedExpression], Seq[GeneratedExpression]) = {
-    val input1AccessExprs = for (i <- 0 until input1.getArity)
-      yield generateInputAccess(input1, input1Term, i, input1PojoFieldMapping)
-
-    val input2AccessExprs = input2 match {
-      case Some(ti) => for (i <- 0 until ti.getArity)
-        // use generateFieldAccess instead of generateInputAccess to avoid the generated table
-        // function's field access code is put on the top of function body rather than
-        // the while loop
-        yield generateFieldAccess(ti, input2Term, i, input2PojoFieldMapping)
-      case None => throw new CodeGenException("Type information of input2 must not be null.")
-    }
-    (input1AccessExprs, input2AccessExprs)
-  }
-
-  /**
-    * Generates an expression from a sequence of RexNode. If objects or variables can be reused,
-    * they will be added to reusable code sections internally. The evaluation result
-    * may be stored in the global result variable (see [[outRecordTerm]]).
-    *
-    * @param returnType conversion target type. Type must have the same arity than rexNodes.
-    * @param resultFieldNames result field names necessary for a mapping to POJO fields.
-    * @param rexNodes sequence of RexNode to be converted
-    * @return instance of GeneratedExpression
-    */
-  def generateResultExpression(
-      returnType: TypeInformation[_ <: Any],
-      resultFieldNames: Seq[String],
-      rexNodes: Seq[RexNode])
-    : GeneratedExpression = {
-    val fieldExprs = rexNodes.map(generateExpression)
-    generateResultExpression(fieldExprs, returnType, resultFieldNames)
-  }
-
-  /**
-    * Generates an expression from a sequence of other expressions. If objects or variables can
-    * be reused, they will be added to reusable code sections internally. The evaluation result
-    * may be stored in the global result variable (see [[outRecordTerm]]).
-    *
-    * @param fieldExprs field expressions to be converted
-    * @param returnType conversion target type. Type must have the same arity than fieldExprs.
-    * @param resultFieldNames result field names necessary for a mapping to POJO fields.
-    * @return instance of GeneratedExpression
-    */
-  def generateResultExpression(
-      fieldExprs: Seq[GeneratedExpression],
-      returnType: TypeInformation[_ <: Any],
-      resultFieldNames: Seq[String])
-    : GeneratedExpression = {
-    // initial type check
-    if (returnType.getArity != fieldExprs.length) {
-      throw new CodeGenException("Arity of result type does not match number of expressions.")
-    }
-    if (resultFieldNames.length != fieldExprs.length) {
-      throw new CodeGenException("Arity of result field names does not match number of " +
-        "expressions.")
-    }
-    // type check
-    returnType match {
-      case pt: PojoTypeInfo[_] =>
-        fieldExprs.zipWithIndex foreach {
-          case (fieldExpr, i) if fieldExpr.resultType != pt.getTypeAt(resultFieldNames(i)) =>
-            throw new CodeGenException("Incompatible types of expression and result type.")
-
-          case _ => // ok
-        }
-
-      case ct: CompositeType[_] =>
-        fieldExprs.zipWithIndex foreach {
-          case (fieldExpr, i) if fieldExpr.resultType != ct.getTypeAt(i) =>
-            throw new CodeGenException("Incompatible types of expression and result type.")
-          case _ => // ok
-        }
-
-      case at: AtomicType[_] if at != fieldExprs.head.resultType =>
-        throw new CodeGenException("Incompatible types of expression and result type.")
-
-      case _ => // ok
-    }
-
-    val returnTypeTerm = boxedTypeTermForTypeInfo(returnType)
-    val boxedFieldExprs = fieldExprs.map(generateOutputFieldBoxing)
-
-    // generate result expression
-    returnType match {
-      case ri: RowTypeInfo =>
-        addReusableOutRecord(ri)
-        val resultSetters: String = boxedFieldExprs.zipWithIndex map {
-          case (fieldExpr, i) =>
-            if (nullCheck) {
-              s"""
-              |${fieldExpr.code}
-              |if (${fieldExpr.nullTerm}) {
-              |  $outRecordTerm.setField($i, null);
-              |}
-              |else {
-              |  $outRecordTerm.setField($i, ${fieldExpr.resultTerm});
-              |}
-              |""".stripMargin
-            }
-            else {
-              s"""
-              |${fieldExpr.code}
-              |$outRecordTerm.setField($i, ${fieldExpr.resultTerm});
-              |""".stripMargin
-            }
-        } mkString "\n"
-
-        GeneratedExpression(outRecordTerm, "false", resultSetters, returnType)
-
-      case pt: PojoTypeInfo[_] =>
-        addReusableOutRecord(pt)
-        val resultSetters: String = boxedFieldExprs.zip(resultFieldNames) map {
-          case (fieldExpr, fieldName) =>
-            val accessor = getFieldAccessor(pt.getTypeClass, fieldName)
-
-            accessor match {
-              // Reflective access of primitives/Objects
-              case ObjectPrivateFieldAccessor(field) =>
-                val fieldTerm = addReusablePrivateFieldAccess(pt.getTypeClass, fieldName)
-
-                val defaultIfNull = if (isFieldPrimitive(field)) {
-                  primitiveDefaultValue(fieldExpr.resultType)
-                } else {
-                  "null"
-                }
-
-                if (nullCheck) {
-                  s"""
-                    |${fieldExpr.code}
-                    |if (${fieldExpr.nullTerm}) {
-                    |  ${reflectiveFieldWriteAccess(
-                          fieldTerm,
-                          field,
-                          outRecordTerm,
-                          defaultIfNull)};
-                    |}
-                    |else {
-                    |  ${reflectiveFieldWriteAccess(
-                          fieldTerm,
-                          field,
-                          outRecordTerm,
-                          fieldExpr.resultTerm)};
-                    |}
-                    |""".stripMargin
-                }
-                else {
-                  s"""
-                    |${fieldExpr.code}
-                    |${reflectiveFieldWriteAccess(
-                          fieldTerm,
-                          field,
-                          outRecordTerm,
-                          fieldExpr.resultTerm)};
-                    |""".stripMargin
-                }
-
-              // primitive or Object field access (implicit boxing)
-              case _ =>
-                if (nullCheck) {
-                  s"""
-                    |${fieldExpr.code}
-                    |if (${fieldExpr.nullTerm}) {
-                    |  $outRecordTerm.$fieldName = null;
-                    |}
-                    |else {
-                    |  $outRecordTerm.$fieldName = ${fieldExpr.resultTerm};
-                    |}
-                    |""".stripMargin
-                }
-                else {
-                  s"""
-                    |${fieldExpr.code}
-                    |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm};
-                    |""".stripMargin
-                }
-              }
-          } mkString "\n"
-
-        GeneratedExpression(outRecordTerm, "false", resultSetters, returnType)
-
-      case tup: TupleTypeInfo[_] =>
-        addReusableOutRecord(tup)
-        val resultSetters: String = boxedFieldExprs.zipWithIndex map {
-          case (fieldExpr, i) =>
-            val fieldName = "f" + i
-            if (nullCheck) {
-              s"""
-                |${fieldExpr.code}
-                |if (${fieldExpr.nullTerm}) {
-                |  throw new NullPointerException("Null result cannot be stored in a Tuple.");
-                |}
-                |else {
-                |  $outRecordTerm.$fieldName = ${fieldExpr.resultTerm};
-                |}
-                |""".stripMargin
-            }
-            else {
-              s"""
-                |${fieldExpr.code}
-                |$outRecordTerm.$fieldName = ${fieldExpr.resultTerm};
-                |""".stripMargin
-            }
-        } mkString "\n"
-
-        GeneratedExpression(outRecordTerm, "false", resultSetters, returnType)
-
-      case cc: CaseClassTypeInfo[_] =>
-        val fieldCodes: String = boxedFieldExprs.map(_.code).mkString("\n")
-        val constructorParams: String = boxedFieldExprs.map(_.resultTerm).mkString(", ")
-        val resultTerm = newName(outRecordTerm)
-
-        val nullCheckCode = if (nullCheck) {
-        boxedFieldExprs map { (fieldExpr) =>
-          s"""
-              |if (${fieldExpr.nullTerm}) {
-              |  throw new NullPointerException("Null result cannot be stored in a Case Class.");
-              |}
-              |""".stripMargin
-          } mkString "\n"
-        } else {
-          ""
-        }
-
-        val resultCode =
-          s"""
-            |$fieldCodes
-            |$nullCheckCode
-            |$returnTypeTerm $resultTerm = new $returnTypeTerm($constructorParams);
-            |""".stripMargin
-
-        GeneratedExpression(resultTerm, "false", resultCode, returnType)
-
-      case a: AtomicType[_] =>
-        val fieldExpr = boxedFieldExprs.head
-        val nullCheckCode = if (nullCheck) {
-          s"""
-          |if (${fieldExpr.nullTerm}) {
-          |  throw new NullPointerException("Null result cannot be used for atomic types.");
-          |}
-          |""".stripMargin
-        } else {
-          ""
-        }
-        val resultCode =
-          s"""
-            |${fieldExpr.code}
-            |$nullCheckCode
-            |""".stripMargin
-
-        GeneratedExpression(fieldExpr.resultTerm, "false", resultCode, returnType)
-
-      case _ =>
-        throw new CodeGenException(s"Unsupported result type: $returnType")
-    }
-  }
-
-  // ----------------------------------------------------------------------------------------------
-  // RexVisitor methods
-  // ----------------------------------------------------------------------------------------------
-
-  override def visitInputRef(inputRef: RexInputRef): GeneratedExpression = {
-    // if inputRef index is within size of input1 we work with input1, input2 otherwise
-    val input = if (inputRef.getIndex < input1.getArity) {
-      (input1, input1Term, input1PojoFieldMapping)
-    } else {
-      (input2.getOrElse(throw new CodeGenException("Invalid input access.")),
-        input2Term,
-        input2PojoFieldMapping)
-    }
-
-    val index = if (input._2 == input1Term) {
-      inputRef.getIndex
-    } else {
-      inputRef.getIndex - input1.getArity
-    }
-
-    generateInputAccess(input._1, input._2, index, input._3)
-  }
-
-  override def visitFieldAccess(rexFieldAccess: RexFieldAccess): GeneratedExpression = {
-    val refExpr = rexFieldAccess.getReferenceExpr.accept(this)
-    val index = rexFieldAccess.getField.getIndex
-    val fieldAccessExpr = generateFieldAccess(
-      refExpr.resultType,
-      refExpr.resultTerm,
-      index,
-      input1PojoFieldMapping)
-
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldAccessExpr.resultType)
-    val defaultValue = primitiveDefaultValue(fieldAccessExpr.resultType)
-    val resultCode = if (nullCheck) {
-      s"""
-        |${refExpr.code}
-        |$resultTypeTerm $resultTerm;
-        |boolean $nullTerm;
-        |if (${refExpr.nullTerm}) {
-        |  $resultTerm = $defaultValue;
-        |  $nullTerm = true;
-        |}
-        |else {
-        |  ${fieldAccessExpr.code}
-        |  $resultTerm = ${fieldAccessExpr.resultTerm};
-        |  $nullTerm = ${fieldAccessExpr.nullTerm};
-        |}
-        |""".stripMargin
-    } else {
-      s"""
-        |${refExpr.code}
-        |${fieldAccessExpr.code}
-        |$resultTypeTerm $resultTerm = ${fieldAccessExpr.resultTerm};
-        |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, resultCode, fieldAccessExpr.resultType)
-  }
-
-  override def visitLiteral(literal: RexLiteral): GeneratedExpression = {
-    val resultType = FlinkTypeFactory.toTypeInfo(literal.getType)
-    val value = literal.getValue3
-    // null value with type
-    if (value == null) {
-      return generateNullLiteral(resultType)
-    }
-    // non-null values
-    literal.getType.getSqlTypeName match {
-
-      case BOOLEAN =>
-        generateNonNullLiteral(resultType, literal.getValue3.toString)
-
-      case TINYINT =>
-        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
-        if (decimal.isValidByte) {
-          generateNonNullLiteral(resultType, decimal.byteValue().toString)
-        }
-        else {
-          throw new CodeGenException("Decimal can not be converted to byte.")
-        }
-
-      case SMALLINT =>
-        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
-        if (decimal.isValidShort) {
-          generateNonNullLiteral(resultType, decimal.shortValue().toString)
-        }
-        else {
-          throw new CodeGenException("Decimal can not be converted to short.")
-        }
-
-      case INTEGER =>
-        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
-        if (decimal.isValidInt) {
-          generateNonNullLiteral(resultType, decimal.intValue().toString)
-        }
-        else {
-          throw new CodeGenException("Decimal can not be converted to integer.")
-        }
-
-      case BIGINT =>
-        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
-        if (decimal.isValidLong) {
-          generateNonNullLiteral(resultType, decimal.longValue().toString + "L")
-        }
-        else {
-          throw new CodeGenException("Decimal can not be converted to long.")
-        }
-
-      case FLOAT =>
-        val floatValue = value.asInstanceOf[JBigDecimal].floatValue()
-        floatValue match {
-          case Float.NaN => generateNonNullLiteral(resultType, "java.lang.Float.NaN")
-          case Float.NegativeInfinity =>
-            generateNonNullLiteral(resultType, "java.lang.Float.NEGATIVE_INFINITY")
-          case Float.PositiveInfinity =>
-            generateNonNullLiteral(resultType, "java.lang.Float.POSITIVE_INFINITY")
-          case _ => generateNonNullLiteral(resultType, floatValue.toString + "f")
-        }
-
-      case DOUBLE =>
-        val doubleValue = value.asInstanceOf[JBigDecimal].doubleValue()
-        doubleValue match {
-          case Double.NaN => generateNonNullLiteral(resultType, "java.lang.Double.NaN")
-          case Double.NegativeInfinity =>
-            generateNonNullLiteral(resultType, "java.lang.Double.NEGATIVE_INFINITY")
-          case Double.PositiveInfinity =>
-            generateNonNullLiteral(resultType, "java.lang.Double.POSITIVE_INFINITY")
-          case _ => generateNonNullLiteral(resultType, doubleValue.toString + "d")
-        }
-      case DECIMAL =>
-        val decimalField = addReusableDecimal(value.asInstanceOf[JBigDecimal])
-        generateNonNullLiteral(resultType, decimalField)
-
-      case VARCHAR | CHAR =>
-        generateNonNullLiteral(resultType, "\"" + value.toString + "\"")
-
-      case SYMBOL =>
-        generateSymbol(value.asInstanceOf[Enum[_]])
-
-      case DATE =>
-        generateNonNullLiteral(resultType, value.toString)
-
-      case TIME =>
-        generateNonNullLiteral(resultType, value.toString)
-
-      case TIMESTAMP =>
-        generateNonNullLiteral(resultType, value.toString + "L")
-
-      case typeName if YEAR_INTERVAL_TYPES.contains(typeName) =>
-        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
-        if (decimal.isValidInt) {
-          generateNonNullLiteral(resultType, decimal.intValue().toString)
-        } else {
-          throw new CodeGenException("Decimal can not be converted to interval of months.")
-        }
-
-      case typeName if DAY_INTERVAL_TYPES.contains(typeName) =>
-        val decimal = BigDecimal(value.asInstanceOf[JBigDecimal])
-        if (decimal.isValidLong) {
-          generateNonNullLiteral(resultType, decimal.longValue().toString + "L")
-        } else {
-          throw new CodeGenException("Decimal can not be converted to interval of milliseconds.")
-        }
-
-      case t@_ =>
-        throw new CodeGenException(s"Type not supported: $t")
-    }
-  }
-
-  override def visitCorrelVariable(correlVariable: RexCorrelVariable): GeneratedExpression = {
-    GeneratedExpression(input1Term, NEVER_NULL, NO_CODE, input1)
-  }
-
-  override def visitLocalRef(localRef: RexLocalRef): GeneratedExpression =
-    throw new CodeGenException("Local variables are not supported yet.")
-
-  override def visitRangeRef(rangeRef: RexRangeRef): GeneratedExpression =
-    throw new CodeGenException("Range references are not supported yet.")
-
-  override def visitDynamicParam(dynamicParam: RexDynamicParam): GeneratedExpression =
-    throw new CodeGenException("Dynamic parameter references are not supported yet.")
-
-  override def visitCall(call: RexCall): GeneratedExpression = {
-    val operands = call.getOperands.map(_.accept(this))
-    val resultType = FlinkTypeFactory.toTypeInfo(call.getType)
-
-    call.getOperator match {
-      // arithmetic
-      case PLUS if isNumeric(resultType) =>
-        val left = operands.head
-        val right = operands(1)
-        requireNumeric(left)
-        requireNumeric(right)
-        generateArithmeticOperator("+", nullCheck, resultType, left, right)
-
-      case PLUS | DATETIME_PLUS if isTemporal(resultType) =>
-        val left = operands.head
-        val right = operands(1)
-        requireTemporal(left)
-        requireTemporal(right)
-        generateTemporalPlusMinus(plus = true, nullCheck, left, right)
-
-      case MINUS if isNumeric(resultType) =>
-        val left = operands.head
-        val right = operands(1)
-        requireNumeric(left)
-        requireNumeric(right)
-        generateArithmeticOperator("-", nullCheck, resultType, left, right)
-
-      case MINUS | MINUS_DATE if isTemporal(resultType) =>
-        val left = operands.head
-        val right = operands(1)
-        requireTemporal(left)
-        requireTemporal(right)
-        generateTemporalPlusMinus(plus = false, nullCheck, left, right)
-
-      case MULTIPLY if isNumeric(resultType) =>
-        val left = operands.head
-        val right = operands(1)
-        requireNumeric(left)
-        requireNumeric(right)
-        generateArithmeticOperator("*", nullCheck, resultType, left, right)
-
-      case DIVIDE | DIVIDE_INTEGER if isNumeric(resultType) =>
-        val left = operands.head
-        val right = operands(1)
-        requireNumeric(left)
-        requireNumeric(right)
-        generateArithmeticOperator("/", nullCheck, resultType, left, right)
-
-      case MOD if isNumeric(resultType) =>
-        val left = operands.head
-        val right = operands(1)
-        requireNumeric(left)
-        requireNumeric(right)
-        generateArithmeticOperator("%", nullCheck, resultType, left, right)
-
-      case UNARY_MINUS if isNumeric(resultType) =>
-        val operand = operands.head
-        requireNumeric(operand)
-        generateUnaryArithmeticOperator("-", nullCheck, resultType, operand)
-
-      case UNARY_MINUS if isTimeInterval(resultType) =>
-        val operand = operands.head
-        requireTimeInterval(operand)
-        generateUnaryIntervalPlusMinus(plus = false, nullCheck, operand)
-
-      case UNARY_PLUS if isNumeric(resultType) =>
-        val operand = operands.head
-        requireNumeric(operand)
-        generateUnaryArithmeticOperator("+", nullCheck, resultType, operand)
-
-      case UNARY_PLUS if isTimeInterval(resultType) =>
-        val operand = operands.head
-        requireTimeInterval(operand)
-        generateUnaryIntervalPlusMinus(plus = true, nullCheck, operand)
-
-      // comparison
-      case EQUALS =>
-        val left = operands.head
-        val right = operands(1)
-        generateEquals(nullCheck, left, right)
-
-      case NOT_EQUALS =>
-        val left = operands.head
-        val right = operands(1)
-        generateNotEquals(nullCheck, left, right)
-
-      case GREATER_THAN =>
-        val left = operands.head
-        val right = operands(1)
-        requireComparable(left)
-        requireComparable(right)
-        generateComparison(">", nullCheck, left, right)
-
-      case GREATER_THAN_OR_EQUAL =>
-        val left = operands.head
-        val right = operands(1)
-        requireComparable(left)
-        requireComparable(right)
-        generateComparison(">=", nullCheck, left, right)
-
-      case LESS_THAN =>
-        val left = operands.head
-        val right = operands(1)
-        requireComparable(left)
-        requireComparable(right)
-        generateComparison("<", nullCheck, left, right)
-
-      case LESS_THAN_OR_EQUAL =>
-        val left = operands.head
-        val right = operands(1)
-        requireComparable(left)
-        requireComparable(right)
-        generateComparison("<=", nullCheck, left, right)
-
-      case IS_NULL =>
-        val operand = operands.head
-        generateIsNull(nullCheck, operand)
-
-      case IS_NOT_NULL =>
-        val operand = operands.head
-        generateIsNotNull(nullCheck, operand)
-
-      // logic
-      case AND =>
-        operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) =>
-          requireBoolean(left)
-          requireBoolean(right)
-          generateAnd(nullCheck, left, right)
-        }
-
-      case OR =>
-        operands.reduceLeft { (left: GeneratedExpression, right: GeneratedExpression) =>
-          requireBoolean(left)
-          requireBoolean(right)
-          generateOr(nullCheck, left, right)
-        }
-
-      case NOT =>
-        val operand = operands.head
-        requireBoolean(operand)
-        generateNot(nullCheck, operand)
-
-      case CASE =>
-        generateIfElse(nullCheck, operands, resultType)
-
-      case IS_TRUE =>
-        val operand = operands.head
-        requireBoolean(operand)
-        generateIsTrue(operand)
-
-      case IS_NOT_TRUE =>
-        val operand = operands.head
-        requireBoolean(operand)
-        generateIsNotTrue(operand)
-
-      case IS_FALSE =>
-        val operand = operands.head
-        requireBoolean(operand)
-        generateIsFalse(operand)
-
-      case IS_NOT_FALSE =>
-        val operand = operands.head
-        requireBoolean(operand)
-        generateIsNotFalse(operand)
-
-      // casting
-      case CAST | REINTERPRET =>
-        val operand = operands.head
-        generateCast(nullCheck, operand, resultType)
-
-      // as / renaming
-      case AS =>
-        operands.head
-
-      // string arithmetic
-      case CONCAT =>
-        val left = operands.head
-        val right = operands(1)
-        requireString(left)
-        generateArithmeticOperator("+", nullCheck, resultType, left, right)
-
-      // arrays
-      case ARRAY_VALUE_CONSTRUCTOR =>
-        generateArray(this, resultType, operands)
-
-      case ITEM =>
-        val array = operands.head
-        val index = operands(1)
-        requireArray(array)
-        requireInteger(index)
-        generateArrayElementAt(this, array, index)
-
-      case CARDINALITY =>
-        val array = operands.head
-        requireArray(array)
-        generateArrayCardinality(nullCheck, array)
-
-      case ELEMENT =>
-        val array = operands.head
-        requireArray(array)
-        generateArrayElement(this, array)
-
-      // advanced scalar functions
-      case sqlOperator: SqlOperator =>
-        val callGen = FunctionGenerator.getCallGenerator(
-          sqlOperator,
-          operands.map(_.resultType),
-          resultType)
-        callGen
-          .getOrElse(throw new CodeGenException(s"Unsupported call: $sqlOperator \n" +
-            s"If you think this function should be supported, " +
-            s"you can create an issue and start a discussion for it."))
-          .generate(this, operands)
-
-      // unknown or invalid
-      case call@_ =>
-        throw new CodeGenException(s"Unsupported call: $call")
-    }
-  }
-
-  override def visitOver(over: RexOver): GeneratedExpression =
-    throw new CodeGenException("Aggregate functions over windows are not supported yet.")
-
-  override def visitSubQuery(subQuery: RexSubQuery): GeneratedExpression =
-    throw new CodeGenException("Subqueries are not supported yet.")
-
-  // ----------------------------------------------------------------------------------------------
-  // generator helping methods
-  // ----------------------------------------------------------------------------------------------
-
-  private def generateInputAccess(
-      inputType: TypeInformation[Any],
-      inputTerm: String,
-      index: Int,
-      pojoFieldMapping: Option[Array[Int]])
-    : GeneratedExpression = {
-    // if input has been used before, we can reuse the code that
-    // has already been generated
-    val inputExpr = reusableInputUnboxingExprs.get((inputTerm, index)) match {
-      // input access and unboxing has already been generated
-      case Some(expr) =>
-        expr
-
-      // generate input access and unboxing if necessary
-      case None =>
-        val expr = if (nullableInput) {
-          generateNullableInputFieldAccess(inputType, inputTerm, index, pojoFieldMapping)
-        } else {
-          generateFieldAccess(inputType, inputTerm, index, pojoFieldMapping)
-        }
-
-        reusableInputUnboxingExprs((inputTerm, index)) = expr
-        expr
-    }
-    // hide the generated code as it will be executed only once
-    GeneratedExpression(inputExpr.resultTerm, inputExpr.nullTerm, "", inputExpr.resultType)
-  }
-
-  private def generateNullableInputFieldAccess(
-      inputType: TypeInformation[Any],
-      inputTerm: String,
-      index: Int,
-      pojoFieldMapping: Option[Array[Int]])
-    : GeneratedExpression = {
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-
-    val fieldType = inputType match {
-      case ct: CompositeType[_] =>
-        val fieldIndex = if (ct.isInstanceOf[PojoTypeInfo[_]]) {
-          pojoFieldMapping.get(index)
-        }
-        else {
-          index
-        }
-        ct.getTypeAt(fieldIndex)
-      case at: AtomicType[_] => at
-      case _ => throw new CodeGenException("Unsupported type for input field access.")
-    }
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType)
-    val defaultValue = primitiveDefaultValue(fieldType)
-    val fieldAccessExpr = generateFieldAccess(inputType, inputTerm, index, pojoFieldMapping)
-
-    val inputCheckCode =
-      s"""
-        |$resultTypeTerm $resultTerm;
-        |boolean $nullTerm;
-        |if ($inputTerm == null) {
-        |  $resultTerm = $defaultValue;
-        |  $nullTerm = true;
-        |}
-        |else {
-        |  ${fieldAccessExpr.code}
-        |  $resultTerm = ${fieldAccessExpr.resultTerm};
-        |  $nullTerm = ${fieldAccessExpr.nullTerm};
-        |}
-        |""".stripMargin
-
-    GeneratedExpression(resultTerm, nullTerm, inputCheckCode, fieldType)
-  }
-
-  private def generateFieldAccess(
-      inputType: TypeInformation[_],
-      inputTerm: String,
-      index: Int,
-      pojoFieldMapping: Option[Array[Int]])
-    : GeneratedExpression = {
-    inputType match {
-      case ct: CompositeType[_] =>
-        val fieldIndex = if (ct.isInstanceOf[PojoTypeInfo[_]] && pojoFieldMapping.nonEmpty) {
-          pojoFieldMapping.get(index)
-        }
-        else {
-          index
-        }
-        val accessor = fieldAccessorFor(ct, fieldIndex)
-        val fieldType: TypeInformation[Any] = ct.getTypeAt(fieldIndex)
-        val fieldTypeTerm = boxedTypeTermForTypeInfo(fieldType)
-
-        accessor match {
-          case ObjectFieldAccessor(field) =>
-            // primitive
-            if (isFieldPrimitive(field)) {
-              generateNonNullLiteral(fieldType, s"$inputTerm.${field.getName}")
-            }
-            // Object
-            else {
-              generateInputFieldUnboxing(
-                fieldType,
-                s"($fieldTypeTerm) $inputTerm.${field.getName}")
-            }
-
-          case ObjectGenericFieldAccessor(fieldName) =>
-            // Object
-            val inputCode = s"($fieldTypeTerm) $inputTerm.$fieldName"
-            generateInputFieldUnboxing(fieldType, inputCode)
-
-          case ObjectMethodAccessor(methodName) =>
-            // Object
-            val inputCode = s"($fieldTypeTerm) $inputTerm.$methodName()"
-            generateInputFieldUnboxing(fieldType, inputCode)
-
-          case ProductAccessor(i) =>
-            // Object
-            val inputCode = s"($fieldTypeTerm) $inputTerm.getField($i)"
-            generateInputFieldUnboxing(fieldType, inputCode)
-
-          case ObjectPrivateFieldAccessor(field) =>
-            val fieldTerm = addReusablePrivateFieldAccess(ct.getTypeClass, field.getName)
-            val reflectiveAccessCode = reflectiveFieldReadAccess(fieldTerm, field, inputTerm)
-            // primitive
-            if (isFieldPrimitive(field)) {
-              generateNonNullLiteral(fieldType, reflectiveAccessCode)
-            }
-            // Object
-            else {
-              generateInputFieldUnboxing(fieldType, reflectiveAccessCode)
-            }
-        }
-
-      case at: AtomicType[_] =>
-        val fieldTypeTerm = boxedTypeTermForTypeInfo(at)
-        val inputCode = s"($fieldTypeTerm) $inputTerm"
-        generateInputFieldUnboxing(at, inputCode)
-
-      case _ =>
-        throw new CodeGenException("Unsupported type for input field access.")
-    }
-  }
-
-  private def generateNullLiteral(resultType: TypeInformation[_]): GeneratedExpression = {
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(resultType)
-    val defaultValue = primitiveDefaultValue(resultType)
-
-    if (nullCheck) {
-      val wrappedCode = s"""
-        |$resultTypeTerm $resultTerm = $defaultValue;
-        |boolean $nullTerm = true;
-        |""".stripMargin
-      GeneratedExpression(resultTerm, nullTerm, wrappedCode, resultType)
-    } else {
-      throw new CodeGenException("Null literals are not allowed if nullCheck is disabled.")
-    }
-  }
-
-  private[flink] def generateNonNullLiteral(
-      literalType: TypeInformation[_],
-      literalCode: String)
-    : GeneratedExpression = {
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(literalType)
-
-    val resultCode = if (nullCheck) {
-      s"""
-        |$resultTypeTerm $resultTerm = $literalCode;
-        |boolean $nullTerm = false;
-        |""".stripMargin
-    } else {
-      s"""
-        |$resultTypeTerm $resultTerm = $literalCode;
-        |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, resultCode, literalType)
-  }
-
-  private[flink] def generateSymbol(enum: Enum[_]): GeneratedExpression = {
-    GeneratedExpression(
-      qualifyEnum(enum),
-      "false",
-      "",
-      new GenericTypeInfo(enum.getDeclaringClass))
-  }
-
-  /**
-    * Converts the external boxed format to an internal mostly primitive field representation.
-    * Wrapper types can autoboxed to their corresponding primitive type (Integer -> int). External
-    * objects are converted to their internal representation (Timestamp -> internal timestamp
-    * in long).
-    *
-    * @param fieldType type of field
-    * @param fieldTerm expression term of field to be unboxed
-    * @return internal unboxed field representation
-    */
-  private[flink] def generateInputFieldUnboxing(
-      fieldType: TypeInformation[_],
-      fieldTerm: String)
-    : GeneratedExpression = {
-    val tmpTerm = newName("tmp")
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-    val tmpTypeTerm = boxedTypeTermForTypeInfo(fieldType)
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(fieldType)
-    val defaultValue = primitiveDefaultValue(fieldType)
-
-    // explicit unboxing
-    val unboxedFieldCode = if (isTimePoint(fieldType)) {
-      timePointToInternalCode(fieldType, fieldTerm)
-    } else {
-      fieldTerm
-    }
-
-    val wrappedCode = if (nullCheck && !isReference(fieldType)) {
-      s"""
-        |$tmpTypeTerm $tmpTerm = $unboxedFieldCode;
-        |boolean $nullTerm = $tmpTerm == null;
-        |$resultTypeTerm $resultTerm;
-        |if ($nullTerm) {
-        |  $resultTerm = $defaultValue;
-        |}
-        |else {
-        |  $resultTerm = $tmpTerm;
-        |}
-        |""".stripMargin
-    } else if (nullCheck) {
-      s"""
-        |$resultTypeTerm $resultTerm = $unboxedFieldCode;
-        |boolean $nullTerm = $fieldTerm == null;
-        |""".stripMargin
-    } else {
-      s"""
-        |$resultTypeTerm $resultTerm = $unboxedFieldCode;
-        |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, wrappedCode, fieldType)
-  }
-
-  /**
-    * Converts the internal mostly primitive field representation to an external boxed format.
-    * Primitive types can autoboxed to their corresponding object type (int -> Integer). Internal
-    * representations are converted to their external objects (internal timestamp
-    * in long -> Timestamp).
-    *
-    * @param expr expression to be boxed
-    * @return external boxed field representation
-    */
-  private[flink] def generateOutputFieldBoxing(expr: GeneratedExpression): GeneratedExpression = {
-    expr.resultType match {
-      // convert internal date/time/timestamp to java.sql.* objects
-      case SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIME | SqlTimeTypeInfo.TIMESTAMP =>
-        val resultTerm = newName("result")
-        val resultTypeTerm = boxedTypeTermForTypeInfo(expr.resultType)
-        val convMethod = internalToTimePointCode(expr.resultType, expr.resultTerm)
-
-        val resultCode = if (nullCheck) {
-          s"""
-            |${expr.code}
-            |$resultTypeTerm $resultTerm;
-            |if (${expr.nullTerm}) {
-            |  $resultTerm = null;
-            |}
-            |else {
-            |  $resultTerm = $convMethod;
-            |}
-            |""".stripMargin
-        } else {
-          s"""
-            |${expr.code}
-            |$resultTypeTerm $resultTerm = $convMethod;
-            |""".stripMargin
-        }
-
-        GeneratedExpression(resultTerm, expr.nullTerm, resultCode, expr.resultType)
-
-      // other types are autoboxed or need no boxing
-      case _ => expr
-    }
-  }
-
-  // ----------------------------------------------------------------------------------------------
-  // Reusable code snippets
-  // ----------------------------------------------------------------------------------------------
-
-  /**
-    * Adds a reusable output record to the member area of the generated [[Function]].
-    * The passed [[TypeInformation]] defines the type class to be instantiated.
-    *
-    * @param ti type information of type class to be instantiated during runtime
-    * @return member variable term
-    */
-  def addReusableOutRecord(ti: TypeInformation[_]): Unit = {
-    val statement = ti match {
-      case rt: RowTypeInfo =>
-        s"""
-          |transient ${ti.getTypeClass.getCanonicalName} $outRecordTerm =
-          |    new ${ti.getTypeClass.getCanonicalName}(${rt.getArity});
-          |""".stripMargin
-      case _ =>
-        s"""
-          |${ti.getTypeClass.getCanonicalName} $outRecordTerm =
-          |    new ${ti.getTypeClass.getCanonicalName}();
-          |""".stripMargin
-    }
-    reusableMemberStatements.add(statement)
-  }
-
-  /**
-    * Adds a reusable [[java.lang.reflect.Field]] to the member area of the generated [[Function]].
-    * The field can be used for accessing POJO fields more efficiently during runtime, however,
-    * the field does not have to be public.
-    *
-    * @param clazz class of containing field
-    * @param fieldName name of field to be extracted and instantiated during runtime
-    * @return member variable term
-    */
-  def addReusablePrivateFieldAccess(clazz: Class[_], fieldName: String): String = {
-    val fieldTerm = s"field_${clazz.getCanonicalName.replace('.', '$')}_$fieldName"
-    val fieldExtraction =
-      s"""
-        |transient java.lang.reflect.Field $fieldTerm =
-        |    org.apache.flink.api.java.typeutils.TypeExtractor.getDeclaredField(
-        |      ${clazz.getCanonicalName}.class, "$fieldName");
-        |""".stripMargin
-    reusableMemberStatements.add(fieldExtraction)
-
-    val fieldAccessibility =
-      s"""
-        |$fieldTerm.setAccessible(true);
-        |""".stripMargin
-    reusableInitStatements.add(fieldAccessibility)
-
-    fieldTerm
-  }
-
-  /**
-    * Adds a reusable [[java.math.BigDecimal]] to the member area of the generated [[Function]].
-    *
-    * @param decimal decimal object to be instantiated during runtime
-    * @return member variable term
-    */
-  def addReusableDecimal(decimal: JBigDecimal): String = decimal match {
-    case JBigDecimal.ZERO => "java.math.BigDecimal.ZERO"
-    case JBigDecimal.ONE => "java.math.BigDecimal.ONE"
-    case JBigDecimal.TEN => "java.math.BigDecimal.TEN"
-    case _ =>
-      val fieldTerm = newName("decimal")
-      val fieldDecimal =
-        s"""
-          |transient java.math.BigDecimal $fieldTerm =
-          |    new java.math.BigDecimal("${decimal.toString}");
-          |""".stripMargin
-      reusableMemberStatements.add(fieldDecimal)
-      fieldTerm
-  }
-
-  /**
-    * Adds a reusable [[UserDefinedFunction]] to the member area of the generated [[Function]].
-    * The [[UserDefinedFunction]] must have a default constructor, however, it does not have
-    * to be public.
-    *
-    * @param function [[UserDefinedFunction]] object to be instantiated during runtime
-    * @return member variable term
-    */
-  def addReusableFunction(function: UserDefinedFunction): String = {
-    val classQualifier = function.getClass.getCanonicalName
-    val fieldTerm = s"function_${classQualifier.replace('.', '$')}"
-
-    val fieldFunction =
-      s"""
-        |transient $classQualifier $fieldTerm = null;
-        |""".stripMargin
-    reusableMemberStatements.add(fieldFunction)
-
-    val constructorTerm = s"constructor_${classQualifier.replace('.', '$')}"
-    val constructorAccessibility =
-      s"""
-        |java.lang.reflect.Constructor $constructorTerm =
-        |  $classQualifier.class.getDeclaredConstructor();
-        |$constructorTerm.setAccessible(true);
-        |$fieldTerm = ($classQualifier) $constructorTerm.newInstance();
-       """.stripMargin
-    reusableInitStatements.add(constructorAccessibility)
-    fieldTerm
-  }
-
-  /**
-    * Adds a reusable array to the member area of the generated [[Function]].
-    */
-  def addReusableArray(clazz: Class[_], size: Int): String = {
-    val fieldTerm = newName("array")
-    val classQualifier = clazz.getCanonicalName // works also for int[] etc.
-    val initArray = classQualifier.replaceFirst("\\[", s"[$size")
-    val fieldArray =
-      s"""
-        |transient $classQualifier $fieldTerm =
-        |    new $initArray;
-        |""".stripMargin
-    reusableMemberStatements.add(fieldArray)
-    fieldTerm
-  }
-
-  /**
-    * Adds a reusable timestamp to the beginning of the SAM of the generated [[Function]].
-    */
-  def addReusableTimestamp(): String = {
-    val fieldTerm = s"timestamp"
-
-    val field =
-      s"""
-        |final long $fieldTerm = java.lang.System.currentTimeMillis();
-        |""".stripMargin
-    reusablePerRecordStatements.add(field)
-    fieldTerm
-  }
-
-    /**
-    * Adds a reusable local timestamp to the beginning of the SAM of the generated [[Function]].
-    */
-  def addReusableLocalTimestamp(): String = {
-    val fieldTerm = s"localtimestamp"
-
-    val timestamp = addReusableTimestamp()
-
-    val field =
-      s"""
-        |final long $fieldTerm = $timestamp + java.util.TimeZone.getDefault().getOffset(timestamp);
-        |""".stripMargin
-    reusablePerRecordStatements.add(field)
-    fieldTerm
-  }
-
-  /**
-    * Adds a reusable time to the beginning of the SAM of the generated [[Function]].
-    */
-  def addReusableTime(): String = {
-    val fieldTerm = s"time"
-
-    val timestamp = addReusableTimestamp()
-
-    // adopted from org.apache.calcite.runtime.SqlFunctions.currentTime()
-    val field =
-      s"""
-        |final int $fieldTerm = (int) ($timestamp % ${DateTimeUtils.MILLIS_PER_DAY});
-        |if (time < 0) {
-        |  time += ${DateTimeUtils.MILLIS_PER_DAY};
-        |}
-        |""".stripMargin
-    reusablePerRecordStatements.add(field)
-    fieldTerm
-  }
-
-  /**
-    * Adds a reusable local time to the beginning of the SAM of the generated [[Function]].
-    */
-  def addReusableLocalTime(): String = {
-    val fieldTerm = s"localtime"
-
-    val localtimestamp = addReusableLocalTimestamp()
-
-    // adopted from org.apache.calcite.runtime.SqlFunctions.localTime()
-    val field =
-      s"""
-        |final int $fieldTerm = (int) ($localtimestamp % ${DateTimeUtils.MILLIS_PER_DAY});
-        |""".stripMargin
-    reusablePerRecordStatements.add(field)
-    fieldTerm
-  }
-
-
-  /**
-    * Adds a reusable date to the beginning of the SAM of the generated [[Function]].
-    */
-  def addReusableDate(): String = {
-    val fieldTerm = s"date"
-
-    val timestamp = addReusableTimestamp()
-    val time = addReusableTime()
-
-    // adopted from org.apache.calcite.runtime.SqlFunctions.currentDate()
-    val field =
-      s"""
-        |final int $fieldTerm = (int) ($timestamp / ${DateTimeUtils.MILLIS_PER_DAY});
-        |if ($time < 0) {
-        |  $fieldTerm -= 1;
-        |}
-        |""".stripMargin
-    reusablePerRecordStatements.add(field)
-    fieldTerm
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/Compiler.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/Compiler.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/Compiler.scala
deleted file mode 100644
index fce13ba..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/Compiler.scala
+++ /dev/null
@@ -1,41 +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.api.table.codegen
-
-import org.apache.flink.api.common.InvalidProgramException
-import org.codehaus.commons.compiler.CompileException
-import org.codehaus.janino.SimpleCompiler
-
-trait Compiler[T] {
-
-  @throws(classOf[CompileException])
-  def compile(cl: ClassLoader, name: String, code: String): Class[T] = {
-    require(cl != null, "Classloader must not be null.")
-    val compiler = new SimpleCompiler()
-    compiler.setParentClassLoader(cl)
-    try {
-      compiler.cook(code)
-    } catch {
-      case e: CompileException =>
-        throw new InvalidProgramException("Table program cannot be compiled. " +
-          "This is a bug. Please file an issue.", e)
-    }
-    compiler.getClassLoader.loadClass(name).asInstanceOf[Class[T]]
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/ExpressionReducer.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/ExpressionReducer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/ExpressionReducer.scala
deleted file mode 100644
index 871264e..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/ExpressionReducer.scala
+++ /dev/null
@@ -1,119 +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.api.table.codegen
-
-import java.util
-
-import org.apache.calcite.plan.RelOptPlanner
-import org.apache.calcite.rex.{RexBuilder, RexNode}
-import org.apache.calcite.sql.`type`.SqlTypeName
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.table.typeutils.TypeConverter
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.types.Row
-import org.apache.flink.api.table.{FlinkTypeFactory, TableConfig}
-
-import scala.collection.JavaConverters._
-
-/**
-  * Evaluates constant expressions using Flink's [[CodeGenerator]].
-  */
-class ExpressionReducer(config: TableConfig)
-  extends RelOptPlanner.Executor with Compiler[MapFunction[Row, Row]] {
-
-  private val EMPTY_ROW_INFO = TypeConverter.DEFAULT_ROW_TYPE
-  private val EMPTY_ROW = new Row(0)
-
-  override def reduce(
-    rexBuilder: RexBuilder,
-    constExprs: util.List[RexNode],
-    reducedValues: util.List[RexNode]): Unit = {
-
-    val typeFactory = rexBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-
-    val literals = constExprs.asScala.map(e => (e.getType.getSqlTypeName, e)).flatMap {
-
-      // we need to cast here for RexBuilder.makeLiteral
-      case (SqlTypeName.DATE, e) =>
-        Some(
-          rexBuilder.makeCast(typeFactory.createTypeFromTypeInfo(BasicTypeInfo.INT_TYPE_INFO), e)
-        )
-      case (SqlTypeName.TIME, e) =>
-        Some(
-          rexBuilder.makeCast(typeFactory.createTypeFromTypeInfo(BasicTypeInfo.INT_TYPE_INFO), e)
-        )
-      case (SqlTypeName.TIMESTAMP, e) =>
-        Some(
-          rexBuilder.makeCast(typeFactory.createTypeFromTypeInfo(BasicTypeInfo.LONG_TYPE_INFO), e)
-        )
-
-      // we don't support object literals yet, we skip those constant expressions
-      case (SqlTypeName.ANY, _) | (SqlTypeName.ROW, _) | (SqlTypeName.ARRAY, _) => None
-
-      case (_, e) => Some(e)
-    }
-
-    val literalTypes = literals.map(e => FlinkTypeFactory.toTypeInfo(e.getType))
-    val resultType = new RowTypeInfo(literalTypes: _*)
-
-    // generate MapFunction
-    val generator = new CodeGenerator(config, false, EMPTY_ROW_INFO)
-
-    val result = generator.generateResultExpression(
-      resultType,
-      resultType.getFieldNames,
-      literals)
-
-    val generatedFunction = generator.generateFunction[MapFunction[Row, Row]](
-      "ExpressionReducer",
-      classOf[MapFunction[Row, Row]],
-      s"""
-        |${result.code}
-        |return ${result.resultTerm};
-        |""".stripMargin,
-      resultType.asInstanceOf[TypeInformation[Any]])
-
-    val clazz = compile(getClass.getClassLoader, generatedFunction.name, generatedFunction.code)
-    val function = clazz.newInstance()
-
-    // execute
-    val reduced = function.map(EMPTY_ROW)
-
-    // add the reduced results or keep them unreduced
-    var i = 0
-    var reducedIdx = 0
-    while (i < constExprs.size()) {
-      val unreduced = constExprs.get(i)
-      unreduced.getType.getSqlTypeName match {
-        // we insert the original expression for object literals
-        case SqlTypeName.ANY | SqlTypeName.ROW | SqlTypeName.ARRAY =>
-          reducedValues.add(unreduced)
-        case _ =>
-          val literal = rexBuilder.makeLiteral(
-            reduced.getField(reducedIdx),
-            unreduced.getType,
-            true)
-          reducedValues.add(literal)
-          reducedIdx += 1
-      }
-      i += 1
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/Indenter.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/Indenter.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/Indenter.scala
deleted file mode 100644
index c7d9a2e..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/Indenter.scala
+++ /dev/null
@@ -1,54 +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.api.table.codegen
-
-class IndentStringContext(sc: StringContext) {
-  def j(args: Any*): String = {
-    val sb = new StringBuilder()
-    for ((s, a) <- sc.parts zip args) {
-      sb append s
-
-      val ind = getindent(s)
-      if (ind.nonEmpty) {
-        sb append a.toString.replaceAll("\n", "\n" + ind)
-      } else {
-        sb append a.toString
-      }
-    }
-    if (sc.parts.size > args.size) {
-      sb append sc.parts.last
-    }
-
-    sb.toString()
-  }
-
-  // get white indent after the last new line, if any
-  def getindent(str: String): String = {
-    val lastnl = str.lastIndexOf("\n")
-    if (lastnl == -1) ""
-    else {
-      val ind = str.substring(lastnl + 1)
-      if (ind.trim.isEmpty) ind  // ind is all whitespace. Use this
-      else ""
-    }
-  }
-}
-
-object Indenter {
-  implicit def toISC(sc: StringContext): IndentStringContext = new IndentStringContext(sc)
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/BuiltInMethods.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/BuiltInMethods.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/BuiltInMethods.scala
deleted file mode 100644
index c7c7477..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/BuiltInMethods.scala
+++ /dev/null
@@ -1,39 +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.api.table.codegen.calls
-
-import java.math.{BigDecimal => JBigDecimal}
-
-import org.apache.calcite.linq4j.tree.Types
-import org.apache.calcite.runtime.SqlFunctions
-import org.apache.flink.api.table.functions.utils.MathFunctions
-
-object BuiltInMethods {
-  val LOG10 = Types.lookupMethod(classOf[Math], "log10", classOf[Double])
-  val EXP = Types.lookupMethod(classOf[Math], "exp", classOf[Double])
-  val POWER = Types.lookupMethod(classOf[Math], "pow", classOf[Double], classOf[Double])
-  val POWER_DEC = Types.lookupMethod(
-    classOf[MathFunctions], "power", classOf[Double], classOf[JBigDecimal])
-  val LN = Types.lookupMethod(classOf[Math], "log", classOf[Double])
-  val ABS = Types.lookupMethod(classOf[SqlFunctions], "abs", classOf[Double])
-  val ABS_DEC = Types.lookupMethod(classOf[SqlFunctions], "abs", classOf[JBigDecimal])
-  val LIKE_WITH_ESCAPE = Types.lookupMethod(classOf[SqlFunctions], "like",
-    classOf[String], classOf[String], classOf[String])
-  val SIMILAR_WITH_ESCAPE = Types.lookupMethod(classOf[SqlFunctions], "similar",
-    classOf[String], classOf[String], classOf[String])
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CallGenerator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CallGenerator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CallGenerator.scala
deleted file mode 100644
index 96d0b74..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CallGenerator.scala
+++ /dev/null
@@ -1,74 +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.api.table.codegen.calls
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.codegen.CodeGenUtils._
-import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedExpression}
-
-trait CallGenerator {
-
-  def generate(
-      codeGenerator: CodeGenerator,
-      operands: Seq[GeneratedExpression])
-    : GeneratedExpression
-
-}
-
-object CallGenerator {
-
-  def generateCallIfArgsNotNull(
-      nullCheck: Boolean,
-      returnType: TypeInformation[_],
-      operands: Seq[GeneratedExpression])
-      (call: (Seq[String]) => String)
-    : GeneratedExpression = {
-    val resultTerm = newName("result")
-    val nullTerm = newName("isNull")
-    val resultTypeTerm = primitiveTypeTermForTypeInfo(returnType)
-    val defaultValue = primitiveDefaultValue(returnType)
-
-    val resultCode = if (nullCheck && operands.nonEmpty) {
-      s"""
-        |${operands.map(_.code).mkString("\n")}
-        |boolean $nullTerm = ${operands.map(_.nullTerm).mkString(" || ")};
-        |$resultTypeTerm $resultTerm;
-        |if ($nullTerm) {
-        |  $resultTerm = $defaultValue;
-        |}
-        |else {
-        |  $resultTerm = ${call(operands.map(_.resultTerm))};
-        |}
-        |""".stripMargin
-    } else if (nullCheck && operands.isEmpty) {
-      s"""
-        |${operands.map(_.code).mkString("\n")}
-        |boolean $nullTerm = false;
-        |$resultTypeTerm $resultTerm = ${call(operands.map(_.resultTerm))};
-        |""".stripMargin
-    } else{
-      s"""
-        |${operands.map(_.code).mkString("\n")}
-        |$resultTypeTerm $resultTerm = ${call(operands.map(_.resultTerm))};
-        |""".stripMargin
-    }
-
-    GeneratedExpression(resultTerm, nullTerm, resultCode, returnType)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala
deleted file mode 100644
index 4aaa209..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/codegen/calls/CurrentTimePointCallGen.scala
+++ /dev/null
@@ -1,58 +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.api.table.codegen.calls
-
-import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation}
-import org.apache.flink.api.table.codegen.{CodeGenerator, GeneratedExpression}
-
-/**
-  * Generates function call to determine current time point (as date/time/timestamp) in
-  * local timezone or not.
-  */
-class CurrentTimePointCallGen(
-    targetType: TypeInformation[_],
-    local: Boolean)
-  extends CallGenerator {
-
-  override def generate(
-      codeGenerator: CodeGenerator,
-      operands: Seq[GeneratedExpression])
-    : GeneratedExpression = targetType match {
-    case SqlTimeTypeInfo.TIME if local =>
-      val time = codeGenerator.addReusableLocalTime()
-      codeGenerator.generateNonNullLiteral(targetType, time)
-
-    case SqlTimeTypeInfo.TIMESTAMP if local =>
-      val timestamp = codeGenerator.addReusableLocalTimestamp()
-      codeGenerator.generateNonNullLiteral(targetType, timestamp)
-
-    case SqlTimeTypeInfo.DATE =>
-      val date = codeGenerator.addReusableDate()
-      codeGenerator.generateNonNullLiteral(targetType, date)
-
-    case SqlTimeTypeInfo.TIME =>
-      val time = codeGenerator.addReusableTime()
-      codeGenerator.generateNonNullLiteral(targetType, time)
-
-    case SqlTimeTypeInfo.TIMESTAMP =>
-      val timestamp = codeGenerator.addReusableTimestamp()
-      codeGenerator.generateNonNullLiteral(targetType, timestamp)
-  }
-
-}


[06/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
new file mode 100644
index 0000000..0b2c8fc
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/TableSourceITCase.scala
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch
+
+import java.io.{File, FileOutputStream, OutputStreamWriter}
+
+import org.apache.flink.api.common.io.GenericInputFormat
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.java.{DataSet => JavaSet, ExecutionEnvironment => JavaExecEnv}
+import org.apache.flink.api.scala.ExecutionEnvironment
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.sources.{BatchTableSource, CsvTableSource}
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class TableSourceITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testBatchTableSourceTableAPI(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    tEnv.registerTableSource("MyTestTable", new TestBatchTableSource())
+    val results = tEnv
+      .scan("MyTestTable")
+      .where('amount < 4)
+      .select('amount * 'id, 'name)
+      .collect()
+
+    val expected = Seq(
+      "0,Record_0", "0,Record_16", "0,Record_32", "1,Record_1", "17,Record_17",
+      "36,Record_18", "4,Record_2", "57,Record_19", "9,Record_3").mkString("\n")
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testBatchTableSourceSQL(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    tEnv.registerTableSource("MyTestTable", new TestBatchTableSource())
+    val results = tEnv.sql(
+      "SELECT amount * id, name FROM MyTestTable WHERE amount < 4").collect()
+
+    val expected = Seq(
+      "0,Record_0", "0,Record_16", "0,Record_32", "1,Record_1", "17,Record_17",
+      "36,Record_18", "4,Record_2", "57,Record_19", "9,Record_3").mkString("\n")
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testCsvTableSource(): Unit = {
+
+    val csvRecords = Seq(
+      "First#Id#Score#Last",
+      "Mike#1#12.3#Smith",
+      "Bob#2#45.6#Taylor",
+      "Sam#3#7.89#Miller",
+      "Peter#4#0.12#Smith",
+      "% Just a comment",
+      "Liz#5#34.5#Williams",
+      "Sally#6#6.78#Miller",
+      "Alice#7#90.1#Smith",
+      "Kelly#8#2.34#Williams"
+    )
+
+    val tempFile = File.createTempFile("csv-test", "tmp")
+    tempFile.deleteOnExit()
+    val tmpWriter = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8")
+    tmpWriter.write(csvRecords.mkString("$"))
+    tmpWriter.close()
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val csvTable = new CsvTableSource(
+      tempFile.getAbsolutePath,
+      Array("first", "id", "score", "last"),
+      Array(
+        BasicTypeInfo.STRING_TYPE_INFO,
+        BasicTypeInfo.INT_TYPE_INFO,
+        BasicTypeInfo.DOUBLE_TYPE_INFO,
+        BasicTypeInfo.STRING_TYPE_INFO
+      ),
+      fieldDelim = "#",
+      rowDelim = "$",
+      ignoreFirstLine = true,
+      ignoreComments = "%"
+    )
+
+    tEnv.registerTableSource("csvTable", csvTable)
+    val results = tEnv.sql(
+      "SELECT last, sum(score), max(id) FROM csvTable GROUP BY last").collect()
+
+    val expected = Seq(
+      "Smith,102.52,7",
+      "Taylor,45.6,2",
+      "Miller,14.67,6",
+      "Williams,36.84,8").mkString("\n")
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+}
+
+class TestBatchTableSource extends BatchTableSource[Row] {
+
+  val fieldTypes: Array[TypeInformation[_]] = Array(
+    BasicTypeInfo.STRING_TYPE_INFO,
+    BasicTypeInfo.LONG_TYPE_INFO,
+    BasicTypeInfo.INT_TYPE_INFO
+  )
+
+  /** Returns the data of the table as a [[org.apache.flink.api.java.DataSet]]. */
+  override def getDataSet(execEnv: JavaExecEnv): JavaSet[Row] = {
+    execEnv.createInput(new GeneratingInputFormat(33), getReturnType).setParallelism(1)
+  }
+
+  /** Returns the types of the table fields. */
+  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
+
+  /** Returns the names of the table fields. */
+  override def getFieldsNames: Array[String] = Array("name", "id", "amount")
+
+  /** Returns the [[TypeInformation]] for the return type. */
+  override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*)
+
+  /** Returns the number of fields of the table. */
+  override def getNumberOfFields: Int = 3
+}
+
+class GeneratingInputFormat(val num: Int) extends GenericInputFormat[Row] {
+
+  var cnt = 0L
+
+  override def reachedEnd(): Boolean = cnt >= num
+
+  override def nextRecord(reuse: Row): Row = {
+    reuse.setField(0, s"Record_$cnt")
+    reuse.setField(1, cnt)
+    reuse.setField(2, (cnt % 16).toInt)
+    cnt += 1
+    reuse
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/AggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/AggregationsITCase.scala
new file mode 100644
index 0000000..4f55bee
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/AggregationsITCase.scala
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import TableProgramsTestBase.TableConfigMode
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class AggregationsITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testAggregationTypes(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT sum(_1), min(_1), max(_1), count(_1), avg(_1) FROM MyTable"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "231,1,21,21,11"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testTableAggregation(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT sum(_1) FROM MyTable"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "231"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testDataSetAggregation(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT sum(_1) FROM MyTable"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "231"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testWorkingAggregationDataTypes(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery =
+      "SELECT avg(_1), avg(_2), avg(_3), avg(_4), avg(_5), avg(_6), count(_7), " +
+      "  sum(CAST(_6 AS DECIMAL))" +
+      "FROM MyTable"
+
+    val ds = env.fromElements(
+      (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"),
+      (2: Byte, 2: Short, 2, 2L, 2.0f, 2.0d, "Ciao"))
+    tEnv.registerDataSet("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1,1,1,1,1.5,1.5,2,3.0"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testTableWorkingAggregationDataTypes(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT avg(a), avg(b), avg(c), avg(d), avg(e), avg(f), count(g)" +
+      "FROM MyTable"
+
+    val ds = env.fromElements(
+      (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"),
+      (2: Byte, 2: Short, 2, 2L, 2.0f, 2.0d, "Ciao")).toTable(tEnv, 'a, 'b, 'c, 'd, 'e, 'f, 'g)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1,1,1,1,1.5,1.5,2"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testTableProjection(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT avg(a), sum(a), count(a), avg(b), sum(b) " +
+      "FROM MyTable"
+
+    val ds = env.fromElements((1: Byte, 1: Short), (2: Byte, 2: Short)).toTable(tEnv, 'a, 'b)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1,3,2,1,3"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testTableAggregationWithArithmetic(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT avg(a + 2) + 2, count(b) + 5 " +
+      "FROM MyTable"
+
+    val ds = env.fromElements((1f, "Hello"), (2f, "Ciao")).toTable(tEnv, 'a, 'b)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "5.5,7"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testAggregationWithTwoCount(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT count(_1), count(_2) FROM MyTable"
+
+    val ds = env.fromElements((1f, "Hello"), (2f, "Ciao")).toTable(tEnv)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "2,2"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+
+  @Test
+  def testAggregationAfterProjection(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM " +
+      "(SELECT _1 as a, _2 as b, _3 as c FROM MyTable)"
+
+    val ds = env.fromElements(
+      (1: Byte, 1: Short, 1, 1L, 1.0f, 1.0d, "Hello"),
+      (2: Byte, 2: Short, 2, 2L, 2.0f, 2.0d, "Ciao")).toTable(tEnv)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1,3,2"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testDistinctAggregate(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT sum(_1) as a, count(distinct _3) as b FROM MyTable"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds)
+
+    // must fail. distinct aggregates are not supported
+    tEnv.sql(sqlQuery).toDataSet[Row]
+  }
+
+  @Test(expected = classOf[TableException])
+  def testGroupedDistinctAggregate(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT _2, avg(distinct _1) as a, count(_3) as b FROM MyTable GROUP BY _2"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds)
+
+    // must fail. distinct aggregates are not supported
+    tEnv.sql(sqlQuery).toDataSet[Row]
+  }
+
+  @Test(expected = classOf[TableException])
+  def testGroupingSetAggregate(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT _2, _3, avg(_1) as a FROM MyTable GROUP BY GROUPING SETS (_2, _3)"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds)
+
+    // must fail. grouping sets are not supported
+    tEnv.sql(sqlQuery).toDataSet[Row]
+  }
+
+  @Test
+  def testAggregateEmptyDataSets(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT avg(a), sum(a), count(b) " +
+      "FROM MyTable where a = 4 group by a"
+
+    val sqlQuery2 = "SELECT avg(a), sum(a), count(b) " +
+      "FROM MyTable where a = 4"
+
+    val sqlQuery3 = "SELECT avg(a), sum(a), count(b) " +
+      "FROM MyTable"
+
+    val ds = env.fromElements(
+      (1: Byte, 1: Short),
+      (2: Byte, 2: Short))
+      .toTable(tEnv, 'a, 'b)
+
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+    val result2 = tEnv.sql(sqlQuery2)
+    val result3 = tEnv.sql(sqlQuery3)
+
+    val results = result.toDataSet[Row].collect()
+    val expected = Seq.empty
+    val results2 =  result2.toDataSet[Row].collect()
+    val expected2 = "null,null,0"
+    val results3 = result3.toDataSet[Row].collect()
+    val expected3 = "1,3,2"
+
+    assert(results.equals(expected),
+      "Empty result is expected for grouped set, but actual: " + results)
+    TestBaseUtils.compareResultAsText(results2.asJava, expected2)
+    TestBaseUtils.compareResultAsText(results3.asJava, expected3)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/CalcITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/CalcITCase.scala
new file mode 100644
index 0000000..f3f554b
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/CalcITCase.scala
@@ -0,0 +1,329 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.sql
+
+
+import java.sql.{Date, Time, Timestamp}
+import java.util
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.batch.sql.FilterITCase.MyHashCode
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.table.api.{TableEnvironment, ValidationException}
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.apache.flink.types.Row
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class CalcITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testSelectStarFromTable(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
+      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
+      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
+
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testSelectStarFromDataSet(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
+      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
+      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
+
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testSimpleSelectAll(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT a, b, c FROM MyTable"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" +
+      "4,3,Hello world, how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" +
+      "7,4,Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" +
+      "11,5,Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" +
+      "15,5,Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" +
+      "19,6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
+
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testSelectWithNaming(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT _1 as a, _2 as b FROM MyTable"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1,1\n" + "2,2\n" + "3,2\n" + "4,3\n" + "5,3\n" + "6,3\n" + "7,4\n" +
+      "8,4\n" + "9,4\n" + "10,4\n" + "11,5\n" + "12,5\n" + "13,5\n" + "14,5\n" + "15,5\n" +
+      "16,6\n" + "17,6\n" + "18,6\n" + "19,6\n" + "20,6\n" + "21,6\n"
+
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidFields(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT a, foo FROM MyTable"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", ds)
+
+    tEnv.sql(sqlQuery)
+  }
+
+  @Test
+  def testAllRejectingFilter(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable WHERE false"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testAllPassingFilter(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable WHERE true"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world\n" + "4,3,Hello world, " +
+      "how are you?\n" + "5,3,I am fine.\n" + "6,3,Luke Skywalker\n" + "7,4," +
+      "Comment#1\n" + "8,4,Comment#2\n" + "9,4,Comment#3\n" + "10,4,Comment#4\n" + "11,5," +
+      "Comment#5\n" + "12,5,Comment#6\n" + "13,5,Comment#7\n" + "14,5,Comment#8\n" + "15,5," +
+      "Comment#9\n" + "16,6,Comment#10\n" + "17,6,Comment#11\n" + "18,6,Comment#12\n" + "19," +
+      "6,Comment#13\n" + "20,6,Comment#14\n" + "21,6,Comment#15\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testFilterOnString(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable WHERE c LIKE '%world%'"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "3,2,Hello world\n" + "4,3,Hello world, how are you?\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testFilterOnInteger(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable WHERE MOD(a,2)=0"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "2,2,Hello\n" + "4,3,Hello world, how are you?\n" +
+      "6,3,Luke Skywalker\n" + "8,4," + "Comment#2\n" + "10,4,Comment#4\n" +
+      "12,5,Comment#6\n" + "14,5,Comment#8\n" + "16,6," +
+      "Comment#10\n" + "18,6,Comment#12\n" + "20,6,Comment#14\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testDisjunctivePredicate(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable WHERE a < 2 OR a > 20"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1,1,Hi\n" + "21,6,Comment#15\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testFilterWithAnd(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable WHERE MOD(a,2)<>0 AND MOD(b,2)=0"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "3,2,Hello world\n" + "7,4,Comment#1\n" +
+      "9,4,Comment#3\n" + "17,6,Comment#11\n" +
+      "19,6,Comment#13\n" + "21,6,Comment#15\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testAdvancedDataTypes(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT a, b, c, DATE '1984-07-12', TIME '14:34:24', " +
+      "TIMESTAMP '1984-07-12 14:34:24' FROM MyTable"
+
+    val ds = env.fromElements((
+      Date.valueOf("1984-07-12"),
+      Time.valueOf("14:34:24"),
+      Timestamp.valueOf("1984-07-12 14:34:24")))
+    tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1984-07-12,14:34:24,1984-07-12 14:34:24.0," +
+      "1984-07-12,14:34:24,1984-07-12 14:34:24.0"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testUserDefinedScalarFunction(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    tEnv.registerFunction("hashCode",
+      new org.apache.flink.table.api.java.batch.table.CalcITCase.OldHashCode)
+    tEnv.registerFunction("hashCode", MyHashCode)
+
+    val ds = env.fromElements("a", "b", "c")
+    tEnv.registerDataSet("MyTable", ds, 'text)
+
+    val result = tEnv.sql("SELECT hashCode(text) FROM MyTable")
+
+    val expected = "97\n98\n99"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+}
+
+object FilterITCase {
+  object MyHashCode extends ScalarFunction {
+    def eval(s: String): Int = s.hashCode()
+  }
+}
+
+object CalcITCase {
+
+  @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
+  def parameters(): util.Collection[Array[java.lang.Object]] = {
+    Seq[Array[AnyRef]](
+      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.DEFAULT),
+      Array(TestExecutionMode.COLLECTION, TableProgramsTestBase.NO_NULL)).asJava
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/JoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/JoinITCase.scala
new file mode 100644
index 0000000..344428b
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/JoinITCase.scala
@@ -0,0 +1,366 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.table.api.{TableEnvironment, TableException, ValidationException}
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.apache.flink.types.Row
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class JoinITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testJoin(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE b = e"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testJoinWithFilter(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE b = e AND b < 2"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hi,Hallo\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testJoinWithJoinFilter(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE b = e AND a < 6 AND h < b"
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hello world, how are you?,Hallo Welt wie\n" +
+      "I am fine.,Hallo Welt wie\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testJoinWithMultipleKeys(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE a = d AND b = h"
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt wie gehts?\n" +
+      "Hello world,ABC\n" + "I am fine.,HIJ\n" + "I am fine.,IJK\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testJoinNonExistingKey(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE foo = e"
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    tEnv.sql(sqlQuery)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testJoinNonMatchingKeyTypes(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE a = g"
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    tEnv.sql(sqlQuery).toDataSet[Row].collect()
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testJoinWithAmbiguousFields(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE a = d"
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'c)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    tEnv.sql(sqlQuery)
+  }
+
+  @Test
+  def testJoinWithAlias(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT Table5.c, Table3.c FROM Table3, Table5 WHERE a = d AND a < 4"
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'c)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    val result = tEnv.sql(sqlQuery)
+    val expected = "1,Hi\n" + "2,Hello\n" + "1,Hello\n" +
+      "2,Hello world\n" + "2,Hello world\n" + "3,Hello world\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testJoinNoEqualityPredicate(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c, g FROM Table3, Table5 WHERE d = f"
+
+    val ds1 = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    tEnv.sql(sqlQuery).toDataSet[Row].collect()
+  }
+
+  @Test
+  def testDataSetJoinWithAggregation(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT COUNT(g), COUNT(b) FROM Table3, Table5 WHERE a = d"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env)
+    tEnv.registerDataSet("Table3", ds1, 'a, 'b, 'c)
+    tEnv.registerDataSet("Table5", ds2, 'd, 'e, 'f, 'g, 'h)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "6,6"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testTableJoinWithAggregation(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT COUNT(b), COUNT(g) FROM Table3, Table5 WHERE a = d"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv, 'd, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "6,6"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testFullOuterJoin(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    tEnv.getConfig.setNullCheck(true)
+
+    val sqlQuery = "SELECT c, g FROM Table3 FULL OUTER JOIN Table5 ON b = e"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" +
+      "null,Hallo Welt wie\n" + "null,Hallo Welt wie gehts?\n" + "null,ABC\n" + "null,BCD\n" +
+      "null,CDE\n" + "null,DEF\n" + "null,EFG\n" + "null,FGH\n" + "null,GHI\n" + "null,HIJ\n" +
+      "null,IJK\n" + "null,JKL\n" + "null,KLM"
+
+    val results = tEnv.sql(sqlQuery).toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testLeftOuterJoin(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    tEnv.getConfig.setNullCheck(true)
+
+    val sqlQuery = "SELECT c, g FROM Table5 LEFT OUTER JOIN Table3 ON b = e"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    tEnv.sql(sqlQuery).toDataSet[Row].collect()
+
+    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" +
+      "null,Hallo Welt wie\n" + "null,Hallo Welt wie gehts?\n" + "null,ABC\n" + "null,BCD\n" +
+      "null,CDE\n" + "null,DEF\n" + "null,EFG\n" + "null,FGH\n" + "null,GHI\n" + "null,HIJ\n" +
+      "null,IJK\n" + "null,JKL\n" + "null,KLM"
+    val results = tEnv.sql(sqlQuery).toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testRightOuterJoin(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+    tEnv.getConfig.setNullCheck(true)
+
+    val sqlQuery = "SELECT c, g FROM Table3 RIGHT OUTER JOIN Table5 ON b = e"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env).toTable(tEnv).as('d, 'e, 'f, 'g, 'h)
+    tEnv.registerTable("Table3", ds1)
+    tEnv.registerTable("Table5", ds2)
+
+    tEnv.sql(sqlQuery).toDataSet[Row].collect()
+
+    val expected = "Hi,Hallo\n" + "Hello,Hallo Welt\n" + "Hello world,Hallo Welt\n" +
+      "null,Hallo Welt wie\n" + "null,Hallo Welt wie gehts?\n" + "null,ABC\n" + "null,BCD\n" +
+      "null,CDE\n" + "null,DEF\n" + "null,EFG\n" + "null,FGH\n" + "null,GHI\n" + "null,HIJ\n" +
+      "null,IJK\n" + "null,JKL\n" + "null,KLM"
+    val results = tEnv.sql(sqlQuery).toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testCrossJoin(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val table1 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a1, 'a2, 'a3)
+    val table2 = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('b1, 'b2, 'b3)
+    tEnv.registerTable("A", table1)
+    tEnv.registerTable("B", table2)
+
+    val sqlQuery = "SELECT a1, b1 FROM A CROSS JOIN B"
+    tEnv.sql(sqlQuery).count
+  }
+
+  @Test
+  def testCrossJoinWithLeftSingleRowInput(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val table = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a1, 'a2, 'a3)
+    tEnv.registerTable("A", table)
+
+    val sqlQuery2 = "SELECT * FROM (SELECT count(*) FROM A) CROSS JOIN A"
+    val expected =
+      "3,1,1,Hi\n" +
+      "3,2,2,Hello\n" +
+      "3,3,2,Hello world"
+    val result = tEnv.sql(sqlQuery2).collect()
+    TestBaseUtils.compareResultAsText(result.asJava, expected)
+  }
+
+  @Test
+  def testCrossJoinWithRightSingleRowInput(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val table = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a1, 'a2, 'a3)
+    tEnv.registerTable("A", table)
+
+    val sqlQuery1 = "SELECT * FROM A CROSS JOIN (SELECT count(*) FROM A)"
+    val expected =
+      "1,1,Hi,3\n" +
+      "2,2,Hello,3\n" +
+      "3,2,Hello world,3"
+    val result = tEnv.sql(sqlQuery1).collect()
+    TestBaseUtils.compareResultAsText(result.asJava, expected)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SetOperatorsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SetOperatorsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SetOperatorsITCase.scala
new file mode 100644
index 0000000..cc44e7a
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SetOperatorsITCase.scala
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.util.Random
+
+@RunWith(classOf[Parameterized])
+class SetOperatorsITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testUnionAll(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c FROM t1 UNION ALL (SELECT f FROM t2)"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env)
+    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
+    tEnv.registerDataSet("t2", ds2, 'd, 'e, 'f)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hi\n" + "Hello\n" + "Hello world\n" + "Hi\n" + "Hello\n" + "Hello world\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testUnion(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c FROM t1 UNION (SELECT f FROM t2)"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.getSmall3TupleDataSet(env)
+    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
+    tEnv.registerDataSet("t2", ds2, 'd, 'e, 'f)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hi\n" + "Hello\n" + "Hello world\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testUnionWithFilter(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c FROM (" +
+      "SELECT * FROM t1 UNION ALL (SELECT a, b, c FROM t2))" +
+      "WHERE b < 2"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env)
+    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
+    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'd, 'c, 'e)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hi\n" + "Hallo\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testUnionWithAggregation(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT count(c) FROM (" +
+      "SELECT * FROM t1 UNION ALL (SELECT a, b, c FROM t2))"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env)
+    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
+    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'd, 'c, 'e)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "18"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testExcept(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c FROM t1 EXCEPT (SELECT c FROM t2)"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = env.fromElements((1, 1L, "Hi"))
+    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
+    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'c)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hello\n" + "Hello world\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  @Ignore
+  // calcite sql parser doesn't support EXCEPT ALL
+  def testExceptAll(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c FROM t1 EXCEPT ALL SELECT c FROM t2"
+
+    val data1 = new mutable.MutableList[Int]
+    data1 += (1, 1, 1, 2, 2)
+    val data2 = new mutable.MutableList[Int]
+    data2 += (1, 2, 2, 3)
+    val ds1 = env.fromCollection(data1)
+    val ds2 = env.fromCollection(data2)
+
+    tEnv.registerDataSet("t1", ds1, 'c)
+    tEnv.registerDataSet("t2", ds2, 'c)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1\n1"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testExceptWithFilter(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c FROM (" +
+      "SELECT * FROM t1 EXCEPT (SELECT a, b, c FROM t2))" +
+      "WHERE b < 2"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.get5TupleDataSet(env)
+    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
+    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'd, 'c, 'e)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hi\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testIntersect(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c FROM t1 INTERSECT SELECT c FROM t2"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val data = new mutable.MutableList[(Int, Long, String)]
+    data.+=((1, 1L, "Hi"))
+    data.+=((2, 2L, "Hello"))
+    data.+=((2, 2L, "Hello"))
+    data.+=((3, 2L, "Hello world!"))
+    val ds2 = env.fromCollection(Random.shuffle(data))
+
+    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
+    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'c)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hi\n" + "Hello\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  @Ignore
+  // calcite sql parser doesn't support INTERSECT ALL
+  def testIntersectAll(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c FROM t1 INTERSECT ALL SELECT c FROM t2"
+
+    val data1 = new mutable.MutableList[Int]
+    data1 += (1, 1, 1, 2, 2)
+    val data2 = new mutable.MutableList[Int]
+    data2 += (1, 2, 2, 3)
+    val ds1 = env.fromCollection(data1)
+    val ds2 = env.fromCollection(data2)
+
+    tEnv.registerDataSet("t1", ds1, 'c)
+    tEnv.registerDataSet("t2", ds2, 'c)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "1\n2\n2"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testIntersectWithFilter(): Unit = {
+    val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT c FROM ((SELECT * FROM t1) INTERSECT (SELECT * FROM t2)) WHERE a > 1"
+
+    val ds1 = CollectionDataSets.getSmall3TupleDataSet(env)
+    val ds2 = CollectionDataSets.get3TupleDataSet(env)
+
+    tEnv.registerDataSet("t1", ds1, 'a, 'b, 'c)
+    tEnv.registerDataSet("t2", ds2, 'a, 'b, 'c)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hello\n" + "Hello world\n"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SetOperatorsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SetOperatorsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SetOperatorsTest.scala
new file mode 100644
index 0000000..6c07c6e
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SetOperatorsTest.scala
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+class SetOperatorsTest extends TableTestBase {
+
+  @Test
+  def testExists(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Long, Int, String)]("A", 'a_long, 'a_int, 'a_string)
+    util.addTable[(Long, Int, String)]("B", 'b_long, 'b_int, 'b_string)
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      binaryNode(
+        "DataSetJoin",
+        batchTableNode(0),
+        unaryNode(
+          "DataSetAggregate",
+          unaryNode(
+            "DataSetCalc",
+            binaryNode(
+              "DataSetJoin",
+              unaryNode(
+                "DataSetCalc",
+                batchTableNode(1),
+                term("select", "b_long")
+              ),
+              unaryNode(
+                "DataSetAggregate",
+                unaryNode(
+                  "DataSetCalc",
+                  batchTableNode(0),
+                  term("select", "a_long")
+                ),
+                term("groupBy", "a_long"),
+                term("select", "a_long")
+              ),
+              term("where", "=(a_long, b_long)"),
+              term("join", "b_long", "a_long"),
+              term("joinType", "InnerJoin")
+            ),
+            term("select", "true AS $f0", "a_long")
+          ),
+          term("groupBy", "a_long"),
+          term("select", "a_long", "MIN($f0) AS $f1")
+        ),
+        term("where", "=(a_long, a_long0)"),
+        term("join", "a_long", "a_int", "a_string", "a_long0", "$f1"),
+        term("joinType", "InnerJoin")
+      ),
+      term("select", "a_int", "a_string")
+    )
+
+    util.verifySql(
+      "SELECT a_int, a_string FROM A WHERE EXISTS(SELECT * FROM B WHERE a_long = b_long)",
+      expected
+    )
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SingleRowJoinTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SingleRowJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SingleRowJoinTest.scala
new file mode 100644
index 0000000..ecc685d
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SingleRowJoinTest.scala
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+class SingleRowJoinTest extends TableTestBase {
+
+  @Test
+  def testSingleRowEquiJoin(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, String)]("A", 'a1, 'a2)
+
+    val query =
+      "SELECT a1, a2 " +
+      "FROM A, (SELECT count(a1) AS cnt FROM A) " +
+      "WHERE a1 = cnt"
+
+    val expected =
+      unaryNode(
+        "DataSetCalc",
+        binaryNode(
+          "DataSetSingleRowJoin",
+          batchTableNode(0),
+          unaryNode(
+            "DataSetAggregate",
+            unaryNode(
+              "DataSetUnion",
+              unaryNode(
+                "DataSetValues",
+                unaryNode(
+                  "DataSetCalc",
+                  batchTableNode(0),
+                  term("select", "a1")
+                ),
+                tuples(List(null)),
+                term("values", "a1")
+              ),
+              term("union","a1")
+            ),
+            term("select", "COUNT(a1) AS cnt")
+          ),
+          term("where", "=(CAST(a1), cnt)"),
+          term("join", "a1", "a2", "cnt"),
+          term("joinType", "NestedLoopJoin")
+        ),
+        term("select", "a1", "a2")
+      )
+
+    util.verifySql(query, expected)
+  }
+
+  @Test
+  def testSingleRowNotEquiJoin(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, String)]("A", 'a1, 'a2)
+
+    val query =
+      "SELECT a1, a2 " +
+      "FROM A, (SELECT count(a1) AS cnt FROM A) " +
+      "WHERE a1 < cnt"
+
+    val expected =
+      unaryNode(
+        "DataSetCalc",
+        binaryNode(
+          "DataSetSingleRowJoin",
+          batchTableNode(0),
+          unaryNode(
+            "DataSetAggregate",
+            unaryNode(
+              "DataSetUnion",
+              unaryNode(
+                "DataSetValues",
+                unaryNode(
+                  "DataSetCalc",
+                  batchTableNode(0),
+                  term("select", "a1")
+                ),
+                tuples(List(null)),
+                term("values", "a1")
+              ),
+              term("union", "a1")
+            ),
+            term("select", "COUNT(a1) AS cnt")
+          ),
+          term("where", "<(a1, cnt)"),
+          term("join", "a1", "a2", "cnt"),
+          term("joinType", "NestedLoopJoin")
+        ),
+        term("select", "a1", "a2")
+      )
+
+    util.verifySql(query, expected)
+  }
+
+  @Test
+  def testSingleRowJoinWithComplexPredicate(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Long)]("A", 'a1, 'a2)
+    util.addTable[(Int, Long)]("B", 'b1, 'b2)
+
+    val query =
+      "SELECT a1, a2, b1, b2 " +
+        "FROM A, (SELECT min(b1) AS b1, max(b2) AS b2 FROM B) " +
+        "WHERE a1 < b1 AND a2 = b2"
+
+    val expected = binaryNode(
+      "DataSetSingleRowJoin",
+      batchTableNode(0),
+      unaryNode(
+        "DataSetAggregate",
+        unaryNode(
+          "DataSetUnion",
+          unaryNode(
+            "DataSetValues",
+            batchTableNode(1),
+            tuples(List(null, null)),
+            term("values", "b1", "b2")
+          ),
+          term("union","b1","b2")
+        ),
+        term("select", "MIN(b1) AS b1", "MAX(b2) AS b2")
+      ),
+      term("where", "AND(<(a1, b1)", "=(a2, b2))"),
+      term("join", "a1", "a2", "b1", "b2"),
+      term("joinType", "NestedLoopJoin")
+    )
+
+    util.verifySql(query, expected)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SortITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SortITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SortITCase.scala
new file mode 100644
index 0000000..0f46e9b
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SortITCase.scala
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.sql
+
+import org.apache.flink.api.scala.ExecutionEnvironment
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala.batch.utils.SortTestUtils._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala._
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class SortITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testOrderByMultipleFieldsWithSql(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable ORDER BY _1 DESC, _2 DESC"
+
+    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
+      (- x.productElement(0).asInstanceOf[Int], - x.productElement(1).asInstanceOf[Long]))
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds)
+
+    val expected = sortExpectedly(tupleDataSetStrings)
+    val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
+
+    val result = results
+      .filterNot(_.isEmpty)
+      .sortBy(_.head)(Ordering.by(f=> f.toString))
+      .reduceLeft(_ ++ _)
+
+    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
+  }
+
+  @Test
+  def testOrderByWithOffset(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable ORDER BY _1 DESC OFFSET 2 ROWS"
+
+    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
+      - x.productElement(0).asInstanceOf[Int] )
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds)
+
+    val expected = sortExpectedly(tupleDataSetStrings, 2, 21)
+    val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
+
+    val result = results.
+      filterNot(_.isEmpty)
+      .sortBy(_.head)(Ordering.by(f=> f.toString))
+      .reduceLeft(_ ++ _)
+
+    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
+  }
+
+  @Test
+  def testOrderByWithOffsetAndFetch(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable ORDER BY _1 OFFSET 2 ROWS FETCH NEXT 5 ROWS ONLY"
+
+    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
+      x.productElement(0).asInstanceOf[Int] )
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds)
+
+    val expected = sortExpectedly(tupleDataSetStrings, 2, 7)
+    val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
+
+    val result = results
+      .filterNot(_.isEmpty)
+      .sortBy(_.head)(Ordering.by(f=> f.toString))
+      .reduceLeft(_ ++ _)
+
+    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
+  }
+
+  @Test
+  def testOrderByLimit(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable ORDER BY _2, _1 LIMIT 5"
+
+    implicit def rowOrdering[T <: Product] = Ordering.by((x : T) =>
+      (x.productElement(1).asInstanceOf[Long], x.productElement(0).asInstanceOf[Int]) )
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds)
+
+    val expected = sortExpectedly(tupleDataSetStrings, 0, 5)
+    val results = tEnv.sql(sqlQuery).toDataSet[Row].mapPartition(rows => Seq(rows.toSeq)).collect()
+
+    val result = results
+      .filterNot(_.isEmpty)
+      .sortBy(_.head)(Ordering.by(f=> f.toString))
+      .reduceLeft(_ ++ _)
+
+    TestBaseUtils.compareOrderedResultAsText(result.asJava, expected)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testLimitWithoutOrder(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT * FROM MyTable LIMIT 5"
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds)
+
+    tEnv.sql(sqlQuery).toDataSet[Row].collect()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/TableWithSQLITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/TableWithSQLITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/TableWithSQLITCase.scala
new file mode 100644
index 0000000..aabc62a
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/TableWithSQLITCase.scala
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.util.CollectionDataSets
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit._
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConverters._
+
+@RunWith(classOf[Parameterized])
+class TableWithSQLITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsTestBase(mode, configMode) {
+
+  @Test
+  def testSQLTable(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env)
+    tEnv.registerDataSet("MyTable", ds, 'a, 'b, 'c)
+
+    val sqlQuery = "SELECT * FROM MyTable WHERE a > 9"
+
+    val result = tEnv.sql(sqlQuery).select('a.avg, 'b.sum, 'c.count)
+
+    val expected = "15,65,12"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testTableSQLTable(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val ds = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    val t1 = ds.filter('a > 9)
+
+    tEnv.registerTable("MyTable", t1)
+
+    val sqlQuery = "SELECT avg(a) as a1, sum(b) as b1, count(c) as c1 FROM MyTable"
+
+    val result = tEnv.sql(sqlQuery).select('a1 + 1, 'b1 - 5, 'c1)
+
+    val expected = "16,60,12"
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testMultipleSQLQueries(): Unit = {
+
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val t = CollectionDataSets.get3TupleDataSet(env).toTable(tEnv, 'a, 'b, 'c)
+    tEnv.registerTable("MyTable", t)
+
+    val sqlQuery = "SELECT a as aa FROM MyTable WHERE b = 6"
+    val result1 = tEnv.sql(sqlQuery)
+    tEnv.registerTable("ResTable", result1)
+
+    val sqlQuery2 = "SELECT count(aa) FROM ResTable"
+    val result2 = tEnv.sql(sqlQuery2)
+
+    val expected = "6"
+    val results = result2.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+  @Test
+  def testSelectWithCompositeType(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val sqlQuery = "SELECT MyTable.a2, MyTable.a1._2 FROM MyTable"
+
+    val ds = env.fromElements(((12, true), "Hello")).toTable(tEnv).as('a1, 'a2)
+    tEnv.registerTable("MyTable", ds)
+
+    val result = tEnv.sql(sqlQuery)
+
+    val expected = "Hello,true\n"
+
+    val results = result.toDataSet[Row].collect()
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/UserDefinedTableFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/UserDefinedTableFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/UserDefinedTableFunctionTest.scala
new file mode 100644
index 0000000..e091da2
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/UserDefinedTableFunctionTest.scala
@@ -0,0 +1,238 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.{HierarchyTableFunction, PojoTableFunc, TableFunc2}
+import org.apache.flink.table.utils._
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+class UserDefinedTableFunctionTest extends TableTestBase {
+
+  @Test
+  def testCrossJoin(): Unit = {
+    val util = batchTestUtil()
+    val func1 = new TableFunc1
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    util.addFunction("func1", func1)
+
+    val sqlQuery = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(c)) AS T(s)"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", "func1($cor0.c)"),
+        term("function", func1.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "f0 AS s")
+    )
+
+    util.verifySql(sqlQuery, expected)
+
+    // test overloading
+
+    val sqlQuery2 = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(c, '$')) AS T(s)"
+
+    val expected2 = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", "func1($cor0.c, '$')"),
+        term("function", func1.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "f0 AS s")
+    )
+
+    util.verifySql(sqlQuery2, expected2)
+  }
+
+  @Test
+  def testLeftOuterJoin(): Unit = {
+    val util = batchTestUtil()
+    val func1 = new TableFunc1
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    util.addFunction("func1", func1)
+
+    val sqlQuery = "SELECT c, s FROM MyTable LEFT JOIN LATERAL TABLE(func1(c)) AS T(s) ON TRUE"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", "func1($cor0.c)"),
+        term("function", func1.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
+        term("joinType", "LEFT")
+      ),
+      term("select", "c", "f0 AS s")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testCustomType(): Unit = {
+    val util = batchTestUtil()
+    val func2 = new TableFunc2
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    util.addFunction("func2", func2)
+
+    val sqlQuery = "SELECT c, name, len FROM MyTable, LATERAL TABLE(func2(c)) AS T(name, len)"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", "func2($cor0.c)"),
+        term("function", func2.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
+               "VARCHAR(2147483647) f0, INTEGER f1)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "f0 AS name", "f1 AS len")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testHierarchyType(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = new HierarchyTableFunction
+    util.addFunction("hierarchy", function)
+
+    val sqlQuery = "SELECT c, T.* FROM MyTable, LATERAL TABLE(hierarchy(c)) AS T(name, adult, len)"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", "hierarchy($cor0.c)"),
+        term("function", function.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," +
+               " VARCHAR(2147483647) f0, BOOLEAN f1, INTEGER f2)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "f0 AS name", "f1 AS adult", "f2 AS len")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testPojoType(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = new PojoTableFunc
+    util.addFunction("pojo", function)
+
+    val sqlQuery = "SELECT c, name, age FROM MyTable, LATERAL TABLE(pojo(c))"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", "pojo($cor0.c)"),
+        term("function", function.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," +
+               " INTEGER age, VARCHAR(2147483647) name)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "name", "age")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testFilter(): Unit = {
+    val util = batchTestUtil()
+    val func2 = new TableFunc2
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    util.addFunction("func2", func2)
+
+    val sqlQuery = "SELECT c, name, len FROM MyTable, LATERAL TABLE(func2(c)) AS T(name, len) " +
+      "WHERE len > 2"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", "func2($cor0.c)"),
+        term("function", func2.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
+               "VARCHAR(2147483647) f0, INTEGER f1)"),
+        term("joinType", "INNER"),
+        term("condition", ">($1, 2)")
+      ),
+      term("select", "c", "f0 AS name", "f1 AS len")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+
+  @Test
+  def testScalarFunction(): Unit = {
+    val util = batchTestUtil()
+    val func1 = new TableFunc1
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    util.addFunction("func1", func1)
+
+    val sqlQuery = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(SUBSTRING(c, 2))) AS T(s)"
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", "func1(SUBSTRING($cor0.c, 2))"),
+        term("function", func1.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "f0 AS s")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+}


[36/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/mathExpressions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/mathExpressions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/mathExpressions.scala
deleted file mode 100644
index e0f4691..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/mathExpressions.scala
+++ /dev/null
@@ -1,129 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-import org.apache.calcite.tools.RelBuilder
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.typeutils.TypeCheckUtils
-import org.apache.flink.api.table.validate._
-
-case class Abs(child: Expression) extends UnaryExpression {
-  override private[flink] def resultType: TypeInformation[_] = child.resultType
-
-  override private[flink] def validateInput(): ValidationResult =
-    TypeCheckUtils.assertNumericExpr(child.resultType, "Abs")
-
-  override def toString: String = s"abs($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.ABS, child.toRexNode)
-  }
-}
-
-case class Ceil(child: Expression) extends UnaryExpression {
-  override private[flink] def resultType: TypeInformation[_] = LONG_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult =
-    TypeCheckUtils.assertNumericExpr(child.resultType, "Ceil")
-
-  override def toString: String = s"ceil($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.CEIL, child.toRexNode)
-  }
-}
-
-case class Exp(child: Expression) extends UnaryExpression with InputTypeSpec {
-  override private[flink] def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO
-
-  override private[flink] def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: Nil
-
-  override def toString: String = s"exp($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.EXP, child.toRexNode)
-  }
-}
-
-
-case class Floor(child: Expression) extends UnaryExpression {
-  override private[flink] def resultType: TypeInformation[_] = LONG_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult =
-    TypeCheckUtils.assertNumericExpr(child.resultType, "Floor")
-
-  override def toString: String = s"floor($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.FLOOR, child.toRexNode)
-  }
-}
-
-case class Log10(child: Expression) extends UnaryExpression with InputTypeSpec {
-  override private[flink] def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO
-
-  override private[flink] def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: Nil
-
-  override def toString: String = s"log10($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.LOG10, child.toRexNode)
-  }
-}
-
-case class Ln(child: Expression) extends UnaryExpression with InputTypeSpec {
-  override private[flink] def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO
-
-  override private[flink] def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: Nil
-
-  override def toString: String = s"ln($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.LN, child.toRexNode)
-  }
-}
-
-case class Power(left: Expression, right: Expression) extends BinaryExpression with InputTypeSpec {
-  override private[flink] def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO
-
-  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
-    DOUBLE_TYPE_INFO :: DOUBLE_TYPE_INFO :: Nil
-
-  override def toString: String = s"pow($left, $right)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.POWER, left.toRexNode, right.toRexNode)
-  }
-}
-
-case class Sqrt(child: Expression) extends UnaryExpression with InputTypeSpec {
-  override private[flink] def resultType: TypeInformation[_] = DOUBLE_TYPE_INFO
-
-  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
-    Seq(DOUBLE_TYPE_INFO)
-
-  override def toString: String = s"sqrt($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.POWER, child.toRexNode, Literal(0.5).toRexNode)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ordering.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ordering.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ordering.scala
deleted file mode 100644
index c15d462..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ordering.scala
+++ /dev/null
@@ -1,54 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.tools.RelBuilder
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.validate._
-
-abstract class Ordering extends UnaryExpression {
-  override private[flink] def validateInput(): ValidationResult = {
-    if (!child.isInstanceOf[NamedExpression]) {
-      ValidationFailure(s"Sort should only based on field reference")
-    } else {
-      ValidationSuccess
-    }
-  }
-}
-
-case class Asc(child: Expression) extends Ordering {
-  override def toString: String = s"($child).asc"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    child.toRexNode
-  }
-
-  override private[flink] def resultType: TypeInformation[_] = child.resultType
-}
-
-case class Desc(child: Expression) extends Ordering {
-  override def toString: String = s"($child).desc"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.desc(child.toRexNode)
-  }
-
-  override private[flink] def resultType: TypeInformation[_] = child.resultType
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/package.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/package.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/package.scala
deleted file mode 100644
index 2e5d0b2..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/package.scala
+++ /dev/null
@@ -1,29 +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.api.table
-
-/**
- * This package contains the base class of AST nodes and all the expression language AST classes.
- * Expression trees should not be manually constructed by users. They are implicitly constructed
- * from the implicit DSL conversions in
- * [[org.apache.flink.api.scala.table.ImplicitExpressionConversions]] and
- * [[org.apache.flink.api.scala.table.ImplicitExpressionOperations]]. For the Java API,
- * expression trees should be generated from a string parser that parses expressions and creates
- * AST nodes.
- */
-package object expressions

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/stringExpressions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/stringExpressions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/stringExpressions.scala
deleted file mode 100644
index 56b5b5e..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/stringExpressions.scala
+++ /dev/null
@@ -1,279 +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.api.table.expressions
-
-import scala.collection.JavaConversions._
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.expressions.TrimMode.TrimMode
-import org.apache.flink.api.table.validate._
-
-/**
-  * Returns the length of this `str`.
-  */
-case class CharLength(child: Expression) extends UnaryExpression {
-  override private[flink] def resultType: TypeInformation[_] = INT_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (child.resultType == STRING_TYPE_INFO) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(s"CharLength operator requires String input, " +
-        s"but $child is of type ${child.resultType}")
-    }
-  }
-
-  override def toString: String = s"($child).charLength()"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.CHAR_LENGTH, child.toRexNode)
-  }
-}
-
-/**
-  * Returns str with the first letter of each word in uppercase.
-  * All other letters are in lowercase. Words are delimited by white space.
-  */
-case class InitCap(child: Expression) extends UnaryExpression {
-  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (child.resultType == STRING_TYPE_INFO) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(s"InitCap operator requires String input, " + 
-        s"but $child is of type ${child.resultType}")
-    }
-  }
-
-  override def toString: String = s"($child).initCap()"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.INITCAP, child.toRexNode)
-  }
-}
-
-/**
-  * Returns true if `str` matches `pattern`.
-  */
-case class Like(str: Expression, pattern: Expression) extends BinaryExpression {
-  private[flink] def left: Expression = str
-  private[flink] def right: Expression = pattern
-
-  override private[flink] def resultType: TypeInformation[_] = BOOLEAN_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (str.resultType == STRING_TYPE_INFO && pattern.resultType == STRING_TYPE_INFO) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(s"Like operator requires (String, String) input, " +
-        s"but ($str, $pattern) is of type (${str.resultType}, ${pattern.resultType})")
-    }
-  }
-
-  override def toString: String = s"($str).like($pattern)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.LIKE, children.map(_.toRexNode))
-  }
-}
-
-/**
-  * Returns str with all characters changed to lowercase.
-  */
-case class Lower(child: Expression) extends UnaryExpression {
-  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (child.resultType == STRING_TYPE_INFO) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(s"Lower operator requires String input, " +
-        s"but $child is of type ${child.resultType}")
-    }
-  }
-
-  override def toString: String = s"($child).toLowerCase()"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.LOWER, child.toRexNode)
-  }
-}
-
-/**
-  * Returns true if `str` is similar to `pattern`.
-  */
-case class Similar(str: Expression, pattern: Expression) extends BinaryExpression {
-  private[flink] def left: Expression = str
-  private[flink] def right: Expression = pattern
-
-  override private[flink] def resultType: TypeInformation[_] = BOOLEAN_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (str.resultType == STRING_TYPE_INFO && pattern.resultType == STRING_TYPE_INFO) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(s"Similar operator requires (String, String) input, " +
-        s"but ($str, $pattern) is of type (${str.resultType}, ${pattern.resultType})")
-    }
-  }
-
-  override def toString: String = s"($str).similarTo($pattern)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.SIMILAR_TO, children.map(_.toRexNode))
-  }
-}
-
-/**
-  * Returns substring of `str` from `begin`(inclusive) for `length`.
-  */
-case class Substring(
-    str: Expression,
-    begin: Expression,
-    length: Expression) extends Expression with InputTypeSpec {
-
-  def this(str: Expression, begin: Expression) = this(str, begin, CharLength(str))
-
-  override private[flink] def children: Seq[Expression] = str :: begin :: length :: Nil
-
-  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
-
-  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
-    Seq(STRING_TYPE_INFO, INT_TYPE_INFO, INT_TYPE_INFO)
-
-  override def toString: String = s"($str).substring($begin, $length)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.SUBSTRING, children.map(_.toRexNode))
-  }
-}
-
-/**
-  * Trim `trimString` from `str` according to `trimMode`.
-  */
-case class Trim(
-    trimMode: Expression,
-    trimString: Expression,
-    str: Expression) extends Expression {
-
-  override private[flink] def children: Seq[Expression] = trimMode :: trimString :: str :: Nil
-
-  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    trimMode match {
-      case SymbolExpression(_: TrimMode) =>
-        if (trimString.resultType != STRING_TYPE_INFO) {
-          ValidationFailure(s"String expected for trimString, get ${trimString.resultType}")
-        } else if (str.resultType != STRING_TYPE_INFO) {
-          ValidationFailure(s"String expected for str, get ${str.resultType}")
-        } else {
-          ValidationSuccess
-        }
-      case _ => ValidationFailure("TrimMode symbol expected.")
-    }
-  }
-
-  override def toString: String = s"($str).trim($trimMode, $trimString)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.TRIM, children.map(_.toRexNode))
-  }
-}
-
-/**
-  * Enumeration of trim flags.
-  */
-object TrimConstants {
-  val TRIM_DEFAULT_CHAR = Literal(" ")
-}
-
-/**
-  * Returns str with all characters changed to uppercase.
-  */
-case class Upper(child: Expression) extends UnaryExpression with InputTypeSpec {
-
-  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
-
-  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
-    Seq(STRING_TYPE_INFO)
-
-  override def toString: String = s"($child).upperCase()"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.UPPER, child.toRexNode)
-  }
-}
-
-/**
-  * Returns the position of string needle in string haystack.
-  */
-case class Position(needle: Expression, haystack: Expression)
-    extends Expression with InputTypeSpec {
-
-  override private[flink] def children: Seq[Expression] = Seq(needle, haystack)
-
-  override private[flink] def resultType: TypeInformation[_] = INT_TYPE_INFO
-
-  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO)
-
-  override def toString: String = s"($needle).position($haystack)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.POSITION, needle.toRexNode, haystack.toRexNode)
-  }
-}
-
-/**
-  * Replaces a substring of a string with a replacement string.
-  * Starting at a position for a given length.
-  */
-case class Overlay(
-    str: Expression,
-    replacement: Expression,
-    starting: Expression,
-    position: Expression)
-  extends Expression with InputTypeSpec {
-
-  def this(str: Expression, replacement: Expression, starting: Expression) =
-    this(str, replacement, starting, CharLength(replacement))
-
-  override private[flink] def children: Seq[Expression] =
-    Seq(str, replacement, starting, position)
-
-  override private[flink] def resultType: TypeInformation[_] = STRING_TYPE_INFO
-
-  override private[flink] def expectedTypes: Seq[TypeInformation[_]] =
-    Seq(STRING_TYPE_INFO, STRING_TYPE_INFO, INT_TYPE_INFO, INT_TYPE_INFO)
-
-  override def toString: String = s"($str).overlay($replacement, $starting, $position)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(
-      SqlStdOperatorTable.OVERLAY,
-      str.toRexNode,
-      replacement.toRexNode,
-      starting.toRexNode,
-      position.toRexNode)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/symbols.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/symbols.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/symbols.scala
deleted file mode 100644
index dfa8820..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/symbols.scala
+++ /dev/null
@@ -1,132 +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.api.table.expressions
-
-import org.apache.calcite.avatica.util.{TimeUnit, TimeUnitRange}
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.sql.fun.SqlTrimFunction
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.TypeInformation
-
-import scala.language.{existentials, implicitConversions}
-
-/**
-  * General expression class to represent a symbol.
-  */
-case class SymbolExpression(symbol: TableSymbol) extends LeafExpression {
-
-  override private[flink] def resultType: TypeInformation[_] =
-    throw new UnsupportedOperationException("This should not happen. A symbol has no result type.")
-
-  def toExpr = this // triggers implicit conversion
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    // dirty hack to pass Java enums to Java from Scala
-    val enum = symbol.enum.asInstanceOf[Enum[T] forSome { type T <: Enum[T] }]
-    relBuilder.getRexBuilder.makeFlag(enum)
-  }
-
-  override def toString: String = s"${symbol.symbols}.${symbol.name}"
-
-}
-
-/**
-  * Symbol that wraps a Calcite symbol in form of a Java enum.
-  */
-trait TableSymbol {
-  def symbols: TableSymbols
-  def name: String
-  def enum: Enum[_]
-}
-
-/**
-  * Enumeration of symbols.
-  */
-abstract class TableSymbols extends Enumeration {
-
-  class TableSymbolValue(e: Enum[_]) extends Val(e.name()) with TableSymbol {
-    override def symbols: TableSymbols = TableSymbols.this
-
-    override def enum: Enum[_] = e
-
-    override def name: String = toString()
-  }
-
-  protected final def Value(enum: Enum[_]): TableSymbolValue = new TableSymbolValue(enum)
-
-  implicit def symbolToExpression(symbol: TableSymbolValue): SymbolExpression =
-    SymbolExpression(symbol)
-
-}
-
-/**
-  * Units for working with time intervals.
-  */
-object TimeIntervalUnit extends TableSymbols {
-
-  type TimeIntervalUnit = TableSymbolValue
-
-  val YEAR = Value(TimeUnitRange.YEAR)
-  val YEAR_TO_MONTH = Value(TimeUnitRange.YEAR_TO_MONTH)
-  val MONTH = Value(TimeUnitRange.MONTH)
-  val DAY = Value(TimeUnitRange.DAY)
-  val DAY_TO_HOUR = Value(TimeUnitRange.DAY_TO_HOUR)
-  val DAY_TO_MINUTE = Value(TimeUnitRange.DAY_TO_MINUTE)
-  val DAY_TO_SECOND = Value(TimeUnitRange.DAY_TO_SECOND)
-  val HOUR = Value(TimeUnitRange.HOUR)
-  val HOUR_TO_MINUTE = Value(TimeUnitRange.HOUR_TO_MINUTE)
-  val HOUR_TO_SECOND = Value(TimeUnitRange.HOUR_TO_SECOND)
-  val MINUTE = Value(TimeUnitRange.MINUTE)
-  val MINUTE_TO_SECOND = Value(TimeUnitRange.MINUTE_TO_SECOND)
-  val SECOND = Value(TimeUnitRange.SECOND)
-
-}
-
-/**
-  * Units for working with time points.
-  */
-object TimePointUnit extends TableSymbols {
-
-  type TimePointUnit = TableSymbolValue
-
-  val YEAR = Value(TimeUnit.YEAR)
-  val MONTH = Value(TimeUnit.MONTH)
-  val DAY = Value(TimeUnit.DAY)
-  val HOUR = Value(TimeUnit.HOUR)
-  val MINUTE = Value(TimeUnit.MINUTE)
-  val SECOND = Value(TimeUnit.SECOND)
-  val QUARTER = Value(TimeUnit.QUARTER)
-  val WEEK = Value(TimeUnit.WEEK)
-  val MILLISECOND = Value(TimeUnit.MILLISECOND)
-  val MICROSECOND = Value(TimeUnit.MICROSECOND)
-
-}
-
-/**
-  * Modes for trimming strings.
-  */
-object TrimMode extends TableSymbols {
-
-  type TrimMode = TableSymbolValue
-
-  val BOTH = Value(SqlTrimFunction.Flag.BOTH)
-  val LEADING = Value(SqlTrimFunction.Flag.LEADING)
-  val TRAILING = Value(SqlTrimFunction.Flag.TRAILING)
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala
deleted file mode 100644
index cd5ca0a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/time.scala
+++ /dev/null
@@ -1,377 +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.api.table.expressions
-
-import org.apache.calcite.avatica.util.{TimeUnit, TimeUnitRange}
-import org.apache.calcite.rex._
-import org.apache.calcite.sql.`type`.SqlTypeName
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation}
-import org.apache.flink.api.table.FlinkRelBuilder
-import org.apache.flink.api.table.expressions.ExpressionUtils.{divide, getFactor, mod}
-import org.apache.flink.api.table.expressions.TimeIntervalUnit.TimeIntervalUnit
-import org.apache.flink.api.table.typeutils.TypeCheckUtils.isTimeInterval
-import org.apache.flink.api.table.typeutils.{TimeIntervalTypeInfo, TypeCheckUtils}
-import org.apache.flink.api.table.validate.{ValidationResult, ValidationFailure, ValidationSuccess}
-
-import scala.collection.JavaConversions._
-
-case class Extract(timeIntervalUnit: Expression, temporal: Expression) extends Expression {
-
-  override private[flink] def children: Seq[Expression] = timeIntervalUnit :: temporal :: Nil
-
-  override private[flink] def resultType: TypeInformation[_] = LONG_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (!TypeCheckUtils.isTemporal(temporal.resultType)) {
-      return ValidationFailure(s"Extract operator requires Temporal input, " +
-        s"but $temporal is of type ${temporal.resultType}")
-    }
-
-    timeIntervalUnit match {
-      case SymbolExpression(TimeIntervalUnit.YEAR)
-           | SymbolExpression(TimeIntervalUnit.MONTH)
-           | SymbolExpression(TimeIntervalUnit.DAY)
-        if temporal.resultType == SqlTimeTypeInfo.DATE
-          || temporal.resultType == SqlTimeTypeInfo.TIMESTAMP
-          || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MILLIS
-          || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MONTHS =>
-        ValidationSuccess
-
-      case SymbolExpression(TimeIntervalUnit.HOUR)
-           | SymbolExpression(TimeIntervalUnit.MINUTE)
-           | SymbolExpression(TimeIntervalUnit.SECOND)
-        if temporal.resultType == SqlTimeTypeInfo.TIME
-          || temporal.resultType == SqlTimeTypeInfo.TIMESTAMP
-          || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MILLIS =>
-        ValidationSuccess
-
-      case _ =>
-        ValidationFailure(s"Extract operator does not support unit '$timeIntervalUnit' for input" +
-          s" of type '${temporal.resultType}'.")
-    }
-  }
-
-  override def toString: String = s"($temporal).extract($timeIntervalUnit)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    // get wrapped Calcite unit
-    val timeUnitRange = timeIntervalUnit
-      .asInstanceOf[SymbolExpression]
-      .symbol
-      .enum
-      .asInstanceOf[TimeUnitRange]
-
-    // convert RexNodes
-    convertExtract(
-      timeIntervalUnit.toRexNode,
-      timeUnitRange,
-      temporal.toRexNode,
-      relBuilder.asInstanceOf[FlinkRelBuilder])
-  }
-
-  /**
-    * Standard conversion of the EXTRACT operator.
-    * Source: [[org.apache.calcite.sql2rel.StandardConvertletTable#convertExtract()]]
-    */
-  private def convertExtract(
-      timeUnitRangeRexNode: RexNode,
-      timeUnitRange: TimeUnitRange,
-      temporal: RexNode,
-      relBuilder: FlinkRelBuilder)
-    : RexNode = {
-
-    // TODO convert this into Table API expressions to make the code more readable
-    val rexBuilder = relBuilder.getRexBuilder
-    val resultType = relBuilder.getTypeFactory().createTypeFromTypeInfo(LONG_TYPE_INFO)
-    var result = rexBuilder.makeReinterpretCast(
-      resultType,
-      temporal,
-      rexBuilder.makeLiteral(false))
-
-    val unit = timeUnitRange.startUnit
-    val sqlTypeName = temporal.getType.getSqlTypeName
-    unit match {
-      case TimeUnit.YEAR | TimeUnit.MONTH | TimeUnit.DAY =>
-        sqlTypeName match {
-          case SqlTypeName.TIMESTAMP =>
-            result = divide(rexBuilder, result, TimeUnit.DAY.multiplier)
-            return rexBuilder.makeCall(
-              resultType,
-              SqlStdOperatorTable.EXTRACT_DATE,
-              Seq(timeUnitRangeRexNode, result))
-
-          case SqlTypeName.DATE =>
-            return rexBuilder.makeCall(
-              resultType,
-              SqlStdOperatorTable.EXTRACT_DATE,
-              Seq(timeUnitRangeRexNode, result))
-
-          case _ => // do nothing
-        }
-
-      case _ => // do nothing
-    }
-
-    result = mod(rexBuilder, resultType, result, getFactor(unit))
-    result = divide(rexBuilder, result, unit.multiplier)
-    result
-  }
-}
-
-abstract class TemporalCeilFloor(
-    timeIntervalUnit: Expression,
-    temporal: Expression)
-  extends Expression {
-
-  override private[flink] def children: Seq[Expression] = timeIntervalUnit :: temporal :: Nil
-
-  override private[flink] def resultType: TypeInformation[_] = temporal.resultType
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (!TypeCheckUtils.isTimePoint(temporal.resultType)) {
-      return ValidationFailure(s"Temporal ceil/floor operator requires Time Point input, " +
-        s"but $temporal is of type ${temporal.resultType}")
-    }
-    val unit = timeIntervalUnit match {
-      case SymbolExpression(u: TimeIntervalUnit) => Some(u)
-      case _ => None
-    }
-    if (unit.isEmpty) {
-      return ValidationFailure(s"Temporal ceil/floor operator requires Time Interval Unit " +
-        s"input, but $timeIntervalUnit is of type ${timeIntervalUnit.resultType}")
-    }
-
-    (unit.get, temporal.resultType) match {
-      case (TimeIntervalUnit.YEAR | TimeIntervalUnit.MONTH,
-          SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIMESTAMP) =>
-        ValidationSuccess
-      case (TimeIntervalUnit.DAY, SqlTimeTypeInfo.TIMESTAMP) =>
-        ValidationSuccess
-      case (TimeIntervalUnit.HOUR | TimeIntervalUnit.MINUTE | TimeIntervalUnit.SECOND,
-          SqlTimeTypeInfo.TIME | SqlTimeTypeInfo.TIMESTAMP) =>
-        ValidationSuccess
-      case _ =>
-        ValidationFailure(s"Temporal ceil/floor operator does not support " +
-          s"unit '$timeIntervalUnit' for input of type '${temporal.resultType}'.")
-    }
-  }
-}
-
-case class TemporalFloor(
-    timeIntervalUnit: Expression,
-    temporal: Expression)
-  extends TemporalCeilFloor(
-    timeIntervalUnit,
-    temporal) {
-
-  override def toString: String = s"($temporal).floor($timeIntervalUnit)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.FLOOR, temporal.toRexNode, timeIntervalUnit.toRexNode)
-  }
-}
-
-case class TemporalCeil(
-    timeIntervalUnit: Expression,
-    temporal: Expression)
-  extends TemporalCeilFloor(
-    timeIntervalUnit,
-    temporal) {
-
-  override def toString: String = s"($temporal).ceil($timeIntervalUnit)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.CEIL, temporal.toRexNode, timeIntervalUnit.toRexNode)
-  }
-}
-
-abstract class CurrentTimePoint(
-    targetType: TypeInformation[_],
-    local: Boolean)
-  extends LeafExpression {
-
-  override private[flink] def resultType: TypeInformation[_] = targetType
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (!TypeCheckUtils.isTimePoint(targetType)) {
-      ValidationFailure(s"CurrentTimePoint operator requires Time Point target type, " +
-        s"but get $targetType.")
-    } else if (local && targetType == SqlTimeTypeInfo.DATE) {
-      ValidationFailure(s"Localized CurrentTimePoint operator requires Time or Timestamp target " +
-        s"type, but get $targetType.")
-    } else {
-      ValidationSuccess
-    }
-  }
-
-  override def toString: String = if (local) {
-    s"local$targetType()"
-  } else {
-    s"current$targetType()"
-  }
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    val operator = targetType match {
-      case SqlTimeTypeInfo.TIME if local => SqlStdOperatorTable.LOCALTIME
-      case SqlTimeTypeInfo.TIMESTAMP if local => SqlStdOperatorTable.LOCALTIMESTAMP
-      case SqlTimeTypeInfo.DATE => SqlStdOperatorTable.CURRENT_DATE
-      case SqlTimeTypeInfo.TIME => SqlStdOperatorTable.CURRENT_TIME
-      case SqlTimeTypeInfo.TIMESTAMP => SqlStdOperatorTable.CURRENT_TIMESTAMP
-    }
-    relBuilder.call(operator)
-  }
-}
-
-case class CurrentDate() extends CurrentTimePoint(SqlTimeTypeInfo.DATE, local = false)
-
-case class CurrentTime() extends CurrentTimePoint(SqlTimeTypeInfo.TIME, local = false)
-
-case class CurrentTimestamp() extends CurrentTimePoint(SqlTimeTypeInfo.TIMESTAMP, local = false)
-
-case class LocalTime() extends CurrentTimePoint(SqlTimeTypeInfo.TIME, local = true)
-
-case class LocalTimestamp() extends CurrentTimePoint(SqlTimeTypeInfo.TIMESTAMP, local = true)
-
-/**
-  * Extracts the quarter of a year from a SQL date.
-  */
-case class Quarter(child: Expression) extends UnaryExpression with InputTypeSpec {
-
-  override private[flink] def expectedTypes: Seq[TypeInformation[_]] = Seq(SqlTimeTypeInfo.DATE)
-
-  override private[flink] def resultType: TypeInformation[_] = LONG_TYPE_INFO
-
-  override def toString: String = s"($child).quarter()"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    /**
-      * Standard conversion of the QUARTER operator.
-      * Source: [[org.apache.calcite.sql2rel.StandardConvertletTable#convertQuarter()]]
-      */
-    Plus(
-      Div(
-        Minus(
-          Extract(TimeIntervalUnit.MONTH, child),
-          Literal(1L)),
-        Literal(TimeUnit.QUARTER.multiplier.longValue())),
-      Literal(1L)
-    ).toRexNode
-  }
-}
-
-/**
-  * Determines whether two anchored time intervals overlap.
-  */
-case class TemporalOverlaps(
-    leftTimePoint: Expression,
-    leftTemporal: Expression,
-    rightTimePoint: Expression,
-    rightTemporal: Expression)
-  extends Expression {
-
-  override private[flink] def children: Seq[Expression] =
-    Seq(leftTimePoint, leftTemporal, rightTimePoint, rightTemporal)
-
-  override private[flink] def resultType: TypeInformation[_] = BOOLEAN_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (!TypeCheckUtils.isTimePoint(leftTimePoint.resultType)) {
-      return ValidationFailure(s"TemporalOverlaps operator requires leftTimePoint to be of type " +
-        s"Time Point, but get ${leftTimePoint.resultType}.")
-    }
-    if (!TypeCheckUtils.isTimePoint(rightTimePoint.resultType)) {
-      return ValidationFailure(s"TemporalOverlaps operator requires rightTimePoint to be of " +
-        s"type Time Point, but get ${rightTimePoint.resultType}.")
-    }
-    if (leftTimePoint.resultType != rightTimePoint.resultType) {
-      return ValidationFailure(s"TemporalOverlaps operator requires leftTimePoint and " +
-        s"rightTimePoint to be of same type.")
-    }
-
-    // leftTemporal is point, then it must be comparable with leftTimePoint
-    if (TypeCheckUtils.isTimePoint(leftTemporal.resultType)) {
-      if (leftTemporal.resultType != leftTimePoint.resultType) {
-        return ValidationFailure(s"TemporalOverlaps operator requires leftTemporal and " +
-          s"leftTimePoint to be of same type if leftTemporal is of type Time Point.")
-      }
-    } else if (!isTimeInterval(leftTemporal.resultType)) {
-      return ValidationFailure(s"TemporalOverlaps operator requires leftTemporal to be of " +
-        s"type Time Point or Time Interval.")
-    }
-
-    // rightTemporal is point, then it must be comparable with rightTimePoint
-    if (TypeCheckUtils.isTimePoint(rightTemporal.resultType)) {
-      if (rightTemporal.resultType != rightTimePoint.resultType) {
-        return ValidationFailure(s"TemporalOverlaps operator requires rightTemporal and " +
-          s"rightTimePoint to be of same type if rightTemporal is of type Time Point.")
-      }
-    } else if (!isTimeInterval(rightTemporal.resultType)) {
-      return ValidationFailure(s"TemporalOverlaps operator requires rightTemporal to be of " +
-        s"type Time Point or Time Interval.")
-    }
-    ValidationSuccess
-  }
-
-  override def toString: String = s"temporalOverlaps(${children.mkString(", ")})"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    convertOverlaps(
-      leftTimePoint.toRexNode,
-      leftTemporal.toRexNode,
-      rightTimePoint.toRexNode,
-      rightTemporal.toRexNode,
-      relBuilder.asInstanceOf[FlinkRelBuilder])
-  }
-
-  /**
-    * Standard conversion of the OVERLAPS operator.
-    * Source: [[org.apache.calcite.sql2rel.StandardConvertletTable#convertOverlaps()]]
-    */
-  private def convertOverlaps(
-      leftP: RexNode,
-      leftT: RexNode,
-      rightP: RexNode,
-      rightT: RexNode,
-      relBuilder: FlinkRelBuilder)
-    : RexNode = {
-    // leftT = leftP + leftT if leftT is an interval
-    val convLeftT = if (isTimeInterval(leftTemporal.resultType)) {
-        relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, leftP, leftT)
-      } else {
-        leftT
-      }
-    // rightT = rightP + rightT if rightT is an interval
-    val convRightT = if (isTimeInterval(rightTemporal.resultType)) {
-        relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, rightP, rightT)
-      } else {
-        rightT
-      }
-    // leftT >= rightP
-    val leftPred = relBuilder.call(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, convLeftT, rightP)
-    // rightT >= leftP
-    val rightPred = relBuilder.call(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, convRightT, leftP)
-
-    // leftT >= rightP and rightT >= leftP
-    relBuilder.call(SqlStdOperatorTable.AND, leftPred, rightPred)
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/windowProperties.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/windowProperties.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/windowProperties.scala
deleted file mode 100644
index 8386c46..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/windowProperties.scala
+++ /dev/null
@@ -1,57 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.SqlTimeTypeInfo
-import org.apache.flink.api.table.FlinkRelBuilder.NamedWindowProperty
-import org.apache.flink.api.table.validate.{ValidationFailure, ValidationSuccess}
-
-abstract class WindowProperty(child: Expression) extends UnaryExpression {
-
-  override def toString = s"WindowProperty($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode =
-    throw new UnsupportedOperationException("WindowProperty cannot be transformed to RexNode.")
-
-  override private[flink] def validateInput() =
-    if (child.isInstanceOf[WindowReference]) {
-      ValidationSuccess
-    } else {
-      ValidationFailure("Child must be a window reference.")
-    }
-
-  private[flink] def toNamedWindowProperty(name: String)(implicit relBuilder: RelBuilder)
-    : NamedWindowProperty = NamedWindowProperty(name, this)
-}
-
-case class WindowStart(child: Expression) extends WindowProperty(child) {
-
-  override private[flink] def resultType = SqlTimeTypeInfo.TIMESTAMP
-
-  override def toString: String = s"start($child)"
-}
-
-case class WindowEnd(child: Expression) extends WindowProperty(child) {
-
-  override private[flink] def resultType = SqlTimeTypeInfo.TIMESTAMP
-
-  override def toString: String = s"end($child)"
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/ScalarFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/ScalarFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/ScalarFunction.scala
deleted file mode 100644
index 2e16096..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/ScalarFunction.scala
+++ /dev/null
@@ -1,100 +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.api.table.functions
-
-import org.apache.flink.api.common.functions.InvalidTypesException
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.TypeExtractor
-import org.apache.flink.api.table.ValidationException
-import org.apache.flink.api.table.expressions.{Expression, ScalarFunctionCall}
-
-/**
-  * Base class for a user-defined scalar function. A user-defined scalar functions maps zero, one,
-  * or multiple scalar values to a new scalar value.
-  *
-  * The behavior of a [[ScalarFunction]] can be defined by implementing a custom evaluation
-  * method. An evaluation method must be declared publicly and named "eval". Evaluation methods
-  * can also be overloaded by implementing multiple methods named "eval".
-  *
-  * User-defined functions must have a default constructor and must be instantiable during runtime.
-  *
-  * By default the result type of an evaluation method is determined by Flink's type extraction
-  * facilities. This is sufficient for basic types or simple POJOs but might be wrong for more
-  * complex, custom, or composite types. In these cases [[TypeInformation]] of the result type
-  * can be manually defined by overriding [[getResultType()]].
-  *
-  * Internally, the Table/SQL API code generation works with primitive values as much as possible.
-  * If a user-defined scalar function should not introduce much overhead during runtime, it is
-  * recommended to declare parameters and result types as primitive types instead of their boxed
-  * classes. DATE/TIME is equal to int, TIMESTAMP is equal to long.
-  */
-abstract class ScalarFunction extends UserDefinedFunction {
-
-  /**
-    * Creates a call to a [[ScalarFunction]] in Scala Table API.
-    *
-    * @param params actual parameters of function
-    * @return [[Expression]] in form of a [[ScalarFunctionCall]]
-    */
-  final def apply(params: Expression*): Expression = {
-    ScalarFunctionCall(this, params)
-  }
-
-  override def toString: String = getClass.getCanonicalName
-
-  // ----------------------------------------------------------------------------------------------
-
-  /**
-    * Returns the result type of the evaluation method with a given signature.
-    *
-    * This method needs to be overriden in case Flink's type extraction facilities are not
-    * sufficient to extract the [[TypeInformation]] based on the return type of the evaluation
-    * method. Flink's type extraction facilities can handle basic types or
-    * simple POJOs but might be wrong for more complex, custom, or composite types.
-    *
-    * @param signature signature of the method the return type needs to be determined
-    * @return [[TypeInformation]] of result type or null if Flink should determine the type
-    */
-  def getResultType(signature: Array[Class[_]]): TypeInformation[_] = null
-
-  /**
-    * Returns [[TypeInformation]] about the operands of the evaluation method with a given
-    * signature.
-    *
-    * In order to perform operand type inference in SQL (especially when NULL is used) it might be
-    * necessary to determine the parameter [[TypeInformation]] of an evaluation method.
-    * By default Flink's type extraction facilities are used for this but might be wrong for
-    * more complex, custom, or composite types.
-    *
-    * @param signature signature of the method the operand types need to be determined
-    * @return [[TypeInformation]] of  operand types
-    */
-  def getParameterTypes(signature: Array[Class[_]]): Array[TypeInformation[_]] = {
-    signature.map { c =>
-      try {
-        TypeExtractor.getForClass(c)
-      } catch {
-        case ite: InvalidTypesException =>
-          throw new ValidationException(
-            s"Parameter types of scalar function '${this.getClass.getCanonicalName}' cannot be " +
-            s"automatically determined. Please provide type information manually.")
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/TableFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/TableFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/TableFunction.scala
deleted file mode 100644
index ca9aaf1..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/TableFunction.scala
+++ /dev/null
@@ -1,138 +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.api.table.functions
-
-import java.util
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.expressions.{Expression, TableFunctionCall}
-
-/**
-  * Base class for a user-defined table function (UDTF). A user-defined table functions works on
-  * zero, one, or multiple scalar values as input and returns multiple rows as output.
-  *
-  * The behavior of a [[TableFunction]] can be defined by implementing a custom evaluation
-  * method. An evaluation method must be declared publicly, not static and named "eval".
-  * Evaluation methods can also be overloaded by implementing multiple methods named "eval".
-  *
-  * User-defined functions must have a default constructor and must be instantiable during runtime.
-  *
-  * By default the result type of an evaluation method is determined by Flink's type extraction
-  * facilities. This is sufficient for basic types or simple POJOs but might be wrong for more
-  * complex, custom, or composite types. In these cases [[TypeInformation]] of the result type
-  * can be manually defined by overriding [[getResultType()]].
-  *
-  * Internally, the Table/SQL API code generation works with primitive values as much as possible.
-  * If a user-defined table function should not introduce much overhead during runtime, it is
-  * recommended to declare parameters and result types as primitive types instead of their boxed
-  * classes. DATE/TIME is equal to int, TIMESTAMP is equal to long.
-  *
-  * Example:
-  *
-  * {{{
-  *
-  *   public class Split extends TableFunction<String> {
-  *
-  *     // implement an "eval" method with as many parameters as you want
-  *     public void eval(String str) {
-  *       for (String s : str.split(" ")) {
-  *         collect(s);   // use collect(...) to emit an output row
-  *       }
-  *     }
-  *
-  *     // you can overload the eval method here ...
-  *   }
-  *
-  *   val tEnv: TableEnvironment = ...
-  *   val table: Table = ...    // schema: [a: String]
-  *
-  *   // for Scala users
-  *   val split = new Split()
-  *   table.join(split('c) as ('s)).select('a, 's)
-  *
-  *   // for Java users
-  *   tEnv.registerFunction("split", new Split())   // register table function first
-  *   table.join("split(a) as (s)").select("a, s")
-  *
-  *   // for SQL users
-  *   tEnv.registerFunction("split", new Split())   // register table function first
-  *   tEnv.sql("SELECT a, s FROM MyTable, LATERAL TABLE(split(a)) as T(s)")
-  *
-  * }}}
-  *
-  * @tparam T The type of the output row
-  */
-abstract class TableFunction[T] extends UserDefinedFunction {
-
-  /**
-    * Creates a call to a [[TableFunction]] in Scala Table API.
-    *
-    * @param params actual parameters of function
-    * @return [[Expression]] in form of a [[TableFunctionCall]]
-    */
-  final def apply(params: Expression*)(implicit typeInfo: TypeInformation[T]): Expression = {
-    val resultType = if (getResultType == null) {
-      typeInfo
-    } else {
-      getResultType
-    }
-    TableFunctionCall(getClass.getSimpleName, this, params, resultType)
-  }
-
-  override def toString: String = getClass.getCanonicalName
-
-  // ----------------------------------------------------------------------------------------------
-
-  private val rows: util.ArrayList[T] = new util.ArrayList[T]()
-
-  /**
-    * Emit an output row.
-    *
-    * @param row the output row
-    */
-  protected def collect(row: T): Unit = {
-    // cache rows for now, maybe immediately process them further
-    rows.add(row)
-  }
-
-  /**
-    * Internal use. Get an iterator of the buffered rows.
-    */
-  def getRowsIterator = rows.iterator()
-
-  /**
-    * Internal use. Clear buffered rows.
-    */
-  def clear() = rows.clear()
-
-  // ----------------------------------------------------------------------------------------------
-
-  /**
-    * Returns the result type of the evaluation method with a given signature.
-    *
-    * This method needs to be overriden in case Flink's type extraction facilities are not
-    * sufficient to extract the [[TypeInformation]] based on the return type of the evaluation
-    * method. Flink's type extraction facilities can handle basic types or
-    * simple POJOs but might be wrong for more complex, custom, or composite types.
-    *
-    * @return [[TypeInformation]] of result type or null if Flink should determine the type
-    */
-  def getResultType: TypeInformation[T] = null
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/UserDefinedFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/UserDefinedFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/UserDefinedFunction.scala
deleted file mode 100644
index cdf6b07..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/UserDefinedFunction.scala
+++ /dev/null
@@ -1,27 +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.api.table.functions
-
-/**
-  * Base class for all user-defined functions such as scalar functions, table functions,
-  * or aggregation functions.
-  *
-  * User-defined functions must have a default constructor and must be instantiable during runtime.
-  */
-trait UserDefinedFunction {
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/MathFunctions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/MathFunctions.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/MathFunctions.scala
deleted file mode 100644
index 8a0fe65..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/MathFunctions.scala
+++ /dev/null
@@ -1,29 +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.api.table.functions.utils
-
-import java.math.{BigDecimal => JBigDecimal}
-
-class MathFunctions {}
-
-object MathFunctions {
-  def power(a: Double, b: JBigDecimal): Double = {
-    Math.pow(a, b.doubleValue())
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/ScalarSqlFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/ScalarSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/ScalarSqlFunction.scala
deleted file mode 100644
index 7953b25..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/ScalarSqlFunction.scala
+++ /dev/null
@@ -1,182 +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.api.table.functions.utils
-
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.sql._
-import org.apache.calcite.sql.`type`.SqlOperandTypeChecker.Consistency
-import org.apache.calcite.sql.`type`._
-import org.apache.calcite.sql.parser.SqlParserPos
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.functions.ScalarFunction
-import org.apache.flink.api.table.functions.utils.ScalarSqlFunction.{createOperandTypeChecker, createOperandTypeInference, createReturnTypeInference}
-import org.apache.flink.api.table.functions.utils.UserDefinedFunctionUtils.{getResultType, getSignature, getSignatures, signatureToString, signaturesToString}
-import org.apache.flink.api.table.{FlinkTypeFactory, ValidationException}
-
-import scala.collection.JavaConverters._
-
-/**
-  * Calcite wrapper for user-defined scalar functions.
-  *
-  * @param name function name (used by SQL parser)
-  * @param scalarFunction scalar function to be called
-  * @param typeFactory type factory for converting Flink's between Calcite's types
-  */
-class ScalarSqlFunction(
-    name: String,
-    scalarFunction: ScalarFunction,
-    typeFactory: FlinkTypeFactory)
-  extends SqlFunction(
-    new SqlIdentifier(name, SqlParserPos.ZERO),
-    createReturnTypeInference(name, scalarFunction, typeFactory),
-    createOperandTypeInference(scalarFunction, typeFactory),
-    createOperandTypeChecker(name, scalarFunction),
-    null,
-    SqlFunctionCategory.USER_DEFINED_FUNCTION) {
-
-  def getScalarFunction = scalarFunction
-
-}
-
-object ScalarSqlFunction {
-
-  private[flink] def createReturnTypeInference(
-      name: String,
-      scalarFunction: ScalarFunction,
-      typeFactory: FlinkTypeFactory)
-    : SqlReturnTypeInference = {
-    /**
-      * Return type inference based on [[ScalarFunction]] given information.
-      */
-    new SqlReturnTypeInference {
-      override def inferReturnType(opBinding: SqlOperatorBinding): RelDataType = {
-        val parameters = opBinding
-          .collectOperandTypes()
-          .asScala
-          .map { operandType =>
-            if (operandType.getSqlTypeName == SqlTypeName.NULL) {
-              null
-            } else {
-              FlinkTypeFactory.toTypeInfo(operandType)
-            }
-          }
-        val foundSignature = getSignature(scalarFunction, parameters)
-        if (foundSignature.isEmpty) {
-          throw new ValidationException(
-            s"Given parameters of function '$name' do not match any signature. \n" +
-              s"Actual: ${signatureToString(parameters)} \n" +
-              s"Expected: ${signaturesToString(scalarFunction)}")
-        }
-        val resultType = getResultType(scalarFunction, foundSignature.get)
-        typeFactory.createTypeFromTypeInfo(resultType)
-      }
-    }
-  }
-
-  private[flink] def createOperandTypeInference(
-      scalarFunction: ScalarFunction,
-      typeFactory: FlinkTypeFactory)
-    : SqlOperandTypeInference = {
-    /**
-      * Operand type inference based on [[ScalarFunction]] given information.
-      */
-    new SqlOperandTypeInference {
-      override def inferOperandTypes(
-          callBinding: SqlCallBinding,
-          returnType: RelDataType,
-          operandTypes: Array[RelDataType]): Unit = {
-
-        val operandTypeInfo = getOperandTypeInfo(callBinding)
-
-        val foundSignature = getSignature(scalarFunction, operandTypeInfo)
-          .getOrElse(throw new ValidationException(s"Operand types of could not be inferred."))
-
-        val inferredTypes = scalarFunction
-          .getParameterTypes(foundSignature)
-          .map(typeFactory.createTypeFromTypeInfo)
-
-        inferredTypes.zipWithIndex.foreach {
-          case (inferredType, i) =>
-            operandTypes(i) = inferredType
-        }
-      }
-    }
-  }
-
-  private[flink] def createOperandTypeChecker(
-      name: String,
-      scalarFunction: ScalarFunction)
-    : SqlOperandTypeChecker = {
-
-    val signatures = getSignatures(scalarFunction)
-
-    /**
-      * Operand type checker based on [[ScalarFunction]] given information.
-      */
-    new SqlOperandTypeChecker {
-      override def getAllowedSignatures(op: SqlOperator, opName: String): String = {
-        s"$opName[${signaturesToString(scalarFunction)}]"
-      }
-
-      override def getOperandCountRange: SqlOperandCountRange = {
-        val signatureLengths = signatures.map(_.length)
-        SqlOperandCountRanges.between(signatureLengths.min, signatureLengths.max)
-      }
-
-      override def checkOperandTypes(
-          callBinding: SqlCallBinding,
-          throwOnFailure: Boolean)
-        : Boolean = {
-        val operandTypeInfo = getOperandTypeInfo(callBinding)
-
-        val foundSignature = getSignature(scalarFunction, operandTypeInfo)
-
-        if (foundSignature.isEmpty) {
-          if (throwOnFailure) {
-            throw new ValidationException(
-              s"Given parameters of function '$name' do not match any signature. \n" +
-                s"Actual: ${signatureToString(operandTypeInfo)} \n" +
-                s"Expected: ${signaturesToString(scalarFunction)}")
-          } else {
-            false
-          }
-        } else {
-          true
-        }
-      }
-
-      override def isOptional(i: Int): Boolean = false
-
-      override def getConsistency: Consistency = Consistency.NONE
-
-    }
-  }
-
-  private[flink] def getOperandTypeInfo(callBinding: SqlCallBinding): Seq[TypeInformation[_]] = {
-    val operandTypes = for (i <- 0 until callBinding.getOperandCount)
-      yield callBinding.getOperandType(i)
-    operandTypes.map { operandType =>
-      if (operandType.getSqlTypeName == SqlTypeName.NULL) {
-        null
-      } else {
-        FlinkTypeFactory.toTypeInfo(operandType)
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/TableSqlFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/TableSqlFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/TableSqlFunction.scala
deleted file mode 100644
index 738238d..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/TableSqlFunction.scala
+++ /dev/null
@@ -1,120 +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.api.table.functions.utils
-
-import com.google.common.base.Predicate
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.sql._
-import org.apache.calcite.sql.`type`._
-import org.apache.calcite.sql.parser.SqlParserPos
-import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction
-import org.apache.calcite.util.Util
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.functions.TableFunction
-import org.apache.flink.api.table.FlinkTypeFactory
-import org.apache.flink.api.table.plan.schema.FlinkTableFunctionImpl
-
-import scala.collection.JavaConverters._
-import java.util
-
-/**
-  * Calcite wrapper for user-defined table functions.
-  */
-class TableSqlFunction(
-    name: String,
-    udtf: TableFunction[_],
-    rowTypeInfo: TypeInformation[_],
-    returnTypeInference: SqlReturnTypeInference,
-    operandTypeInference: SqlOperandTypeInference,
-    operandTypeChecker: SqlOperandTypeChecker,
-    paramTypes: util.List[RelDataType],
-    functionImpl: FlinkTableFunctionImpl[_])
-  extends SqlUserDefinedTableFunction(
-    new SqlIdentifier(name, SqlParserPos.ZERO),
-    returnTypeInference,
-    operandTypeInference,
-    operandTypeChecker,
-    paramTypes,
-    functionImpl) {
-
-  /**
-    * Get the user-defined table function.
-    */
-  def getTableFunction = udtf
-
-  /**
-    * Get the type information of the table returned by the table function.
-    */
-  def getRowTypeInfo = rowTypeInfo
-
-  /**
-    * Get additional mapping information if the returned table type is a POJO
-    * (POJO types have no deterministic field order).
-    */
-  def getPojoFieldMapping = functionImpl.fieldIndexes
-
-}
-
-object TableSqlFunction {
-
-  /**
-    * Util function to create a [[TableSqlFunction]].
-    *
-    * @param name function name (used by SQL parser)
-    * @param udtf user-defined table function to be called
-    * @param rowTypeInfo the row type information generated by the table function
-    * @param typeFactory type factory for converting Flink's between Calcite's types
-    * @param functionImpl Calcite table function schema
-    * @return [[TableSqlFunction]]
-    */
-  def apply(
-    name: String,
-    udtf: TableFunction[_],
-    rowTypeInfo: TypeInformation[_],
-    typeFactory: FlinkTypeFactory,
-    functionImpl: FlinkTableFunctionImpl[_]): TableSqlFunction = {
-
-    val argTypes: util.List[RelDataType] = new util.ArrayList[RelDataType]
-    val typeFamilies: util.List[SqlTypeFamily] = new util.ArrayList[SqlTypeFamily]
-    // derives operands' data types and type families
-    functionImpl.getParameters.asScala.foreach{ o =>
-      val relType: RelDataType = o.getType(typeFactory)
-      argTypes.add(relType)
-      typeFamilies.add(Util.first(relType.getSqlTypeName.getFamily, SqlTypeFamily.ANY))
-    }
-    // derives whether the 'input'th parameter of a method is optional.
-    val optional: Predicate[Integer] = new Predicate[Integer]() {
-      def apply(input: Integer): Boolean = {
-        functionImpl.getParameters.get(input).isOptional
-      }
-    }
-    // create type check for the operands
-    val typeChecker: FamilyOperandTypeChecker = OperandTypes.family(typeFamilies, optional)
-
-    new TableSqlFunction(
-      name,
-      udtf,
-      rowTypeInfo,
-      ReturnTypes.CURSOR,
-      InferTypes.explicit(argTypes),
-      typeChecker,
-      argTypes,
-      functionImpl)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/UserDefinedFunctionUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/UserDefinedFunctionUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/UserDefinedFunctionUtils.scala
deleted file mode 100644
index 4899691..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/functions/utils/UserDefinedFunctionUtils.scala
+++ /dev/null
@@ -1,338 +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.api.table.functions.utils
-
-import java.lang.reflect.{Method, Modifier}
-import java.sql.{Date, Time, Timestamp}
-
-import com.google.common.primitives.Primitives
-import org.apache.calcite.sql.SqlFunction
-import org.apache.flink.api.common.functions.InvalidTypesException
-import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.java.typeutils.TypeExtractor
-import org.apache.flink.api.table.{FlinkTypeFactory, TableException, ValidationException}
-import org.apache.flink.api.table.functions.{ScalarFunction, TableFunction, UserDefinedFunction}
-import org.apache.flink.api.table.plan.schema.FlinkTableFunctionImpl
-import org.apache.flink.util.InstantiationUtil
-
-object UserDefinedFunctionUtils {
-
-  /**
-    * Instantiates a user-defined function.
-    */
-  def instantiate[T <: UserDefinedFunction](clazz: Class[T]): T = {
-    val constructor = clazz.getDeclaredConstructor()
-    constructor.setAccessible(true)
-    constructor.newInstance()
-  }
-
-  /**
-    * Checks if a user-defined function can be easily instantiated.
-    */
-  def checkForInstantiation(clazz: Class[_]): Unit = {
-    if (!InstantiationUtil.isPublic(clazz)) {
-      throw ValidationException("Function class is not public.")
-    }
-    else if (!InstantiationUtil.isProperClass(clazz)) {
-      throw ValidationException("Function class is no proper class, it is either abstract," +
-        " an interface, or a primitive type.")
-    }
-    else if (InstantiationUtil.isNonStaticInnerClass(clazz)) {
-      throw ValidationException("The class is an inner class, but not statically accessible.")
-    }
-
-    // check for default constructor (can be private)
-    clazz
-      .getDeclaredConstructors
-      .find(_.getParameterTypes.isEmpty)
-      .getOrElse(throw ValidationException("Function class needs a default constructor."))
-  }
-
-  /**
-    * Check whether this is a Scala object. It is forbidden to use [[TableFunction]] implemented
-    * by a Scala object, since concurrent risks.
-    */
-  def checkNotSingleton(clazz: Class[_]): Unit = {
-    // TODO it is not a good way to check singleton. Maybe improve it further.
-    if (clazz.getFields.map(_.getName) contains "MODULE$") {
-      throw new ValidationException(
-        s"TableFunction implemented by class ${clazz.getCanonicalName} " +
-          s"is a Scala object, it is forbidden since concurrent risks.")
-    }
-  }
-
-  // ----------------------------------------------------------------------------------------------
-  // Utilities for eval methods
-  // ----------------------------------------------------------------------------------------------
-
-  /**
-    * Returns signatures matching the given signature of [[TypeInformation]].
-    * Elements of the signature can be null (act as a wildcard).
-    */
-  def getSignature(
-      function: UserDefinedFunction,
-      signature: Seq[TypeInformation[_]])
-    : Option[Array[Class[_]]] = {
-    // We compare the raw Java classes not the TypeInformation.
-    // TypeInformation does not matter during runtime (e.g. within a MapFunction).
-    val actualSignature = typeInfoToClass(signature)
-    val signatures = getSignatures(function)
-
-    signatures
-      // go over all signatures and find one matching actual signature
-      .find { curSig =>
-      // match parameters of signature to actual parameters
-      actualSignature.length == curSig.length &&
-        curSig.zipWithIndex.forall { case (clazz, i) =>
-          parameterTypeEquals(actualSignature(i), clazz)
-        }
-    }
-  }
-
-  /**
-    * Returns eval method matching the given signature of [[TypeInformation]].
-    */
-  def getEvalMethod(
-      function: UserDefinedFunction,
-      signature: Seq[TypeInformation[_]])
-    : Option[Method] = {
-    // We compare the raw Java classes not the TypeInformation.
-    // TypeInformation does not matter during runtime (e.g. within a MapFunction).
-    val actualSignature = typeInfoToClass(signature)
-    val evalMethods = checkAndExtractEvalMethods(function)
-
-    evalMethods
-      // go over all eval methods and find one matching
-      .find { cur =>
-      val signatures = cur.getParameterTypes
-      // match parameters of signature to actual parameters
-      actualSignature.length == signatures.length &&
-        signatures.zipWithIndex.forall { case (clazz, i) =>
-          parameterTypeEquals(actualSignature(i), clazz)
-        }
-    }
-  }
-
-  /**
-    * Extracts "eval" methods and throws a [[ValidationException]] if no implementation
-    * can be found.
-    */
-  def checkAndExtractEvalMethods(function: UserDefinedFunction): Array[Method] = {
-    val methods = function
-      .getClass
-      .getDeclaredMethods
-      .filter { m =>
-        val modifiers = m.getModifiers
-        m.getName == "eval" &&
-          Modifier.isPublic(modifiers) &&
-          !Modifier.isAbstract(modifiers) &&
-          !(function.isInstanceOf[TableFunction[_]] && Modifier.isStatic(modifiers))
-      }
-
-    if (methods.isEmpty) {
-      throw new ValidationException(
-        s"Function class '${function.getClass.getCanonicalName}' does not implement at least " +
-          s"one method named 'eval' which is public, not abstract and " +
-          s"(in case of table functions) not static.")
-    } else {
-      methods
-    }
-  }
-
-  def getSignatures(function: UserDefinedFunction): Array[Array[Class[_]]] = {
-    checkAndExtractEvalMethods(function).map(_.getParameterTypes)
-  }
-
-  // ----------------------------------------------------------------------------------------------
-  // Utilities for SQL functions
-  // ----------------------------------------------------------------------------------------------
-
-  /**
-    * Create [[SqlFunction]] for a [[ScalarFunction]]
-    * @param name function name
-    * @param function scalar function
-    * @param typeFactory type factory
-    * @return the ScalarSqlFunction
-    */
-  def createScalarSqlFunction(
-      name: String,
-      function: ScalarFunction,
-      typeFactory: FlinkTypeFactory)
-    : SqlFunction = {
-    new ScalarSqlFunction(name, function, typeFactory)
-  }
-
-  /**
-    * Create [[SqlFunction]]s for a [[TableFunction]]'s every eval method
-    * @param name function name
-    * @param tableFunction table function
-    * @param resultType the type information of returned table
-    * @param typeFactory type factory
-    * @return the TableSqlFunction
-    */
-  def createTableSqlFunctions(
-      name: String,
-      tableFunction: TableFunction[_],
-      resultType: TypeInformation[_],
-      typeFactory: FlinkTypeFactory)
-    : Seq[SqlFunction] = {
-    val (fieldNames, fieldIndexes, _) = UserDefinedFunctionUtils.getFieldInfo(resultType)
-    val evalMethods = checkAndExtractEvalMethods(tableFunction)
-
-    evalMethods.map { method =>
-      val function = new FlinkTableFunctionImpl(resultType, fieldIndexes, fieldNames, method)
-      TableSqlFunction(name, tableFunction, resultType, typeFactory, function)
-    }
-  }
-
-  // ----------------------------------------------------------------------------------------------
-  // Utilities for scalar functions
-  // ----------------------------------------------------------------------------------------------
-
-  /**
-    * Internal method of [[ScalarFunction#getResultType()]] that does some pre-checking and uses
-    * [[TypeExtractor]] as default return type inference.
-    */
-  def getResultType(
-      function: ScalarFunction,
-      signature: Array[Class[_]])
-    : TypeInformation[_] = {
-    // find method for signature
-    val evalMethod = checkAndExtractEvalMethods(function)
-      .find(m => signature.sameElements(m.getParameterTypes))
-      .getOrElse(throw new ValidationException("Given signature is invalid."))
-
-    val userDefinedTypeInfo = function.getResultType(signature)
-    if (userDefinedTypeInfo != null) {
-      userDefinedTypeInfo
-    } else {
-      try {
-        TypeExtractor.getForClass(evalMethod.getReturnType)
-      } catch {
-        case ite: InvalidTypesException =>
-          throw new ValidationException(
-            s"Return type of scalar function '${function.getClass.getCanonicalName}' cannot be " +
-              s"automatically determined. Please provide type information manually.")
-      }
-    }
-  }
-
-  /**
-    * Returns the return type of the evaluation method matching the given signature.
-    */
-  def getResultTypeClass(
-      function: ScalarFunction,
-      signature: Array[Class[_]])
-    : Class[_] = {
-    // find method for signature
-    val evalMethod = checkAndExtractEvalMethods(function)
-      .find(m => signature.sameElements(m.getParameterTypes))
-      .getOrElse(throw new IllegalArgumentException("Given signature is invalid."))
-    evalMethod.getReturnType
-  }
-
-  // ----------------------------------------------------------------------------------------------
-  // Miscellaneous
-  // ----------------------------------------------------------------------------------------------
-
-  /**
-    * Returns field names and field positions for a given [[TypeInformation]].
-    *
-    * Field names are automatically extracted for
-    * [[org.apache.flink.api.common.typeutils.CompositeType]].
-    *
-    * @param inputType The TypeInformation to extract the field names and positions from.
-    * @return A tuple of two arrays holding the field names and corresponding field positions.
-    */
-  def getFieldInfo(inputType: TypeInformation[_])
-    : (Array[String], Array[Int], Array[TypeInformation[_]]) = {
-
-    val fieldNames: Array[String] = inputType match {
-      case t: CompositeType[_] => t.getFieldNames
-      case a: AtomicType[_] => Array("f0")
-      case tpe =>
-        throw new TableException(s"Currently only CompositeType and AtomicType are supported. " +
-          s"Type $tpe lacks explicit field naming")
-    }
-    val fieldIndexes = fieldNames.indices.toArray
-    val fieldTypes: Array[TypeInformation[_]] = fieldNames.map { i =>
-      inputType match {
-        case t: CompositeType[_] => t.getTypeAt(i).asInstanceOf[TypeInformation[_]]
-        case a: AtomicType[_] => a.asInstanceOf[TypeInformation[_]]
-        case tpe =>
-          throw new TableException(s"Currently only CompositeType and AtomicType are supported.")
-      }
-    }
-    (fieldNames, fieldIndexes, fieldTypes)
-  }
-
-  /**
-    * Prints one signature consisting of classes.
-    */
-  def signatureToString(signature: Array[Class[_]]): String =
-  signature.map { clazz =>
-    if (clazz == null) {
-      "null"
-    } else {
-      clazz.getCanonicalName
-    }
-  }.mkString("(", ", ", ")")
-
-  /**
-    * Prints one signature consisting of TypeInformation.
-    */
-  def signatureToString(signature: Seq[TypeInformation[_]]): String = {
-    signatureToString(typeInfoToClass(signature))
-  }
-
-  /**
-    * Prints all eval methods signatures of a class.
-    */
-  def signaturesToString(function: UserDefinedFunction): String = {
-    getSignatures(function).map(signatureToString).mkString(", ")
-  }
-
-  /**
-    * Extracts type classes of [[TypeInformation]] in a null-aware way.
-    */
-  private def typeInfoToClass(typeInfos: Seq[TypeInformation[_]]): Array[Class[_]] =
-  typeInfos.map { typeInfo =>
-    if (typeInfo == null) {
-      null
-    } else {
-      typeInfo.getTypeClass
-    }
-  }.toArray
-
-
-  /**
-    * Compares parameter candidate classes with expected classes. If true, the parameters match.
-    * Candidate can be null (acts as a wildcard).
-    */
-  private def parameterTypeEquals(candidate: Class[_], expected: Class[_]): Boolean =
-  candidate == null ||
-    candidate == expected ||
-    expected.isPrimitive && Primitives.wrap(expected) == candidate ||
-    candidate == classOf[Date] && expected == classOf[Int] ||
-    candidate == classOf[Time] && expected == classOf[Int] ||
-    candidate == classOf[Timestamp] && expected == classOf[Long]
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/package.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/package.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/package.scala
deleted file mode 100644
index bdcb22c..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/package.scala
+++ /dev/null
@@ -1,34 +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.api
-
-/**
- * == Table API ==
- *
- * This package contains the generic part of the Table API. It can be used with Flink Streaming
- * and Flink Batch. From Scala as well as from Java.
- *
- * When using the Table API, as user creates a [[org.apache.flink.api.table.Table]] from
- * a DataSet or DataStream. On this relational operations can be performed. A table can also
- * be converted back to a DataSet or DataStream.
- *
- * Packages [[org.apache.flink.api.scala.table]] and [[org.apache.flink.api.java.table]] contain
- * the language specific part of the API. Refer to these packages for documentation on how
- * the Table API can be used in Java and Scala.
- */
-package object table


[37/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala
deleted file mode 100644
index c960a79..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionParser.scala
+++ /dev/null
@@ -1,489 +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.api.table.expressions
-
-import org.apache.calcite.avatica.util.DateTimeUtils.{MILLIS_PER_DAY, MILLIS_PER_HOUR, MILLIS_PER_MINUTE, MILLIS_PER_SECOND}
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation}
-import org.apache.flink.api.table.ExpressionParserException
-import org.apache.flink.api.table.expressions.ExpressionUtils.{toMilliInterval, toMonthInterval}
-import org.apache.flink.api.table.expressions.TimeIntervalUnit.TimeIntervalUnit
-import org.apache.flink.api.table.expressions.TimePointUnit.TimePointUnit
-import org.apache.flink.api.table.expressions.TrimMode.TrimMode
-import org.apache.flink.api.table.typeutils.TimeIntervalTypeInfo
-
-import scala.language.implicitConversions
-import scala.util.parsing.combinator.{JavaTokenParsers, PackratParsers}
-
-/**
- * Parser for expressions inside a String. This parses exactly the same expressions that
- * would be accepted by the Scala Expression DSL.
- *
- * See [[org.apache.flink.api.scala.table.ImplicitExpressionConversions]] and
- * [[org.apache.flink.api.scala.table.ImplicitExpressionOperations]] for the constructs
- * available in the Scala Expression DSL. This parser must be kept in sync with the Scala DSL
- * lazy valined in the above files.
- */
-object ExpressionParser extends JavaTokenParsers with PackratParsers {
-  case class Keyword(key: String)
-
-  // Convert the keyword into an case insensitive Parser
-  implicit def keyword2Parser(kw: Keyword): Parser[String] = {
-    ("""(?i)\Q""" + kw.key + """\E""").r
-  }
-
-  // Keyword
-
-  lazy val ARRAY: Keyword = Keyword("Array")
-  lazy val AS: Keyword = Keyword("as")
-  lazy val COUNT: Keyword = Keyword("count")
-  lazy val AVG: Keyword = Keyword("avg")
-  lazy val MIN: Keyword = Keyword("min")
-  lazy val MAX: Keyword = Keyword("max")
-  lazy val SUM: Keyword = Keyword("sum")
-  lazy val START: Keyword = Keyword("start")
-  lazy val END: Keyword = Keyword("end")
-  lazy val CAST: Keyword = Keyword("cast")
-  lazy val NULL: Keyword = Keyword("Null")
-  lazy val IF: Keyword = Keyword("?")
-  lazy val ASC: Keyword = Keyword("asc")
-  lazy val DESC: Keyword = Keyword("desc")
-  lazy val TO_DATE: Keyword = Keyword("toDate")
-  lazy val TO_TIME: Keyword = Keyword("toTime")
-  lazy val TO_TIMESTAMP: Keyword = Keyword("toTimestamp")
-  lazy val TRIM: Keyword = Keyword("trim")
-  lazy val EXTRACT: Keyword = Keyword("extract")
-  lazy val FLOOR: Keyword = Keyword("floor")
-  lazy val CEIL: Keyword = Keyword("ceil")
-  lazy val YEARS: Keyword = Keyword("years")
-  lazy val YEAR: Keyword = Keyword("year")
-  lazy val MONTHS: Keyword = Keyword("months")
-  lazy val MONTH: Keyword = Keyword("month")
-  lazy val DAYS: Keyword = Keyword("days")
-  lazy val DAY: Keyword = Keyword("day")
-  lazy val HOURS: Keyword = Keyword("hours")
-  lazy val HOUR: Keyword = Keyword("hour")
-  lazy val MINUTES: Keyword = Keyword("minutes")
-  lazy val MINUTE: Keyword = Keyword("minute")
-  lazy val SECONDS: Keyword = Keyword("seconds")
-  lazy val SECOND: Keyword = Keyword("second")
-  lazy val MILLIS: Keyword = Keyword("millis")
-  lazy val MILLI: Keyword = Keyword("milli")
-  lazy val ROWS: Keyword = Keyword("rows")
-  lazy val STAR: Keyword = Keyword("*")
-  lazy val GET: Keyword = Keyword("get")
-  lazy val FLATTEN: Keyword = Keyword("flatten")
-
-  def functionIdent: ExpressionParser.Parser[String] =
-    not(ARRAY) ~ not(AS) ~ not(COUNT) ~ not(AVG) ~ not(MIN) ~ not(MAX) ~
-      not(SUM) ~ not(START) ~ not(END)~ not(CAST) ~ not(NULL) ~
-      not(IF) ~> super.ident
-
-  // symbols
-
-  lazy val timeIntervalUnit: PackratParser[Expression] = TimeIntervalUnit.values map {
-    case unit: TimeIntervalUnit => literal(unit.toString) ^^^ unit.toExpr
-  } reduceLeft(_ | _)
-
-  lazy val timePointUnit: PackratParser[Expression] = TimePointUnit.values map {
-    case unit: TimePointUnit => literal(unit.toString) ^^^ unit.toExpr
-  } reduceLeft(_ | _)
-
-  lazy val trimMode: PackratParser[Expression] = TrimMode.values map {
-    case mode: TrimMode => literal(mode.toString) ^^^ mode.toExpr
-  } reduceLeft(_ | _)
-
-  // data types
-
-  lazy val dataType: PackratParser[TypeInformation[_]] =
-    "BYTE" ^^ { ti => BasicTypeInfo.BYTE_TYPE_INFO } |
-      "SHORT" ^^ { ti => BasicTypeInfo.SHORT_TYPE_INFO } |
-      "INTERVAL_MONTHS" ^^ {
-        ti => TimeIntervalTypeInfo.INTERVAL_MONTHS.asInstanceOf[TypeInformation[_]]
-      } |
-      "INTERVAL_MILLIS" ^^ {
-        ti => TimeIntervalTypeInfo.INTERVAL_MILLIS.asInstanceOf[TypeInformation[_]]
-      } |
-      "INT" ^^ { ti => BasicTypeInfo.INT_TYPE_INFO } |
-      "LONG" ^^ { ti => BasicTypeInfo.LONG_TYPE_INFO } |
-      "FLOAT" ^^ { ti => BasicTypeInfo.FLOAT_TYPE_INFO } |
-      "DOUBLE" ^^ { ti => BasicTypeInfo.DOUBLE_TYPE_INFO } |
-      ("BOOLEAN" | "BOOL") ^^ { ti => BasicTypeInfo.BOOLEAN_TYPE_INFO } |
-      "STRING" ^^ { ti => BasicTypeInfo.STRING_TYPE_INFO } |
-      "DATE" ^^ { ti => SqlTimeTypeInfo.DATE.asInstanceOf[TypeInformation[_]] } |
-      "TIMESTAMP" ^^ { ti => SqlTimeTypeInfo.TIMESTAMP } |
-      "TIME" ^^ { ti => SqlTimeTypeInfo.TIME } |
-      "DECIMAL" ^^ { ti => BasicTypeInfo.BIG_DEC_TYPE_INFO }
-
-  // Literals
-
-  // same as floatingPointNumber but we do not allow trailing dot "12.d" or "2."
-  lazy val floatingPointNumberFlink: Parser[String] =
-    """-?(\d+(\.\d+)?|\d*\.\d+)([eE][+-]?\d+)?[fFdD]?""".r
-
-  lazy val numberLiteral: PackratParser[Expression] =
-    (wholeNumber <~ ("l" | "L")) ^^ { n => Literal(n.toLong) } |
-      (decimalNumber <~ ("p" | "P")) ^^ { n => Literal(BigDecimal(n)) } |
-      (floatingPointNumberFlink | decimalNumber) ^^ {
-        n =>
-          if (n.matches("""-?\d+""")) {
-            Literal(n.toInt)
-          } else if (n.endsWith("f") || n.endsWith("F")) {
-            Literal(n.toFloat)
-          } else {
-            Literal(n.toDouble)
-          }
-      }
-
-  lazy val singleQuoteStringLiteral: Parser[Expression] =
-    ("'" + """([^'\p{Cntrl}\\]|\\[\\'"bfnrt]|\\u[a-fA-F0-9]{4})*""" + "'").r ^^ {
-      str => Literal(str.substring(1, str.length - 1))
-    }
-
-  lazy val stringLiteralFlink: PackratParser[Expression] = super.stringLiteral ^^ {
-    str => Literal(str.substring(1, str.length - 1))
-  }
-
-  lazy val boolLiteral: PackratParser[Expression] = ("true" | "false") ^^ {
-    str => Literal(str.toBoolean)
-  }
-
-  lazy val nullLiteral: PackratParser[Expression] = NULL ~ "(" ~> dataType <~ ")" ^^ {
-    dt => Null(dt)
-  }
-
-  lazy val literalExpr: PackratParser[Expression] =
-    numberLiteral | stringLiteralFlink | singleQuoteStringLiteral | boolLiteral | nullLiteral
-
-  lazy val fieldReference: PackratParser[NamedExpression] = (STAR | ident) ^^ {
-    sym => UnresolvedFieldReference(sym)
-  }
-
-  lazy val atom: PackratParser[Expression] =
-    ( "(" ~> expression <~ ")" ) | literalExpr | fieldReference
-
-  // suffix operators
-
-  lazy val suffixSum: PackratParser[Expression] =
-    composite <~ "." ~ SUM ~ opt("()") ^^ { e => Sum(e) }
-
-  lazy val suffixMin: PackratParser[Expression] =
-    composite <~ "." ~ MIN ~ opt("()") ^^ { e => Min(e) }
-
-  lazy val suffixMax: PackratParser[Expression] =
-    composite <~ "." ~ MAX ~ opt("()") ^^ { e => Max(e) }
-
-  lazy val suffixCount: PackratParser[Expression] =
-    composite <~ "." ~ COUNT ~ opt("()") ^^ { e => Count(e) }
-
-  lazy val suffixAvg: PackratParser[Expression] =
-    composite <~ "." ~ AVG ~ opt("()") ^^ { e => Avg(e) }
-
-  lazy val suffixStart: PackratParser[Expression] =
-    composite <~ "." ~ START ~ opt("()") ^^ { e => WindowStart(e) }
-
-  lazy val suffixEnd: PackratParser[Expression] =
-    composite <~ "." ~ END ~ opt("()") ^^ { e => WindowEnd(e) }
-
-  lazy val suffixCast: PackratParser[Expression] =
-    composite ~ "." ~ CAST ~ "(" ~ dataType ~ ")" ^^ {
-    case e ~ _ ~ _ ~ _ ~ dt ~ _ => Cast(e, dt)
-  }
-
-  lazy val suffixAs: PackratParser[Expression] =
-    composite ~ "." ~ AS ~ "(" ~ rep1sep(fieldReference, ",") ~ ")" ^^ {
-    case e ~ _ ~ _ ~ _ ~ target ~ _ => Alias(e, target.head.name, target.tail.map(_.name))
-  }
-
-  lazy val suffixTrim = composite ~ "." ~ TRIM ~ "(" ~ trimMode ~ "," ~ expression ~ ")" ^^ {
-    case operand ~ _ ~ _ ~ _ ~ mode ~ _ ~ trimCharacter ~ _ => Trim(mode, trimCharacter, operand)
-  }
-
-  lazy val suffixTrimWithoutArgs = composite <~ "." ~ TRIM ~ opt("()") ^^ {
-    e => Trim(TrimMode.BOTH, TrimConstants.TRIM_DEFAULT_CHAR, e)
-  }
-
-  lazy val suffixIf: PackratParser[Expression] =
-    composite ~ "." ~ IF ~ "(" ~ expression ~ "," ~ expression ~ ")" ^^ {
-    case condition ~ _ ~ _ ~ _ ~ ifTrue ~ _ ~ ifFalse ~ _ => If(condition, ifTrue, ifFalse)
-  }
-
-  lazy val suffixExtract = composite ~ "." ~ EXTRACT ~ "(" ~ timeIntervalUnit ~ ")" ^^ {
-    case operand ~ _  ~ _ ~ _ ~ unit ~ _ => Extract(unit, operand)
-  }
-
-  lazy val suffixFloor = composite ~ "." ~ FLOOR ~ "(" ~ timeIntervalUnit ~ ")" ^^ {
-    case operand ~ _  ~ _ ~ _ ~ unit ~ _ => TemporalFloor(unit, operand)
-  }
-
-  lazy val suffixCeil = composite ~ "." ~ CEIL ~ "(" ~ timeIntervalUnit ~ ")" ^^ {
-    case operand ~ _  ~ _ ~ _ ~ unit ~ _ => TemporalCeil(unit, operand)
-  }
-
-  lazy val suffixFunctionCall =
-    composite ~ "." ~ functionIdent ~ "(" ~ repsep(expression, ",") ~ ")" ^^ {
-    case operand ~ _ ~ name ~ _ ~ args ~ _ => Call(name.toUpperCase, operand :: args)
-  }
-
-  lazy val suffixFunctionCallOneArg = composite ~ "." ~ functionIdent ^^ {
-    case operand ~ _ ~ name => Call(name.toUpperCase, Seq(operand))
-  }
-
-  lazy val suffixAsc : PackratParser[Expression] =
-    atom <~ "." ~ ASC ~ opt("()") ^^ { e => Asc(e) }
-
-  lazy val suffixDesc : PackratParser[Expression] =
-    atom <~ "." ~ DESC ~ opt("()") ^^ { e => Desc(e) }
-
-  lazy val suffixToDate: PackratParser[Expression] =
-    composite <~ "." ~ TO_DATE ~ opt("()") ^^ { e => Cast(e, SqlTimeTypeInfo.DATE) }
-
-  lazy val suffixToTimestamp: PackratParser[Expression] =
-    composite <~ "." ~ TO_TIMESTAMP ~ opt("()") ^^ { e => Cast(e, SqlTimeTypeInfo.TIMESTAMP) }
-
-  lazy val suffixToTime: PackratParser[Expression] =
-    composite <~ "." ~ TO_TIME ~ opt("()") ^^ { e => Cast(e, SqlTimeTypeInfo.TIME) }
-
-  lazy val suffixTimeInterval : PackratParser[Expression] =
-    composite ~ "." ~ (YEARS | MONTHS | DAYS | HOURS | MINUTES | SECONDS | MILLIS |
-      YEAR | MONTH | DAY | HOUR | MINUTE | SECOND | MILLI) ^^ {
-
-    case expr ~ _ ~ (YEARS.key | YEAR.key) => toMonthInterval(expr, 12)
-
-    case expr ~ _ ~ (MONTHS.key | MONTH.key) => toMonthInterval(expr, 1)
-
-    case expr ~ _ ~ (DAYS.key | DAY.key) => toMilliInterval(expr, MILLIS_PER_DAY)
-
-    case expr ~ _ ~ (HOURS.key | HOUR.key) => toMilliInterval(expr, MILLIS_PER_HOUR)
-
-    case expr ~ _ ~ (MINUTES.key | MINUTE.key) => toMilliInterval(expr, MILLIS_PER_MINUTE)
-
-    case expr ~ _ ~ (SECONDS.key | SECOND.key) => toMilliInterval(expr, MILLIS_PER_SECOND)
-
-    case expr ~ _ ~ (MILLIS.key | MILLI.key)=> toMilliInterval(expr, 1)
-  }
-
-  lazy val suffixRowInterval : PackratParser[Expression] =
-    composite <~ "." ~ ROWS ^^ { e => ExpressionUtils.toRowInterval(e) }
-
-  lazy val suffixGet: PackratParser[Expression] =
-    composite ~ "." ~ GET ~ "(" ~ literalExpr ~ ")" ^^ {
-      case e ~ _ ~ _ ~ _ ~ index ~ _ =>
-        GetCompositeField(e, index.asInstanceOf[Literal].value)
-  }
-
-  lazy val suffixFlattening: PackratParser[Expression] =
-    composite <~ "." ~ FLATTEN ~ opt("()") ^^ { e => Flattening(e) }
-
-  lazy val suffixed: PackratParser[Expression] =
-    suffixTimeInterval | suffixRowInterval | suffixSum | suffixMin | suffixMax | suffixStart |
-      suffixEnd | suffixCount | suffixAvg | suffixCast | suffixAs | suffixTrim |
-      suffixTrimWithoutArgs | suffixIf | suffixAsc | suffixDesc | suffixToDate |
-      suffixToTimestamp | suffixToTime | suffixExtract | suffixFloor | suffixCeil |
-      suffixGet | suffixFlattening |
-      suffixFunctionCall | suffixFunctionCallOneArg // function call must always be at the end
-
-  // prefix operators
-
-  lazy val prefixArray: PackratParser[Expression] =
-    ARRAY ~ "(" ~> repsep(expression, ",") <~ ")" ^^ { elements => ArrayConstructor(elements) }
-
-  lazy val prefixSum: PackratParser[Expression] =
-    SUM ~ "(" ~> expression <~ ")" ^^ { e => Sum(e) }
-
-  lazy val prefixMin: PackratParser[Expression] =
-    MIN ~ "(" ~> expression <~ ")" ^^ { e => Min(e) }
-
-  lazy val prefixMax: PackratParser[Expression] =
-    MAX ~ "(" ~> expression <~ ")" ^^ { e => Max(e) }
-
-  lazy val prefixCount: PackratParser[Expression] =
-    COUNT ~ "(" ~> expression <~ ")" ^^ { e => Count(e) }
-
-  lazy val prefixAvg: PackratParser[Expression] =
-    AVG ~ "(" ~> expression <~ ")" ^^ { e => Avg(e) }
-
-  lazy val prefixStart: PackratParser[Expression] =
-    START ~ "(" ~> expression <~ ")" ^^ { e => WindowStart(e) }
-
-  lazy val prefixEnd: PackratParser[Expression] =
-    END ~ "(" ~> expression <~ ")" ^^ { e => WindowEnd(e) }
-
-  lazy val prefixCast: PackratParser[Expression] =
-    CAST ~ "(" ~ expression ~ "," ~ dataType ~ ")" ^^ {
-    case _ ~ _ ~ e ~ _ ~ dt ~ _ => Cast(e, dt)
-  }
-
-  lazy val prefixAs: PackratParser[Expression] =
-    AS ~ "(" ~ expression ~ "," ~ rep1sep(fieldReference, ",") ~ ")" ^^ {
-    case _ ~ _ ~ e ~ _ ~ target ~ _ => Alias(e, target.head.name, target.tail.map(_.name))
-  }
-
-  lazy val prefixIf: PackratParser[Expression] =
-      IF ~ "(" ~ expression ~ "," ~ expression ~ "," ~ expression ~ ")" ^^ {
-    case _ ~ _ ~ condition ~ _ ~ ifTrue ~ _ ~ ifFalse ~ _ => If(condition, ifTrue, ifFalse)
-  }
-
-  lazy val prefixFunctionCall = functionIdent ~ "(" ~ repsep(expression, ",") ~ ")" ^^ {
-    case name ~ _ ~ args ~ _ => Call(name.toUpperCase, args)
-  }
-
-  lazy val prefixFunctionCallOneArg = functionIdent ~ "(" ~ expression ~ ")" ^^ {
-    case name ~ _ ~ arg ~ _ => Call(name.toUpperCase, Seq(arg))
-  }
-
-  lazy val prefixTrim = TRIM ~ "(" ~ trimMode ~ "," ~ expression ~ "," ~ expression ~ ")" ^^ {
-    case _ ~ _ ~ mode ~ _ ~ trimCharacter ~ _ ~ operand ~ _ => Trim(mode, trimCharacter, operand)
-  }
-
-  lazy val prefixTrimWithoutArgs = TRIM ~ "(" ~ expression ~ ")" ^^ {
-    case _ ~ _ ~ operand ~ _ => Trim(TrimMode.BOTH, TrimConstants.TRIM_DEFAULT_CHAR, operand)
-  }
-
-  lazy val prefixExtract = EXTRACT ~ "(" ~ expression ~ "," ~ timeIntervalUnit ~ ")" ^^ {
-    case _ ~ _ ~ operand ~ _ ~ unit ~ _ => Extract(unit, operand)
-  }
-
-  lazy val prefixFloor = FLOOR ~ "(" ~ expression ~ "," ~ timeIntervalUnit ~ ")" ^^ {
-    case _ ~ _ ~ operand ~ _ ~ unit ~ _ => TemporalFloor(unit, operand)
-  }
-
-  lazy val prefixCeil = CEIL ~ "(" ~ expression ~ "," ~ timeIntervalUnit ~ ")" ^^ {
-    case _ ~ _ ~ operand ~ _ ~ unit ~ _ => TemporalCeil(unit, operand)
-  }
-
-  lazy val prefixGet: PackratParser[Expression] =
-    GET ~ "(" ~ composite ~ ","  ~ literalExpr ~ ")" ^^ {
-      case _ ~ _ ~ e ~ _ ~ index ~ _ =>
-        GetCompositeField(e, index.asInstanceOf[Literal].value)
-  }
-
-  lazy val prefixFlattening: PackratParser[Expression] =
-    FLATTEN ~ "(" ~> composite <~ ")" ^^ { e => Flattening(e) }
-
-  lazy val prefixed: PackratParser[Expression] =
-    prefixArray | prefixSum | prefixMin | prefixMax | prefixCount | prefixAvg |
-      prefixStart | prefixEnd |
-      prefixCast | prefixAs | prefixTrim | prefixTrimWithoutArgs | prefixIf | prefixExtract |
-      prefixFloor | prefixCeil | prefixGet | prefixFlattening |
-      prefixFunctionCall | prefixFunctionCallOneArg // function call must always be at the end
-
-  // suffix/prefix composite
-
-  lazy val composite: PackratParser[Expression] = suffixed | prefixed | atom |
-    failure("Composite expression expected.")
-
-  // unary ops
-
-  lazy val unaryNot: PackratParser[Expression] = "!" ~> composite ^^ { e => Not(e) }
-
-  lazy val unaryMinus: PackratParser[Expression] = "-" ~> composite ^^ { e => UnaryMinus(e) }
-
-  lazy val unary = composite | unaryNot | unaryMinus |
-    failure("Unary expression expected.")
-
-  // arithmetic
-
-  lazy val product = unary * (
-    "*" ^^^ { (a:Expression, b:Expression) => Mul(a,b) } |
-    "/" ^^^ { (a:Expression, b:Expression) => Div(a,b) } |
-    "%" ^^^ { (a:Expression, b:Expression) => Mod(a,b) } ) |
-    failure("Product expected.")
-
-  lazy val term = product * (
-    "+" ^^^ { (a:Expression, b:Expression) => Plus(a,b) } |
-    "-" ^^^ { (a:Expression, b:Expression) => Minus(a,b) } ) |
-    failure("Term expected.")
-
-  // Comparison
-
-  lazy val equalTo: PackratParser[Expression] = term ~ ("===" | "==" | "=") ~ term ^^ {
-    case l ~ _ ~ r => EqualTo(l, r)
-  }
-
-  lazy val notEqualTo: PackratParser[Expression] = term ~ ("!==" | "!=" | "<>") ~ term ^^ {
-    case l ~ _ ~ r => NotEqualTo(l, r)
-  }
-
-  lazy val greaterThan: PackratParser[Expression] = term ~ ">" ~ term ^^ {
-    case l ~ _ ~ r => GreaterThan(l, r)
-  }
-
-  lazy val greaterThanOrEqual: PackratParser[Expression] = term ~ ">=" ~ term ^^ {
-    case l ~ _ ~ r => GreaterThanOrEqual(l, r)
-  }
-
-  lazy val lessThan: PackratParser[Expression] = term ~ "<" ~ term ^^ {
-    case l ~ _ ~ r => LessThan(l, r)
-  }
-
-  lazy val lessThanOrEqual: PackratParser[Expression] = term ~ "<=" ~ term ^^ {
-    case l ~ _ ~ r => LessThanOrEqual(l, r)
-  }
-
-  lazy val comparison: PackratParser[Expression] =
-    equalTo | notEqualTo |
-    greaterThan | greaterThanOrEqual |
-    lessThan | lessThanOrEqual | term |
-    failure("Comparison expected.")
-
-  // logic
-
-  lazy val logic = comparison * (
-    "&&" ^^^ { (a:Expression, b:Expression) => And(a,b) } |
-    "||" ^^^ { (a:Expression, b:Expression) => Or(a,b) } ) |
-    failure("Logic expected.")
-
-  // alias
-
-  lazy val alias: PackratParser[Expression] = logic ~ AS ~ fieldReference ^^ {
-      case e ~ _ ~ name => Alias(e, name.name)
-  } | logic ~ AS ~ "(" ~ rep1sep(fieldReference, ",") ~ ")" ^^ {
-    case e ~ _ ~ _ ~ names ~ _ => Alias(e, names.head.name, names.tail.map(_.name))
-  } | logic
-
-  lazy val expression: PackratParser[Expression] = alias |
-    failure("Invalid expression.")
-
-  lazy val expressionList: Parser[List[Expression]] = rep1sep(expression, ",")
-
-  def parseExpressionList(expression: String): List[Expression] = {
-    parseAll(expressionList, expression) match {
-      case Success(lst, _) => lst
-
-      case NoSuccess(msg, next) =>
-        throwError(msg, next)
-    }
-  }
-
-  def parseExpression(exprString: String): Expression = {
-    parseAll(expression, exprString) match {
-      case Success(lst, _) => lst
-
-      case NoSuccess(msg, next) =>
-        throwError(msg, next)
-    }
-  }
-
-  private def throwError(msg: String, next: Input): Nothing = {
-    val improvedMsg = msg.replace("string matching regex `\\z'", "End of expression")
-
-    throw ExpressionParserException(
-      s"""Could not parse expression at column ${next.pos.column}: $improvedMsg
-        |${next.pos.longString}""".stripMargin)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionUtils.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionUtils.scala
deleted file mode 100644
index 8657534..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/ExpressionUtils.scala
+++ /dev/null
@@ -1,154 +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.api.table.expressions
-
-import java.lang.{Boolean => JBoolean, Byte => JByte, Short => JShort, Integer => JInteger, Long => JLong, Float => JFloat, Double => JDouble}
-import java.math.{BigDecimal => JBigDecimal}
-import java.sql.{Date, Time, Timestamp}
-
-import org.apache.calcite.avatica.util.TimeUnit
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rex.{RexBuilder, RexNode}
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.api.table.ValidationException
-import org.apache.flink.api.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo}
-
-object ExpressionUtils {
-
-  private[flink] def toMonthInterval(expr: Expression, multiplier: Int): Expression = expr match {
-    case Literal(value: Int, BasicTypeInfo.INT_TYPE_INFO) =>
-      Literal(value * multiplier, TimeIntervalTypeInfo.INTERVAL_MONTHS)
-    case _ =>
-      Cast(Mul(expr, Literal(multiplier)), TimeIntervalTypeInfo.INTERVAL_MONTHS)
-  }
-
-  private[flink] def toMilliInterval(expr: Expression, multiplier: Long): Expression = expr match {
-    case Literal(value: Int, BasicTypeInfo.INT_TYPE_INFO) =>
-      Literal(value * multiplier, TimeIntervalTypeInfo.INTERVAL_MILLIS)
-    case Literal(value: Long, BasicTypeInfo.LONG_TYPE_INFO) =>
-      Literal(value * multiplier, TimeIntervalTypeInfo.INTERVAL_MILLIS)
-    case _ =>
-      Cast(Mul(expr, Literal(multiplier)), TimeIntervalTypeInfo.INTERVAL_MILLIS)
-  }
-
-  private[flink] def toRowInterval(expr: Expression): Expression = expr match {
-    case Literal(value: Int, BasicTypeInfo.INT_TYPE_INFO) =>
-      Literal(value.toLong, RowIntervalTypeInfo.INTERVAL_ROWS)
-    case Literal(value: Long, BasicTypeInfo.LONG_TYPE_INFO) =>
-      Literal(value, RowIntervalTypeInfo.INTERVAL_ROWS)
-    case _ =>
-      throw new IllegalArgumentException("Invalid value for row interval literal.")
-  }
-
-  private[flink] def convertArray(array: Array[_]): Expression = {
-    def createArray(): Expression = {
-      ArrayConstructor(array.map(Literal(_)))
-    }
-
-    array match {
-      // primitives
-      case _: Array[Boolean] => createArray()
-      case _: Array[Byte] => createArray()
-      case _: Array[Short] => createArray()
-      case _: Array[Int] => createArray()
-      case _: Array[Long] => createArray()
-      case _: Array[Float] => createArray()
-      case _: Array[Double] => createArray()
-
-      // boxed types
-      case _: Array[JBoolean] => createArray()
-      case _: Array[JByte] => createArray()
-      case _: Array[JShort] => createArray()
-      case _: Array[JInteger] => createArray()
-      case _: Array[JLong] => createArray()
-      case _: Array[JFloat] => createArray()
-      case _: Array[JDouble] => createArray()
-
-      // others
-      case _: Array[String] => createArray()
-      case _: Array[JBigDecimal] => createArray()
-      case _: Array[Date] => createArray()
-      case _: Array[Time] => createArray()
-      case _: Array[Timestamp] => createArray()
-      case bda: Array[BigDecimal] => ArrayConstructor(bda.map { bd => Literal(bd.bigDecimal) })
-
-      case _ =>
-        // nested
-        if (array.length > 0 && array.head.isInstanceOf[Array[_]]) {
-          ArrayConstructor(array.map { na => convertArray(na.asInstanceOf[Array[_]]) })
-        } else {
-          throw ValidationException("Unsupported array type.")
-        }
-    }
-  }
-
-  // ----------------------------------------------------------------------------------------------
-  // RexNode conversion functions (see org.apache.calcite.sql2rel.StandardConvertletTable)
-  // ----------------------------------------------------------------------------------------------
-
-  /**
-    * Copy of [[org.apache.calcite.sql2rel.StandardConvertletTable#getFactor()]].
-    */
-  private[flink] def getFactor(unit: TimeUnit): JBigDecimal = unit match {
-    case TimeUnit.DAY => java.math.BigDecimal.ONE
-    case TimeUnit.HOUR => TimeUnit.DAY.multiplier
-    case TimeUnit.MINUTE => TimeUnit.HOUR.multiplier
-    case TimeUnit.SECOND => TimeUnit.MINUTE.multiplier
-    case TimeUnit.YEAR => java.math.BigDecimal.ONE
-    case TimeUnit.MONTH => TimeUnit.YEAR.multiplier
-    case _ => throw new IllegalArgumentException("Invalid start unit.")
-  }
-
-  /**
-    * Copy of [[org.apache.calcite.sql2rel.StandardConvertletTable#mod()]].
-    */
-  private[flink] def mod(
-      rexBuilder: RexBuilder,
-      resType: RelDataType,
-      res: RexNode,
-      value: JBigDecimal)
-    : RexNode = {
-    if (value == JBigDecimal.ONE) return res
-    rexBuilder.makeCall(SqlStdOperatorTable.MOD, res, rexBuilder.makeExactLiteral(value, resType))
-  }
-
-  /**
-    * Copy of [[org.apache.calcite.sql2rel.StandardConvertletTable#divide()]].
-    */
-  private[flink] def divide(rexBuilder: RexBuilder, res: RexNode, value: JBigDecimal): RexNode = {
-    if (value == JBigDecimal.ONE) return res
-    if (value.compareTo(JBigDecimal.ONE) < 0 && value.signum == 1) {
-      try {
-        val reciprocal = JBigDecimal.ONE.divide(value, JBigDecimal.ROUND_UNNECESSARY)
-        return rexBuilder.makeCall(
-          SqlStdOperatorTable.MULTIPLY,
-          res,
-          rexBuilder.makeExactLiteral(reciprocal))
-      } catch {
-        case e: ArithmeticException => // ignore
-      }
-    }
-    rexBuilder.makeCall(
-      SqlStdOperatorTable.DIVIDE_INTEGER,
-      res,
-      rexBuilder.makeExactLiteral(value))
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/InputTypeSpec.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/InputTypeSpec.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/InputTypeSpec.scala
deleted file mode 100644
index 67e44a1..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/InputTypeSpec.scala
+++ /dev/null
@@ -1,57 +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.api.table.expressions
-
-import scala.collection.mutable
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.validate._
-
-/**
-  * Expressions that have specification on its inputs.
-  */
-trait InputTypeSpec extends Expression {
-
-  /**
-    * Input type specification for each child.
-    *
-    * For example, [[Power]] expecting both of the children be of Double Type should use:
-    * {{{
-    *   def expectedTypes: Seq[TypeInformation[_]] = DOUBLE_TYPE_INFO :: DOUBLE_TYPE_INFO :: Nil
-    * }}}
-    */
-  private[flink] def expectedTypes: Seq[TypeInformation[_]]
-
-  override private[flink] def validateInput(): ValidationResult = {
-    val typeMismatches = mutable.ArrayBuffer.empty[String]
-    children.zip(expectedTypes).zipWithIndex.foreach { case ((e, tpe), i) =>
-      if (e.resultType != tpe) {
-        typeMismatches += s"expecting $tpe on ${i}th input, get ${e.resultType}"
-      }
-    }
-    if (typeMismatches.isEmpty) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(
-        s"""|$this fails on input type checking: ${typeMismatches.mkString("[", ", ", "]")}.
-            |Operand should be casted to proper type
-            |""".stripMargin)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/aggregations.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/aggregations.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/aggregations.scala
deleted file mode 100644
index 259f7e5..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/aggregations.scala
+++ /dev/null
@@ -1,100 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-import org.apache.calcite.tools.RelBuilder
-import org.apache.calcite.tools.RelBuilder.AggCall
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.api.table.typeutils.TypeCheckUtils
-
-abstract sealed class Aggregation extends UnaryExpression {
-
-  override def toString = s"Aggregate($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode =
-    throw new UnsupportedOperationException("Aggregate cannot be transformed to RexNode")
-
-  /**
-    * Convert Aggregate to its counterpart in Calcite, i.e. AggCall
-    */
-  private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall
-}
-
-case class Sum(child: Expression) extends Aggregation {
-  override def toString = s"sum($child)"
-
-  override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
-    relBuilder.aggregateCall(SqlStdOperatorTable.SUM, false, null, name, child.toRexNode)
-  }
-
-  override private[flink] def resultType = child.resultType
-
-  override private[flink] def validateInput() =
-    TypeCheckUtils.assertNumericExpr(child.resultType, "sum")
-}
-
-case class Min(child: Expression) extends Aggregation {
-  override def toString = s"min($child)"
-
-  override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
-    relBuilder.aggregateCall(SqlStdOperatorTable.MIN, false, null, name, child.toRexNode)
-  }
-
-  override private[flink] def resultType = child.resultType
-
-  override private[flink] def validateInput() =
-    TypeCheckUtils.assertOrderableExpr(child.resultType, "min")
-}
-
-case class Max(child: Expression) extends Aggregation {
-  override def toString = s"max($child)"
-
-  override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
-    relBuilder.aggregateCall(SqlStdOperatorTable.MAX, false, null, name, child.toRexNode)
-  }
-
-  override private[flink] def resultType = child.resultType
-
-  override private[flink] def validateInput() =
-    TypeCheckUtils.assertOrderableExpr(child.resultType, "max")
-}
-
-case class Count(child: Expression) extends Aggregation {
-  override def toString = s"count($child)"
-
-  override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
-    relBuilder.aggregateCall(SqlStdOperatorTable.COUNT, false, null, name, child.toRexNode)
-  }
-
-  override private[flink] def resultType = BasicTypeInfo.LONG_TYPE_INFO
-}
-
-case class Avg(child: Expression) extends Aggregation {
-  override def toString = s"avg($child)"
-
-  override private[flink] def toAggCall(name: String)(implicit relBuilder: RelBuilder): AggCall = {
-    relBuilder.aggregateCall(SqlStdOperatorTable.AVG, false, null, name, child.toRexNode)
-  }
-
-  override private[flink] def resultType = child.resultType
-
-  override private[flink] def validateInput() =
-    TypeCheckUtils.assertNumericExpr(child.resultType, "avg")
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/arithmetic.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/arithmetic.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/arithmetic.scala
deleted file mode 100644
index 8702886..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/arithmetic.scala
+++ /dev/null
@@ -1,158 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.sql.`type`.IntervalSqlType
-import org.apache.calcite.sql.SqlOperator
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.table.typeutils.TypeCheckUtils._
-import org.apache.flink.api.table.typeutils.TypeCoercion
-import org.apache.flink.api.table.validate._
-
-import scala.collection.JavaConversions._
-
-abstract class BinaryArithmetic extends BinaryExpression {
-  private[flink] def sqlOperator: SqlOperator
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(sqlOperator, children.map(_.toRexNode))
-  }
-
-  override private[flink] def resultType: TypeInformation[_] =
-    TypeCoercion.widerTypeOf(left.resultType, right.resultType) match {
-      case Some(t) => t
-      case None =>
-        throw new RuntimeException("This should never happen.")
-    }
-
-  // TODO: tighten this rule once we implemented type coercion rules during validation
-  override private[flink] def validateInput(): ValidationResult = {
-    if (!isNumeric(left.resultType) || !isNumeric(right.resultType)) {
-      ValidationFailure(s"$this requires both operands Numeric, get " +
-        s"$left : ${left.resultType} and $right : ${right.resultType}")
-    } else {
-      ValidationSuccess
-    }
-  }
-}
-
-case class Plus(left: Expression, right: Expression) extends BinaryArithmetic {
-  override def toString = s"($left + $right)"
-
-  private[flink] val sqlOperator = SqlStdOperatorTable.PLUS
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    if(isString(left.resultType)) {
-      val castedRight = Cast(right, BasicTypeInfo.STRING_TYPE_INFO)
-      relBuilder.call(SqlStdOperatorTable.CONCAT, left.toRexNode, castedRight.toRexNode)
-    } else if(isString(right.resultType)) {
-      val castedLeft = Cast(left, BasicTypeInfo.STRING_TYPE_INFO)
-      relBuilder.call(SqlStdOperatorTable.CONCAT, castedLeft.toRexNode, right.toRexNode)
-    } else if (isTimeInterval(left.resultType) && left.resultType == right.resultType) {
-      relBuilder.call(SqlStdOperatorTable.PLUS, left.toRexNode, right.toRexNode)
-    } else if (isTimeInterval(left.resultType) && isTemporal(right.resultType)) {
-      // Calcite has a bug that can't apply INTERVAL + DATETIME (INTERVAL at left)
-      // we manually switch them here
-      relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, right.toRexNode, left.toRexNode)
-    } else if (isTemporal(left.resultType) && isTemporal(right.resultType)) {
-      relBuilder.call(SqlStdOperatorTable.DATETIME_PLUS, left.toRexNode, right.toRexNode)
-    } else {
-      val castedLeft = Cast(left, resultType)
-      val castedRight = Cast(right, resultType)
-      relBuilder.call(SqlStdOperatorTable.PLUS, castedLeft.toRexNode, castedRight.toRexNode)
-    }
-  }
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (isString(left.resultType) || isString(right.resultType)) {
-      ValidationSuccess
-    } else if (isTimeInterval(left.resultType) && left.resultType == right.resultType) {
-      ValidationSuccess
-    } else if (isTimePoint(left.resultType) && isTimeInterval(right.resultType)) {
-      ValidationSuccess
-    } else if (isTimeInterval(left.resultType) && isTimePoint(right.resultType)) {
-      ValidationSuccess
-    } else if (isNumeric(left.resultType) && isNumeric(right.resultType)) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(
-        s"$this requires Numeric, String, Intervals of same type, " +
-        s"or Interval and a time point input, " +
-        s"get $left : ${left.resultType} and $right : ${right.resultType}")
-    }
-  }
-}
-
-case class UnaryMinus(child: Expression) extends UnaryExpression {
-  override def toString = s"-($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.UNARY_MINUS, child.toRexNode)
-  }
-
-  override private[flink] def resultType = child.resultType
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (isNumeric(child.resultType)) {
-      ValidationSuccess
-    } else if (isTimeInterval(child.resultType)) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(s"$this requires Numeric, or Interval input, get ${child.resultType}")
-    }
-  }
-}
-
-case class Minus(left: Expression, right: Expression) extends BinaryArithmetic {
-  override def toString = s"($left - $right)"
-
-  private[flink] val sqlOperator = SqlStdOperatorTable.MINUS
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (isTimeInterval(left.resultType) && left.resultType == right.resultType) {
-      ValidationSuccess
-    } else if (isTimePoint(left.resultType) && isTimeInterval(right.resultType)) {
-      ValidationSuccess
-    } else if (isTimeInterval(left.resultType) && isTimePoint(right.resultType)) {
-      ValidationSuccess
-    } else {
-      super.validateInput()
-    }
-  }
-}
-
-case class Div(left: Expression, right: Expression) extends BinaryArithmetic {
-  override def toString = s"($left / $right)"
-
-  private[flink] val sqlOperator = SqlStdOperatorTable.DIVIDE
-}
-
-case class Mul(left: Expression, right: Expression) extends BinaryArithmetic {
-  override def toString = s"($left * $right)"
-
-  private[flink] val sqlOperator = SqlStdOperatorTable.MULTIPLY
-}
-
-case class Mod(left: Expression, right: Expression) extends BinaryArithmetic {
-  override def toString = s"($left % $right)"
-
-  private[flink] val sqlOperator = SqlStdOperatorTable.MOD
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/array.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/array.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/array.scala
deleted file mode 100644
index 78084de..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/array.scala
+++ /dev/null
@@ -1,146 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo.INT_TYPE_INFO
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, PrimitiveArrayTypeInfo, TypeInformation}
-import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
-import org.apache.flink.api.table.FlinkRelBuilder
-import org.apache.flink.api.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
-
-import scala.collection.JavaConverters._
-
-case class ArrayConstructor(elements: Seq[Expression]) extends Expression {
-
-  override private[flink] def children: Seq[Expression] = elements
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    val relDataType = relBuilder
-      .asInstanceOf[FlinkRelBuilder]
-      .getTypeFactory
-      .createTypeFromTypeInfo(resultType)
-    val values = elements.map(_.toRexNode).toList.asJava
-    relBuilder
-      .getRexBuilder
-      .makeCall(relDataType, SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR, values)
-  }
-
-  override def toString = s"array(${elements.mkString(", ")})"
-
-  override private[flink] def resultType = ObjectArrayTypeInfo.getInfoFor(elements.head.resultType)
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (elements.isEmpty) {
-      return ValidationFailure("Empty arrays are not supported yet.")
-    }
-    val elementType = elements.head.resultType
-    if (!elements.forall(_.resultType == elementType)) {
-      ValidationFailure("Not all elements of the array have the same type.")
-    } else {
-      ValidationSuccess
-    }
-  }
-}
-
-case class ArrayElementAt(array: Expression, index: Expression) extends Expression {
-
-  override private[flink] def children: Seq[Expression] = Seq(array, index)
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder
-      .getRexBuilder
-      .makeCall(SqlStdOperatorTable.ITEM, array.toRexNode, index.toRexNode)
-  }
-
-  override def toString = s"($array).at($index)"
-
-  override private[flink] def resultType = array.resultType match {
-    case oati: ObjectArrayTypeInfo[_, _] => oati.getComponentInfo
-    case pati: PrimitiveArrayTypeInfo[_] => pati.getComponentType
-  }
-
-  override private[flink] def validateInput(): ValidationResult = {
-    array.resultType match {
-      case _: ObjectArrayTypeInfo[_, _] | _: PrimitiveArrayTypeInfo[_] =>
-        if (index.resultType == INT_TYPE_INFO) {
-          // check for common user mistake
-          index match {
-            case Literal(value: Int, INT_TYPE_INFO) if value < 1 =>
-              ValidationFailure(
-                s"Array element access needs an index starting at 1 but was $value.")
-            case _ => ValidationSuccess
-          }
-        } else {
-          ValidationFailure(
-            s"Array element access needs an integer index but was '${index.resultType}'.")
-        }
-      case other@_ => ValidationFailure(s"Array expected but was '$other'.")
-    }
-  }
-}
-
-case class ArrayCardinality(array: Expression) extends Expression {
-
-  override private[flink] def children: Seq[Expression] = Seq(array)
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder
-      .getRexBuilder
-      .makeCall(SqlStdOperatorTable.CARDINALITY, array.toRexNode)
-  }
-
-  override def toString = s"($array).cardinality()"
-
-  override private[flink] def resultType = BasicTypeInfo.INT_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    array.resultType match {
-      case _: ObjectArrayTypeInfo[_, _] | _: PrimitiveArrayTypeInfo[_] => ValidationSuccess
-      case other@_ => ValidationFailure(s"Array expected but was '$other'.")
-    }
-  }
-}
-
-case class ArrayElement(array: Expression) extends Expression {
-
-  override private[flink] def children: Seq[Expression] = Seq(array)
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder
-      .getRexBuilder
-      .makeCall(SqlStdOperatorTable.ELEMENT, array.toRexNode)
-  }
-
-  override def toString = s"($array).element()"
-
-  override private[flink] def resultType = array.resultType match {
-    case oati: ObjectArrayTypeInfo[_, _] => oati.getComponentInfo
-    case pati: PrimitiveArrayTypeInfo[_] => pati.getComponentType
-  }
-
-  override private[flink] def validateInput(): ValidationResult = {
-    array.resultType match {
-      case _: ObjectArrayTypeInfo[_, _] | _: PrimitiveArrayTypeInfo[_] => ValidationSuccess
-      case other@_ => ValidationFailure(s"Array expected but was '$other'.")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/call.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/call.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/call.scala
deleted file mode 100644
index 3bb9dac..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/call.scala
+++ /dev/null
@@ -1,159 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.functions.{ScalarFunction, TableFunction}
-import org.apache.flink.api.table.functions.utils.UserDefinedFunctionUtils._
-import org.apache.flink.api.table.plan.logical.{LogicalNode, LogicalTableFunctionCall}
-import org.apache.flink.api.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
-import org.apache.flink.api.table.{FlinkTypeFactory, UnresolvedException, ValidationException}
-
-/**
-  * General expression for unresolved function calls. The function can be a built-in
-  * scalar function or a user-defined scalar function.
-  */
-case class Call(functionName: String, args: Seq[Expression]) extends Expression {
-
-  override private[flink] def children: Seq[Expression] = args
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    throw UnresolvedException(s"trying to convert UnresolvedFunction $functionName to RexNode")
-  }
-
-  override def toString = s"\\$functionName(${args.mkString(", ")})"
-
-  override private[flink] def resultType =
-    throw UnresolvedException(s"calling resultType on UnresolvedFunction $functionName")
-
-  override private[flink] def validateInput(): ValidationResult =
-    ValidationFailure(s"Unresolved function call: $functionName")
-}
-
-/**
-  * Expression for calling a user-defined scalar functions.
-  *
-  * @param scalarFunction scalar function to be called (might be overloaded)
-  * @param parameters actual parameters that determine target evaluation method
-  */
-case class ScalarFunctionCall(
-    scalarFunction: ScalarFunction,
-    parameters: Seq[Expression])
-  extends Expression {
-
-  private var foundSignature: Option[Array[Class[_]]] = None
-
-  override private[flink] def children: Seq[Expression] = parameters
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    val typeFactory = relBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-    relBuilder.call(
-      createScalarSqlFunction(
-        scalarFunction.getClass.getCanonicalName,
-        scalarFunction,
-        typeFactory),
-      parameters.map(_.toRexNode): _*)
-  }
-
-  override def toString =
-    s"${scalarFunction.getClass.getCanonicalName}(${parameters.mkString(", ")})"
-
-  override private[flink] def resultType = getResultType(scalarFunction, foundSignature.get)
-
-  override private[flink] def validateInput(): ValidationResult = {
-    val signature = children.map(_.resultType)
-    // look for a signature that matches the input types
-    foundSignature = getSignature(scalarFunction, signature)
-    if (foundSignature.isEmpty) {
-      ValidationFailure(s"Given parameters do not match any signature. \n" +
-        s"Actual: ${signatureToString(signature)} \n" +
-        s"Expected: ${signaturesToString(scalarFunction)}")
-    } else {
-      ValidationSuccess
-    }
-  }
-}
-
-/**
-  *
-  * Expression for calling a user-defined table function with actual parameters.
-  *
-  * @param functionName function name
-  * @param tableFunction user-defined table function
-  * @param parameters actual parameters of function
-  * @param resultType type information of returned table
-  */
-case class TableFunctionCall(
-    functionName: String,
-    tableFunction: TableFunction[_],
-    parameters: Seq[Expression],
-    resultType: TypeInformation[_])
-  extends Expression {
-
-  private var aliases: Option[Seq[String]] = None
-
-  override private[flink] def children: Seq[Expression] = parameters
-
-  /**
-    * Assigns an alias for this table function's returned fields that the following operator
-    * can refer to.
-    *
-    * @param aliasList alias for this table function's returned fields
-    * @return this table function call
-    */
-  private[flink] def as(aliasList: Option[Seq[String]]): TableFunctionCall = {
-    this.aliases = aliasList
-    this
-  }
-
-  /**
-    * Converts an API class to a logical node for planning.
-    */
-  private[flink] def toLogicalTableFunctionCall(child: LogicalNode): LogicalTableFunctionCall = {
-    val originNames = getFieldInfo(resultType)._1
-
-    // determine the final field names
-    val fieldNames = if (aliases.isDefined) {
-      val aliasList = aliases.get
-      if (aliasList.length != originNames.length) {
-        throw ValidationException(
-          s"List of column aliases must have same degree as table; " +
-            s"the returned table of function '$functionName' has ${originNames.length} " +
-            s"columns (${originNames.mkString(",")}), " +
-            s"whereas alias list has ${aliasList.length} columns")
-      } else {
-        aliasList.toArray
-      }
-    } else {
-      originNames
-    }
-
-    LogicalTableFunctionCall(
-      functionName,
-      tableFunction,
-      parameters,
-      resultType,
-      fieldNames,
-      child)
-  }
-
-  override def toString =
-    s"${tableFunction.getClass.getCanonicalName}(${parameters.mkString(", ")})"
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/cast.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/cast.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/cast.scala
deleted file mode 100644
index 2232a91..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/cast.scala
+++ /dev/null
@@ -1,53 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.FlinkTypeFactory
-import org.apache.flink.api.table.typeutils.TypeCoercion
-import org.apache.flink.api.table.validate._
-
-case class Cast(child: Expression, resultType: TypeInformation[_]) extends UnaryExpression {
-
-  override def toString = s"$child.cast($resultType)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    val typeFactory = relBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-    relBuilder
-      .getRexBuilder
-      // we use abstract cast here because RelBuilder.cast() has to many side effects
-      .makeAbstractCast(
-        typeFactory.createTypeFromTypeInfo(resultType),
-        child.toRexNode)
-  }
-
-  override private[flink] def makeCopy(anyRefs: Array[AnyRef]): this.type = {
-    val child: Expression = anyRefs.head.asInstanceOf[Expression]
-    copy(child, resultType).asInstanceOf[this.type]
-  }
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (TypeCoercion.canCast(child.resultType, resultType)) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(s"Unsupported cast from ${child.resultType} to $resultType")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala
deleted file mode 100644
index 5a150f8..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/comparison.scala
+++ /dev/null
@@ -1,160 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.sql.SqlOperator
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.table.typeutils.TypeCheckUtils.{isComparable, isNumeric}
-import org.apache.flink.api.table.validate._
-
-import scala.collection.JavaConversions._
-
-abstract class BinaryComparison extends BinaryExpression {
-  private[flink] def sqlOperator: SqlOperator
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(sqlOperator, children.map(_.toRexNode))
-  }
-
-  override private[flink] def resultType = BOOLEAN_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult =
-    (left.resultType, right.resultType) match {
-      case (lType, rType) if isNumeric(lType) && isNumeric(rType) => ValidationSuccess
-      case (lType, rType) if isComparable(lType) && lType == rType => ValidationSuccess
-      case (lType, rType) =>
-        ValidationFailure(
-          s"Comparison is only supported for numeric types and " +
-            s"comparable types of same type, got $lType and $rType")
-    }
-}
-
-case class EqualTo(left: Expression, right: Expression) extends BinaryComparison {
-  override def toString = s"$left === $right"
-
-  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.EQUALS
-
-  override private[flink] def validateInput(): ValidationResult =
-    (left.resultType, right.resultType) match {
-      case (lType, rType) if isNumeric(lType) && isNumeric(rType) => ValidationSuccess
-      case (lType, rType) if lType == rType => ValidationSuccess
-      case (lType, rType) =>
-        ValidationFailure(s"Equality predicate on incompatible types: $lType and $rType")
-    }
-}
-
-case class NotEqualTo(left: Expression, right: Expression) extends BinaryComparison {
-  override def toString = s"$left !== $right"
-
-  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.NOT_EQUALS
-
-  override private[flink] def validateInput(): ValidationResult =
-    (left.resultType, right.resultType) match {
-      case (lType, rType) if isNumeric(lType) && isNumeric(rType) => ValidationSuccess
-      case (lType, rType) if lType == rType => ValidationSuccess
-      case (lType, rType) =>
-        ValidationFailure(s"Inequality predicate on incompatible types: $lType and $rType")
-    }
-}
-
-case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison {
-  override def toString = s"$left > $right"
-
-  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.GREATER_THAN
-}
-
-case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
-  override def toString = s"$left >= $right"
-
-  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.GREATER_THAN_OR_EQUAL
-}
-
-case class LessThan(left: Expression, right: Expression) extends BinaryComparison {
-  override def toString = s"$left < $right"
-
-  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.LESS_THAN
-}
-
-case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
-  override def toString = s"$left <= $right"
-
-  private[flink] val sqlOperator: SqlOperator = SqlStdOperatorTable.LESS_THAN_OR_EQUAL
-}
-
-case class IsNull(child: Expression) extends UnaryExpression {
-  override def toString = s"($child).isNull"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.isNull(child.toRexNode)
-  }
-
-  override private[flink] def resultType = BOOLEAN_TYPE_INFO
-}
-
-case class IsNotNull(child: Expression) extends UnaryExpression {
-  override def toString = s"($child).isNotNull"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.isNotNull(child.toRexNode)
-  }
-
-  override private[flink] def resultType = BOOLEAN_TYPE_INFO
-}
-
-case class IsTrue(child: Expression) extends UnaryExpression {
-  override def toString = s"($child).isTrue"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.IS_TRUE, child.toRexNode)
-  }
-
-  override private[flink] def resultType = BOOLEAN_TYPE_INFO
-}
-
-case class IsFalse(child: Expression) extends UnaryExpression {
-  override def toString = s"($child).isFalse"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.IS_FALSE, child.toRexNode)
-  }
-
-  override private[flink] def resultType = BOOLEAN_TYPE_INFO
-}
-
-case class IsNotTrue(child: Expression) extends UnaryExpression {
-  override def toString = s"($child).isNotTrue"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.IS_NOT_TRUE, child.toRexNode)
-  }
-
-  override private[flink] def resultType = BOOLEAN_TYPE_INFO
-}
-
-case class IsNotFalse(child: Expression) extends UnaryExpression {
-  override def toString = s"($child).isNotFalse"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.call(SqlStdOperatorTable.IS_NOT_FALSE, child.toRexNode)
-  }
-
-  override private[flink] def resultType = BOOLEAN_TYPE_INFO
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/composite.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/composite.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/composite.scala
deleted file mode 100644
index ee1eb46..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/composite.scala
+++ /dev/null
@@ -1,106 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.api.table.UnresolvedException
-import org.apache.flink.api.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
-
-/**
-  * Flattening of composite types. All flattenings are resolved into
-  * `GetCompositeField` expressions.
-  */
-case class Flattening(child: Expression) extends UnaryExpression {
-
-  override def toString = s"$child.flatten()"
-
-  override private[flink] def resultType: TypeInformation[_] =
-    throw UnresolvedException(s"Invalid call to on ${this.getClass}.")
-
-  override private[flink] def validateInput(): ValidationResult =
-    ValidationFailure(s"Unresolved flattening of $child")
-}
-
-case class GetCompositeField(child: Expression, key: Any) extends UnaryExpression {
-
-  private var fieldIndex: Option[Int] = None
-
-  override def toString = s"$child.get($key)"
-
-  override private[flink] def validateInput(): ValidationResult = {
-    // check for composite type
-    if (!child.resultType.isInstanceOf[CompositeType[_]]) {
-      return ValidationFailure(s"Cannot access field of non-composite type '${child.resultType}'.")
-    }
-    val compositeType = child.resultType.asInstanceOf[CompositeType[_]]
-
-    // check key
-    key match {
-      case name: String =>
-        val index = compositeType.getFieldIndex(name)
-        if (index < 0) {
-          ValidationFailure(s"Field name '$name' could not be found.")
-        } else {
-          fieldIndex = Some(index)
-          ValidationSuccess
-        }
-      case index: Int =>
-        if (index >= compositeType.getArity) {
-          ValidationFailure(s"Field index '$index' exceeds arity.")
-        } else {
-          fieldIndex = Some(index)
-          ValidationSuccess
-        }
-      case _ =>
-        ValidationFailure(s"Invalid key '$key'.")
-    }
-  }
-
-  override private[flink] def resultType: TypeInformation[_] =
-    child.resultType.asInstanceOf[CompositeType[_]].getTypeAt(fieldIndex.get)
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder
-      .getRexBuilder
-      .makeFieldAccess(child.toRexNode, fieldIndex.get)
-  }
-
-  override private[flink] def makeCopy(anyRefs: Array[AnyRef]): this.type = {
-    val child: Expression = anyRefs.head.asInstanceOf[Expression]
-    copy(child, key).asInstanceOf[this.type]
-  }
-
-  /**
-    * Gives a meaningful alias if possible (e.g. a$mypojo$field).
-    */
-  private[flink] def aliasName(): Option[String] = child match {
-    case gcf: GetCompositeField =>
-      val alias = gcf.aliasName()
-      if (alias.isDefined) {
-        Some(s"${alias.get}$$$key")
-      } else {
-        None
-      }
-    case c: ResolvedFieldReference => Some(s"${c.name}$$$key")
-    case _ => None
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/fieldExpression.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/fieldExpression.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/fieldExpression.scala
deleted file mode 100644
index e651bb3..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/fieldExpression.scala
+++ /dev/null
@@ -1,152 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
-import org.apache.flink.api.table.{UnresolvedException, ValidationException}
-import org.apache.flink.api.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
-
-trait NamedExpression extends Expression {
-  private[flink] def name: String
-  private[flink] def toAttribute: Attribute
-}
-
-abstract class Attribute extends LeafExpression with NamedExpression {
-  override private[flink] def toAttribute: Attribute = this
-
-  private[flink] def withName(newName: String): Attribute
-}
-
-case class UnresolvedFieldReference(name: String) extends Attribute {
-
-  override def toString = "\"" + name
-
-  override private[flink] def withName(newName: String): Attribute =
-    UnresolvedFieldReference(newName)
-
-  override private[flink] def resultType: TypeInformation[_] =
-    throw UnresolvedException(s"Calling resultType on ${this.getClass}.")
-
-  override private[flink] def validateInput(): ValidationResult =
-    ValidationFailure(s"Unresolved reference $name.")
-}
-
-case class ResolvedFieldReference(
-    name: String,
-    resultType: TypeInformation[_]) extends Attribute {
-
-  override def toString = s"'$name"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.field(name)
-  }
-
-  override private[flink] def withName(newName: String): Attribute = {
-    if (newName == name) {
-      this
-    } else {
-      ResolvedFieldReference(newName, resultType)
-    }
-  }
-}
-
-case class Alias(child: Expression, name: String, extraNames: Seq[String] = Seq())
-    extends UnaryExpression with NamedExpression {
-
-  override def toString = s"$child as '$name"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.alias(child.toRexNode, name)
-  }
-
-  override private[flink] def resultType: TypeInformation[_] = child.resultType
-
-  override private[flink] def makeCopy(anyRefs: Array[AnyRef]): this.type = {
-    val child: Expression = anyRefs.head.asInstanceOf[Expression]
-    copy(child, name, extraNames).asInstanceOf[this.type]
-  }
-
-  override private[flink] def toAttribute: Attribute = {
-    if (valid) {
-      ResolvedFieldReference(name, child.resultType)
-    } else {
-      UnresolvedFieldReference(name)
-    }
-  }
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (name == "*") {
-      ValidationFailure("Alias can not accept '*' as name.")
-    } else if (extraNames.nonEmpty) {
-      ValidationFailure("Invalid call to Alias with multiple names.")
-    } else {
-      ValidationSuccess
-    }
-  }
-}
-
-case class UnresolvedAlias(child: Expression) extends UnaryExpression with NamedExpression {
-
-  override private[flink] def name: String =
-    throw UnresolvedException("Invalid call to name on UnresolvedAlias")
-
-  override private[flink] def toAttribute: Attribute =
-    throw UnresolvedException("Invalid call to toAttribute on UnresolvedAlias")
-
-  override private[flink] def resultType: TypeInformation[_] =
-    throw UnresolvedException("Invalid call to resultType on UnresolvedAlias")
-
-  override private[flink] lazy val valid = false
-}
-
-case class RowtimeAttribute() extends Attribute {
-  override private[flink] def withName(newName: String): Attribute = {
-    if (newName == "rowtime") {
-      this
-    } else {
-      throw new ValidationException("Cannot rename streaming rowtime attribute.")
-    }
-  }
-
-  override private[flink] def name: String = "rowtime"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    throw new UnsupportedOperationException("A rowtime attribute can not be used solely.")
-  }
-
-  override private[flink] def resultType: TypeInformation[_] = BasicTypeInfo.LONG_TYPE_INFO
-}
-
-case class WindowReference(name: String) extends Attribute {
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode =
-    throw new UnsupportedOperationException("A window reference can not be used solely.")
-
-  override private[flink] def resultType: TypeInformation[_] =
-    throw new UnsupportedOperationException("A window reference has no result type.")
-
-  override private[flink] def withName(newName: String): Attribute = {
-    if (newName == name) {
-      this
-    } else {
-      throw new ValidationException("Cannot rename window reference.")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/literals.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/literals.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/literals.scala
deleted file mode 100644
index 6382abe..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/literals.scala
+++ /dev/null
@@ -1,120 +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.api.table.expressions
-
-import java.sql.{Date, Time, Timestamp}
-import java.util.{Calendar, TimeZone}
-
-import org.apache.calcite.avatica.util.TimeUnit
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.sql.SqlIntervalQualifier
-import org.apache.calcite.sql.`type`.SqlTypeName
-import org.apache.calcite.sql.parser.SqlParserPos
-import org.apache.calcite.tools.RelBuilder
-import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, SqlTimeTypeInfo, TypeInformation}
-import org.apache.flink.api.table.FlinkTypeFactory
-import org.apache.flink.api.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo}
-
-object Literal {
-  private[flink] def apply(l: Any): Literal = l match {
-    case i: Int => Literal(i, BasicTypeInfo.INT_TYPE_INFO)
-    case s: Short => Literal(s, BasicTypeInfo.SHORT_TYPE_INFO)
-    case b: Byte => Literal(b, BasicTypeInfo.BYTE_TYPE_INFO)
-    case l: Long => Literal(l, BasicTypeInfo.LONG_TYPE_INFO)
-    case d: Double => Literal(d, BasicTypeInfo.DOUBLE_TYPE_INFO)
-    case f: Float => Literal(f, BasicTypeInfo.FLOAT_TYPE_INFO)
-    case str: String => Literal(str, BasicTypeInfo.STRING_TYPE_INFO)
-    case bool: Boolean => Literal(bool, BasicTypeInfo.BOOLEAN_TYPE_INFO)
-    case javaDec: java.math.BigDecimal => Literal(javaDec, BasicTypeInfo.BIG_DEC_TYPE_INFO)
-    case scalaDec: scala.math.BigDecimal =>
-      Literal(scalaDec.bigDecimal, BasicTypeInfo.BIG_DEC_TYPE_INFO)
-    case sqlDate: Date => Literal(sqlDate, SqlTimeTypeInfo.DATE)
-    case sqlTime: Time => Literal(sqlTime, SqlTimeTypeInfo.TIME)
-    case sqlTimestamp: Timestamp => Literal(sqlTimestamp, SqlTimeTypeInfo.TIMESTAMP)
-  }
-}
-
-case class Literal(value: Any, resultType: TypeInformation[_]) extends LeafExpression {
-  override def toString = resultType match {
-    case _: BasicTypeInfo[_] => value.toString
-    case _@SqlTimeTypeInfo.DATE => value.toString + ".toDate"
-    case _@SqlTimeTypeInfo.TIME => value.toString + ".toTime"
-    case _@SqlTimeTypeInfo.TIMESTAMP => value.toString + ".toTimestamp"
-    case _@TimeIntervalTypeInfo.INTERVAL_MILLIS => value.toString + ".millis"
-    case _@TimeIntervalTypeInfo.INTERVAL_MONTHS => value.toString + ".months"
-    case _@RowIntervalTypeInfo.INTERVAL_ROWS => value.toString + ".rows"
-    case _ => s"Literal($value, $resultType)"
-  }
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    resultType match {
-      case BasicTypeInfo.BIG_DEC_TYPE_INFO =>
-        val bigDecValue = value.asInstanceOf[java.math.BigDecimal]
-        val decType = relBuilder.getTypeFactory.createSqlType(SqlTypeName.DECIMAL)
-        relBuilder.getRexBuilder.makeExactLiteral(bigDecValue, decType)
-
-      // date/time
-      case SqlTimeTypeInfo.DATE =>
-        relBuilder.getRexBuilder.makeDateLiteral(dateToCalendar)
-      case SqlTimeTypeInfo.TIME =>
-        relBuilder.getRexBuilder.makeTimeLiteral(dateToCalendar, 0)
-      case SqlTimeTypeInfo.TIMESTAMP =>
-        relBuilder.getRexBuilder.makeTimestampLiteral(dateToCalendar, 3)
-
-      case TimeIntervalTypeInfo.INTERVAL_MONTHS =>
-        val interval = java.math.BigDecimal.valueOf(value.asInstanceOf[Int])
-        val intervalQualifier = new SqlIntervalQualifier(
-          TimeUnit.YEAR,
-          TimeUnit.MONTH,
-          SqlParserPos.ZERO)
-        relBuilder.getRexBuilder.makeIntervalLiteral(interval, intervalQualifier)
-
-      case TimeIntervalTypeInfo.INTERVAL_MILLIS =>
-        val interval = java.math.BigDecimal.valueOf(value.asInstanceOf[Long])
-        val intervalQualifier = new SqlIntervalQualifier(
-          TimeUnit.DAY,
-          TimeUnit.SECOND,
-          SqlParserPos.ZERO)
-        relBuilder.getRexBuilder.makeIntervalLiteral(interval, intervalQualifier)
-
-      case _ => relBuilder.literal(value)
-    }
-  }
-
-  private def dateToCalendar: Calendar = {
-    val date = value.asInstanceOf[java.util.Date]
-    val cal = Calendar.getInstance()
-    val t = date.getTime
-    // according to Calcite's SqlFunctions.internalToXXX methods
-    cal.setTimeInMillis(t + TimeZone.getDefault.getOffset(t))
-    cal
-  }
-}
-
-case class Null(resultType: TypeInformation[_]) extends LeafExpression {
-  override def toString = s"null"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    val rexBuilder = relBuilder.getRexBuilder
-    val typeFactory = relBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-    rexBuilder
-      .makeCast(
-        typeFactory.createTypeFromTypeInfo(resultType),
-        rexBuilder.constantNull())
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala
deleted file mode 100644
index 9c8e279..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/expressions/logic.scala
+++ /dev/null
@@ -1,107 +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.api.table.expressions
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-import org.apache.calcite.tools.RelBuilder
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.api.table.validate._
-
-abstract class BinaryPredicate extends BinaryExpression {
-  override private[flink] def resultType = BasicTypeInfo.BOOLEAN_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (left.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO &&
-        right.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(s"$this only accepts children of Boolean type, " +
-        s"get $left : ${left.resultType} and $right : ${right.resultType}")
-    }
-  }
-}
-
-case class Not(child: Expression) extends UnaryExpression {
-
-  override def toString = s"!($child)"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.not(child.toRexNode)
-  }
-
-  override private[flink] def resultType = BasicTypeInfo.BOOLEAN_TYPE_INFO
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (child.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(s"Not operator requires a boolean expression as input, " +
-        s"but $child is of type ${child.resultType}")
-    }
-  }
-}
-
-case class And(left: Expression, right: Expression) extends BinaryPredicate {
-
-  override def toString = s"$left && $right"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.and(left.toRexNode, right.toRexNode)
-  }
-}
-
-case class Or(left: Expression, right: Expression) extends BinaryPredicate {
-
-  override def toString = s"$left || $right"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    relBuilder.or(left.toRexNode, right.toRexNode)
-  }
-}
-
-case class If(
-    condition: Expression,
-    ifTrue: Expression,
-    ifFalse: Expression)
-  extends Expression {
-  private[flink] def children = Seq(condition, ifTrue, ifFalse)
-
-  override private[flink] def resultType = ifTrue.resultType
-
-  override def toString = s"($condition)? $ifTrue : $ifFalse"
-
-  override private[flink] def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
-    val c = condition.toRexNode
-    val t = ifTrue.toRexNode
-    val f = ifFalse.toRexNode
-    relBuilder.call(SqlStdOperatorTable.CASE, c, t, f)
-  }
-
-  override private[flink] def validateInput(): ValidationResult = {
-    if (condition.resultType == BasicTypeInfo.BOOLEAN_TYPE_INFO &&
-        ifTrue.resultType == ifFalse.resultType) {
-      ValidationSuccess
-    } else {
-      ValidationFailure(
-        s"If should have boolean condition and same type of ifTrue and ifFalse, get " +
-          s"(${condition.resultType}, ${ifTrue.resultType}, ${ifFalse.resultType})")
-    }
-  }
-}


[10/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala
deleted file mode 100644
index 3eee4d4..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/CalcITCase.scala
+++ /dev/null
@@ -1,286 +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.api.scala.stream.table
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.stream.utils.{StreamITCase, StreamTestData}
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.expressions.Literal
-import org.apache.flink.api.table.{TableEnvironment, TableException}
-import org.apache.flink.types.Row
-import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.junit.Assert._
-import org.junit.Test
-
-import scala.collection.mutable
-
-class CalcITCase extends StreamingMultipleProgramsTestBase {
-
-  @Test
-  def testSimpleSelectAll(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).select('_1, '_2, '_3)
-
-    val results = ds.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList(
-        "1,1,Hi",
-        "2,2,Hello",
-        "3,2,Hello world")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testSelectFirst(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).select('_1)
-
-    val results = ds.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("1", "2", "3")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testSimpleSelectWithNaming(): Unit = {
-
-    // verify ProjectMergeRule.
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv)
-      .select('_1 as 'a, '_2 as 'b, '_1 as 'c)
-      .select('a, 'b)
-
-    val results = ds.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList(
-      "1,1", "2,2", "3,2", "4,3", "5,3", "6,3", "7,4",
-      "8,4", "9,4", "10,4", "11,5", "12,5", "13,5", "14,5", "15,5",
-      "16,6", "17,6", "18,6", "19,6", "20,6", "21,6")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testSimpleSelectAllWithAs(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
-      .select('a, 'b, 'c)
-
-    val results = ds.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList(
-        "1,1,Hi",
-        "2,2,Hello",
-        "3,2,Hello world")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testAsWithToFewFields(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b)
-
-    val results = ds.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("no")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testAsWithToManyFields(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c, 'd)
-
-    val results = ds.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("no")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test(expected = classOf[TableException])
-  def testAsWithAmbiguousFields(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'b)
-
-    val results = ds.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("no")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-
-  @Test(expected = classOf[TableException])
-  def testOnlyFieldRefInAs(): Unit = {
-
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b as 'c, 'd)
-
-    val results = ds.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("no")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testSimpleFilter(): Unit = {
-    /*
-     * Test simple filter
-     */
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter('a === 3)
-    val results = filterDs.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("3,2,Hello world")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testAllRejectingFilter(): Unit = {
-    /*
-     * Test all-rejecting filter
-     */
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter( Literal(false) )
-    val results = filterDs.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    assertEquals(true, StreamITCase.testResults.isEmpty)
-  }
-
-  @Test
-  def testAllPassingFilter(): Unit = {
-    /*
-     * Test all-passing filter
-     */
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter( Literal(true) )
-    val results = filterDs.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList(
-        "1,1,Hi",
-        "2,2,Hello",
-        "3,2,Hello world")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testFilterOnIntegerTupleField(): Unit = {
-    /*
-     * Test filter on Integer tuple field.
-     */
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter( 'a % 2 === 0 )
-    val results = filterDs.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList(
-      "2,2,Hello", "4,3,Hello world, how are you?",
-      "6,3,Luke Skywalker", "8,4,Comment#2", "10,4,Comment#4",
-      "12,5,Comment#6", "14,5,Comment#8", "16,6,Comment#10",
-      "18,6,Comment#12", "20,6,Comment#14")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testNotEquals(): Unit = {
-    /*
-     * Test filter on Integer tuple field.
-     */
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    StreamITCase.testResults = mutable.MutableList()
-    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
-
-    val filterDs = ds.filter( 'a % 2 !== 0)
-    val results = filterDs.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-    val expected = mutable.MutableList(
-      "1,1,Hi", "3,2,Hello world",
-      "5,3,I am fine.", "7,4,Comment#1", "9,4,Comment#3",
-      "11,5,Comment#5", "13,5,Comment#7", "15,5,Comment#9",
-      "17,6,Comment#11", "19,6,Comment#13", "21,6,Comment#15")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/GroupWindowTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/GroupWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/GroupWindowTest.scala
deleted file mode 100644
index 9c2d6b3..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/GroupWindowTest.scala
+++ /dev/null
@@ -1,734 +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.api.scala.stream.table
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table._
-import org.apache.flink.api.table.expressions.{RowtimeAttribute, WindowReference}
-import org.apache.flink.api.table.plan.logical._
-import org.apache.flink.api.table.utils.TableTestBase
-import org.apache.flink.api.table.utils.TableTestUtil.{streamTableNode, term, unaryNode}
-import org.junit.{Ignore, Test}
-
-class GroupWindowTest extends TableTestBase {
-
-  // batch windows are not supported yet
-  @Test(expected = classOf[ValidationException])
-  def testInvalidBatchWindow(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .groupBy('string)
-      .window(Session withGap 100.milli as 'string)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidWindowProperty(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .groupBy('string)
-      .select('string, 'string.start) // property in non windowed table
-  }
-
-  @Test(expected = classOf[TableException])
-  def testInvalidRowtime1(): Unit = {
-    val util = streamTestUtil()
-    // rowtime attribute must not be a field name
-    util.addTable[(Long, Int, String)]('rowtime, 'long, 'int, 'string)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidRowtime2(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .select('string, 'int as 'rowtime) // rowtime attribute must not be an alias
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidRowtime3(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table.as('rowtime, 'myint, 'mystring) // rowtime attribute must not be an alias
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidRowtime4(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .groupBy('string)
-      // only rowtime is a valid time attribute in a stream environment
-      .window(Tumble over 50.milli on 'string)
-      .select('string, 'int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidTumblingSize(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .groupBy('string)
-      .window(Tumble over "WRONG") // string is not a valid interval
-      .select('string, 'int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidSlidingSize(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .groupBy('string)
-      .window(Slide over "WRONG" every "WRONG") // string is not a valid interval
-      .select('string, 'int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidSlidingSlide(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .groupBy('string)
-      .window(Slide over 12.rows every 1.minute) // row and time intervals may not be mixed
-      .select('string, 'int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidSessionGap(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .groupBy('string)
-      .window(Session withGap 10.rows) // row interval is not valid for session windows
-      .select('string, 'int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidWindowAlias1(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .groupBy('string)
-      .window(Session withGap 100.milli as 1 + 1) // expression instead of a symbol
-      .select('string, 'int.count)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testInvalidWindowAlias2(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    table
-      .groupBy('string)
-      .window(Session withGap 100.milli as 'string) // field name "string" is already present
-      .select('string, 'int.count)
-  }
-
-  @Test
-  def testProcessingTimeTumblingGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Tumble over 50.milli)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term("window", ProcessingTimeTumblingGroupWindow(None, 50.milli)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testProcessingTimeTumblingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Tumble over 2.rows)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term("window", ProcessingTimeTumblingGroupWindow(None, 2.rows)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testEventTimeTumblingGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Tumble over 5.milli on 'rowtime)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term("window", EventTimeTumblingGroupWindow(None, RowtimeAttribute(), 5.milli)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  @Ignore // see comments in DataStreamAggregate
-  def testEventTimeTumblingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Tumble over 2.rows on 'rowtime)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      streamTableNode(0),
-      term("groupBy", "string"),
-      term("window", EventTimeTumblingGroupWindow(None, RowtimeAttribute(), 2.rows)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testProcessingTimeSlidingGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Slide over 50.milli every 50.milli)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term("window", ProcessingTimeSlidingGroupWindow(None, 50.milli, 50.milli)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testProcessingTimeSlidingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Slide over 2.rows every 1.rows)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term("window", ProcessingTimeSlidingGroupWindow(None, 2.rows, 1.rows)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testEventTimeSlidingGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Slide over 8.milli every 10.milli on 'rowtime)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term("window", EventTimeSlidingGroupWindow(None, RowtimeAttribute(), 8.milli, 10.milli)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  @Ignore // see comments in DataStreamAggregate
-  def testEventTimeSlidingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Slide over 2.rows every 1.rows on 'rowtime)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      streamTableNode(0),
-      term("groupBy", "string"),
-      term("window", EventTimeSlidingGroupWindow(None, RowtimeAttribute(), 2.rows, 1.rows)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testEventTimeSessionGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Session withGap 7.milli on 'rowtime)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term("window", EventTimeSessionGroupWindow(None, RowtimeAttribute(), 7.milli)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testAllProcessingTimeTumblingGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Tumble over 50.milli)
-      .select('string, 'int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term("window", ProcessingTimeTumblingGroupWindow(None, 50.milli)),
-      term("select", "string", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testAllProcessingTimeTumblingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Tumble over 2.rows)
-      .select('int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "int")
-      ),
-      term("window", ProcessingTimeTumblingGroupWindow(None, 2.rows)),
-      term("select", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testAllEventTimeTumblingGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Tumble over 5.milli on 'rowtime)
-      .select('int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "int")
-      ),
-      term("window", EventTimeTumblingGroupWindow(None, RowtimeAttribute(), 5.milli)),
-      term("select", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  @Ignore // see comments in DataStreamAggregate
-  def testAllEventTimeTumblingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Tumble over 2.rows on 'rowtime)
-      .select('int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "int")
-      ),
-      term("window", EventTimeTumblingGroupWindow(None, RowtimeAttribute(), 2.rows)),
-      term("select", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-
-  @Test
-  def testAllProcessingTimeSlidingGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Slide over 50.milli every 50.milli)
-      .select('int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "int")
-      ),
-      term("window", ProcessingTimeSlidingGroupWindow(None, 50.milli, 50.milli)),
-      term("select", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testAllProcessingTimeSlidingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Slide over 2.rows every 1.rows)
-      .select('int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "int")
-      ),
-      term("window", ProcessingTimeSlidingGroupWindow(None, 2.rows, 1.rows)),
-      term("select", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testAllEventTimeSlidingGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Slide over 8.milli every 10.milli on 'rowtime)
-      .select('int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "int")
-      ),
-      term("window", EventTimeSlidingGroupWindow(None, RowtimeAttribute(), 8.milli, 10.milli)),
-      term("select", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  @Ignore // see comments in DataStreamAggregate
-  def testAllEventTimeSlidingGroupWindowOverCount(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Slide over 2.rows every 1.rows on 'rowtime)
-      .select('int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "int")
-      ),
-      term("window", EventTimeSlidingGroupWindow(None, RowtimeAttribute(), 2.rows, 1.rows)),
-      term("select", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testAllEventTimeSessionGroupWindowOverTime(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .window(Session withGap 7.milli on 'rowtime)
-      .select('int.count)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "int")
-      ),
-      term("window", EventTimeSessionGroupWindow(None, RowtimeAttribute(), 7.milli)),
-      term("select", "COUNT(int) AS TMP_0")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testTumbleWindowStartEnd(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Tumble over 5.milli on 'rowtime as 'w)
-      .select('string, 'int.count, 'w.start, 'w.end)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term("window",
-        EventTimeTumblingGroupWindow(
-          Some(WindowReference("w")),
-          RowtimeAttribute(),
-          5.milli)),
-      term("select",
-        "string",
-        "COUNT(int) AS TMP_0",
-        "start(WindowReference(w)) AS TMP_1",
-        "end(WindowReference(w)) AS TMP_2")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testSlideWindowStartEnd(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Slide over 10.milli every 5.milli on 'rowtime as 'w)
-      .select('string, 'int.count, 'w.start, 'w.end)
-
-    val expected = unaryNode(
-      "DataStreamAggregate",
-      unaryNode(
-        "DataStreamCalc",
-        streamTableNode(0),
-        term("select", "string", "int")
-      ),
-      term("groupBy", "string"),
-      term("window",
-        EventTimeSlidingGroupWindow(
-          Some(WindowReference("w")),
-          RowtimeAttribute(),
-          10.milli,
-          5.milli)),
-      term("select",
-        "string",
-        "COUNT(int) AS TMP_0",
-        "start(WindowReference(w)) AS TMP_1",
-        "end(WindowReference(w)) AS TMP_2")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testSessionWindowStartWithTwoEnd(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Session withGap 3.milli on 'rowtime as 'w)
-      .select('w.end as 'we1, 'string, 'int.count as 'cnt, 'w.start as 'ws, 'w.end as 'we2)
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamAggregate",
-        unaryNode(
-          "DataStreamCalc",
-          streamTableNode(0),
-          term("select", "string", "int")
-        ),
-        term("groupBy", "string"),
-        term("window",
-          EventTimeSessionGroupWindow(
-            Some(WindowReference("w")),
-            RowtimeAttribute(),
-            3.milli)),
-        term("select",
-          "string",
-          "COUNT(int) AS TMP_1",
-          "end(WindowReference(w)) AS TMP_0",
-          "start(WindowReference(w)) AS TMP_2")
-      ),
-      term("select", "TMP_0 AS we1", "string", "TMP_1 AS cnt", "TMP_2 AS ws", "TMP_0 AS we2")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-
-  @Test
-  def testTumbleWindowWithDuplicateAggsAndProps(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
-
-    val windowedTable = table
-      .groupBy('string)
-      .window(Tumble over 5.millis on 'rowtime as 'w)
-      .select('string, 'int.sum + 1 as 's1, 'int.sum + 3 as 's2, 'w.start as 'x, 'w.start as 'x2,
-        'w.end as 'x3, 'w.end)
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamAggregate",
-        unaryNode(
-          "DataStreamCalc",
-          streamTableNode(0),
-          term("select", "string", "int")
-        ),
-        term("groupBy", "string"),
-        term("window",
-          EventTimeTumblingGroupWindow(
-            Some(WindowReference("w")),
-            RowtimeAttribute(),
-            5.millis)),
-        term("select",
-          "string",
-          "SUM(int) AS TMP_0",
-          "start(WindowReference(w)) AS TMP_1",
-          "end(WindowReference(w)) AS TMP_2")
-      ),
-      term("select",
-        "string",
-        "+(CAST(AS(TMP_0, 'TMP_3')), CAST(1)) AS s1",
-        "+(CAST(AS(TMP_0, 'TMP_4')), CAST(3)) AS s2",
-        "TMP_1 AS x",
-        "TMP_1 AS x2",
-        "TMP_2 AS x3",
-        "TMP_2 AS TMP_5")
-    )
-
-    util.verifyTable(windowedTable, expected)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala
deleted file mode 100644
index 5096b53..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnionITCase.scala
+++ /dev/null
@@ -1,125 +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.api.scala.stream.table
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.stream.utils.{StreamITCase, StreamTestData}
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.{TableEnvironment, ValidationException}
-import org.apache.flink.types.Row
-import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.junit.Assert._
-import org.junit.Test
-
-import scala.collection.mutable
-
-class UnionITCase extends StreamingMultipleProgramsTestBase {
-
-  @Test
-  def testUnion(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    StreamITCase.testResults = mutable.MutableList()
-    val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'd, 'e, 'f)
-
-    val unionDs = ds1.unionAll(ds2).select('c)
-
-    val results = unionDs.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList(
-        "Hi", "Hello", "Hello world", "Hi", "Hello", "Hello world")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test
-  def testUnionWithFilter(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    StreamITCase.testResults = mutable.MutableList()
-    val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = StreamTestData.get5TupleDataStream(env).toTable(tEnv, 'a, 'b, 'd, 'c, 'e)
-
-    val unionDs = ds1.unionAll(ds2.select('a, 'b, 'c)).filter('b < 2).select('c)
-
-    val results = unionDs.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    val expected = mutable.MutableList("Hi", "Hallo")
-    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testUnionFieldsNameNotOverlap1(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    StreamITCase.testResults = mutable.MutableList()
-    val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = StreamTestData.get5TupleDataStream(env).toTable(tEnv, 'a, 'b, 'd, 'c, 'e)
-
-    val unionDs = ds1.unionAll(ds2)
-
-    val results = unionDs.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    assertEquals(true, StreamITCase.testResults.isEmpty)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testUnionFieldsNameNotOverlap2(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-
-    StreamITCase.testResults = mutable.MutableList()
-    val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
-    val ds2 = StreamTestData.get5TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
-      .select('a, 'b, 'c)
-
-    val unionDs = ds1.unionAll(ds2)
-
-    val results = unionDs.toDataStream[Row]
-    results.addSink(new StreamITCase.StringSink)
-    env.execute()
-
-    println(StreamITCase.testResults)
-    assertEquals(true, StreamITCase.testResults.isEmpty)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testUnionTablesFromDifferentEnvs(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv1 = TableEnvironment.getTableEnvironment(env)
-    val tEnv2 = TableEnvironment.getTableEnvironment(env)
-
-    val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv1, 'a, 'b, 'c)
-    val ds2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv2, 'a, 'b, 'c)
-
-    // Must fail. Tables are bound to different TableEnvironments.
-    ds1.unionAll(ds2)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala
deleted file mode 100644
index 6d1a62e..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UnsupportedOpsTest.scala
+++ /dev/null
@@ -1,112 +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.api.scala.stream.table
-
-import org.apache.flink.api.scala.stream.utils.StreamTestData
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.{TableEnvironment, ValidationException}
-import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
-import org.junit.Test
-
-class UnsupportedOpsTest extends StreamingMultipleProgramsTestBase {
-
-  @Test(expected = classOf[ValidationException])
-  def testSelectWithAggregation(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).select('_1.min)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testDistinct(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).distinct()
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testSort(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).orderBy('_1.desc)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testJoin(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    t1.join(t2)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testUnion(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    t1.union(t2)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testIntersect(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    t1.intersect(t2)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testIntersectAll(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    t1.intersectAll(t2)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testMinus(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    t1.minus(t2)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testMinusAll(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    t1.minusAll(t2)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testLimit(): Unit = {
-    val env = StreamExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
-    t1.limit(0,5)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UserDefinedTableFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UserDefinedTableFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UserDefinedTableFunctionTest.scala
deleted file mode 100644
index 305f1db..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/table/UserDefinedTableFunctionTest.scala
+++ /dev/null
@@ -1,385 +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.api.scala.stream.table
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.types.Row
-import org.apache.flink.api.table.expressions.utils._
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.{TableEnvironment, TableException, Types, ValidationException}
-import org.apache.flink.api.table.utils.TableTestUtil._
-import org.apache.flink.api.table.utils._
-import org.apache.flink.streaming.api.datastream.{DataStream => JDataStream}
-import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaExecutionEnv}
-import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment => ScalaExecutionEnv}
-import org.junit.Assert.{assertTrue, fail}
-import org.junit.Test
-import org.mockito.Mockito._
-
-class UserDefinedTableFunctionTest extends TableTestBase {
-
-  @Test
-  def testJavaScalaTableAPIEquality(): Unit = {
-    // mock
-    val ds = mock(classOf[DataStream[Row]])
-    val jDs = mock(classOf[JDataStream[Row]])
-    val typeInfo = new RowTypeInfo(Seq(Types.INT, Types.LONG, Types.STRING): _*)
-    when(ds.javaStream).thenReturn(jDs)
-    when(jDs.getType).thenReturn(typeInfo)
-
-    // Scala environment
-    val env = mock(classOf[ScalaExecutionEnv])
-    val tableEnv = TableEnvironment.getTableEnvironment(env)
-    val in1 = ds.toTable(tableEnv).as('a, 'b, 'c)
-
-    // Java environment
-    val javaEnv = mock(classOf[JavaExecutionEnv])
-    val javaTableEnv = TableEnvironment.getTableEnvironment(javaEnv)
-    val in2 = javaTableEnv.fromDataStream(jDs).as("a, b, c")
-
-    // test cross join
-    val func1 = new TableFunc1
-    javaTableEnv.registerFunction("func1", func1)
-    var scalaTable = in1.join(func1('c) as 's).select('c, 's)
-    var javaTable = in2.join("func1(c).as(s)").select("c, s")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test left outer join
-    scalaTable = in1.leftOuterJoin(func1('c) as 's).select('c, 's)
-    javaTable = in2.leftOuterJoin("as(func1(c), s)").select("c, s")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test overloading
-    scalaTable = in1.join(func1('c, "$") as 's).select('c, 's)
-    javaTable = in2.join("func1(c, '$') as (s)").select("c, s")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test custom result type
-    val func2 = new TableFunc2
-    javaTableEnv.registerFunction("func2", func2)
-    scalaTable = in1.join(func2('c) as ('name, 'len)).select('c, 'name, 'len)
-    javaTable = in2.join("func2(c).as(name, len)").select("c, name, len")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test hierarchy generic type
-    val hierarchy = new HierarchyTableFunction
-    javaTableEnv.registerFunction("hierarchy", hierarchy)
-    scalaTable = in1.join(hierarchy('c) as ('name, 'adult, 'len))
-      .select('c, 'name, 'len, 'adult)
-    javaTable = in2.join("AS(hierarchy(c), name, adult, len)")
-      .select("c, name, len, adult")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test pojo type
-    val pojo = new PojoTableFunc
-    javaTableEnv.registerFunction("pojo", pojo)
-    scalaTable = in1.join(pojo('c))
-      .select('c, 'name, 'age)
-    javaTable = in2.join("pojo(c)")
-      .select("c, name, age")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test with filter
-    scalaTable = in1.join(func2('c) as ('name, 'len))
-      .select('c, 'name, 'len).filter('len > 2)
-    javaTable = in2.join("func2(c) as (name, len)")
-      .select("c, name, len").filter("len > 2")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // test with scalar function
-    scalaTable = in1.join(func1('c.substring(2)) as 's)
-      .select('a, 'c, 's)
-    javaTable = in2.join("func1(substring(c, 2)) as (s)")
-      .select("a, c, s")
-    verifyTableEquals(scalaTable, javaTable)
-
-    // check scala object is forbidden
-    expectExceptionThrown(
-      tableEnv.registerFunction("func3", ObjectTableFunction), "Scala object")
-    expectExceptionThrown(
-      javaTableEnv.registerFunction("func3", ObjectTableFunction), "Scala object")
-    expectExceptionThrown(
-      in1.join(ObjectTableFunction('a, 1)), "Scala object")
-
-  }
-
-  @Test
-  def testInvalidTableFunction(): Unit = {
-    // mock
-    val util = streamTestUtil()
-    val t = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val tEnv = TableEnvironment.getTableEnvironment(mock(classOf[JavaExecutionEnv]))
-
-    //=================== check scala object is forbidden =====================
-    // Scala table environment register
-    expectExceptionThrown(util.addFunction("udtf", ObjectTableFunction), "Scala object")
-    // Java table environment register
-    expectExceptionThrown(tEnv.registerFunction("udtf", ObjectTableFunction), "Scala object")
-    // Scala Table API directly call
-    expectExceptionThrown(t.join(ObjectTableFunction('a, 1)), "Scala object")
-
-
-    //============ throw exception when table function is not registered =========
-    // Java Table API call
-    expectExceptionThrown(t.join("nonexist(a)"), "Undefined function: NONEXIST")
-    // SQL API call
-    expectExceptionThrown(
-      util.tEnv.sql("SELECT * FROM MyTable, LATERAL TABLE(nonexist(a))"),
-      "No match found for function signature nonexist(<NUMERIC>)")
-
-
-    //========= throw exception when the called function is a scalar function ====
-    util.addFunction("func0", Func0)
-    // Java Table API call
-    expectExceptionThrown(
-      t.join("func0(a)"),
-      "only accept expressions that define table functions",
-      classOf[TableException])
-    // SQL API call
-    // NOTE: it doesn't throw an exception but an AssertionError, maybe a Calcite bug
-    expectExceptionThrown(
-      util.tEnv.sql("SELECT * FROM MyTable, LATERAL TABLE(func0(a))"),
-      null,
-      classOf[AssertionError])
-
-    //========== throw exception when the parameters is not correct ===============
-    // Java Table API call
-    util.addFunction("func2", new TableFunc2)
-    expectExceptionThrown(
-      t.join("func2(c, c)"),
-      "Given parameters of function 'FUNC2' do not match any signature")
-    // SQL API call
-    expectExceptionThrown(
-      util.tEnv.sql("SELECT * FROM MyTable, LATERAL TABLE(func2(c, c))"),
-      "No match found for function signature func2(<CHARACTER>, <CHARACTER>)")
-  }
-
-  @Test
-  def testCrossJoin(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = util.addFunction("func1", new TableFunc1)
-
-    val result1 = table.join(function('c) as 's).select('c, 's)
-
-    val expected1 = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", s"$function($$2)"),
-        term("function", function),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "s")
-    )
-
-    util.verifyTable(result1, expected1)
-
-    // test overloading
-
-    val result2 = table.join(function('c, "$") as 's).select('c, 's)
-
-    val expected2 = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", s"$function($$2, '$$')"),
-        term("function", function),
-        term("rowType",
-             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "s")
-    )
-
-    util.verifyTable(result2, expected2)
-  }
-
-  @Test
-  def testLeftOuterJoin(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = util.addFunction("func1", new TableFunc1)
-
-    val result = table.leftOuterJoin(function('c) as 's).select('c, 's)
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", s"$function($$2)"),
-        term("function", function),
-        term("rowType",
-          "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
-        term("joinType", "LEFT")
-      ),
-      term("select", "c", "s")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testCustomType(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = util.addFunction("func2", new TableFunc2)
-
-    val result = table.join(function('c) as ('name, 'len)).select('c, 'name, 'len)
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", s"$function($$2)"),
-        term("function", function),
-        term("rowType",
-          "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
-           "VARCHAR(2147483647) name, INTEGER len)"),
-        term("joinType", "INNER")
-      ),
-      term("select", "c", "name", "len")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testHierarchyType(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = util.addFunction("hierarchy", new HierarchyTableFunction)
-
-    val result = table.join(function('c) as ('name, 'adult, 'len))
-
-    val expected = unaryNode(
-      "DataStreamCorrelate",
-      streamTableNode(0),
-      term("invocation", s"$function($$2)"),
-      term("function", function),
-      term("rowType",
-        "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," +
-        " VARCHAR(2147483647) name, BOOLEAN adult, INTEGER len)"),
-      term("joinType", "INNER")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testPojoType(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = util.addFunction("pojo", new PojoTableFunc)
-
-    val result = table.join(function('c))
-
-    val expected = unaryNode(
-      "DataStreamCorrelate",
-      streamTableNode(0),
-      term("invocation", s"$function($$2)"),
-      term("function", function),
-      term("rowType",
-        "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
-         "INTEGER age, VARCHAR(2147483647) name)"),
-      term("joinType", "INNER")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testFilter(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = util.addFunction("func2", new TableFunc2)
-
-    val result = table
-      .join(function('c) as ('name, 'len))
-      .select('c, 'name, 'len)
-      .filter('len > 2)
-
-    val expected = unaryNode(
-      "DataStreamCalc",
-      unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation", s"$function($$2)"),
-        term("function", function),
-        term("rowType",
-          "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
-          "VARCHAR(2147483647) name, INTEGER len)"),
-        term("joinType", "INNER"),
-        term("condition", ">($1, 2)")
-      ),
-      term("select", "c", "name", "len")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testScalarFunction(): Unit = {
-    val util = streamTestUtil()
-    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-    val function = util.addFunction("func1", new TableFunc1)
-
-    val result = table.join(function('c.substring(2)) as 's)
-
-    val expected = unaryNode(
-        "DataStreamCorrelate",
-        streamTableNode(0),
-        term("invocation",  s"$function(SUBSTRING($$2, 2, CHAR_LENGTH($$2)))"),
-        term("function", function),
-        term("rowType",
-          "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
-        term("joinType", "INNER")
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  private def expectExceptionThrown(
-      function: => Unit,
-      keywords: String,
-      clazz: Class[_ <: Throwable] = classOf[ValidationException])
-    : Unit = {
-    try {
-      function
-      fail(s"Expected a $clazz, but no exception is thrown.")
-    } catch {
-      case e if e.getClass == clazz =>
-        if (keywords != null) {
-          assertTrue(
-            s"The exception message '${e.getMessage}' doesn't contain keyword '$keywords'",
-            e.getMessage.contains(keywords))
-        }
-      case e: Throwable => fail(s"Expected throw ${clazz.getSimpleName}, but is $e.")
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamITCase.scala
deleted file mode 100644
index 4fd3cd7..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamITCase.scala
+++ /dev/null
@@ -1,49 +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.api.scala.stream.utils
-
-import java.util.Collections
-
-import org.apache.flink.types.Row
-import org.junit.Assert._
-import scala.collection.mutable
-import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
-import scala.collection.JavaConverters._
-
-object StreamITCase {
-
-  var testResults = mutable.MutableList.empty[String]
-
-  def clear = {
-    StreamITCase.testResults.clear()
-  }
-
-  def compareWithList(expected: java.util.List[String]): Unit = {
-    Collections.sort(expected)
-    assertEquals(expected.asScala, StreamITCase.testResults.sorted)
-  }
-
-  final class StringSink extends RichSinkFunction[Row]() {
-    def invoke(value: Row) {
-      testResults.synchronized {
-        testResults += value.toString 
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamTestData.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamTestData.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamTestData.scala
deleted file mode 100644
index 321b8ac..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/scala/stream/utils/StreamTestData.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.api.scala.stream.utils
-
-import org.apache.flink.api.scala._
-import scala.collection.mutable
-import org.apache.flink.streaming.api.scala.DataStream
-import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
-
-object StreamTestData {
-
-  def getSmall3TupleDataStream(env: StreamExecutionEnvironment): DataStream[(Int, Long, String)] = {
-    val data = new mutable.MutableList[(Int, Long, String)]
-    data.+=((1, 1L, "Hi"))
-    data.+=((2, 2L, "Hello"))
-    data.+=((3, 2L, "Hello world"))
-    env.fromCollection(data)
-  }
-
-  def get3TupleDataStream(env: StreamExecutionEnvironment): DataStream[(Int, Long, String)] = {
-    val data = new mutable.MutableList[(Int, Long, String)]
-    data.+=((1, 1L, "Hi"))
-    data.+=((2, 2L, "Hello"))
-    data.+=((3, 2L, "Hello world"))
-    data.+=((4, 3L, "Hello world, how are you?"))
-    data.+=((5, 3L, "I am fine."))
-    data.+=((6, 3L, "Luke Skywalker"))
-    data.+=((7, 4L, "Comment#1"))
-    data.+=((8, 4L, "Comment#2"))
-    data.+=((9, 4L, "Comment#3"))
-    data.+=((10, 4L, "Comment#4"))
-    data.+=((11, 5L, "Comment#5"))
-    data.+=((12, 5L, "Comment#6"))
-    data.+=((13, 5L, "Comment#7"))
-    data.+=((14, 5L, "Comment#8"))
-    data.+=((15, 5L, "Comment#9"))
-    data.+=((16, 6L, "Comment#10"))
-    data.+=((17, 6L, "Comment#11"))
-    data.+=((18, 6L, "Comment#12"))
-    data.+=((19, 6L, "Comment#13"))
-    data.+=((20, 6L, "Comment#14"))
-    data.+=((21, 6L, "Comment#15"))
-    env.fromCollection(data)
-  }
-
-  def get5TupleDataStream(env: StreamExecutionEnvironment):
-      DataStream[(Int, Long, Int, String, Long)] = {
-
-    val data = new mutable.MutableList[(Int, Long, Int, String, Long)]
-    data.+=((1, 1L, 0, "Hallo", 1L))
-    data.+=((2, 2L, 1, "Hallo Welt", 2L))
-    data.+=((2, 3L, 2, "Hallo Welt wie", 1L))
-    data.+=((3, 4L, 3, "Hallo Welt wie gehts?", 2L))
-    data.+=((3, 5L, 4, "ABC", 2L))
-    data.+=((3, 6L, 5, "BCD", 3L))
-    data.+=((4, 7L, 6, "CDE", 2L))
-    data.+=((4, 8L, 7, "DEF", 1L))
-    data.+=((4, 9L, 8, "EFG", 1L))
-    data.+=((4, 10L, 9, "FGH", 2L))
-    data.+=((5, 11L, 10, "GHI", 1L))
-    data.+=((5, 12L, 11, "HIJ", 3L))
-    data.+=((5, 13L, 12, "IJK", 3L))
-    data.+=((5, 14L, 13, "JKL", 2L))
-    data.+=((5, 15L, 14, "KLM", 2L))
-    env.fromCollection(data)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/AggregationTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/AggregationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/AggregationTest.scala
deleted file mode 100644
index 6c9d2e8..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/AggregationTest.scala
+++ /dev/null
@@ -1,261 +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.api.table
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.utils.TableTestBase
-import org.apache.flink.api.table.utils.TableTestUtil._
-import org.junit.Test
-
-/**
-  * Test for testing aggregate plans.
-  */
-class AggregationTest extends TableTestBase {
-
-  @Test
-  def testAggregateQueryBatchSQL(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM MyTable"
-
-    val setValues = unaryNode(
-      "DataSetValues",
-      batchTableNode(0),
-      tuples(List(null,null,null)),
-      term("values","a","b","c")
-    )
-    val union = unaryNode(
-      "DataSetUnion",
-      setValues,
-      term("union","a","b","c")
-    )
-
-    val aggregate = unaryNode(
-      "DataSetAggregate",
-      union,
-      term("select",
-        "AVG(a) AS EXPR$0",
-        "SUM(b) AS EXPR$1",
-        "COUNT(c) AS EXPR$2")
-    )
-    util.verifySql(sqlQuery, aggregate)
-  }
-
-  @Test
-  def testAggregateWithFilterQueryBatchSQL(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM MyTable WHERE a = 1"
-
-    val calcNode = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select", "a", "b", "c"),
-      term("where", "=(a, 1)")
-    )
-
-    val setValues =  unaryNode(
-        "DataSetValues",
-        calcNode,
-        tuples(List(null,null,null)),
-        term("values","a","b","c")
-    )
-
-    val union = unaryNode(
-      "DataSetUnion",
-      setValues,
-      term("union","a","b","c")
-    )
-
-    val aggregate = unaryNode(
-      "DataSetAggregate",
-      union,
-      term("select",
-        "AVG(a) AS EXPR$0",
-        "SUM(b) AS EXPR$1",
-        "COUNT(c) AS EXPR$2")
-    )
-    util.verifySql(sqlQuery, aggregate)
-  }
-
-  @Test
-  def testAggregateGroupQueryBatchSQL(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM MyTable GROUP BY a"
-
-    val aggregate = unaryNode(
-        "DataSetAggregate",
-        batchTableNode(0),
-        term("groupBy", "a"),
-        term("select",
-          "a",
-          "AVG(a) AS EXPR$0",
-          "SUM(b) AS EXPR$1",
-          "COUNT(c) AS EXPR$2")
-    )
-    val expected = unaryNode(
-        "DataSetCalc",
-        aggregate,
-        term("select",
-          "EXPR$0",
-          "EXPR$1",
-          "EXPR$2")
-    )
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testAggregateGroupWithFilterQueryBatchSQL(): Unit = {
-    val util = batchTestUtil()
-    util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
-
-    val sqlQuery = "SELECT avg(a), sum(b), count(c) FROM MyTable WHERE a = 1 GROUP BY a"
-
-    val calcNode = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select","a", "b", "c") ,
-      term("where","=(a, 1)")
-    )
-
-    val aggregate = unaryNode(
-        "DataSetAggregate",
-        calcNode,
-        term("groupBy", "a"),
-        term("select",
-          "a",
-          "AVG(a) AS EXPR$0",
-          "SUM(b) AS EXPR$1",
-          "COUNT(c) AS EXPR$2")
-    )
-    val expected = unaryNode(
-        "DataSetCalc",
-        aggregate,
-        term("select",
-          "EXPR$0",
-          "EXPR$1",
-          "EXPR$2")
-    )
-    util.verifySql(sqlQuery, expected)
-  }
-
-  @Test
-  def testAggregateGroupWithFilterTableApi(): Unit = {
-
-    val util = batchTestUtil()
-    val sourceTable = util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
-
-    val resultTable = sourceTable.groupBy('a)
-      .select('a, 'a.avg, 'b.sum, 'c.count)
-      .where('a === 1)
-
-    val calcNode = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select", "a", "b", "c"),
-      term("where", "=(a, 1)")
-    )
-
-    val expected = unaryNode(
-      "DataSetAggregate",
-      calcNode,
-      term("groupBy", "a"),
-      term("select",
-        "a",
-        "AVG(a) AS TMP_0",
-        "SUM(b) AS TMP_1",
-        "COUNT(c) AS TMP_2")
-    )
-
-    util.verifyTable(resultTable,expected)
-  }
-
-  @Test
-  def testAggregateTableApi(): Unit = {
-    val util = batchTestUtil()
-    val sourceTable = util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
-    val resultTable = sourceTable.select('a.avg,'b.sum,'c.count)
-
-    val setValues = unaryNode(
-      "DataSetValues",
-      batchTableNode(0),
-      tuples(List(null,null,null)),
-      term("values","a","b","c")
-    )
-    val union = unaryNode(
-      "DataSetUnion",
-      setValues,
-      term("union","a","b","c")
-    )
-
-    val expected = unaryNode(
-      "DataSetAggregate",
-      union,
-      term("select",
-        "AVG(a) AS TMP_0",
-        "SUM(b) AS TMP_1",
-        "COUNT(c) AS TMP_2")
-    )
-    util.verifyTable(resultTable, expected)
-  }
-
-  @Test
-  def testAggregateWithFilterTableApi(): Unit = {
-    val util = batchTestUtil()
-    val sourceTable = util.addTable[(Int, Long, Int)]("MyTable", 'a, 'b, 'c)
-
-    val resultTable = sourceTable.select('a,'b,'c).where('a === 1)
-      .select('a.avg,'b.sum,'c.count)
-
-    val calcNode = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select", "a", "b", "c"),
-      term("where", "=(a, 1)")
-    )
-
-    val setValues =  unaryNode(
-      "DataSetValues",
-      calcNode,
-      tuples(List(null,null,null)),
-      term("values","a","b","c")
-    )
-
-    val union = unaryNode(
-      "DataSetUnion",
-      setValues,
-      term("union","a","b","c")
-    )
-
-    val expected = unaryNode(
-      "DataSetAggregate",
-      union,
-      term("select",
-        "AVG(a) AS TMP_0",
-        "SUM(b) AS TMP_1",
-        "COUNT(c) AS TMP_2")
-    )
-
-    util.verifyTable(resultTable, expected)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/CalciteConfigBuilderTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/CalciteConfigBuilderTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/CalciteConfigBuilderTest.scala
deleted file mode 100644
index 2b0d446..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/CalciteConfigBuilderTest.scala
+++ /dev/null
@@ -1,208 +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.api.table
-
-import org.apache.calcite.rel.rules.{CalcSplitRule, CalcMergeRule, FilterMergeRule}
-import org.apache.calcite.sql.fun.{SqlStdOperatorTable, OracleSqlOperatorTable}
-import org.apache.calcite.tools.RuleSets
-import org.junit.Test
-import org.junit.Assert._
-
-import scala.collection.JavaConverters._
-
-class CalciteConfigBuilderTest {
-
-  @Test
-  def testDefaultRules(): Unit = {
-
-    val cc: CalciteConfig = new CalciteConfigBuilder()
-      .build()
-
-    assertEquals(false, cc.replacesRuleSet)
-    assertFalse(cc.getRuleSet.isDefined)
-  }
-
-  @Test
-  def testReplaceRules(): Unit = {
-
-    val cc: CalciteConfig = new CalciteConfigBuilder()
-      .replaceRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
-      .build()
-
-    assertEquals(true, cc.replacesRuleSet)
-    assertTrue(cc.getRuleSet.isDefined)
-    val cSet = cc.getRuleSet.get.iterator().asScala.toSet
-    assertEquals(1, cSet.size)
-    assertTrue(cSet.contains(FilterMergeRule.INSTANCE))
-  }
-
-  @Test
-  def testReplaceAddRules(): Unit = {
-
-    val cc: CalciteConfig = new CalciteConfigBuilder()
-      .replaceRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
-      .addRuleSet(RuleSets.ofList(CalcMergeRule.INSTANCE, CalcSplitRule.INSTANCE))
-      .build()
-
-    assertEquals(true, cc.replacesRuleSet)
-    assertTrue(cc.getRuleSet.isDefined)
-    val cSet = cc.getRuleSet.get.iterator().asScala.toSet
-    assertEquals(3, cSet.size)
-    assertTrue(cSet.contains(FilterMergeRule.INSTANCE))
-    assertTrue(cSet.contains(CalcMergeRule.INSTANCE))
-    assertTrue(cSet.contains(CalcSplitRule.INSTANCE))
-  }
-
-  @Test
-  def testAddRules(): Unit = {
-
-    val cc: CalciteConfig = new CalciteConfigBuilder()
-      .addRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
-      .build()
-
-    assertEquals(false, cc.replacesRuleSet)
-    assertTrue(cc.getRuleSet.isDefined)
-    val cSet = cc.getRuleSet.get.iterator().asScala.toSet
-    assertEquals(1, cSet.size)
-    assertTrue(cSet.contains(FilterMergeRule.INSTANCE))
-  }
-
-  @Test
-  def testAddAddRules(): Unit = {
-
-    val cc: CalciteConfig = new CalciteConfigBuilder()
-      .addRuleSet(RuleSets.ofList(FilterMergeRule.INSTANCE))
-      .addRuleSet(RuleSets.ofList(CalcMergeRule.INSTANCE, CalcSplitRule.INSTANCE))
-      .build()
-
-    assertEquals(false, cc.replacesRuleSet)
-    assertTrue(cc.getRuleSet.isDefined)
-    val cSet = cc.getRuleSet.get.iterator().asScala.toSet
-    assertEquals(3, cSet.size)
-    assertTrue(cSet.contains(FilterMergeRule.INSTANCE))
-    assertTrue(cSet.contains(CalcMergeRule.INSTANCE))
-    assertTrue(cSet.contains(CalcSplitRule.INSTANCE))
-  }
-
-  @Test
-  def testDefaultOperatorTable(): Unit = {
-
-    val cc: CalciteConfig = new CalciteConfigBuilder()
-      .build()
-
-    assertEquals(false, cc.replacesSqlOperatorTable)
-    assertFalse(cc.getSqlOperatorTable.isDefined)
-  }
-
-  def testReplaceOperatorTable(): Unit = {
-
-    val oracleTable = new OracleSqlOperatorTable
-
-    val cc: CalciteConfig = new CalciteConfigBuilder()
-      .replaceSqlOperatorTable(oracleTable)
-      .build()
-
-    val oracleOps = oracleTable.getOperatorList.asScala
-
-    assertEquals(true, cc.replacesSqlOperatorTable)
-    assertTrue(cc.getSqlOperatorTable.isDefined)
-    val ops = cc.getSqlOperatorTable.get.getOperatorList
-      .asScala.toSet
-    assertEquals(oracleOps.size, ops.size)
-    for (o <- oracleOps) {
-      assertTrue(ops.contains(o))
-    }
-  }
-
-  def testReplaceAddOperatorTable(): Unit = {
-
-    val oracleTable = new OracleSqlOperatorTable
-    val stdTable = new SqlStdOperatorTable
-
-    val cc: CalciteConfig = new CalciteConfigBuilder()
-      .replaceSqlOperatorTable(oracleTable)
-      .addSqlOperatorTable(stdTable)
-      .build()
-
-    val oracleOps = oracleTable.getOperatorList.asScala
-    val stdOps = stdTable.getOperatorList.asScala
-
-    assertEquals(true, cc.replacesSqlOperatorTable)
-    assertTrue(cc.getSqlOperatorTable.isDefined)
-    val ops = cc.getSqlOperatorTable.get.getOperatorList
-      .asScala.toSet
-    assertEquals(oracleOps.size + stdOps.size, ops.size)
-    for (o <- oracleOps) {
-      assertTrue(ops.contains(o))
-    }
-    for (o <- stdOps) {
-      assertTrue(ops.contains(o))
-    }
-
-  }
-
-  def testAddOperatorTable(): Unit = {
-
-    val oracleTable = new OracleSqlOperatorTable
-
-    val cc: CalciteConfig = new CalciteConfigBuilder()
-      .addSqlOperatorTable(oracleTable)
-      .build()
-
-    val oracleOps = oracleTable.getOperatorList.asScala
-
-    assertEquals(false, cc.replacesSqlOperatorTable)
-    assertTrue(cc.getSqlOperatorTable.isDefined)
-    val ops = cc.getSqlOperatorTable.get.getOperatorList
-      .asScala.toSet
-    assertEquals(oracleOps.size, ops.size)
-    for (o <- oracleOps) {
-      assertTrue(ops.contains(o))
-    }
-  }
-
-  def testAddAddOperatorTable(): Unit = {
-
-    val oracleTable = new OracleSqlOperatorTable
-    val stdTable = new SqlStdOperatorTable
-
-    val cc: CalciteConfig = new CalciteConfigBuilder()
-      .addSqlOperatorTable(oracleTable)
-      .addSqlOperatorTable(stdTable)
-      .build()
-
-    val oracleOps = oracleTable.getOperatorList.asScala
-    val stdOps = stdTable.getOperatorList.asScala
-
-    assertEquals(false, cc.replacesSqlOperatorTable)
-    assertTrue(cc.getSqlOperatorTable.isDefined)
-    val ops = cc.getSqlOperatorTable.get.getOperatorList
-      .asScala.toSet
-    assertEquals(oracleOps.size + stdOps.size, ops.size)
-    for (o <- oracleOps) {
-      assertTrue(ops.contains(o))
-    }
-    for (o <- stdOps) {
-      assertTrue(ops.contains(o))
-    }
-
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/CompositeFlatteningTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/CompositeFlatteningTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/CompositeFlatteningTest.scala
deleted file mode 100644
index f14b9d8..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/CompositeFlatteningTest.scala
+++ /dev/null
@@ -1,146 +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.api.table
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala.createTypeInformation
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.CompositeFlatteningTest.{TestCaseClass, giveMeCaseClass}
-import org.apache.flink.api.table.functions.ScalarFunction
-import org.apache.flink.api.table.utils.TableTestBase
-import org.apache.flink.api.table.utils.TableTestUtil._
-import org.junit.Test
-
-
-class CompositeFlatteningTest extends TableTestBase {
-
-  @Test(expected = classOf[ValidationException])
-  def testDuplicateFlattening(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[((Int, Long), (String, Boolean), String)]("MyTable", 'a, 'b, 'c)
-
-    table.select('a.flatten(), 'a.flatten())
-  }
-
-  @Test
-  def testMultipleFlatteningsTable(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[((Int, Long), (String, Boolean), String)]("MyTable", 'a, 'b, 'c)
-
-    val result = table.select('a.flatten(), 'c, 'b.flatten())
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select",
-        "a._1 AS a$_1",
-        "a._2 AS a$_2",
-        "c",
-        "b._1 AS b$_1",
-        "b._2 AS b$_2"
-      )
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testMultipleFlatteningsSql(): Unit = {
-    val util = batchTestUtil()
-    val table = util.addTable[((Int, Long), (String, Boolean), String)]("MyTable", 'a, 'b, 'c)
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select",
-        "a._1 AS _1",
-        "a._2 AS _2",
-        "c",
-        "b._1 AS _10",
-        "b._2 AS _20"
-      )
-    )
-
-    util.verifySql(
-      "SELECT MyTable.a.*, c, MyTable.b.* FROM MyTable",
-      expected)
-  }
-
-  @Test
-  def testNestedFlattenings(): Unit = {
-    val util = batchTestUtil()
-    val table = util
-      .addTable[((((String, TestCaseClass), Boolean), String), String)]("MyTable", 'a, 'b)
-
-    val result = table.select('a.flatten(), 'b.flatten())
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select",
-        "a._1 AS a$_1",
-        "a._2 AS a$_2",
-        "b"
-      )
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-  @Test
-  def testScalarFunctionAccess(): Unit = {
-    val util = batchTestUtil()
-    val table = util
-      .addTable[(String, Int)]("MyTable", 'a, 'b)
-
-    val result = table.select(
-      giveMeCaseClass().get("my"),
-      giveMeCaseClass().get("clazz"),
-      giveMeCaseClass().flatten())
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select",
-        "org.apache.flink.api.table.CompositeFlatteningTest.giveMeCaseClass$().my AS _c0",
-        "org.apache.flink.api.table.CompositeFlatteningTest.giveMeCaseClass$().clazz AS _c1",
-        "org.apache.flink.api.table.CompositeFlatteningTest.giveMeCaseClass$().my AS _c2",
-        "org.apache.flink.api.table.CompositeFlatteningTest.giveMeCaseClass$().clazz AS _c3"
-      )
-    )
-
-    util.verifyTable(result, expected)
-  }
-
-}
-
-object CompositeFlatteningTest {
-
-  case class TestCaseClass(my: String, clazz: Int)
-
-  object giveMeCaseClass extends ScalarFunction {
-    def eval(): TestCaseClass = {
-      TestCaseClass("hello", 42)
-    }
-
-    override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = {
-      createTypeInformation[TestCaseClass]
-    }
-  }
-}


[30/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala
deleted file mode 100644
index c1c79ec..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregate.scala
+++ /dev/null
@@ -1,170 +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.api.table.runtime.aggregate
-
-import java.math.BigDecimal
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.types.Row
-
-abstract class MinAggregate[T](implicit ord: Ordering[T]) extends Aggregate[T] {
-
-  protected var minIndex: Int = _
-
-  /**
-   * Initiate the intermediate aggregate value in Row.
-   *
-   * @param intermediate The intermediate aggregate row to initiate.
-   */
-  override def initiate(intermediate: Row): Unit = {
-    intermediate.setField(minIndex, null)
-  }
-
-  /**
-   * Accessed in MapFunction, prepare the input of partial aggregate.
-   *
-   * @param value
-   * @param partial
-   */
-  override def prepare(value: Any, partial: Row): Unit = {
-    if (value == null) {
-      initiate(partial)
-    } else {
-      partial.setField(minIndex, value)
-    }
-  }
-
-  /**
-   * Accessed in CombineFunction and GroupReduceFunction, merge partial
-   * aggregate result into aggregate buffer.
-   *
-   * @param partial
-   * @param buffer
-   */
-  override def merge(partial: Row, buffer: Row): Unit = {
-    val partialValue = partial.getField(minIndex).asInstanceOf[T]
-    if (partialValue != null) {
-      val bufferValue = buffer.getField(minIndex).asInstanceOf[T]
-      if (bufferValue != null) {
-        val min : T = if (ord.compare(partialValue, bufferValue) < 0) partialValue else bufferValue
-        buffer.setField(minIndex, min)
-      } else {
-        buffer.setField(minIndex, partialValue)
-      }
-    }
-  }
-
-  /**
-   * Return the final aggregated result based on aggregate buffer.
-   *
-   * @param buffer
-   * @return
-   */
-  override def evaluate(buffer: Row): T = {
-    buffer.getField(minIndex).asInstanceOf[T]
-  }
-
-  override def supportPartial: Boolean = true
-
-  override def setAggOffsetInRow(aggOffset: Int): Unit = {
-    minIndex = aggOffset
-  }
-}
-
-class ByteMinAggregate extends MinAggregate[Byte] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.BYTE_TYPE_INFO)
-
-}
-
-class ShortMinAggregate extends MinAggregate[Short] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.SHORT_TYPE_INFO)
-
-}
-
-class IntMinAggregate extends MinAggregate[Int] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.INT_TYPE_INFO)
-
-}
-
-class LongMinAggregate extends MinAggregate[Long] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO)
-
-}
-
-class FloatMinAggregate extends MinAggregate[Float] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.FLOAT_TYPE_INFO)
-
-}
-
-class DoubleMinAggregate extends MinAggregate[Double] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.DOUBLE_TYPE_INFO)
-
-}
-
-class BooleanMinAggregate extends MinAggregate[Boolean] {
-
-  override def intermediateDataType = Array(BasicTypeInfo.BOOLEAN_TYPE_INFO)
-
-}
-
-class DecimalMinAggregate extends Aggregate[BigDecimal] {
-
-  protected var minIndex: Int = _
-
-  override def intermediateDataType = Array(BasicTypeInfo.BIG_DEC_TYPE_INFO)
-
-  override def initiate(intermediate: Row): Unit = {
-    intermediate.setField(minIndex, null)
-  }
-
-  override def prepare(value: Any, partial: Row): Unit = {
-    if (value == null) {
-      initiate(partial)
-    } else {
-      partial.setField(minIndex, value)
-    }
-  }
-
-  override def merge(partial: Row, buffer: Row): Unit = {
-    val partialValue = partial.getField(minIndex).asInstanceOf[BigDecimal]
-    if (partialValue != null) {
-      val bufferValue = buffer.getField(minIndex).asInstanceOf[BigDecimal]
-      if (bufferValue != null) {
-        val min = if (partialValue.compareTo(bufferValue) < 0) partialValue else bufferValue
-        buffer.setField(minIndex, min)
-      } else {
-        buffer.setField(minIndex, partialValue)
-      }
-    }
-  }
-
-  override def evaluate(buffer: Row): BigDecimal = {
-    buffer.getField(minIndex).asInstanceOf[BigDecimal]
-  }
-
-  override def supportPartial: Boolean = true
-
-  override def setAggOffsetInRow(aggOffset: Int): Unit = {
-    minIndex = aggOffset
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala
deleted file mode 100644
index 16f1608..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregate.scala
+++ /dev/null
@@ -1,130 +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.api.table.runtime.aggregate
-
-import java.math.BigDecimal
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo
-import org.apache.flink.types.Row
-
-abstract class SumAggregate[T: Numeric]
-  extends Aggregate[T] {
-
-  private val numeric = implicitly[Numeric[T]]
-  protected var sumIndex: Int = _
-
-  override def initiate(partial: Row): Unit = {
-    partial.setField(sumIndex, null)
-  }
-
-  override def merge(partial1: Row, buffer: Row): Unit = {
-    val partialValue = partial1.getField(sumIndex).asInstanceOf[T]
-    if (partialValue != null) {
-      val bufferValue = buffer.getField(sumIndex).asInstanceOf[T]
-      if (bufferValue != null) {
-        buffer.setField(sumIndex, numeric.plus(partialValue, bufferValue))
-      } else {
-        buffer.setField(sumIndex, partialValue)
-      }
-    }
-  }
-
-  override def evaluate(buffer: Row): T = {
-    buffer.getField(sumIndex).asInstanceOf[T]
-  }
-
-  override def prepare(value: Any, partial: Row): Unit = {
-    if (value == null) {
-      initiate(partial)
-    } else {
-      val input = value.asInstanceOf[T]
-      partial.setField(sumIndex, input)
-    }
-  }
-
-  override def supportPartial: Boolean = true
-
-  override def setAggOffsetInRow(aggOffset: Int): Unit = {
-    sumIndex = aggOffset
-  }
-}
-
-class ByteSumAggregate extends SumAggregate[Byte] {
-  override def intermediateDataType = Array(BasicTypeInfo.BYTE_TYPE_INFO)
-}
-
-class ShortSumAggregate extends SumAggregate[Short] {
-  override def intermediateDataType = Array(BasicTypeInfo.SHORT_TYPE_INFO)
-}
-
-class IntSumAggregate extends SumAggregate[Int] {
-  override def intermediateDataType = Array(BasicTypeInfo.INT_TYPE_INFO)
-}
-
-class LongSumAggregate extends SumAggregate[Long] {
-  override def intermediateDataType = Array(BasicTypeInfo.LONG_TYPE_INFO)
-}
-
-class FloatSumAggregate extends SumAggregate[Float] {
-  override def intermediateDataType = Array(BasicTypeInfo.FLOAT_TYPE_INFO)
-}
-
-class DoubleSumAggregate extends SumAggregate[Double] {
-  override def intermediateDataType = Array(BasicTypeInfo.DOUBLE_TYPE_INFO)
-}
-
-class DecimalSumAggregate extends Aggregate[BigDecimal] {
-
-  protected var sumIndex: Int = _
-
-  override def intermediateDataType = Array(BasicTypeInfo.BIG_DEC_TYPE_INFO)
-
-  override def initiate(partial: Row): Unit = {
-    partial.setField(sumIndex, null)
-  }
-
-  override def merge(partial1: Row, buffer: Row): Unit = {
-    val partialValue = partial1.getField(sumIndex).asInstanceOf[BigDecimal]
-    if (partialValue != null) {
-      val bufferValue = buffer.getField(sumIndex).asInstanceOf[BigDecimal]
-      if (bufferValue != null) {
-        buffer.setField(sumIndex, partialValue.add(bufferValue))
-      } else {
-        buffer.setField(sumIndex, partialValue)
-      }
-    }
-  }
-
-  override def evaluate(buffer: Row): BigDecimal = {
-    buffer.getField(sumIndex).asInstanceOf[BigDecimal]
-  }
-
-  override def prepare(value: Any, partial: Row): Unit = {
-    if (value == null) {
-      initiate(partial)
-    } else {
-      val input = value.asInstanceOf[BigDecimal]
-      partial.setField(sumIndex, input)
-    }
-  }
-
-  override def supportPartial: Boolean = true
-
-  override def setAggOffsetInRow(aggOffset: Int): Unit = {
-    sumIndex = aggOffset
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/TimeWindowPropertyCollector.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/TimeWindowPropertyCollector.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/TimeWindowPropertyCollector.scala
deleted file mode 100644
index 417c1f1..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/runtime/aggregate/TimeWindowPropertyCollector.scala
+++ /dev/null
@@ -1,54 +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.api.table.runtime.aggregate
-
-import org.apache.calcite.runtime.SqlFunctions
-import org.apache.flink.types.Row
-import org.apache.flink.streaming.api.windowing.windows.TimeWindow
-import org.apache.flink.util.Collector
-
-/**
-  * Adds TimeWindow properties to specified fields of a row before it emits the row to a wrapped
-  * collector.
-  */
-class TimeWindowPropertyCollector(windowStartOffset: Option[Int], windowEndOffset: Option[Int])
-    extends Collector[Row] {
-
-  var wrappedCollector: Collector[Row] = _
-  var timeWindow: TimeWindow = _
-
-  override def collect(record: Row): Unit = {
-
-    val lastFieldPos = record.getArity - 1
-
-    if (windowStartOffset.isDefined) {
-      record.setField(
-        lastFieldPos + windowStartOffset.get,
-        SqlFunctions.internalToTimestamp(timeWindow.getStart))
-    }
-    if (windowEndOffset.isDefined) {
-      record.setField(
-        lastFieldPos + windowEndOffset.get,
-        SqlFunctions.internalToTimestamp(timeWindow.getEnd))
-    }
-    wrappedCollector.collect(record)
-  }
-
-  override def close(): Unit = wrappedCollector.close()
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/BatchTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/BatchTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/BatchTableSink.scala
deleted file mode 100644
index 27dbe8e..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/BatchTableSink.scala
+++ /dev/null
@@ -1,31 +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.api.table.sinks
-
-import org.apache.flink.api.java.DataSet
-
-/** Defines an external [[TableSink]] to emit a batch [[org.apache.flink.api.table.Table]].
-  *
-  * @tparam T Type of [[DataSet]] that this [[TableSink]] expects and supports.
-  */
-trait BatchTableSink[T] extends TableSink[T] {
-
-  /** Emits the DataSet. */
-  def emitDataSet(dataSet: DataSet[T]): Unit
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/CsvTableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/CsvTableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/CsvTableSink.scala
deleted file mode 100644
index 5038d9b..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/CsvTableSink.scala
+++ /dev/null
@@ -1,86 +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.api.table.sinks
-
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.DataSet
-import org.apache.flink.types.Row
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.streaming.api.datastream.DataStream
-
-/**
-  * A simple [[TableSink]] to emit data as CSV files.
-  *
-  * @param path The output path to write the Table to.
-  * @param fieldDelim The field delimiter, ',' by default.
-  */
-class CsvTableSink(
-    path: String,
-    fieldDelim: String = ",")
-  extends TableSinkBase[Row] with BatchTableSink[Row] with StreamTableSink[Row] {
-
-  override def emitDataSet(dataSet: DataSet[Row]): Unit = {
-    dataSet
-      .map(new CsvFormatter(fieldDelim))
-      .writeAsText(path)
-  }
-
-  override def emitDataStream(dataStream: DataStream[Row]): Unit = {
-    dataStream
-      .map(new CsvFormatter(fieldDelim))
-      .writeAsText(path)
-  }
-
-  override protected def copy: TableSinkBase[Row] = {
-    new CsvTableSink(path, fieldDelim)
-  }
-
-  override def getOutputType: TypeInformation[Row] = {
-    new RowTypeInfo(getFieldTypes: _*)
-  }
-}
-
-/**
-  * Formats a [[Row]] into a [[String]] with fields separated by the field delimiter.
-  *
-  * @param fieldDelim The field delimiter.
-  */
-class CsvFormatter(fieldDelim: String) extends MapFunction[Row, String] {
-  override def map(row: Row): String = {
-
-    val builder = new StringBuilder
-
-    // write first value
-    val v = row.getField(0)
-    if (v != null) {
-      builder.append(v.toString)
-    }
-
-    // write following values
-    for (i <- 1 until row.getArity) {
-      builder.append(fieldDelim)
-      val v = row.getField(i)
-      if (v != null) {
-        builder.append(v.toString)
-      }
-    }
-    builder.mkString
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/TableSink.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/TableSink.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/TableSink.scala
deleted file mode 100644
index 3dfc6f1..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/TableSink.scala
+++ /dev/null
@@ -1,58 +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.api.table.sinks
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-
-/** A [[TableSink]] specifies how to emit a [[org.apache.flink.api.table.Table]] to an external
-  * system or location.
-  *
-  * The interface is generic such that it can support different storage locations and formats.
-  *
-  * @tparam T The return type of the [[TableSink]].
-  */
-trait TableSink[T] {
-
-  /**
-    * Return the type expected by this [[TableSink]].
-    *
-    * This type should depend on the types returned by [[getFieldNames]].
-    *
-    * @return The type expected by this [[TableSink]].
-    */
-  def getOutputType: TypeInformation[T]
-
-  /** Returns the names of the table fields. */
-  def getFieldNames: Array[String]
-
-  /** Returns the types of the table fields. */
-  def getFieldTypes: Array[TypeInformation[_]]
-
-  /**
-    * Return a copy of this [[TableSink]] configured with the field names and types of the
-    * [[org.apache.flink.api.table.Table]] to emit.
-    *
-    * @param fieldNames The field names of the table to emit.
-    * @param fieldTypes The field types of the table to emit.
-    * @return A copy of this [[TableSink]] configured with the field names and types of the
-    *         [[org.apache.flink.api.table.Table]] to emit.
-    */
-  def configure(fieldNames: Array[String],
-                fieldTypes: Array[TypeInformation[_]]): TableSink[T]
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/TableSinkBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/TableSinkBase.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/TableSinkBase.scala
deleted file mode 100644
index 612ee0a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sinks/TableSinkBase.scala
+++ /dev/null
@@ -1,67 +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.api.table.sinks
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-
-trait TableSinkBase[T] extends TableSink[T] {
-
-  private var fieldNames: Option[Array[String]] = None
-  private var fieldTypes: Option[Array[TypeInformation[_]]] = None
-
-  /** Return a deep copy of the [[TableSink]]. */
-  protected def copy: TableSinkBase[T]
-
-  /**
-    * Return the field names of the [[org.apache.flink.api.table.Table]] to emit. */
-  def getFieldNames: Array[String] = {
-    fieldNames match {
-      case Some(n) => n
-      case None => throw new IllegalStateException(
-        "TableSink must be configured to retrieve field names.")
-    }
-  }
-
-  /** Return the field types of the [[org.apache.flink.api.table.Table]] to emit. */
-  def getFieldTypes: Array[TypeInformation[_]] = {
-    fieldTypes match {
-      case Some(t) => t
-      case None => throw new IllegalStateException(
-        "TableSink must be configured to retrieve field types.")
-    }
-  }
-
-  /**
-    * Return a copy of this [[TableSink]] configured with the field names and types of the
-    * [[org.apache.flink.api.table.Table]] to emit.
-    *
-    * @param fieldNames The field names of the table to emit.
-    * @param fieldTypes The field types of the table to emit.
-    * @return A copy of this [[TableSink]] configured with the field names and types of the
-    *         [[org.apache.flink.api.table.Table]] to emit.
-    */
-  final def configure(fieldNames: Array[String],
-                      fieldTypes: Array[TypeInformation[_]]): TableSink[T] = {
-
-    val configuredSink = this.copy
-    configuredSink.fieldNames = Some(fieldNames)
-    configuredSink.fieldTypes = Some(fieldTypes)
-
-    configuredSink
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/BatchTableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/BatchTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/BatchTableSource.scala
deleted file mode 100644
index 74e4cd6..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/BatchTableSource.scala
+++ /dev/null
@@ -1,36 +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.api.table.sources
-
-import org.apache.flink.api.java.{ExecutionEnvironment, DataSet}
-
-/** Defines an external batch table and provides access to its data.
-  *
-  * @tparam T Type of the [[DataSet]] created by this [[TableSource]].
-  */
-trait BatchTableSource[T] extends TableSource[T] {
-
-  /**
-    * Returns the data of the table as a [[DataSet]].
-    *
-    * NOTE: This method is for internal use only for defining a [[TableSource]].
-    *       Do not use it in Table API programs.
-    */
-  def getDataSet(execEnv: ExecutionEnvironment): DataSet[T]
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala
deleted file mode 100644
index b60575a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/CsvTableSource.scala
+++ /dev/null
@@ -1,123 +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.api.table.sources
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.io.CsvInputFormat
-import org.apache.flink.api.java.{DataSet, ExecutionEnvironment}
-import org.apache.flink.api.table.TableException
-import org.apache.flink.types.Row
-import org.apache.flink.api.java.io.RowCsvInputFormat
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.core.fs.Path
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
-
-/**
-  * A [[BatchTableSource]] and [[StreamTableSource]] for simple CSV files with a
-  * (logically) unlimited number of fields.
-  *
-  * @param path The path to the CSV file.
-  * @param fieldNames The names of the table fields.
-  * @param fieldTypes The types of the table fields.
-  * @param fieldDelim The field delimiter, "," by default.
-  * @param rowDelim The row delimiter, "\n" by default.
-  * @param quoteCharacter An optional quote character for String values, null by default.
-  * @param ignoreFirstLine Flag to ignore the first line, false by default.
-  * @param ignoreComments An optional prefix to indicate comments, null by default.
-  * @param lenient Flag to skip records with parse error instead to fail, false by default.
-  */
-class CsvTableSource(
-    path: String,
-    fieldNames: Array[String],
-    fieldTypes: Array[TypeInformation[_]],
-    fieldDelim: String = CsvInputFormat.DEFAULT_FIELD_DELIMITER,
-    rowDelim: String = CsvInputFormat.DEFAULT_LINE_DELIMITER,
-    quoteCharacter: Character = null,
-    ignoreFirstLine: Boolean = false,
-    ignoreComments: String = null,
-    lenient: Boolean = false)
-  extends BatchTableSource[Row]
-  with StreamTableSource[Row] {
-
-  /**
-    * A [[BatchTableSource]] and [[StreamTableSource]] for simple CSV files with a
-    * (logically) unlimited number of fields.
-    *
-    * @param path The path to the CSV file.
-    * @param fieldNames The names of the table fields.
-    * @param fieldTypes The types of the table fields.
-    */
-  def this(path: String, fieldNames: Array[String], fieldTypes: Array[TypeInformation[_]]) =
-    this(path, fieldNames, fieldTypes, CsvInputFormat.DEFAULT_FIELD_DELIMITER,
-      CsvInputFormat.DEFAULT_LINE_DELIMITER, null, false, null, false)
-
-  if (fieldNames.length != fieldTypes.length) {
-    throw TableException("Number of field names and field types must be equal.")
-  }
-
-  private val returnType = new RowTypeInfo(fieldTypes: _*)
-
-  /**
-    * Returns the data of the table as a [[DataSet]] of [[Row]].
-    *
-    * NOTE: This method is for internal use only for defining a [[TableSource]].
-    *       Do not use it in Table API programs.
-    */
-  override def getDataSet(execEnv: ExecutionEnvironment): DataSet[Row] = {
-    execEnv.createInput(createCsvInput(), returnType)
-  }
-
-  /** Returns the types of the table fields. */
-  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
-
-  /** Returns the names of the table fields. */
-  override def getFieldsNames: Array[String] = fieldNames
-
-  /** Returns the number of fields of the table. */
-  override def getNumberOfFields: Int = fieldNames.length
-
-  /** Returns the [[RowTypeInfo]] for the return type of the [[CsvTableSource]]. */
-  override def getReturnType: RowTypeInfo = returnType
-
-  /**
-    * Returns the data of the table as a [[DataStream]] of [[Row]].
-    *
-    * NOTE: This method is for internal use only for defining a [[TableSource]].
-    *       Do not use it in Table API programs.
-    */
-  override def getDataStream(streamExecEnv: StreamExecutionEnvironment): DataStream[Row] = {
-    streamExecEnv.createInput(createCsvInput(), returnType)
-  }
-
-  private def createCsvInput(): RowCsvInputFormat = {
-    val inputFormat = new RowCsvInputFormat(new Path(path), returnType, rowDelim, fieldDelim)
-
-    inputFormat.setSkipFirstLineAsHeader(ignoreFirstLine)
-    inputFormat.setLenient(lenient)
-    if (quoteCharacter != null) {
-      inputFormat.enableQuotedStringParsing(quoteCharacter)
-    }
-    if (ignoreComments != null) {
-      inputFormat.setCommentPrefix(ignoreComments)
-    }
-
-    inputFormat
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/ProjectableTableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/ProjectableTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/ProjectableTableSource.scala
deleted file mode 100644
index c04138a..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/ProjectableTableSource.scala
+++ /dev/null
@@ -1,38 +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.api.table.sources
-
-/**
-  * Adds support for projection push-down to a [[TableSource]].
-  * A [[TableSource]] extending this interface is able to project the fields of the return table.
-  *
-  * @tparam T The return type of the [[ProjectableTableSource]].
-  */
-trait ProjectableTableSource[T] {
-
-  /**
-    * Creates a copy of the [[ProjectableTableSource]] that projects its output on the specified
-    * fields.
-    *
-    * @param fields The indexes of the fields to return.
-    * @return A copy of the [[ProjectableTableSource]] that projects its output.
-    */
-  def projectFields(fields: Array[Int]): ProjectableTableSource[T]
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/StreamTableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/StreamTableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/StreamTableSource.scala
deleted file mode 100644
index cdae0b3..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/StreamTableSource.scala
+++ /dev/null
@@ -1,37 +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.api.table.sources
-
-import org.apache.flink.streaming.api.datastream.DataStream
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
-
-/** Defines an external stream table and provides access to its data.
-  *
-  * @tparam T Type of the [[DataStream]] created by this [[TableSource]].
-  */
-trait StreamTableSource[T] extends TableSource[T] {
-
-  /**
-    * Returns the data of the table as a [[DataStream]].
-    *
-    * NOTE: This method is for internal use only for defining a [[TableSource]].
-    *       Do not use it in Table API programs.
-    */
-  def getDataStream(execEnv: StreamExecutionEnvironment): DataStream[T]
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/TableSource.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/TableSource.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/TableSource.scala
deleted file mode 100644
index e1ada62..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/sources/TableSource.scala
+++ /dev/null
@@ -1,41 +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.api.table.sources
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-
-/** Defines an external table by providing schema information, i.e., field names and types.
-  *
-  * @tparam T The return type of the [[TableSource]].
-  */
-trait TableSource[T] {
-
-  /** Returns the number of fields of the table. */
-  def getNumberOfFields: Int
-
-  /** Returns the names of the table fields. */
-  def getFieldsNames: Array[String]
-
-  /** Returns the types of the table fields. */
-  def getFieldTypes: Array[TypeInformation[_]]
-
-  /** Returns the [[TypeInformation]] for the return type of the [[TableSource]]. */
-  def getReturnType: TypeInformation[T]
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala
deleted file mode 100644
index 94c8e8c..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/table.scala
+++ /dev/null
@@ -1,922 +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.api.table
-
-import org.apache.calcite.rel.RelNode
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.operators.join.JoinType
-import org.apache.flink.api.table.expressions._
-import org.apache.flink.api.table.plan.ProjectionTranslator._
-import org.apache.flink.api.table.plan.logical.{Minus, _}
-import org.apache.flink.api.table.sinks.TableSink
-
-import scala.collection.JavaConverters._
-
-/**
-  * A Table is the core component of the Table API.
-  * Similar to how the batch and streaming APIs have DataSet and DataStream,
-  * the Table API is built around [[Table]].
-  *
-  * Use the methods of [[Table]] to transform data. Use [[TableEnvironment]] to convert a [[Table]]
-  * back to a DataSet or DataStream.
-  *
-  * When using Scala a [[Table]] can also be converted using implicit conversions.
-  *
-  * Example:
-  *
-  * {{{
-  *   val env = ExecutionEnvironment.getExecutionEnvironment
-  *   val tEnv = TableEnvironment.getTableEnvironment(env)
-  *
-  *   val set: DataSet[(String, Int)] = ...
-  *   val table = set.toTable(tEnv, 'a, 'b)
-  *   ...
-  *   val table2 = ...
-  *   val set2: DataSet[MyType] = table2.toDataSet[MyType]
-  * }}}
-  *
-  * Operations such as [[join]], [[select]], [[where]] and [[groupBy]] either take arguments
-  * in a Scala DSL or as an expression String. Please refer to the documentation for the expression
-  * syntax.
-  *
-  * @param tableEnv The [[TableEnvironment]] to which the table is bound.
-  * @param logicalPlan logical representation
-  */
-class Table(
-    private[flink] val tableEnv: TableEnvironment,
-    private[flink] val logicalPlan: LogicalNode) {
-
-  def relBuilder = tableEnv.getRelBuilder
-
-  def getRelNode: RelNode = logicalPlan.toRelNode(relBuilder)
-
-  /**
-    * Performs a selection operation. Similar to an SQL SELECT statement. The field expressions
-    * can contain complex expressions and aggregations.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.select('key, 'value.avg + " The average" as 'average)
-    * }}}
-    */
-  def select(fields: Expression*): Table = {
-    val expandedFields = expandProjectList(fields, logicalPlan, tableEnv)
-    val (aggNames, propNames) = extractAggregationsAndProperties(expandedFields, tableEnv)
-    if (propNames.nonEmpty) {
-      throw ValidationException("Window properties can only be used on windowed tables.")
-    }
-
-    if (aggNames.nonEmpty) {
-      val projectsOnAgg = replaceAggregationsAndProperties(
-        expandedFields, tableEnv, aggNames, propNames)
-      val projectFields = extractFieldReferences(expandedFields)
-
-      new Table(tableEnv,
-        Project(projectsOnAgg,
-          Aggregate(Nil, aggNames.map(a => Alias(a._1, a._2)).toSeq,
-            Project(projectFields, logicalPlan).validate(tableEnv)
-          ).validate(tableEnv)
-        ).validate(tableEnv)
-      )
-    } else {
-      new Table(tableEnv,
-        Project(expandedFields.map(UnresolvedAlias), logicalPlan).validate(tableEnv))
-    }
-  }
-
-  /**
-    * Performs a selection operation. Similar to an SQL SELECT statement. The field expressions
-    * can contain complex expressions and aggregations.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.select("key, value.avg + ' The average' as average")
-    * }}}
-    */
-  def select(fields: String): Table = {
-    val fieldExprs = ExpressionParser.parseExpressionList(fields)
-    select(fieldExprs: _*)
-  }
-
-  /**
-    * Renames the fields of the expression result. Use this to disambiguate fields before
-    * joining to operations.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.as('a, 'b)
-    * }}}
-    */
-  def as(fields: Expression*): Table = {
-    new Table(tableEnv, AliasNode(fields, logicalPlan).validate(tableEnv))
-  }
-
-  /**
-    * Renames the fields of the expression result. Use this to disambiguate fields before
-    * joining to operations.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.as("a, b")
-    * }}}
-    */
-  def as(fields: String): Table = {
-    val fieldExprs = ExpressionParser.parseExpressionList(fields)
-    as(fieldExprs: _*)
-  }
-
-  /**
-    * Filters out elements that don't pass the filter predicate. Similar to a SQL WHERE
-    * clause.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.filter('name === "Fred")
-    * }}}
-    */
-  def filter(predicate: Expression): Table = {
-    new Table(tableEnv, Filter(predicate, logicalPlan).validate(tableEnv))
-  }
-
-  /**
-    * Filters out elements that don't pass the filter predicate. Similar to a SQL WHERE
-    * clause.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.filter("name = 'Fred'")
-    * }}}
-    */
-  def filter(predicate: String): Table = {
-    val predicateExpr = ExpressionParser.parseExpression(predicate)
-    filter(predicateExpr)
-  }
-
-  /**
-    * Filters out elements that don't pass the filter predicate. Similar to a SQL WHERE
-    * clause.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.where('name === "Fred")
-    * }}}
-    */
-  def where(predicate: Expression): Table = {
-    filter(predicate)
-  }
-
-  /**
-    * Filters out elements that don't pass the filter predicate. Similar to a SQL WHERE
-    * clause.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.where("name = 'Fred'")
-    * }}}
-    */
-  def where(predicate: String): Table = {
-    filter(predicate)
-  }
-
-  /**
-    * Groups the elements on some grouping keys. Use this before a selection with aggregations
-    * to perform the aggregation on a per-group basis. Similar to a SQL GROUP BY statement.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.groupBy('key).select('key, 'value.avg)
-    * }}}
-    */
-  def groupBy(fields: Expression*): GroupedTable = {
-    new GroupedTable(this, fields)
-  }
-
-  /**
-    * Groups the elements on some grouping keys. Use this before a selection with aggregations
-    * to perform the aggregation on a per-group basis. Similar to a SQL GROUP BY statement.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.groupBy("key").select("key, value.avg")
-    * }}}
-    */
-  def groupBy(fields: String): GroupedTable = {
-    val fieldsExpr = ExpressionParser.parseExpressionList(fields)
-    groupBy(fieldsExpr: _*)
-  }
-
-  /**
-    * Removes duplicate values and returns only distinct (different) values.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.select("key, value").distinct()
-    * }}}
-    */
-  def distinct(): Table = {
-    new Table(tableEnv, Distinct(logicalPlan).validate(tableEnv))
-  }
-
-  /**
-    * Joins two [[Table]]s. Similar to an SQL join. The fields of the two joined
-    * operations must not overlap, use [[as]] to rename fields if necessary. You can use
-    * where and select clauses after a join to further specify the behaviour of the join.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]].
-    *
-    * Example:
-    *
-    * {{{
-    *   left.join(right).where('a === 'b && 'c > 3).select('a, 'b, 'd)
-    * }}}
-    */
-  def join(right: Table): Table = {
-    join(right, None, JoinType.INNER)
-  }
-
-  /**
-    * Joins two [[Table]]s. Similar to an SQL join. The fields of the two joined
-    * operations must not overlap, use [[as]] to rename fields if necessary.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]].
-    *
-    * Example:
-    *
-    * {{{
-    *   left.join(right, "a = b")
-    * }}}
-    */
-  def join(right: Table, joinPredicate: String): Table = {
-    join(right, joinPredicate, JoinType.INNER)
-  }
-
-  /**
-    * Joins two [[Table]]s. Similar to an SQL join. The fields of the two joined
-    * operations must not overlap, use [[as]] to rename fields if necessary.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]].
-    *
-    * Example:
-    *
-    * {{{
-    *   left.join(right, 'a === 'b).select('a, 'b, 'd)
-    * }}}
-    */
-  def join(right: Table, joinPredicate: Expression): Table = {
-    join(right, Some(joinPredicate), JoinType.INNER)
-  }
-
-  /**
-    * Joins two [[Table]]s. Similar to an SQL left outer join. The fields of the two joined
-    * operations must not overlap, use [[as]] to rename fields if necessary.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
-    * have nullCheck enabled.
-    *
-    * Example:
-    *
-    * {{{
-    *   left.leftOuterJoin(right, "a = b").select('a, 'b, 'd)
-    * }}}
-    */
-  def leftOuterJoin(right: Table, joinPredicate: String): Table = {
-    join(right, joinPredicate, JoinType.LEFT_OUTER)
-  }
-
-  /**
-    * Joins two [[Table]]s. Similar to an SQL left outer join. The fields of the two joined
-    * operations must not overlap, use [[as]] to rename fields if necessary.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
-    * have nullCheck enabled.
-    *
-    * Example:
-    *
-    * {{{
-    *   left.leftOuterJoin(right, 'a === 'b).select('a, 'b, 'd)
-    * }}}
-    */
-  def leftOuterJoin(right: Table, joinPredicate: Expression): Table = {
-    join(right, Some(joinPredicate), JoinType.LEFT_OUTER)
-  }
-
-  /**
-    * Joins two [[Table]]s. Similar to an SQL right outer join. The fields of the two joined
-    * operations must not overlap, use [[as]] to rename fields if necessary.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
-    * have nullCheck enabled.
-    *
-    * Example:
-    *
-    * {{{
-    *   left.rightOuterJoin(right, "a = b").select('a, 'b, 'd)
-    * }}}
-    */
-  def rightOuterJoin(right: Table, joinPredicate: String): Table = {
-    join(right, joinPredicate, JoinType.RIGHT_OUTER)
-  }
-
-  /**
-    * Joins two [[Table]]s. Similar to an SQL right outer join. The fields of the two joined
-    * operations must not overlap, use [[as]] to rename fields if necessary.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
-    * have nullCheck enabled.
-    *
-    * Example:
-    *
-    * {{{
-    *   left.rightOuterJoin(right, 'a === 'b).select('a, 'b, 'd)
-    * }}}
-    */
-  def rightOuterJoin(right: Table, joinPredicate: Expression): Table = {
-    join(right, Some(joinPredicate), JoinType.RIGHT_OUTER)
-  }
-
-  /**
-    * Joins two [[Table]]s. Similar to an SQL full outer join. The fields of the two joined
-    * operations must not overlap, use [[as]] to rename fields if necessary.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
-    * have nullCheck enabled.
-    *
-    * Example:
-    *
-    * {{{
-    *   left.fullOuterJoin(right, "a = b").select('a, 'b, 'd)
-    * }}}
-    */
-  def fullOuterJoin(right: Table, joinPredicate: String): Table = {
-    join(right, joinPredicate, JoinType.FULL_OUTER)
-  }
-
-  /**
-    * Joins two [[Table]]s. Similar to an SQL full outer join. The fields of the two joined
-    * operations must not overlap, use [[as]] to rename fields if necessary.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]] and its [[TableConfig]] must
-    * have nullCheck enabled.
-    *
-    * Example:
-    *
-    * {{{
-    *   left.fullOuterJoin(right, 'a === 'b).select('a, 'b, 'd)
-    * }}}
-    */
-  def fullOuterJoin(right: Table, joinPredicate: Expression): Table = {
-    join(right, Some(joinPredicate), JoinType.FULL_OUTER)
-  }
-
-  private def join(right: Table, joinPredicate: String, joinType: JoinType): Table = {
-    val joinPredicateExpr = ExpressionParser.parseExpression(joinPredicate)
-    join(right, Some(joinPredicateExpr), joinType)
-  }
-
-  private def join(right: Table, joinPredicate: Option[Expression], joinType: JoinType): Table = {
-    // check that right table belongs to the same TableEnvironment
-    if (right.tableEnv != this.tableEnv) {
-      throw new ValidationException("Only tables from the same TableEnvironment can be joined.")
-    }
-    new Table(
-      tableEnv,
-      Join(this.logicalPlan, right.logicalPlan, joinType, joinPredicate, correlated = false)
-        .validate(tableEnv))
-  }
-
-  /**
-    * Minus of two [[Table]]s with duplicate records removed.
-    * Similar to a SQL EXCEPT clause. Minus returns records from the left table that do not
-    * exist in the right table. Duplicate records in the left table are returned
-    * exactly once, i.e., duplicates are removed. Both tables must have identical field types.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]].
-    *
-    * Example:
-    *
-    * {{{
-    *   left.minus(right)
-    * }}}
-    */
-  def minus(right: Table): Table = {
-    // check that right table belongs to the same TableEnvironment
-    if (right.tableEnv != this.tableEnv) {
-      throw new ValidationException("Only tables from the same TableEnvironment can be " +
-        "subtracted.")
-    }
-    new Table(tableEnv, Minus(logicalPlan, right.logicalPlan, all = false)
-      .validate(tableEnv))
-  }
-
-  /**
-    * Minus of two [[Table]]s. Similar to an SQL EXCEPT ALL.
-    * Similar to a SQL EXCEPT ALL clause. MinusAll returns the records that do not exist in
-    * the right table. A record that is present n times in the left table and m times
-    * in the right table is returned (n - m) times, i.e., as many duplicates as are present
-    * in the right table are removed. Both tables must have identical field types.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]].
-    *
-    * Example:
-    *
-    * {{{
-    *   left.minusAll(right)
-    * }}}
-    */
-  def minusAll(right: Table): Table = {
-    // check that right table belongs to the same TableEnvironment
-    if (right.tableEnv != this.tableEnv) {
-      throw new ValidationException("Only tables from the same TableEnvironment can be " +
-        "subtracted.")
-    }
-    new Table(tableEnv, Minus(logicalPlan, right.logicalPlan, all = true)
-      .validate(tableEnv))
-  }
-
-  /**
-    * Unions two [[Table]]s with duplicate records removed.
-    * Similar to an SQL UNION. The fields of the two union operations must fully overlap.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]].
-    *
-    * Example:
-    *
-    * {{{
-    *   left.union(right)
-    * }}}
-    */
-  def union(right: Table): Table = {
-    // check that right table belongs to the same TableEnvironment
-    if (right.tableEnv != this.tableEnv) {
-      throw new ValidationException("Only tables from the same TableEnvironment can be unioned.")
-    }
-    new Table(tableEnv, Union(logicalPlan, right.logicalPlan, all = false).validate(tableEnv))
-  }
-
-  /**
-    * Unions two [[Table]]s. Similar to an SQL UNION ALL. The fields of the two union operations
-    * must fully overlap.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]].
-    *
-    * Example:
-    *
-    * {{{
-    *   left.unionAll(right)
-    * }}}
-    */
-  def unionAll(right: Table): Table = {
-    // check that right table belongs to the same TableEnvironment
-    if (right.tableEnv != this.tableEnv) {
-      throw new ValidationException("Only tables from the same TableEnvironment can be unioned.")
-    }
-    new Table(tableEnv, Union(logicalPlan, right.logicalPlan, all = true).validate(tableEnv))
-  }
-
-  /**
-    * Intersects two [[Table]]s with duplicate records removed. Intersect returns records that
-    * exist in both tables. If a record is present in one or both tables more than once, it is
-    * returned just once, i.e., the resulting table has no duplicate records. Similar to an
-    * SQL INTERSECT. The fields of the two intersect operations must fully overlap.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]].
-    *
-    * Example:
-    *
-    * {{{
-    *   left.intersect(right)
-    * }}}
-    */
-  def intersect(right: Table): Table = {
-    // check that right table belongs to the same TableEnvironment
-    if (right.tableEnv != this.tableEnv) {
-      throw new ValidationException(
-        "Only tables from the same TableEnvironment can be intersected.")
-    }
-    new Table(tableEnv, Intersect(logicalPlan, right.logicalPlan, all = false).validate(tableEnv))
-  }
-
-  /**
-    * Intersects two [[Table]]s. IntersectAll returns records that exist in both tables.
-    * If a record is present in both tables more than once, it is returned as many times as it
-    * is present in both tables, i.e., the resulting table might have duplicate records. Similar
-    * to an SQL INTERSECT ALL. The fields of the two intersect operations must fully overlap.
-    *
-    * Note: Both tables must be bound to the same [[TableEnvironment]].
-    *
-    * Example:
-    *
-    * {{{
-    *   left.intersectAll(right)
-    * }}}
-    */
-  def intersectAll(right: Table): Table = {
-    // check that right table belongs to the same TableEnvironment
-    if (right.tableEnv != this.tableEnv) {
-      throw new ValidationException(
-        "Only tables from the same TableEnvironment can be intersected.")
-    }
-    new Table(tableEnv, Intersect(logicalPlan, right.logicalPlan, all = true).validate(tableEnv))
-  }
-
-  /**
-    * Sorts the given [[Table]]. Similar to SQL ORDER BY.
-    * The resulting Table is globally sorted across all parallel partitions.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.orderBy('name.desc)
-    * }}}
-    */
-  def orderBy(fields: Expression*): Table = {
-    val order: Seq[Ordering] = fields.map {
-      case o: Ordering => o
-      case e => Asc(e)
-    }
-    new Table(tableEnv, Sort(order, logicalPlan).validate(tableEnv))
-  }
-
-  /**
-    * Sorts the given [[Table]]. Similar to SQL ORDER BY.
-    * The resulting Table is sorted globally sorted across all parallel partitions.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.orderBy("name.desc")
-    * }}}
-    */
-  def orderBy(fields: String): Table = {
-    val parsedFields = ExpressionParser.parseExpressionList(fields)
-    orderBy(parsedFields: _*)
-  }
-
-  /**
-    * Limits a sorted result from an offset position.
-    * Similar to a SQL LIMIT clause. Limit is technically part of the Order By operator and
-    * thus must be preceded by it.
-    *
-    * Example:
-    *
-    * {{{
-    *   // returns unlimited number of records beginning with the 4th record
-    *   tab.orderBy('name.desc).limit(3)
-    * }}}
-    *
-    * @param offset number of records to skip
-    */
-  def limit(offset: Int): Table = {
-    new Table(tableEnv, Limit(offset = offset, child = logicalPlan).validate(tableEnv))
-  }
-
-  /**
-    * Limits a sorted result to a specified number of records from an offset position.
-    * Similar to a SQL LIMIT clause. Limit is technically part of the Order By operator and
-    * thus must be preceded by it.
-    *
-    * Example:
-    *
-    * {{{
-    *   // returns 5 records beginning with the 4th record
-    *   tab.orderBy('name.desc).limit(3, 5)
-    * }}}
-    *
-    * @param offset number of records to skip
-    * @param fetch number of records to be returned
-    */
-  def limit(offset: Int, fetch: Int): Table = {
-    new Table(tableEnv, Limit(offset, fetch, logicalPlan).validate(tableEnv))
-  }
-
-  /**
-    * Joins this [[Table]] to a user-defined [[org.apache.calcite.schema.TableFunction]]. Similar
-    * to an SQL cross join, but it works with a table function. It returns rows from the outer
-    * table (table on the left of the operator) that produces matching values from the table
-    * function (which is defined in the expression on the right side of the operator).
-    *
-    * Example:
-    *
-    * {{{
-    *   class MySplitUDTF extends TableFunction[String] {
-    *     def eval(str: String): Unit = {
-    *       str.split("#").foreach(collect)
-    *     }
-    *   }
-    *
-    *   val split = new MySplitUDTF()
-    *   table.join(split('c) as ('s)).select('a,'b,'c,'s)
-    * }}}
-    */
-  def join(udtf: Expression): Table = {
-    joinUdtfInternal(udtf, JoinType.INNER)
-  }
-
-  /**
-    * Joins this [[Table]] to a user-defined [[org.apache.calcite.schema.TableFunction]]. Similar
-    * to an SQL cross join, but it works with a table function. It returns rows from the outer
-    * table (table on the left of the operator) that produces matching values from the table
-    * function (which is defined in the expression on the right side of the operator).
-    *
-    * Example:
-    *
-    * {{{
-    *   class MySplitUDTF extends TableFunction<String> {
-    *     public void eval(String str) {
-    *       str.split("#").forEach(this::collect);
-    *     }
-    *   }
-    *
-    *   TableFunction<String> split = new MySplitUDTF();
-    *   tableEnv.registerFunction("split", split);
-    *
-    *   table.join("split(c) as (s)").select("a, b, c, s");
-    * }}}
-    */
-  def join(udtf: String): Table = {
-    joinUdtfInternal(udtf, JoinType.INNER)
-  }
-
-  /**
-    * Joins this [[Table]] to a user-defined [[org.apache.calcite.schema.TableFunction]]. Similar
-    * to an SQL left outer join with ON TRUE, but it works with a table function. It returns all
-    * the rows from the outer table (table on the left of the operator), and rows that do not match
-    * the condition from the table function (which is defined in the expression on the right
-    * side of the operator). Rows with no matching condition are filled with null values.
-    *
-    * Example:
-    *
-    * {{{
-    *   class MySplitUDTF extends TableFunction[String] {
-    *     def eval(str: String): Unit = {
-    *       str.split("#").foreach(collect)
-    *     }
-    *   }
-    *
-    *   val split = new MySplitUDTF()
-    *   table.leftOuterJoin(split('c) as ('s)).select('a,'b,'c,'s)
-    * }}}
-    */
-  def leftOuterJoin(udtf: Expression): Table = {
-    joinUdtfInternal(udtf, JoinType.LEFT_OUTER)
-  }
-
-  /**
-    * Joins this [[Table]] to a user-defined [[org.apache.calcite.schema.TableFunction]]. Similar
-    * to an SQL left outer join with ON TRUE, but it works with a table function. It returns all
-    * the rows from the outer table (table on the left of the operator), and rows that do not match
-    * the condition from the table function (which is defined in the expression on the right
-    * side of the operator). Rows with no matching condition are filled with null values.
-    *
-    * Example:
-    *
-    * {{{
-    *   class MySplitUDTF extends TableFunction<String> {
-    *     public void eval(String str) {
-    *       str.split("#").forEach(this::collect);
-    *     }
-    *   }
-    *
-    *   TableFunction<String> split = new MySplitUDTF();
-    *   tableEnv.registerFunction("split", split);
-    *
-    *   table.leftOuterJoin("split(c) as (s)").select("a, b, c, s");
-    * }}}
-    */
-  def leftOuterJoin(udtf: String): Table = {
-    joinUdtfInternal(udtf, JoinType.LEFT_OUTER)
-  }
-
-  private def joinUdtfInternal(udtfString: String, joinType: JoinType): Table = {
-    val udtf = ExpressionParser.parseExpression(udtfString)
-    joinUdtfInternal(udtf, joinType)
-  }
-
-  private def joinUdtfInternal(udtf: Expression, joinType: JoinType): Table = {
-    var alias: Option[Seq[String]] = None
-
-    // unwrap an Expression until we get a TableFunctionCall
-    def unwrap(expr: Expression): TableFunctionCall = expr match {
-      case Alias(child, name, extraNames) =>
-        alias = Some(Seq(name) ++ extraNames)
-        unwrap(child)
-      case Call(name, args) =>
-        val function = tableEnv.getFunctionCatalog.lookupFunction(name, args)
-        unwrap(function)
-      case c: TableFunctionCall => c
-      case _ =>
-        throw new TableException(
-          "Cross/Outer Apply operators only accept expressions that define table functions.")
-    }
-
-    val call = unwrap(udtf)
-      .as(alias)
-      .toLogicalTableFunctionCall(this.logicalPlan)
-      .validate(tableEnv)
-
-    new Table(
-      tableEnv,
-      Join(this.logicalPlan, call, joinType, None, correlated = true).validate(tableEnv))
-  }
-
-  /**
-    * 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.api.table.sinks.BatchTableSink]], a streaming [[Table]] requires a
-    * [[org.apache.flink.api.table.sinks.StreamTableSink]].
-    *
-    * @param sink The [[TableSink]] to which the [[Table]] is written.
-    * @tparam T The data type that the [[TableSink]] expects.
-    */
-  def writeToSink[T](sink: TableSink[T]): Unit = {
-
-    // get schema information of table
-    val rowType = getRelNode.getRowType
-    val fieldNames: Array[String] = rowType.getFieldNames.asScala.toArray
-    val fieldTypes: Array[TypeInformation[_]] = rowType.getFieldList.asScala
-      .map(field => FlinkTypeFactory.toTypeInfo(field.getType)).toArray
-
-    // configure the table sink
-    val configuredSink = sink.configure(fieldNames, fieldTypes)
-
-    // emit the table to the configured table sink
-    tableEnv.writeToSink(this, configuredSink)
-  }
-
-  /**
-    * Groups the records of a table by assigning them to windows defined by a time or row interval.
-    *
-    * For streaming tables of infinite size, grouping into windows is required to define finite
-    * groups on which group-based aggregates can be computed.
-    *
-    * For batch tables of finite size, windowing essentially provides shortcuts for time-based
-    * groupBy.
-    *
-    * __Note__: window on non-grouped streaming table is a non-parallel operation, i.e., all data
-    * will be processed by a single operator.
-    *
-    * @param groupWindow group-window that specifies how elements are grouped.
-    * @return A windowed table.
-    */
-  def window(groupWindow: GroupWindow): GroupWindowedTable = {
-    if (tableEnv.isInstanceOf[BatchTableEnvironment]) {
-      throw new ValidationException(s"Windows on batch tables are currently not supported.")
-    }
-    new GroupWindowedTable(this, Seq(), groupWindow)
-  }
-}
-
-/**
-  * A table that has been grouped on a set of grouping keys.
-  */
-class GroupedTable(
-  private[flink] val table: Table,
-  private[flink] val groupKey: Seq[Expression]) {
-
-  /**
-    * Performs a selection operation on a grouped table. Similar to an SQL SELECT statement.
-    * The field expressions can contain complex expressions and aggregations.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.groupBy('key).select('key, 'value.avg + " The average" as 'average)
-    * }}}
-    */
-  def select(fields: Expression*): Table = {
-    val (aggNames, propNames) = extractAggregationsAndProperties(fields, table.tableEnv)
-    if (propNames.nonEmpty) {
-      throw ValidationException("Window properties can only be used on windowed tables.")
-    }
-
-    val projectsOnAgg = replaceAggregationsAndProperties(
-      fields, table.tableEnv, aggNames, propNames)
-    val projectFields = extractFieldReferences(fields ++ groupKey)
-
-    new Table(table.tableEnv,
-      Project(projectsOnAgg,
-        Aggregate(groupKey, aggNames.map(a => Alias(a._1, a._2)).toSeq,
-          Project(projectFields, table.logicalPlan).validate(table.tableEnv)
-        ).validate(table.tableEnv)
-      ).validate(table.tableEnv))
-  }
-
-  /**
-    * Performs a selection operation on a grouped table. Similar to an SQL SELECT statement.
-    * The field expressions can contain complex expressions and aggregations.
-    *
-    * Example:
-    *
-    * {{{
-    *   tab.groupBy("key").select("key, value.avg + ' The average' as average")
-    * }}}
-    */
-  def select(fields: String): Table = {
-    val fieldExprs = ExpressionParser.parseExpressionList(fields)
-    select(fieldExprs: _*)
-  }
-
-  /**
-    * Groups the records of a table by assigning them to windows defined by a time or row interval.
-    *
-    * For streaming tables of infinite size, grouping into windows is required to define finite
-    * groups on which group-based aggregates can be computed.
-    *
-    * For batch tables of finite size, windowing essentially provides shortcuts for time-based
-    * groupBy.
-    *
-    * @param groupWindow group-window that specifies how elements are grouped.
-    * @return A windowed table.
-    */
-  def window(groupWindow: GroupWindow): GroupWindowedTable = {
-    if (table.tableEnv.isInstanceOf[BatchTableEnvironment]) {
-      throw new ValidationException(s"Windows on batch tables are currently not supported.")
-    }
-    new GroupWindowedTable(table, groupKey, groupWindow)
-  }
-}
-
-class GroupWindowedTable(
-    private[flink] val table: Table,
-    private[flink] val groupKey: Seq[Expression],
-    private[flink] val window: GroupWindow) {
-
-  /**
-    * Performs a selection operation on a windowed table. Similar to an SQL SELECT statement.
-    * The field expressions can contain complex expressions and aggregations.
-    *
-    * Example:
-    *
-    * {{{
-    *   groupWindowTable.select('key, 'window.start, 'value.avg + " The average" as 'average)
-    * }}}
-    */
-  def select(fields: Expression*): Table = {
-    val (aggNames, propNames) = extractAggregationsAndProperties(fields, table.tableEnv)
-    val projectsOnAgg = replaceAggregationsAndProperties(
-      fields, table.tableEnv, aggNames, propNames)
-
-    val projectFields = (table.tableEnv, window) match {
-      // event time can be arbitrary field in batch environment
-      case (_: BatchTableEnvironment, w: EventTimeWindow) =>
-        extractFieldReferences(fields ++ groupKey ++ Seq(w.timeField))
-      case (_, _) =>
-        extractFieldReferences(fields ++ groupKey)
-    }
-
-    new Table(table.tableEnv,
-      Project(
-        projectsOnAgg,
-        WindowAggregate(
-          groupKey,
-          window.toLogicalWindow,
-          propNames.map(a => Alias(a._1, a._2)).toSeq,
-          aggNames.map(a => Alias(a._1, a._2)).toSeq,
-          Project(projectFields, table.logicalPlan).validate(table.tableEnv)
-        ).validate(table.tableEnv)
-      ).validate(table.tableEnv))
-  }
-
-  /**
-    * Performs a selection operation on a group-windows table. Similar to an SQL SELECT statement.
-    * The field expressions can contain complex expressions and aggregations.
-    *
-    * Example:
-    *
-    * {{{
-    *   groupWindowTable.select("key, window.start, value.avg + ' The average' as average")
-    * }}}
-    */
-  def select(fields: String): Table = {
-    val fieldExprs = ExpressionParser.parseExpressionList(fields)
-    select(fieldExprs: _*)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala
deleted file mode 100644
index 8eecc74..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/trees/TreeNode.scala
+++ /dev/null
@@ -1,115 +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.api.table.trees
-
-import org.apache.commons.lang.ClassUtils
-
-/**
- * Generic base class for trees that can be transformed and traversed.
- */
-abstract class TreeNode[A <: TreeNode[A]] extends Product { self: A =>
-
-  /**
-   * List of child nodes that should be considered when doing transformations. Other values
-   * in the Product will not be transformed, only handed through.
-   */
-  private[flink] def children: Seq[A]
-
-  /**
-   * Tests for equality by first testing for reference equality.
-   */
-  private[flink] def fastEquals(other: TreeNode[_]): Boolean = this.eq(other) || this == other
-
-  /**
-    * Do tree transformation in post order.
-    */
-  private[flink] def postOrderTransform(rule: PartialFunction[A, A]): A = {
-    def childrenTransform(rule: PartialFunction[A, A]): A = {
-      var changed = false
-      val newArgs = productIterator.map {
-        case arg: TreeNode[_] if children.contains(arg) =>
-          val newChild = arg.asInstanceOf[A].postOrderTransform(rule)
-          if (!(newChild fastEquals arg)) {
-            changed = true
-            newChild
-          } else {
-            arg
-          }
-        case args: Traversable[_] => args.map {
-          case arg: TreeNode[_] if children.contains(arg) =>
-            val newChild = arg.asInstanceOf[A].postOrderTransform(rule)
-            if (!(newChild fastEquals arg)) {
-              changed = true
-              newChild
-            } else {
-              arg
-            }
-          case other => other
-        }
-        case nonChild: AnyRef => nonChild
-        case null => null
-      }.toArray
-      if (changed) makeCopy(newArgs) else this
-    }
-
-    val afterChildren = childrenTransform(rule)
-    if (afterChildren fastEquals this) {
-      rule.applyOrElse(this, identity[A])
-    } else {
-      rule.applyOrElse(afterChildren, identity[A])
-    }
-  }
-
-  /**
-    * Runs the given function first on the node and then recursively on all its children.
-    */
-  private[flink] def preOrderVisit(f: A => Unit): Unit = {
-    f(this)
-    children.foreach(_.preOrderVisit(f))
-  }
-
-  /**
-   * Creates a new copy of this expression with new children. This is used during transformation
-   * if children change.
-   */
-  private[flink] def makeCopy(newArgs: Array[AnyRef]): A = {
-    val ctors = getClass.getConstructors.filter(_.getParameterTypes.size > 0)
-    if (ctors.isEmpty) {
-      throw new RuntimeException(s"No valid constructor for ${getClass.getSimpleName}")
-    }
-
-    val defaultCtor = ctors.find { ctor =>
-      if (ctor.getParameterTypes.size != newArgs.length) {
-        false
-      } else if (newArgs.contains(null)) {
-        false
-      } else {
-        val argsClasses: Array[Class[_]] = newArgs.map(_.getClass)
-        ClassUtils.isAssignable(argsClasses, ctor.getParameterTypes)
-      }
-    }.getOrElse(ctors.maxBy(_.getParameterTypes.size))
-
-    try {
-      defaultCtor.newInstance(newArgs: _*).asInstanceOf[A]
-    } catch {
-      case e: Throwable =>
-        throw new RuntimeException(
-          s"Fail to copy treeNode ${getClass.getName}: ${e.getStackTraceString}")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/InternalTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/InternalTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/InternalTypeInfo.scala
deleted file mode 100644
index 2f896ec..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/InternalTypeInfo.scala
+++ /dev/null
@@ -1,81 +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.api.table.typeutils
-
-import java.util.Objects
-
-import org.apache.flink.api.common.ExecutionConfig
-import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
-import org.apache.flink.api.common.typeutils.{TypeComparator, TypeSerializer}
-import org.apache.flink.util.Preconditions._
-
-/**
-  * TypeInformation for internal types of the Table API that are for translation purposes only
-  * and should not be contained in final plan.
-  */
-@SerialVersionUID(-13064574364925255L)
-abstract class InternalTypeInfo[T](val clazz: Class[T])
-  extends TypeInformation[T]
-  with AtomicType[T] {
-
-  checkNotNull(clazz)
-
-  override def isBasicType: Boolean =
-    throw new UnsupportedOperationException("This type is for internal use only.")
-
-  override def isTupleType: Boolean =
-    throw new UnsupportedOperationException("This type is for internal use only.")
-
-  override def getArity: Int =
-    throw new UnsupportedOperationException("This type is for internal use only.")
-
-  override def getTotalFields: Int =
-    throw new UnsupportedOperationException("This type is for internal use only.")
-
-  override def getTypeClass: Class[T] = clazz
-
-  override def isKeyType: Boolean =
-    throw new UnsupportedOperationException("This type is for internal use only.")
-
-  override def createSerializer(config: ExecutionConfig): TypeSerializer[T] =
-    throw new UnsupportedOperationException("This type is for internal use only.")
-
-  override def createComparator(
-      sortOrderAscending: Boolean,
-      executionConfig: ExecutionConfig)
-    : TypeComparator[T] =
-    throw new UnsupportedOperationException("This type is for internal use only.")
-
-  // ----------------------------------------------------------------------------------------------
-
-  override def hashCode: Int = Objects.hash(clazz)
-
-  def canEqual(obj: Any): Boolean
-
-  override def equals(obj: Any): Boolean = {
-    obj match {
-      case other: InternalTypeInfo[_] =>
-        other.canEqual(this) && (this.clazz eq other.clazz)
-      case _ =>
-        false
-    }
-  }
-
-  override def toString: String = s"InternalTypeInfo"
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowIntervalTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowIntervalTypeInfo.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowIntervalTypeInfo.scala
deleted file mode 100644
index 4dc83d0..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/typeutils/RowIntervalTypeInfo.scala
+++ /dev/null
@@ -1,36 +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.api.table.typeutils
-
-/**
-  * TypeInformation for row intervals.
-  */
-@SerialVersionUID(-1306179424364925258L)
-class RowIntervalTypeInfo extends InternalTypeInfo[Long](classOf[Long]) {
-
-  def canEqual(obj: Any): Boolean = obj.isInstanceOf[RowIntervalTypeInfo]
-
-  override def toString: String = s"RowIntervalTypeInfo"
-}
-
-object RowIntervalTypeInfo {
-
-  val INTERVAL_ROWS = new RowIntervalTypeInfo()
-
-}


[33/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamAggregate.scala
deleted file mode 100644
index 7133773..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamAggregate.scala
+++ /dev/null
@@ -1,358 +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.api.table.plan.nodes.datastream
-
-import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.AggregateCall
-import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.tuple.Tuple
-import org.apache.flink.api.table.FlinkRelBuilder.NamedWindowProperty
-import org.apache.flink.api.table.expressions._
-import org.apache.flink.api.table.plan.logical._
-import org.apache.flink.api.table.plan.nodes.FlinkAggregate
-import org.apache.flink.api.table.plan.nodes.datastream.DataStreamAggregate._
-import org.apache.flink.api.table.runtime.aggregate.AggregateUtil._
-import org.apache.flink.api.table.runtime.aggregate.{Aggregate, _}
-import org.apache.flink.api.table.typeutils.TypeCheckUtils.isTimeInterval
-import org.apache.flink.api.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo, TypeConverter}
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.{FlinkTypeFactory, StreamTableEnvironment}
-import org.apache.flink.types.Row
-import org.apache.flink.streaming.api.datastream.{AllWindowedStream, DataStream, KeyedStream, WindowedStream}
-import org.apache.flink.streaming.api.windowing.assigners._
-import org.apache.flink.streaming.api.windowing.time.Time
-import org.apache.flink.streaming.api.windowing.windows.{Window => DataStreamWindow}
-
-import scala.collection.JavaConverters._
-
-class DataStreamAggregate(
-    window: LogicalWindow,
-    namedProperties: Seq[NamedWindowProperty],
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    inputNode: RelNode,
-    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
-    rowRelDataType: RelDataType,
-    inputType: RelDataType,
-    grouping: Array[Int])
-  extends SingleRel(cluster, traitSet, inputNode)
-  with FlinkAggregate
-  with DataStreamRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataStreamAggregate(
-      window,
-      namedProperties,
-      cluster,
-      traitSet,
-      inputs.get(0),
-      namedAggregates,
-      getRowType,
-      inputType,
-      grouping)
-  }
-
-  override def toString: String = {
-    s"Aggregate(${
-      if (!grouping.isEmpty) {
-        s"groupBy: (${groupingToString(inputType, grouping)}), "
-      } else {
-        ""
-      }
-    }window: ($window), " +
-      s"select: (${
-        aggregationToString(
-          inputType,
-          grouping,
-          getRowType,
-          namedAggregates,
-          namedProperties)
-      }))"
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw)
-      .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty)
-      .item("window", window)
-      .item("select", aggregationToString(
-        inputType,
-        grouping,
-        getRowType,
-        namedAggregates,
-        namedProperties))
-  }
-
-  override def translateToPlan(
-    tableEnv: StreamTableEnvironment,
-    expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {
-
-    val config = tableEnv.getConfig
-    val groupingKeys = grouping.indices.toArray
-    val inputDS = input.asInstanceOf[DataStreamRel].translateToPlan(
-      tableEnv,
-      // tell the input operator that this operator currently only supports Rows as input
-      Some(TypeConverter.DEFAULT_ROW_TYPE))
-
-    // get the output types
-    val fieldTypes: Array[TypeInformation[_]] =
-      getRowType.getFieldList.asScala
-      .map(field => FlinkTypeFactory.toTypeInfo(field.getType))
-      .toArray
-
-    val rowTypeInfo = new RowTypeInfo(fieldTypes: _*)
-
-    val aggString = aggregationToString(
-      inputType,
-      grouping,
-      getRowType,
-      namedAggregates,
-      namedProperties)
-
-    val prepareOpName = s"prepare select: ($aggString)"
-    val keyedAggOpName = s"groupBy: (${groupingToString(inputType, grouping)}), " +
-      s"window: ($window), " +
-      s"select: ($aggString)"
-    val nonKeyedAggOpName = s"window: ($window), select: ($aggString)"
-
-    val mapFunction = AggregateUtil.createPrepareMapFunction(
-      namedAggregates,
-      grouping,
-      inputType)
-
-    val mappedInput = inputDS.map(mapFunction).name(prepareOpName)
-
-    val result: DataStream[Any] = {
-      // check whether all aggregates support partial aggregate
-      if (AggregateUtil.doAllSupportPartialAggregation(
-            namedAggregates.map(_.getKey),
-            inputType,
-            grouping.length)) {
-        // do Incremental Aggregation
-        val reduceFunction = AggregateUtil.createIncrementalAggregateReduceFunction(
-          namedAggregates,
-          inputType,
-          getRowType,
-          grouping)
-        // grouped / keyed aggregation
-        if (groupingKeys.length > 0) {
-          val windowFunction = AggregateUtil.createWindowIncrementalAggregationFunction(
-            window,
-            namedAggregates,
-            inputType,
-            rowRelDataType,
-            grouping,
-            namedProperties)
-
-          val keyedStream = mappedInput.keyBy(groupingKeys: _*)
-          val windowedStream =
-            createKeyedWindowedStream(window, keyedStream)
-            .asInstanceOf[WindowedStream[Row, Tuple, DataStreamWindow]]
-
-          windowedStream
-          .apply(reduceFunction, windowFunction)
-          .returns(rowTypeInfo)
-          .name(keyedAggOpName)
-          .asInstanceOf[DataStream[Any]]
-        }
-        // global / non-keyed aggregation
-        else {
-          val windowFunction = AggregateUtil.createAllWindowIncrementalAggregationFunction(
-            window,
-            namedAggregates,
-            inputType,
-            rowRelDataType,
-            grouping,
-            namedProperties)
-
-          val windowedStream =
-            createNonKeyedWindowedStream(window, mappedInput)
-            .asInstanceOf[AllWindowedStream[Row, DataStreamWindow]]
-
-          windowedStream
-          .apply(reduceFunction, windowFunction)
-          .returns(rowTypeInfo)
-          .name(nonKeyedAggOpName)
-          .asInstanceOf[DataStream[Any]]
-        }
-      }
-      else {
-        // do non-Incremental Aggregation
-        // grouped / keyed aggregation
-        if (groupingKeys.length > 0) {
-
-          val windowFunction = AggregateUtil.createWindowAggregationFunction(
-            window,
-            namedAggregates,
-            inputType,
-            rowRelDataType,
-            grouping,
-            namedProperties)
-
-          val keyedStream = mappedInput.keyBy(groupingKeys: _*)
-          val windowedStream =
-            createKeyedWindowedStream(window, keyedStream)
-            .asInstanceOf[WindowedStream[Row, Tuple, DataStreamWindow]]
-
-          windowedStream
-          .apply(windowFunction)
-          .returns(rowTypeInfo)
-          .name(keyedAggOpName)
-          .asInstanceOf[DataStream[Any]]
-        }
-        // global / non-keyed aggregation
-        else {
-          val windowFunction = AggregateUtil.createAllWindowAggregationFunction(
-            window,
-            namedAggregates,
-            inputType,
-            rowRelDataType,
-            grouping,
-            namedProperties)
-
-          val windowedStream =
-            createNonKeyedWindowedStream(window, mappedInput)
-            .asInstanceOf[AllWindowedStream[Row, DataStreamWindow]]
-
-          windowedStream
-          .apply(windowFunction)
-          .returns(rowTypeInfo)
-          .name(nonKeyedAggOpName)
-          .asInstanceOf[DataStream[Any]]
-        }
-      }
-    }
-    // if the expected type is not a Row, inject a mapper to convert to the expected type
-    expectedType match {
-      case Some(typeInfo) if typeInfo.getTypeClass != classOf[Row] =>
-        val mapName = s"convert: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
-        result.map(getConversionMapper(
-          config = config,
-          nullableInput = false,
-          inputType = rowTypeInfo.asInstanceOf[TypeInformation[Any]],
-          expectedType = expectedType.get,
-          conversionOperatorName = "DataStreamAggregateConversion",
-          fieldNames = getRowType.getFieldNames.asScala
-        ))
-          .name(mapName)
-      case _ => result
-    }
-  }
-}
-object DataStreamAggregate {
-
-
-  private def createKeyedWindowedStream(groupWindow: LogicalWindow, stream: KeyedStream[Row, Tuple])
-    : WindowedStream[Row, Tuple, _ <: DataStreamWindow] = groupWindow match {
-
-    case ProcessingTimeTumblingGroupWindow(_, size) if isTimeInterval(size.resultType) =>
-      stream.window(TumblingProcessingTimeWindows.of(asTime(size)))
-
-    case ProcessingTimeTumblingGroupWindow(_, size) =>
-      stream.countWindow(asCount(size))
-
-    case EventTimeTumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
-      stream.window(TumblingEventTimeWindows.of(asTime(size)))
-
-    case EventTimeTumblingGroupWindow(_, _, size) =>
-      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
-      // before applying the  windowing logic. Otherwise, this would be the same as a
-      // ProcessingTimeTumblingGroupWindow
-      throw new UnsupportedOperationException("Event-time grouping windows on row intervals are " +
-        "currently not supported.")
-
-    case ProcessingTimeSlidingGroupWindow(_, size, slide) if isTimeInterval(size.resultType) =>
-      stream.window(SlidingProcessingTimeWindows.of(asTime(size), asTime(slide)))
-
-    case ProcessingTimeSlidingGroupWindow(_, size, slide) =>
-      stream.countWindow(asCount(size), asCount(slide))
-
-    case EventTimeSlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) =>
-      stream.window(SlidingEventTimeWindows.of(asTime(size), asTime(slide)))
-
-    case EventTimeSlidingGroupWindow(_, _, size, slide) =>
-      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
-      // before applying the  windowing logic. Otherwise, this would be the same as a
-      // ProcessingTimeTumblingGroupWindow
-      throw new UnsupportedOperationException("Event-time grouping windows on row intervals are " +
-        "currently not supported.")
-
-    case ProcessingTimeSessionGroupWindow(_, gap: Expression) =>
-      stream.window(ProcessingTimeSessionWindows.withGap(asTime(gap)))
-
-    case EventTimeSessionGroupWindow(_, _, gap) =>
-      stream.window(EventTimeSessionWindows.withGap(asTime(gap)))
-  }
-
-  private def createNonKeyedWindowedStream(groupWindow: LogicalWindow, stream: DataStream[Row])
-    : AllWindowedStream[Row, _ <: DataStreamWindow] = groupWindow match {
-
-    case ProcessingTimeTumblingGroupWindow(_, size) if isTimeInterval(size.resultType) =>
-      stream.windowAll(TumblingProcessingTimeWindows.of(asTime(size)))
-
-    case ProcessingTimeTumblingGroupWindow(_, size) =>
-      stream.countWindowAll(asCount(size))
-
-    case EventTimeTumblingGroupWindow(_, _, size) if isTimeInterval(size.resultType) =>
-      stream.windowAll(TumblingEventTimeWindows.of(asTime(size)))
-
-    case EventTimeTumblingGroupWindow(_, _, size) =>
-      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
-      // before applying the  windowing logic. Otherwise, this would be the same as a
-      // ProcessingTimeTumblingGroupWindow
-      throw new UnsupportedOperationException("Event-time grouping windows on row intervals are " +
-        "currently not supported.")
-
-    case ProcessingTimeSlidingGroupWindow(_, size, slide) if isTimeInterval(size.resultType) =>
-      stream.windowAll(SlidingProcessingTimeWindows.of(asTime(size), asTime(slide)))
-
-    case ProcessingTimeSlidingGroupWindow(_, size, slide) =>
-      stream.countWindowAll(asCount(size), asCount(slide))
-
-    case EventTimeSlidingGroupWindow(_, _, size, slide) if isTimeInterval(size.resultType) =>
-      stream.windowAll(SlidingEventTimeWindows.of(asTime(size), asTime(slide)))
-
-    case EventTimeSlidingGroupWindow(_, _, size, slide) =>
-      // TODO: EventTimeTumblingGroupWindow should sort the stream on event time
-      // before applying the  windowing logic. Otherwise, this would be the same as a
-      // ProcessingTimeTumblingGroupWindow
-      throw new UnsupportedOperationException("Event-time grouping windows on row intervals are " +
-        "currently not supported.")
-
-    case ProcessingTimeSessionGroupWindow(_, gap) =>
-      stream.windowAll(ProcessingTimeSessionWindows.withGap(asTime(gap)))
-
-    case EventTimeSessionGroupWindow(_, _, gap) =>
-      stream.windowAll(EventTimeSessionWindows.withGap(asTime(gap)))
-  }
-
-  def asTime(expr: Expression): Time = expr match {
-    case Literal(value: Long, TimeIntervalTypeInfo.INTERVAL_MILLIS) => Time.milliseconds(value)
-    case _ => throw new IllegalArgumentException()
-  }
-
-  def asCount(expr: Expression): Long = expr match {
-    case Literal(value: Long, RowIntervalTypeInfo.INTERVAL_ROWS) => value
-    case _ => throw new IllegalArgumentException()
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCalc.scala
deleted file mode 100644
index 5312a5f..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCalc.scala
+++ /dev/null
@@ -1,104 +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.api.table.plan.nodes.datastream
-
-import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
-import org.apache.calcite.rex.RexProgram
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.StreamTableEnvironment
-import org.apache.flink.api.table.codegen.CodeGenerator
-import org.apache.flink.api.table.plan.nodes.FlinkCalc
-import org.apache.flink.api.table.typeutils.TypeConverter._
-import org.apache.flink.api.common.functions.FlatMapFunction
-import org.apache.flink.streaming.api.datastream.DataStream
-
-/**
-  * Flink RelNode which matches along with FlatMapOperator.
-  *
-  */
-class DataStreamCalc(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    input: RelNode,
-    rowRelDataType: RelDataType,
-    calcProgram: RexProgram,
-    ruleDescription: String)
-  extends SingleRel(cluster, traitSet, input)
-  with FlinkCalc
-  with DataStreamRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataStreamCalc(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      getRowType,
-      calcProgram,
-      ruleDescription
-    )
-  }
-
-  override def toString: String = calcToString(calcProgram, getExpressionString)
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw)
-      .item("select", selectionToString(calcProgram, getExpressionString))
-      .itemIf("where",
-        conditionToString(calcProgram, getExpressionString),
-        calcProgram.getCondition != null)
-  }
-
-  override def translateToPlan(
-      tableEnv: StreamTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {
-
-    val config = tableEnv.getConfig
-
-    val inputDataStream = getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
-
-    val returnType = determineReturnType(
-      getRowType,
-      expectedType,
-      config.getNullCheck,
-      config.getEfficientTypeUsage)
-
-    val generator = new CodeGenerator(config, false, inputDataStream.getType)
-
-    val body = functionBody(
-      generator,
-      inputDataStream.getType,
-      getRowType,
-      calcProgram,
-      config,
-      expectedType)
-
-    val genFunction = generator.generateFunction(
-      ruleDescription,
-      classOf[FlatMapFunction[Any, Any]],
-      body,
-      returnType)
-
-    val mapFunc = calcMapFunction(genFunction)
-    inputDataStream.flatMap(mapFunc).name(calcOpName(calcProgram, getExpressionString))
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamConvention.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamConvention.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamConvention.scala
deleted file mode 100644
index 3b6a653..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamConvention.scala
+++ /dev/null
@@ -1,47 +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.api.table.plan.nodes.datastream
-
-import org.apache.calcite.plan._
-
-class DataStreamConvention extends Convention {
-
-  override def toString: String = getName
-
-  override def useAbstractConvertersForConversion(
-    fromTraits: RelTraitSet,
-    toTraits: RelTraitSet): Boolean = false
-
-  override def canConvertConvention(toConvention: Convention): Boolean = false
-
-  def getInterface: Class[_] = classOf[DataStreamRel]
-
-  def getName: String = "DATASTREAM"
-
-  def getTraitDef: RelTraitDef[_ <: RelTrait] = ConventionTraitDef.INSTANCE
-
-  def satisfies(`trait`: RelTrait): Boolean = this eq `trait`
-
-  def register(planner: RelOptPlanner): Unit = { }
-}
-
-object DataStreamConvention {
-
-  val INSTANCE = new DataStreamConvention
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCorrelate.scala
deleted file mode 100644
index 3bfa6e2..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamCorrelate.scala
+++ /dev/null
@@ -1,134 +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.api.table.plan.nodes.datastream
-
-import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.logical.LogicalTableFunctionScan
-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.table.StreamTableEnvironment
-import org.apache.flink.api.table.codegen.CodeGenerator
-import org.apache.flink.api.table.functions.utils.TableSqlFunction
-import org.apache.flink.api.table.plan.nodes.FlinkCorrelate
-import org.apache.flink.api.table.typeutils.TypeConverter._
-import org.apache.flink.streaming.api.datastream.DataStream
-
-/**
-  * Flink RelNode which matches along with join a user defined table function.
-  */
-class DataStreamCorrelate(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    inputNode: RelNode,
-    scan: LogicalTableFunctionScan,
-    condition: Option[RexNode],
-    relRowType: RelDataType,
-    joinRowType: RelDataType,
-    joinType: SemiJoinType,
-    ruleDescription: String)
-  extends SingleRel(cluster, traitSet, inputNode)
-  with FlinkCorrelate
-  with DataStreamRel {
-
-  override def deriveRowType() = relRowType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataStreamCorrelate(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      scan,
-      condition,
-      relRowType,
-      joinRowType,
-      joinType,
-      ruleDescription)
-  }
-
-  override def toString: String = {
-    val rexCall = scan.getCall.asInstanceOf[RexCall]
-    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
-    correlateToString(rexCall, sqlFunction)
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    val rexCall = scan.getCall.asInstanceOf[RexCall]
-    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
-    super.explainTerms(pw)
-      .item("invocation", scan.getCall)
-      .item("function", sqlFunction.getTableFunction.getClass.getCanonicalName)
-      .item("rowType", relRowType)
-      .item("joinType", joinType)
-      .itemIf("condition", condition.orNull, condition.isDefined)
-  }
-
-  override def translateToPlan(
-      tableEnv: StreamTableEnvironment,
-      expectedType: Option[TypeInformation[Any]])
-    : DataStream[Any] = {
-
-    val config = tableEnv.getConfig
-    val returnType = determineReturnType(
-      getRowType,
-      expectedType,
-      config.getNullCheck,
-      config.getEfficientTypeUsage)
-
-    // we do not need to specify input type
-    val inputDS = inputNode.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
-
-    val funcRel = scan.asInstanceOf[LogicalTableFunctionScan]
-    val rexCall = funcRel.getCall.asInstanceOf[RexCall]
-    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
-    val pojoFieldMapping = sqlFunction.getPojoFieldMapping
-    val udtfTypeInfo = sqlFunction.getRowTypeInfo.asInstanceOf[TypeInformation[Any]]
-
-    val generator = new CodeGenerator(
-      config,
-      false,
-      inputDS.getType,
-      Some(udtfTypeInfo),
-      None,
-      Some(pojoFieldMapping))
-
-    val body = functionBody(
-      generator,
-      udtfTypeInfo,
-      getRowType,
-      rexCall,
-      condition,
-      config,
-      joinType,
-      expectedType)
-
-    val genFunction = generator.generateFunction(
-      ruleDescription,
-      classOf[FlatMapFunction[Any, Any]],
-      body,
-      returnType)
-
-    val mapFunc = correlateMapFunction(genFunction)
-
-    inputDS.flatMap(mapFunc).name(correlateOpName(rexCall, sqlFunction, relRowType))
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamRel.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamRel.scala
deleted file mode 100644
index 6cf13a5..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamRel.scala
+++ /dev/null
@@ -1,45 +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.api.table.plan.nodes.datastream
-
-import org.apache.calcite.rel.RelNode
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.{StreamTableEnvironment, TableConfig}
-import org.apache.flink.api.table.plan.nodes.FlinkRel
-import org.apache.flink.streaming.api.datastream.DataStream
-
-trait DataStreamRel extends RelNode with FlinkRel {
-
-  /**
-    * Translates the FlinkRelNode into a Flink operator.
-    *
-    * @param tableEnv The [[StreamTableEnvironment]] of the translated Table.
-    * @param expectedType specifies the type the Flink operator should return. The type must
-    *                     have the same arity as the result. For instance, if the
-    *                     expected type is a RowTypeInfo this method will return a DataSet of
-    *                     type Row. If the expected type is Tuple2, the operator will return
-    *                     a Tuple2 if possible. Row otherwise.
-    * @return DataStream of type expectedType or RowTypeInfo
-    */
-  def translateToPlan(
-    tableEnv: StreamTableEnvironment,
-    expectedType: Option[TypeInformation[Any]] = None) : DataStream[Any]
-
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamScan.scala
deleted file mode 100644
index da83b64..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamScan.scala
+++ /dev/null
@@ -1,64 +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.api.table.plan.nodes.datastream
-
-import org.apache.calcite.plan._
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.StreamTableEnvironment
-import org.apache.flink.api.table.plan.schema.DataStreamTable
-import org.apache.flink.streaming.api.datastream.DataStream
-
-/**
-  * Flink RelNode which matches along with DataStreamSource.
-  * It ensures that types without deterministic field order (e.g. POJOs) are not part of
-  * the plan translation.
-  */
-class DataStreamScan(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    table: RelOptTable,
-    rowRelDataType: RelDataType)
-  extends StreamScan(cluster, traitSet, table) {
-
-  val dataStreamTable: DataStreamTable[Any] = getTable.unwrap(classOf[DataStreamTable[Any]])
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataStreamScan(
-      cluster,
-      traitSet,
-      getTable,
-      getRowType
-    )
-  }
-
-  override def translateToPlan(
-      tableEnv: StreamTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {
-
-    val config = tableEnv.getConfig
-    val inputDataStream: DataStream[Any] = dataStreamTable.dataStream
-
-    convertToExpectedType(inputDataStream, dataStreamTable, expectedType, config)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamUnion.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamUnion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamUnion.scala
deleted file mode 100644
index f490d31..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamUnion.scala
+++ /dev/null
@@ -1,75 +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.api.table.plan.nodes.datastream
-
-import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.{RelNode, RelWriter, BiRel}
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.StreamTableEnvironment
-import org.apache.flink.streaming.api.datastream.DataStream
-
-import scala.collection.JavaConverters._
-
-/**
-  * Flink RelNode which matches along with Union.
-  *
-  */
-class DataStreamUnion(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    leftNode: RelNode,
-    rightNode: RelNode,
-    rowRelDataType: RelDataType)
-  extends BiRel(cluster, traitSet, leftNode, rightNode)
-  with DataStreamRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataStreamUnion(
-      cluster,
-      traitSet,
-      inputs.get(0),
-      inputs.get(1),
-      getRowType
-    )
-  }
-
-  override def explainTerms(pw: RelWriter): RelWriter = {
-    super.explainTerms(pw).item("union", unionSelectionToString)
-  }
-
-  override def toString = {
-    s"Union(union: (${getRowType.getFieldNames.asScala.toList.mkString(", ")}))"
-  }
-
-  override def translateToPlan(
-      tableEnv: StreamTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {
-
-    val leftDataSet = left.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
-    val rightDataSet = right.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
-    leftDataSet.union(rightDataSet)
-  }
-
-  private def unionSelectionToString: String = {
-    getRowType.getFieldNames.asScala.toList.mkString(", ")
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamValues.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamValues.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamValues.scala
deleted file mode 100644
index 3b98653..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/DataStreamValues.scala
+++ /dev/null
@@ -1,97 +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.api.table.plan.nodes.datastream
-
-import com.google.common.collect.ImmutableList
-import org.apache.calcite.plan._
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.Values
-import org.apache.calcite.rex.RexLiteral
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.StreamTableEnvironment
-import org.apache.flink.api.table.codegen.CodeGenerator
-import org.apache.flink.api.table.runtime.io.ValuesInputFormat
-import org.apache.flink.api.table.typeutils.TypeConverter._
-import org.apache.flink.streaming.api.datastream.DataStream
-
-import scala.collection.JavaConverters._
-
-/**
-  * DataStream RelNode for LogicalValues.
-  */
-class DataStreamValues(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    rowRelDataType: RelDataType,
-    tuples: ImmutableList[ImmutableList[RexLiteral]],
-    ruleDescription: String)
-  extends Values(cluster, rowRelDataType, tuples, traitSet)
-  with DataStreamRel {
-
-  override def deriveRowType() = rowRelDataType
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new DataStreamValues(
-      cluster,
-      traitSet,
-      getRowType,
-      getTuples,
-      ruleDescription
-    )
-  }
-
-  override def translateToPlan(
-      tableEnv: StreamTableEnvironment,
-      expectedType: Option[TypeInformation[Any]])
-    : DataStream[Any] = {
-
-    val config = tableEnv.getConfig
-
-    val returnType = determineReturnType(
-      getRowType,
-      expectedType,
-      config.getNullCheck,
-      config.getEfficientTypeUsage)
-
-    val generator = new CodeGenerator(config)
-
-    // generate code for every record
-    val generatedRecords = getTuples.asScala.map { r =>
-      generator.generateResultExpression(
-        returnType,
-        getRowType.getFieldNames.asScala,
-        r.asScala)
-    }
-
-    // generate input format
-    val generatedFunction = generator.generateValuesInputFormat(
-      ruleDescription,
-      generatedRecords.map(_.code),
-      returnType)
-
-    val inputFormat = new ValuesInputFormat[Any](
-      generatedFunction.name,
-      generatedFunction.code,
-      generatedFunction.returnType)
-
-    tableEnv.execEnv.createInput(inputFormat, returnType).asInstanceOf[DataStream[Any]]
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala
deleted file mode 100644
index b13770e..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamScan.scala
+++ /dev/null
@@ -1,107 +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.api.table.plan.nodes.datastream
-
-import org.apache.calcite.plan._
-import org.apache.calcite.rel.`type`.RelDataType
-import org.apache.calcite.rel.core.TableScan
-import org.apache.flink.api.common.functions.MapFunction
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.typeutils.PojoTypeInfo
-import org.apache.flink.api.table.TableConfig
-import org.apache.flink.api.table.codegen.CodeGenerator
-import org.apache.flink.api.table.plan.schema.FlinkTable
-import org.apache.flink.api.table.runtime.MapRunner
-import org.apache.flink.api.table.typeutils.TypeConverter.determineReturnType
-import org.apache.flink.streaming.api.datastream.DataStream
-
-import scala.collection.JavaConversions._
-import scala.collection.JavaConverters._
-
-abstract class StreamScan(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    table: RelOptTable)
-  extends TableScan(cluster, traitSet, table)
-  with DataStreamRel {
-
-  protected def convertToExpectedType(
-      input: DataStream[Any],
-      flinkTable: FlinkTable[_],
-      expectedType: Option[TypeInformation[Any]],
-      config: TableConfig): DataStream[Any] = {
-
-    val inputType = input.getType
-
-    expectedType match {
-
-      // special case:
-      // if efficient type usage is enabled and no expected type is set
-      // we can simply forward the DataSet to the next operator.
-      // however, we cannot forward PojoTypes as their fields don't have an order
-      case None if config.getEfficientTypeUsage && !inputType.isInstanceOf[PojoTypeInfo[_]] =>
-        input
-
-      case _ =>
-        val determinedType = determineReturnType(
-          getRowType,
-          expectedType,
-          config.getNullCheck,
-          config.getEfficientTypeUsage)
-
-        // conversion
-        if (determinedType != inputType) {
-          val generator = new CodeGenerator(
-            config,
-            nullableInput = false,
-            input.getType,
-            flinkTable.fieldIndexes)
-
-          val conversion = generator.generateConverterResultExpression(
-            determinedType,
-            getRowType.getFieldNames)
-
-          val body =
-            s"""
-               |${conversion.code}
-               |return ${conversion.resultTerm};
-               |""".stripMargin
-
-          val genFunction = generator.generateFunction(
-            "DataSetSourceConversion",
-            classOf[MapFunction[Any, Any]],
-            body,
-            determinedType)
-
-          val mapFunc = new MapRunner[Any, Any](
-            genFunction.name,
-            genFunction.code,
-            genFunction.returnType)
-
-          val opName = s"from: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
-
-          input.map(mapFunc).name(opName)
-        }
-        // no conversion necessary, forward
-        else {
-          input
-        }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala
deleted file mode 100644
index 8201070..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/datastream/StreamTableSourceScan.scala
+++ /dev/null
@@ -1,62 +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.api.table.plan.nodes.datastream
-
-import org.apache.calcite.plan._
-import org.apache.calcite.rel.RelNode
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.plan.schema.TableSourceTable
-import org.apache.flink.api.table.sources.StreamTableSource
-import org.apache.flink.api.table.{FlinkTypeFactory, StreamTableEnvironment}
-import org.apache.flink.streaming.api.datastream.DataStream
-
-/** Flink RelNode to read data from an external source defined by a [[StreamTableSource]]. */
-class StreamTableSourceScan(
-    cluster: RelOptCluster,
-    traitSet: RelTraitSet,
-    table: RelOptTable,
-    tableSource: StreamTableSource[_])
-  extends StreamScan(cluster, traitSet, table) {
-
-  override def deriveRowType() = {
-    val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
-    flinkTypeFactory.buildRowDataType(tableSource.getFieldsNames, tableSource.getFieldTypes)
-  }
-
-  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
-    new StreamTableSourceScan(
-      cluster,
-      traitSet,
-      getTable,
-      tableSource
-    )
-  }
-
-  override def translateToPlan(
-      tableEnv: StreamTableEnvironment,
-      expectedType: Option[TypeInformation[Any]]): DataStream[Any] = {
-
-    val config = tableEnv.getConfig
-    val inputDataStream: DataStream[Any] = tableSource
-      .getDataStream(tableEnv.execEnv).asInstanceOf[DataStream[Any]]
-
-    convertToExpectedType(inputDataStream, new TableSourceTable(tableSource), expectedType, config)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/EnumerableToLogicalTableScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/EnumerableToLogicalTableScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/EnumerableToLogicalTableScan.scala
deleted file mode 100644
index ee515c9..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/EnumerableToLogicalTableScan.scala
+++ /dev/null
@@ -1,49 +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.api.table.plan.rules
-
-import org.apache.calcite.plan.RelOptRule.{any, operand}
-import org.apache.calcite.adapter.enumerable.EnumerableTableScan
-import org.apache.calcite.plan.{RelOptRuleCall, RelOptRule, RelOptRuleOperand}
-import org.apache.calcite.rel.logical.LogicalTableScan
-
-/**
- * Rule that converts an EnumerableTableScan into a LogicalTableScan.
- * We need this rule because Calcite creates an EnumerableTableScan
- * when parsing a SQL query. We convert it into a LogicalTableScan
- * so we can merge the optimization process with any plan that might be created
- * by the Table API.
- */
-class EnumerableToLogicalTableScan(
-    operand: RelOptRuleOperand,
-    description: String) extends RelOptRule(operand, description) {
-
-  override def onMatch(call: RelOptRuleCall): Unit = {
-    val oldRel = call.rel(0).asInstanceOf[EnumerableTableScan]
-    val table = oldRel.getTable
-    val newRel = LogicalTableScan.create(oldRel.getCluster, table)
-    call.transformTo(newRel)
-  }
-}
-
-object EnumerableToLogicalTableScan {
-  val INSTANCE = new EnumerableToLogicalTableScan(
-      operand(classOf[EnumerableTableScan], any),
-    "EnumerableToLogicalTableScan")
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala
deleted file mode 100644
index 183065c..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/FlinkRuleSets.scala
+++ /dev/null
@@ -1,161 +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.api.table.plan.rules
-
-import org.apache.calcite.rel.rules._
-import org.apache.calcite.tools.{RuleSets, RuleSet}
-import org.apache.flink.api.table.plan.rules.dataSet._
-import org.apache.flink.api.table.plan.rules.datastream._
-import org.apache.flink.api.table.plan.rules.datastream.{DataStreamCalcRule, DataStreamScanRule, DataStreamUnionRule}
-
-object FlinkRuleSets {
-
-  /**
-    * RuleSet to optimize plans for batch / DataSet execution
-    */
-  val DATASET_OPT_RULES: RuleSet = RuleSets.ofList(
-
-    // convert a logical table scan to a relational expression
-    TableScanRule.INSTANCE,
-    EnumerableToLogicalTableScan.INSTANCE,
-
-    // push a filter into a join
-    FilterJoinRule.FILTER_ON_JOIN,
-    // push filter into the children of a join
-    FilterJoinRule.JOIN,
-    // push filter through an aggregation
-    FilterAggregateTransposeRule.INSTANCE,
-
-    // aggregation and projection rules
-    AggregateProjectMergeRule.INSTANCE,
-    AggregateProjectPullUpConstantsRule.INSTANCE,
-    // push a projection past a filter or vice versa
-    ProjectFilterTransposeRule.INSTANCE,
-    FilterProjectTransposeRule.INSTANCE,
-    // push a projection to the children of a join
-    ProjectJoinTransposeRule.INSTANCE,
-    // remove identity project
-    ProjectRemoveRule.INSTANCE,
-    // reorder sort and projection
-    SortProjectTransposeRule.INSTANCE,
-    ProjectSortTransposeRule.INSTANCE,
-
-    // join rules
-    JoinPushExpressionsRule.INSTANCE,
-
-    // remove union with only a single child
-    UnionEliminatorRule.INSTANCE,
-    // convert non-all union into all-union + distinct
-    UnionToDistinctRule.INSTANCE,
-
-    // remove aggregation if it does not aggregate and input is already distinct
-    AggregateRemoveRule.INSTANCE,
-    // push aggregate through join
-    AggregateJoinTransposeRule.EXTENDED,
-    // aggregate union rule
-    AggregateUnionAggregateRule.INSTANCE,
-
-    // remove unnecessary sort rule
-    SortRemoveRule.INSTANCE,
-
-    // simplify expressions rules
-    ReduceExpressionsRule.FILTER_INSTANCE,
-    ReduceExpressionsRule.PROJECT_INSTANCE,
-    ReduceExpressionsRule.CALC_INSTANCE,
-    ReduceExpressionsRule.JOIN_INSTANCE,
-
-    // prune empty results rules
-    PruneEmptyRules.AGGREGATE_INSTANCE,
-    PruneEmptyRules.FILTER_INSTANCE,
-    PruneEmptyRules.JOIN_LEFT_INSTANCE,
-    PruneEmptyRules.JOIN_RIGHT_INSTANCE,
-    PruneEmptyRules.PROJECT_INSTANCE,
-    PruneEmptyRules.SORT_INSTANCE,
-    PruneEmptyRules.UNION_INSTANCE,
-
-    // calc rules
-    FilterCalcMergeRule.INSTANCE,
-    ProjectCalcMergeRule.INSTANCE,
-    FilterToCalcRule.INSTANCE,
-    ProjectToCalcRule.INSTANCE,
-    CalcMergeRule.INSTANCE,
-
-    // translate to Flink DataSet nodes
-    DataSetAggregateRule.INSTANCE,
-    DataSetAggregateWithNullValuesRule.INSTANCE,
-    DataSetCalcRule.INSTANCE,
-    DataSetJoinRule.INSTANCE,
-    DataSetSingleRowJoinRule.INSTANCE,
-    DataSetScanRule.INSTANCE,
-    DataSetUnionRule.INSTANCE,
-    DataSetIntersectRule.INSTANCE,
-    DataSetMinusRule.INSTANCE,
-    DataSetSortRule.INSTANCE,
-    DataSetValuesRule.INSTANCE,
-    DataSetCorrelateRule.INSTANCE,
-    BatchTableSourceScanRule.INSTANCE,
-    // project pushdown optimization
-    PushProjectIntoBatchTableSourceScanRule.INSTANCE
-  )
-
-  /**
-  * RuleSet to optimize plans for stream / DataStream execution
-  */
-  val DATASTREAM_OPT_RULES: RuleSet = RuleSets.ofList(
-
-      // convert a logical table scan to a relational expression
-      TableScanRule.INSTANCE,
-      EnumerableToLogicalTableScan.INSTANCE,
-
-      // calc rules
-      FilterToCalcRule.INSTANCE,
-      ProjectToCalcRule.INSTANCE,
-      FilterCalcMergeRule.INSTANCE,
-      ProjectCalcMergeRule.INSTANCE,
-      CalcMergeRule.INSTANCE,
-
-      // prune empty results rules
-      PruneEmptyRules.FILTER_INSTANCE,
-      PruneEmptyRules.PROJECT_INSTANCE,
-      PruneEmptyRules.UNION_INSTANCE,
-
-      // push and merge projection rules
-      ProjectFilterTransposeRule.INSTANCE,
-      FilterProjectTransposeRule.INSTANCE,
-      ProjectRemoveRule.INSTANCE,
-
-      // simplify expressions rules
-      ReduceExpressionsRule.FILTER_INSTANCE,
-      ReduceExpressionsRule.PROJECT_INSTANCE,
-      ReduceExpressionsRule.CALC_INSTANCE,
-
-      // merge and push unions rules
-      UnionEliminatorRule.INSTANCE,
-
-      // translate to DataStream nodes
-      DataStreamAggregateRule.INSTANCE,
-      DataStreamCalcRule.INSTANCE,
-      DataStreamScanRule.INSTANCE,
-      DataStreamUnionRule.INSTANCE,
-      DataStreamValuesRule.INSTANCE,
-      DataStreamCorrelateRule.INSTANCE,
-      StreamTableSourceScanRule.INSTANCE
-  )
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/BatchTableSourceScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/BatchTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/BatchTableSourceScanRule.scala
deleted file mode 100644
index 8e3d8bb..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/BatchTableSourceScanRule.scala
+++ /dev/null
@@ -1,73 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.core.TableScan
-import org.apache.calcite.rel.logical.LogicalTableScan
-import org.apache.flink.api.table.plan.nodes.dataset.{BatchTableSourceScan, DataSetConvention}
-import org.apache.flink.api.table.plan.schema.TableSourceTable
-import org.apache.flink.api.table.sources.BatchTableSource
-
-/** Rule to convert a [[LogicalTableScan]] into a [[BatchTableSourceScan]]. */
-class BatchTableSourceScanRule
-  extends ConverterRule(
-      classOf[LogicalTableScan],
-      Convention.NONE,
-      DataSetConvention.INSTANCE,
-      "BatchTableSourceScanRule")
-  {
-
-  /** Rule must only match if TableScan targets a [[BatchTableSource]] */
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val scan: TableScan = call.rel(0).asInstanceOf[TableScan]
-    val dataSetTable = scan.getTable.unwrap(classOf[TableSourceTable])
-    dataSetTable match {
-      case tst: TableSourceTable =>
-        tst.tableSource match {
-          case _: BatchTableSource[_] =>
-            true
-          case _ =>
-            false
-        }
-      case _ =>
-        false
-    }
-  }
-
-  def convert(rel: RelNode): RelNode = {
-    val scan: TableScan = rel.asInstanceOf[TableScan]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-
-    val tableSource = scan.getTable.unwrap(classOf[TableSourceTable]).tableSource
-      .asInstanceOf[BatchTableSource[_]]
-    new BatchTableSourceScan(
-      rel.getCluster,
-      traitSet,
-      scan.getTable,
-      tableSource
-    )
-  }
-}
-
-object BatchTableSourceScanRule {
-  val INSTANCE: RelOptRule = new BatchTableSourceScanRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateRule.scala
deleted file mode 100644
index 0311c48..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateRule.scala
+++ /dev/null
@@ -1,79 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{RelOptRuleCall, Convention, RelOptRule, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.LogicalAggregate
-import org.apache.flink.api.table.TableException
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetAggregate, DataSetConvention}
-import scala.collection.JavaConversions._
-
-class DataSetAggregateRule
-  extends ConverterRule(
-      classOf[LogicalAggregate],
-      Convention.NONE,
-      DataSetConvention.INSTANCE,
-      "DataSetAggregateRule")
-  {
-
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val agg: LogicalAggregate = call.rel(0).asInstanceOf[LogicalAggregate]
-
-    //for non grouped agg sets should attach null row to source data
-    //need apply DataSetAggregateWithNullValuesRule
-    if (agg.getGroupSet.isEmpty) {
-      return false
-    }
-
-    // check if we have distinct aggregates
-    val distinctAggs = agg.getAggCallList.exists(_.isDistinct)
-    if (distinctAggs) {
-      throw TableException("DISTINCT aggregates are currently not supported.")
-    }
-
-    // check if we have grouping sets
-    val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet
-    if (groupSets || agg.indicator) {
-      throw TableException("GROUPING SETS are currently not supported.")
-    }
-
-    !distinctAggs && !groupSets && !agg.indicator
-  }
-
-  override def convert(rel: RelNode): RelNode = {
-    val agg: LogicalAggregate = rel.asInstanceOf[LogicalAggregate]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-    val convInput: RelNode = RelOptRule.convert(agg.getInput, DataSetConvention.INSTANCE)
-
-    new DataSetAggregate(
-      rel.getCluster,
-      traitSet,
-      convInput,
-      agg.getNamedAggCalls,
-      rel.getRowType,
-      agg.getInput.getRowType,
-      agg.getGroupSet.toArray)
-    }
-  }
-
-object DataSetAggregateRule {
-  val INSTANCE: RelOptRule = new DataSetAggregateRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala
deleted file mode 100644
index 3bf3e0c..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetAggregateWithNullValuesRule.scala
+++ /dev/null
@@ -1,97 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan._
-import scala.collection.JavaConversions._
-import com.google.common.collect.ImmutableList
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.{LogicalValues, LogicalUnion, LogicalAggregate}
-import org.apache.calcite.rex.RexLiteral
-import org.apache.flink.api.table._
-import org.apache.flink.types.Row
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetAggregate, DataSetConvention}
-
-/**
-  * Rule for insert [[Row]] with null records into a [[DataSetAggregate]]
-  * Rule apply for non grouped aggregate query
-  */
-class DataSetAggregateWithNullValuesRule
-  extends ConverterRule(
-    classOf[LogicalAggregate],
-    Convention.NONE,
-    DataSetConvention.INSTANCE,
-    "DataSetAggregateWithNullValuesRule")
-{
-
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val agg: LogicalAggregate = call.rel(0).asInstanceOf[LogicalAggregate]
-
-    //for grouped agg sets shouldn't attach of null row
-    //need apply other rules. e.g. [[DataSetAggregateRule]]
-    if (!agg.getGroupSet.isEmpty) {
-      return false
-    }
-
-    // check if we have distinct aggregates
-    val distinctAggs = agg.getAggCallList.exists(_.isDistinct)
-    if (distinctAggs) {
-      throw TableException("DISTINCT aggregates are currently not supported.")
-    }
-
-    // check if we have grouping sets
-    val groupSets = agg.getGroupSets.size() == 0 || agg.getGroupSets.get(0) != agg.getGroupSet
-    if (groupSets || agg.indicator) {
-      throw TableException("GROUPING SETS are currently not supported.")
-    }
-    !distinctAggs && !groupSets && !agg.indicator
-  }
-
-  override def convert(rel: RelNode): RelNode = {
-    val agg: LogicalAggregate = rel.asInstanceOf[LogicalAggregate]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-    val cluster: RelOptCluster = rel.getCluster
-
-    val fieldTypes = agg.getInput.getRowType.getFieldList.map(_.getType)
-    val nullLiterals: ImmutableList[ImmutableList[RexLiteral]] =
-      ImmutableList.of(ImmutableList.copyOf[RexLiteral](
-        for (fieldType <- fieldTypes)
-          yield {
-            cluster.getRexBuilder.
-              makeLiteral(null, fieldType, false).asInstanceOf[RexLiteral]
-          }))
-
-    val logicalValues = LogicalValues.create(cluster, agg.getInput.getRowType, nullLiterals)
-    val logicalUnion = LogicalUnion.create(List(logicalValues, agg.getInput), true)
-
-    new DataSetAggregate(
-      cluster,
-      traitSet,
-      RelOptRule.convert(logicalUnion, DataSetConvention.INSTANCE),
-      agg.getNamedAggCalls,
-      rel.getRowType,
-      agg.getInput.getRowType,
-      agg.getGroupSet.toArray
-    )
-  }
-}
-
-object DataSetAggregateWithNullValuesRule {
-  val INSTANCE: RelOptRule = new DataSetAggregateWithNullValuesRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetCalcRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetCalcRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetCalcRule.scala
deleted file mode 100644
index 88e74a9..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetCalcRule.scala
+++ /dev/null
@@ -1,52 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.LogicalCalc
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetCalc, DataSetConvention}
-
-class DataSetCalcRule
-  extends ConverterRule(
-      classOf[LogicalCalc],
-      Convention.NONE,
-      DataSetConvention.INSTANCE,
-      "DataSetCalcRule")
-  {
-
-    def convert(rel: RelNode): RelNode = {
-      val calc: LogicalCalc = rel.asInstanceOf[LogicalCalc]
-      val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-      val convInput: RelNode = RelOptRule.convert(calc.getInput, DataSetConvention.INSTANCE)
-
-      new DataSetCalc(
-        rel.getCluster,
-        traitSet,
-        convInput,
-        rel.getRowType,
-        calc.getProgram,
-        description)
-    }
-  }
-
-object DataSetCalcRule {
-  val INSTANCE: RelOptRule = new DataSetCalcRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetCorrelateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetCorrelateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetCorrelateRule.scala
deleted file mode 100644
index 39756be..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetCorrelateRule.scala
+++ /dev/null
@@ -1,91 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.volcano.RelSubset
-import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.{LogicalFilter, LogicalCorrelate, LogicalTableFunctionScan}
-import org.apache.calcite.rex.RexNode
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetCorrelate}
-
-/**
-  * Rule to convert a LogicalCorrelate into a DataSetCorrelate.
-  */
-class DataSetCorrelateRule
-  extends ConverterRule(
-      classOf[LogicalCorrelate],
-      Convention.NONE,
-      DataSetConvention.INSTANCE,
-      "DataSetCorrelateRule") {
-
-    override def matches(call: RelOptRuleCall): Boolean = {
-      val join: LogicalCorrelate = call.rel(0).asInstanceOf[LogicalCorrelate]
-      val right = join.getRight.asInstanceOf[RelSubset].getOriginal
-
-
-      right match {
-        // right node is a table function
-        case scan: LogicalTableFunctionScan => true
-        // a filter is pushed above the table function
-        case filter: LogicalFilter =>
-          filter
-            .getInput.asInstanceOf[RelSubset]
-            .getOriginal
-            .isInstanceOf[LogicalTableFunctionScan]
-        case _ => false
-      }
-    }
-
-    override def convert(rel: RelNode): RelNode = {
-      val join: LogicalCorrelate = rel.asInstanceOf[LogicalCorrelate]
-      val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-      val convInput: RelNode = RelOptRule.convert(join.getInput(0), DataSetConvention.INSTANCE)
-      val right: RelNode = join.getInput(1)
-
-      def convertToCorrelate(relNode: RelNode, condition: Option[RexNode]): DataSetCorrelate = {
-        relNode match {
-          case rel: RelSubset =>
-            convertToCorrelate(rel.getRelList.get(0), condition)
-
-          case filter: LogicalFilter =>
-            convertToCorrelate(
-              filter.getInput.asInstanceOf[RelSubset].getOriginal,
-              Some(filter.getCondition))
-
-          case scan: LogicalTableFunctionScan =>
-            new DataSetCorrelate(
-              rel.getCluster,
-              traitSet,
-              convInput,
-              scan,
-              condition,
-              rel.getRowType,
-              join.getRowType,
-              join.getJoinType,
-              description)
-        }
-      }
-      convertToCorrelate(right, None)
-    }
-  }
-
-object DataSetCorrelateRule {
-  val INSTANCE: RelOptRule = new DataSetCorrelateRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetIntersectRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetIntersectRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetIntersectRule.scala
deleted file mode 100644
index c0e3269..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetIntersectRule.scala
+++ /dev/null
@@ -1,54 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.LogicalIntersect
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetIntersect, DataSetConvention}
-
-class DataSetIntersectRule
-  extends ConverterRule(
-    classOf[LogicalIntersect],
-    Convention.NONE,
-    DataSetConvention.INSTANCE,
-    "DataSetIntersectRule")
-{
-
-  def convert(rel: RelNode): RelNode = {
-
-    val intersect: LogicalIntersect = rel.asInstanceOf[LogicalIntersect]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-    val convLeft: RelNode = RelOptRule.convert(intersect.getInput(0), DataSetConvention.INSTANCE)
-    val convRight: RelNode = RelOptRule.convert(intersect.getInput(1), DataSetConvention.INSTANCE)
-
-    new DataSetIntersect(
-      rel.getCluster,
-      traitSet,
-      convLeft,
-      convRight,
-      rel.getRowType,
-      intersect.all)
-  }
-}
-
-object DataSetIntersectRule {
-  val INSTANCE: RelOptRule = new DataSetIntersectRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala
deleted file mode 100644
index 3fab8bf..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetJoinRule.scala
+++ /dev/null
@@ -1,73 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.LogicalJoin
-
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetJoin, DataSetConvention}
-
-import scala.collection.JavaConversions._
-
-class DataSetJoinRule
-  extends ConverterRule(
-      classOf[LogicalJoin],
-      Convention.NONE,
-      DataSetConvention.INSTANCE,
-      "DataSetJoinRule") {
-
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val join: LogicalJoin = call.rel(0).asInstanceOf[LogicalJoin]
-
-    val joinInfo = join.analyzeCondition
-
-    // joins require an equi-condition or a conjunctive predicate with at least one equi-condition
-    !joinInfo.pairs().isEmpty
-  }
-
-  override def convert(rel: RelNode): RelNode = {
-
-    val join: LogicalJoin = rel.asInstanceOf[LogicalJoin]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-    val convLeft: RelNode = RelOptRule.convert(join.getInput(0), DataSetConvention.INSTANCE)
-    val convRight: RelNode = RelOptRule.convert(join.getInput(1), DataSetConvention.INSTANCE)
-    val joinInfo = join.analyzeCondition
-
-    new DataSetJoin(
-      rel.getCluster,
-      traitSet,
-      convLeft,
-      convRight,
-      rel.getRowType,
-      join.getCondition,
-      join.getRowType,
-      joinInfo,
-      joinInfo.pairs.toList,
-      join.getJoinType,
-      null,
-      description)
-  }
-
-}
-
-object DataSetJoinRule {
-  val INSTANCE: RelOptRule = new DataSetJoinRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetMinusRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetMinusRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetMinusRule.scala
deleted file mode 100644
index 44bead0..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetMinusRule.scala
+++ /dev/null
@@ -1,56 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.logical.LogicalMinus
-import org.apache.calcite.rel.rules.UnionToDistinctRule
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetMinus}
-
-class DataSetMinusRule
-  extends ConverterRule(
-    classOf[LogicalMinus],
-    Convention.NONE,
-    DataSetConvention.INSTANCE,
-    "DataSetMinusRule")
-{
-
-  def convert(rel: RelNode): RelNode = {
-
-    val minus: LogicalMinus = rel.asInstanceOf[LogicalMinus]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-    val convLeft: RelNode = RelOptRule.convert(minus.getInput(0), DataSetConvention.INSTANCE)
-    val convRight: RelNode = RelOptRule.convert(minus.getInput(1), DataSetConvention.INSTANCE)
-
-    new DataSetMinus(
-      rel.getCluster,
-      traitSet,
-      convLeft,
-      convRight,
-      rel.getRowType,
-      minus.all)
-  }
-}
-
-object DataSetMinusRule {
-  val INSTANCE: RelOptRule = new DataSetMinusRule
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetScanRule.scala
deleted file mode 100644
index 7477690..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetScanRule.scala
+++ /dev/null
@@ -1,66 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.{RelOptRuleCall, Convention, RelOptRule, RelTraitSet}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.core.TableScan
-import org.apache.calcite.rel.logical.LogicalTableScan
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetScan}
-import org.apache.flink.api.table.plan.schema.DataSetTable
-
-class DataSetScanRule
-  extends ConverterRule(
-      classOf[LogicalTableScan],
-      Convention.NONE,
-      DataSetConvention.INSTANCE,
-      "DataSetScanRule")
-  {
-
-  /**
-   * If the input is not a DataSetTable, we want the TableScanRule to match instead
-   */
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val scan: TableScan = call.rel(0).asInstanceOf[TableScan]
-    val dataSetTable = scan.getTable.unwrap(classOf[DataSetTable[Any]])
-    dataSetTable match {
-      case _: DataSetTable[Any] =>
-        true
-      case _ =>
-        false
-    }
-  }
-
-  def convert(rel: RelNode): RelNode = {
-    val scan: TableScan = rel.asInstanceOf[TableScan]
-    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-
-    new DataSetScan(
-      rel.getCluster,
-      traitSet,
-      scan.getTable,
-      rel.getRowType
-    )
-  }
-}
-
-object DataSetScanRule {
-  val INSTANCE: RelOptRule = new DataSetScanRule
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala
deleted file mode 100644
index 8109fcf..0000000
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala
+++ /dev/null
@@ -1,82 +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.api.table.plan.rules.dataSet
-
-import org.apache.calcite.plan.volcano.RelSubset
-import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall}
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rel.convert.ConverterRule
-import org.apache.calcite.rel.core.JoinRelType
-import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalJoin}
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetConvention, DataSetSingleRowJoin}
-
-class DataSetSingleRowJoinRule
-  extends ConverterRule(
-      classOf[LogicalJoin],
-      Convention.NONE,
-      DataSetConvention.INSTANCE,
-      "DataSetSingleRowCrossRule") {
-
-  override def matches(call: RelOptRuleCall): Boolean = {
-    val join = call.rel(0).asInstanceOf[LogicalJoin]
-
-    if (isInnerJoin(join)) {
-      isGlobalAggregation(join.getRight.asInstanceOf[RelSubset].getOriginal) ||
-        isGlobalAggregation(join.getLeft.asInstanceOf[RelSubset].getOriginal)
-    } else {
-      false
-    }
-  }
-
-  private def isInnerJoin(join: LogicalJoin) = {
-    join.getJoinType == JoinRelType.INNER
-  }
-
-  private def isGlobalAggregation(node: RelNode) = {
-    node.isInstanceOf[LogicalAggregate] &&
-      isSingleRow(node.asInstanceOf[LogicalAggregate])
-  }
-
-  private def isSingleRow(agg: LogicalAggregate) = {
-    agg.getGroupSet.isEmpty
-  }
-
-  override def convert(rel: RelNode): RelNode = {
-    val join = rel.asInstanceOf[LogicalJoin]
-    val traitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
-    val dataSetLeftNode = RelOptRule.convert(join.getLeft, DataSetConvention.INSTANCE)
-    val dataSetRightNode = RelOptRule.convert(join.getRight, DataSetConvention.INSTANCE)
-    val leftIsSingle = isGlobalAggregation(join.getLeft.asInstanceOf[RelSubset].getOriginal)
-
-    new DataSetSingleRowJoin(
-      rel.getCluster,
-      traitSet,
-      dataSetLeftNode,
-      dataSetRightNode,
-      leftIsSingle,
-      rel.getRowType,
-      join.getCondition,
-      join.getRowType,
-      description)
-  }
-}
-
-object DataSetSingleRowJoinRule {
-  val INSTANCE: RelOptRule = new DataSetSingleRowJoinRule
-}


[08/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/SqlExpressionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/SqlExpressionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/SqlExpressionTest.scala
deleted file mode 100644
index 56f40ea..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/SqlExpressionTest.scala
+++ /dev/null
@@ -1,170 +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.api.table.expressions
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.types.Row
-import org.apache.flink.api.table.expressions.utils.ExpressionTestBase
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.junit.{Ignore, Test}
-
-/**
-  * Tests all SQL expressions that are currently supported according to the documentation.
-  * This tests should be kept in sync with the documentation to reduce confusion due to the
-  * large amount of SQL functions.
-  *
-  * The tests do not test every parameter combination of a function.
-  * They are rather a function existence test and simple functional test.
-  *
-  * The tests are split up and ordered like the sections in the documentation.
-  */
-class SqlExpressionTest extends ExpressionTestBase {
-
-  @Test
-  def testComparisonFunctions(): Unit = {
-    testSqlApi("1 = 1", "true")
-    testSqlApi("1 <> 1", "false")
-    testSqlApi("5 > 2", "true")
-    testSqlApi("2 >= 2", "true")
-    testSqlApi("5 < 2", "false")
-    testSqlApi("2 <= 2", "true")
-    testSqlApi("1 IS NULL", "false")
-    testSqlApi("1 IS NOT NULL", "true")
-    testSqlApi("NULLIF(1,1) IS DISTINCT FROM NULLIF(1,1)", "false")
-    testSqlApi("NULLIF(1,1) IS NOT DISTINCT FROM NULLIF(1,1)", "true")
-    testSqlApi("NULLIF(1,1) IS NOT DISTINCT FROM NULLIF(1,1)", "true")
-    testSqlApi("12 BETWEEN 11 AND 13", "true")
-    testSqlApi("12 BETWEEN ASYMMETRIC 13 AND 11", "false")
-    testSqlApi("12 BETWEEN SYMMETRIC 13 AND 11", "true")
-    testSqlApi("12 NOT BETWEEN 11 AND 13", "false")
-    testSqlApi("12 NOT BETWEEN ASYMMETRIC 13 AND 11", "true")
-    testSqlApi("12 NOT BETWEEN SYMMETRIC 13 AND 11", "false")
-    testSqlApi("'TEST' LIKE '%EST'", "true")
-    //testSqlApi("'%EST' LIKE '.%EST' ESCAPE '.'", "true") // TODO
-    testSqlApi("'TEST' NOT LIKE '%EST'", "false")
-    //testSqlApi("'%EST' NOT LIKE '.%EST' ESCAPE '.'", "false") // TODO
-    testSqlApi("'TEST' SIMILAR TO '.EST'", "true")
-    //testSqlApi("'TEST' SIMILAR TO ':.EST' ESCAPE ':'", "true") // TODO
-    testSqlApi("'TEST' NOT SIMILAR TO '.EST'", "false")
-    //testSqlApi("'TEST' NOT SIMILAR TO ':.EST' ESCAPE ':'", "false") // TODO
-    testSqlApi("'TEST' IN ('west', 'TEST', 'rest')", "true")
-    testSqlApi("'TEST' IN ('west', 'rest')", "false")
-    testSqlApi("'TEST' NOT IN ('west', 'TEST', 'rest')", "false")
-    testSqlApi("'TEST' NOT IN ('west', 'rest')", "true")
-
-    // sub-query functions are not listed here
-  }
-
-  @Test
-  def testLogicalFunctions(): Unit = {
-    testSqlApi("TRUE OR FALSE", "true")
-    testSqlApi("TRUE AND FALSE", "false")
-    testSqlApi("NOT TRUE", "false")
-    testSqlApi("TRUE IS FALSE", "false")
-    testSqlApi("TRUE IS NOT FALSE", "true")
-    testSqlApi("TRUE IS TRUE", "true")
-    testSqlApi("TRUE IS NOT TRUE", "false")
-    testSqlApi("NULLIF(TRUE,TRUE) IS UNKNOWN", "true")
-    testSqlApi("NULLIF(TRUE,TRUE) IS NOT UNKNOWN", "false")
-  }
-
-  @Test
-  def testArithmeticFunctions(): Unit = {
-    testSqlApi("+5", "5")
-    testSqlApi("-5", "-5")
-    testSqlApi("5+5", "10")
-    testSqlApi("5-5", "0")
-    testSqlApi("5*5", "25")
-    testSqlApi("5/5", "1")
-    testSqlApi("POWER(5, 5)", "3125.0")
-    testSqlApi("ABS(-5)", "5")
-    testSqlApi("MOD(-26, 5)", "-1")
-    testSqlApi("SQRT(4)", "2.0")
-    testSqlApi("LN(1)", "0.0")
-    testSqlApi("LOG10(1)", "0.0")
-    testSqlApi("EXP(0)", "1.0")
-    testSqlApi("CEIL(2.5)", "3")
-    testSqlApi("FLOOR(2.5)", "2")
-  }
-
-  @Test
-  def testStringFunctions(): Unit = {
-    testSqlApi("'test' || 'string'", "teststring")
-    testSqlApi("CHAR_LENGTH('string')", "6")
-    testSqlApi("CHARACTER_LENGTH('string')", "6")
-    testSqlApi("UPPER('string')", "STRING")
-    testSqlApi("LOWER('STRING')", "string")
-    testSqlApi("POSITION('STR' IN 'STRING')", "1")
-    testSqlApi("TRIM(BOTH ' STRING ')", "STRING")
-    testSqlApi("TRIM(LEADING 'x' FROM 'xxxxSTRINGxxxx')", "STRINGxxxx")
-    testSqlApi("TRIM(TRAILING 'x' FROM 'xxxxSTRINGxxxx')", "xxxxSTRING")
-    testSqlApi(
-      "OVERLAY('This is a old string' PLACING 'new' FROM 11 FOR 3)",
-      "This is a new string")
-    testSqlApi("SUBSTRING('hello world', 2)", "ello world")
-    testSqlApi("SUBSTRING('hello world', 2, 3)", "ell")
-    testSqlApi("INITCAP('hello world')", "Hello World")
-  }
-
-  @Test
-  def testConditionalFunctions(): Unit = {
-    testSqlApi("CASE 2 WHEN 1, 2 THEN 2 ELSE 3 END", "2")
-    testSqlApi("CASE WHEN 1 = 2 THEN 2 WHEN 1 = 1 THEN 3 ELSE 3 END", "3")
-    testSqlApi("NULLIF(1, 1)", "null")
-    testSqlApi("COALESCE(NULL, 5)", "5")
-  }
-
-  @Test
-  def testTypeConversionFunctions(): Unit = {
-    testSqlApi("CAST(2 AS DOUBLE)", "2.0")
-  }
-
-  @Test
-  def testValueConstructorFunctions(): Unit = {
-    // TODO we need a special code path that flattens ROW types
-    // testSqlApi("ROW('hello world', 12)", "hello world") // test base only returns field 0
-    // testSqlApi("('hello world', 12)", "hello world") // test base only returns field 0
-    testSqlApi("ARRAY[TRUE, FALSE][2]", "false")
-    testSqlApi("ARRAY[TRUE, TRUE]", "[true, true]")
-  }
-
-  @Test
-  def testDateTimeFunctions(): Unit = {
-    testSqlApi("DATE '1990-10-14'", "1990-10-14")
-    testSqlApi("TIME '12:12:12'", "12:12:12")
-    testSqlApi("TIMESTAMP '1990-10-14 12:12:12.123'", "1990-10-14 12:12:12.123")
-    testSqlApi("INTERVAL '10 00:00:00.004' DAY TO SECOND", "+10 00:00:00.004")
-    testSqlApi("INTERVAL '10 00:12' DAY TO MINUTE", "+10 00:12:00.000")
-    testSqlApi("INTERVAL '2-10' YEAR TO MONTH", "+2-10")
-    testSqlApi("EXTRACT(DAY FROM DATE '1990-12-01')", "1")
-    testSqlApi("EXTRACT(DAY FROM INTERVAL '19 12:10:10.123' DAY TO SECOND(3))", "19")
-    testSqlApi("QUARTER(DATE '2016-04-12')", "2")
-  }
-
-  @Test
-  def testArrayFunctions(): Unit = {
-    testSqlApi("CARDINALITY(ARRAY[TRUE, TRUE, FALSE])", "3")
-    testSqlApi("ELEMENT(ARRAY['HELLO WORLD'])", "HELLO WORLD")
-  }
-
-  override def testData: Any = new Row(0)
-
-  override def typeInfo: TypeInformation[Any] =
-    new RowTypeInfo().asInstanceOf[TypeInformation[Any]]
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/TemporalTypesTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/TemporalTypesTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/TemporalTypesTest.scala
deleted file mode 100644
index bd771ba..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/TemporalTypesTest.scala
+++ /dev/null
@@ -1,573 +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.api.table.expressions
-
-import java.sql.{Date, Time, Timestamp}
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.expressions.utils.ExpressionTestBase
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.Types
-import org.apache.flink.types.Row
-import org.junit.Test
-
-class TemporalTypesTest extends ExpressionTestBase {
-
-  @Test
-  def testTimePointLiterals(): Unit = {
-    testAllApis(
-      "1990-10-14".toDate,
-      "'1990-10-14'.toDate",
-      "DATE '1990-10-14'",
-      "1990-10-14")
-
-    testTableApi(
-      Date.valueOf("2040-09-11"),
-      "'2040-09-11'.toDate",
-      "2040-09-11")
-
-    testAllApis(
-      "1500-04-30".cast(Types.DATE),
-      "'1500-04-30'.cast(DATE)",
-      "CAST('1500-04-30' AS DATE)",
-      "1500-04-30")
-
-    testAllApis(
-      "15:45:59".toTime,
-      "'15:45:59'.toTime",
-      "TIME '15:45:59'",
-      "15:45:59")
-
-    testTableApi(
-      Time.valueOf("00:00:00"),
-      "'00:00:00'.toTime",
-      "00:00:00")
-
-    testAllApis(
-      "1:30:00".cast(Types.TIME),
-      "'1:30:00'.cast(TIME)",
-      "CAST('1:30:00' AS TIME)",
-      "01:30:00")
-
-    testAllApis(
-      "1990-10-14 23:00:00.123".toTimestamp,
-      "'1990-10-14 23:00:00.123'.toTimestamp",
-      "TIMESTAMP '1990-10-14 23:00:00.123'",
-      "1990-10-14 23:00:00.123")
-
-    testTableApi(
-      Timestamp.valueOf("2040-09-11 00:00:00.000"),
-      "'2040-09-11 00:00:00.000'.toTimestamp",
-      "2040-09-11 00:00:00.0")
-
-    testAllApis(
-      "1500-04-30 12:00:00".cast(Types.TIMESTAMP),
-      "'1500-04-30 12:00:00'.cast(TIMESTAMP)",
-      "CAST('1500-04-30 12:00:00' AS TIMESTAMP)",
-      "1500-04-30 12:00:00.0")
-  }
-
-  @Test
-  def testTimeIntervalLiterals(): Unit = {
-    testAllApis(
-      1.year,
-      "1.year",
-      "INTERVAL '1' YEAR",
-      "+1-00")
-
-    testAllApis(
-      1.month,
-      "1.month",
-      "INTERVAL '1' MONTH",
-      "+0-01")
-
-    testAllApis(
-      12.days,
-      "12.days",
-      "INTERVAL '12' DAY",
-      "+12 00:00:00.000")
-
-    testAllApis(
-      1.hour,
-      "1.hour",
-      "INTERVAL '1' HOUR",
-      "+0 01:00:00.000")
-
-    testAllApis(
-      3.minutes,
-      "3.minutes",
-      "INTERVAL '3' MINUTE",
-      "+0 00:03:00.000")
-
-    testAllApis(
-      3.seconds,
-      "3.seconds",
-      "INTERVAL '3' SECOND",
-      "+0 00:00:03.000")
-
-    testAllApis(
-      3.millis,
-      "3.millis",
-      "INTERVAL '0.003' SECOND",
-      "+0 00:00:00.003")
-  }
-
-  @Test
-  def testTimePointInput(): Unit = {
-    testAllApis(
-      'f0,
-      "f0",
-      "f0",
-      "1990-10-14")
-
-    testAllApis(
-      'f1,
-      "f1",
-      "f1",
-      "10:20:45")
-
-    testAllApis(
-      'f2,
-      "f2",
-      "f2",
-      "1990-10-14 10:20:45.123")
-  }
-
-  @Test
-  def testTimeIntervalInput(): Unit = {
-    testAllApis(
-      'f9,
-      "f9",
-      "f9",
-      "+2-00")
-
-    testAllApis(
-      'f10,
-      "f10",
-      "f10",
-      "+0 00:00:12.000")
-  }
-
-  @Test
-  def testTimePointCasting(): Unit = {
-    testAllApis(
-      'f0.cast(Types.TIMESTAMP),
-      "f0.cast(TIMESTAMP)",
-      "CAST(f0 AS TIMESTAMP)",
-      "1990-10-14 00:00:00.0")
-
-    testAllApis(
-      'f1.cast(Types.TIMESTAMP),
-      "f1.cast(TIMESTAMP)",
-      "CAST(f1 AS TIMESTAMP)",
-      "1970-01-01 10:20:45.0")
-
-    testAllApis(
-      'f2.cast(Types.DATE),
-      "f2.cast(DATE)",
-      "CAST(f2 AS DATE)",
-      "1990-10-14")
-
-    testAllApis(
-      'f2.cast(Types.TIME),
-      "f2.cast(TIME)",
-      "CAST(f2 AS TIME)",
-      "10:20:45")
-
-    testAllApis(
-      'f2.cast(Types.TIME),
-      "f2.cast(TIME)",
-      "CAST(f2 AS TIME)",
-      "10:20:45")
-
-    testTableApi(
-      'f7.cast(Types.DATE),
-      "f7.cast(DATE)",
-      "2002-11-09")
-
-    testTableApi(
-      'f7.cast(Types.DATE).cast(Types.INT),
-      "f7.cast(DATE).cast(INT)",
-      "12000")
-
-    testTableApi(
-      'f7.cast(Types.TIME),
-      "f7.cast(TIME)",
-      "00:00:12")
-
-    testTableApi(
-      'f7.cast(Types.TIME).cast(Types.INT),
-      "f7.cast(TIME).cast(INT)",
-      "12000")
-
-    testTableApi(
-      'f8.cast(Types.TIMESTAMP),
-      "f8.cast(TIMESTAMP)",
-      "2016-06-27 07:23:33.0")
-
-    testTableApi(
-      'f8.cast(Types.TIMESTAMP).cast(Types.LONG),
-      "f8.cast(TIMESTAMP).cast(LONG)",
-      "1467012213000")
-  }
-
-  @Test
-  def testTimeIntervalCasting(): Unit = {
-    testTableApi(
-      'f7.cast(Types.INTERVAL_MONTHS),
-      "f7.cast(INTERVAL_MONTHS)",
-      "+1000-00")
-
-    testTableApi(
-      'f8.cast(Types.INTERVAL_MILLIS),
-      "f8.cast(INTERVAL_MILLIS)",
-      "+16979 07:23:33.000")
-  }
-
-  @Test
-  def testTimePointComparison(): Unit = {
-    testAllApis(
-      'f0 < 'f3,
-      "f0 < f3",
-      "f0 < f3",
-      "false")
-
-    testAllApis(
-      'f0 < 'f4,
-      "f0 < f4",
-      "f0 < f4",
-      "true")
-
-    testAllApis(
-      'f1 < 'f5,
-      "f1 < f5",
-      "f1 < f5",
-      "false")
-
-    testAllApis(
-      'f0.cast(Types.TIMESTAMP) !== 'f2,
-      "f0.cast(TIMESTAMP) !== f2",
-      "CAST(f0 AS TIMESTAMP) <> f2",
-      "true")
-
-    testAllApis(
-      'f0.cast(Types.TIMESTAMP) === 'f6,
-      "f0.cast(TIMESTAMP) === f6",
-      "CAST(f0 AS TIMESTAMP) = f6",
-      "true")
-  }
-
-  @Test
-  def testTimeIntervalArithmetic(): Unit = {
-
-    // interval months comparison
-
-    testAllApis(
-      12.months < 24.months,
-      "12.months < 24.months",
-      "INTERVAL '12' MONTH < INTERVAL '24' MONTH",
-      "true")
-
-    testAllApis(
-      8.years === 8.years,
-      "8.years === 8.years",
-      "INTERVAL '8' YEAR = INTERVAL '8' YEAR",
-      "true")
-
-    // interval millis comparison
-
-    testAllApis(
-      8.millis > 10.millis,
-      "8.millis > 10.millis",
-      "INTERVAL '0.008' SECOND > INTERVAL '0.010' SECOND",
-      "false")
-
-    testAllApis(
-      8.millis === 8.millis,
-      "8.millis === 8.millis",
-      "INTERVAL '0.008' SECOND = INTERVAL '0.008' SECOND",
-      "true")
-
-    // interval months addition/subtraction
-
-    testAllApis(
-      8.years + 10.months,
-      "8.years + 10.months",
-      "INTERVAL '8' YEAR + INTERVAL '10' MONTH",
-      "+8-10")
-
-    testAllApis(
-      2.years - 12.months,
-      "2.years - 12.months",
-      "INTERVAL '2' YEAR - INTERVAL '12' MONTH",
-      "+1-00")
-
-    testAllApis(
-      -2.years,
-      "-2.years",
-      "-INTERVAL '2' YEAR",
-      "-2-00")
-
-    // interval millis addition/subtraction
-
-    testAllApis(
-      8.hours + 10.minutes + 12.seconds + 5.millis,
-      "8.hours + 10.minutes + 12.seconds + 5.millis",
-      "INTERVAL '8' HOUR + INTERVAL '10' MINUTE + INTERVAL '12.005' SECOND",
-      "+0 08:10:12.005")
-
-    testAllApis(
-      1.minute - 10.seconds,
-      "1.minute - 10.seconds",
-      "INTERVAL '1' MINUTE - INTERVAL '10' SECOND",
-      "+0 00:00:50.000")
-
-    testAllApis(
-      -10.seconds,
-      "-10.seconds",
-      "-INTERVAL '10' SECOND",
-      "-0 00:00:10.000")
-
-    // addition to date
-
-    // interval millis
-    testAllApis(
-      'f0 + 2.days,
-      "f0 + 2.days",
-      "f0 + INTERVAL '2' DAY",
-      "1990-10-16")
-
-    // interval millis
-    testAllApis(
-      30.days + 'f0,
-      "30.days + f0",
-      "INTERVAL '30' DAY + f0",
-      "1990-11-13")
-
-    // interval months
-    testAllApis(
-      'f0 + 2.months,
-      "f0 + 2.months",
-      "f0 + INTERVAL '2' MONTH",
-      "1990-12-14")
-
-    // interval months
-    testAllApis(
-      2.months + 'f0,
-      "2.months + f0",
-      "INTERVAL '2' MONTH + f0",
-      "1990-12-14")
-
-    // addition to time
-
-    // interval millis
-    testAllApis(
-      'f1 + 12.hours,
-      "f1 + 12.hours",
-      "f1 + INTERVAL '12' HOUR",
-      "22:20:45")
-
-    // interval millis
-    testAllApis(
-      12.hours + 'f1,
-      "12.hours + f1",
-      "INTERVAL '12' HOUR + f1",
-      "22:20:45")
-
-    // addition to timestamp
-
-    // interval millis
-    testAllApis(
-      'f2 + 10.days + 4.millis,
-      "f2 + 10.days + 4.millis",
-      "f2 + INTERVAL '10 00:00:00.004' DAY TO SECOND",
-      "1990-10-24 10:20:45.127")
-
-    // interval millis
-    testAllApis(
-      10.days + 'f2 + 4.millis,
-      "10.days + f2 + 4.millis",
-      "INTERVAL '10 00:00:00.004' DAY TO SECOND + f2",
-      "1990-10-24 10:20:45.127")
-
-    // interval months
-    testAllApis(
-      'f2 + 10.years,
-      "f2 + 10.years",
-      "f2 + INTERVAL '10' YEAR",
-      "2000-10-14 10:20:45.123")
-
-    // interval months
-    testAllApis(
-      10.years + 'f2,
-      "10.years + f2",
-      "INTERVAL '10' YEAR + f2",
-      "2000-10-14 10:20:45.123")
-
-    // subtraction from date
-
-    // interval millis
-    testAllApis(
-      'f0 - 2.days,
-      "f0 - 2.days",
-      "f0 - INTERVAL '2' DAY",
-      "1990-10-12")
-
-    // interval millis
-    testAllApis(
-      -30.days + 'f0,
-      "-30.days + f0",
-      "INTERVAL '-30' DAY + f0",
-      "1990-09-14")
-
-    // interval months
-    testAllApis(
-      'f0 - 2.months,
-      "f0 - 2.months",
-      "f0 - INTERVAL '2' MONTH",
-      "1990-08-14")
-
-    // interval months
-    testAllApis(
-      -2.months + 'f0,
-      "-2.months + f0",
-      "-INTERVAL '2' MONTH + f0",
-      "1990-08-14")
-
-    // subtraction from time
-
-    // interval millis
-    testAllApis(
-      'f1 - 12.hours,
-      "f1 - 12.hours",
-      "f1 - INTERVAL '12' HOUR",
-      "22:20:45")
-
-    // interval millis
-    testAllApis(
-      -12.hours + 'f1,
-      "-12.hours + f1",
-      "INTERVAL '-12' HOUR + f1",
-      "22:20:45")
-
-    // subtraction from timestamp
-
-    // interval millis
-    testAllApis(
-      'f2 - 10.days - 4.millis,
-      "f2 - 10.days - 4.millis",
-      "f2 - INTERVAL '10 00:00:00.004' DAY TO SECOND",
-      "1990-10-04 10:20:45.119")
-
-    // interval millis
-    testAllApis(
-      -10.days + 'f2 - 4.millis,
-      "-10.days + f2 - 4.millis",
-      "INTERVAL '-10 00:00:00.004' DAY TO SECOND + f2",
-      "1990-10-04 10:20:45.119")
-
-    // interval months
-    testAllApis(
-      'f2 - 10.years,
-      "f2 - 10.years",
-      "f2 - INTERVAL '10' YEAR",
-      "1980-10-14 10:20:45.123")
-
-    // interval months
-    testAllApis(
-      -10.years + 'f2,
-      "-10.years + f2",
-      "INTERVAL '-10' YEAR + f2",
-      "1980-10-14 10:20:45.123")
-
-    // casting
-
-    testAllApis(
-      -'f9.cast(Types.INTERVAL_MONTHS),
-      "-f9.cast(INTERVAL_MONTHS)",
-      "-CAST(f9 AS INTERVAL YEAR)",
-      "-2-00")
-
-    testAllApis(
-      -'f10.cast(Types.INTERVAL_MILLIS),
-      "-f10.cast(INTERVAL_MILLIS)",
-      "-CAST(f10 AS INTERVAL SECOND)",
-      "-0 00:00:12.000")
-
-    // addition/subtraction of interval millis and interval months
-
-    testAllApis(
-      'f0 + 2.days + 1.month,
-      "f0 + 2.days + 1.month",
-      "f0 + INTERVAL '2' DAY + INTERVAL '1' MONTH",
-      "1990-11-16")
-
-    testAllApis(
-      'f0 - 2.days - 1.month,
-      "f0 - 2.days - 1.month",
-      "f0 - INTERVAL '2' DAY - INTERVAL '1' MONTH",
-      "1990-09-12")
-
-    testAllApis(
-      'f2 + 2.days + 1.month,
-      "f2 + 2.days + 1.month",
-      "f2 + INTERVAL '2' DAY + INTERVAL '1' MONTH",
-      "1990-11-16 10:20:45.123")
-
-    testAllApis(
-      'f2 - 2.days - 1.month,
-      "f2 - 2.days - 1.month",
-      "f2 - INTERVAL '2' DAY - INTERVAL '1' MONTH",
-      "1990-09-12 10:20:45.123")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  def testData = {
-    val testData = new Row(11)
-    testData.setField(0, Date.valueOf("1990-10-14"))
-    testData.setField(1, Time.valueOf("10:20:45"))
-    testData.setField(2, Timestamp.valueOf("1990-10-14 10:20:45.123"))
-    testData.setField(3, Date.valueOf("1990-10-13"))
-    testData.setField(4, Date.valueOf("1990-10-15"))
-    testData.setField(5, Time.valueOf("00:00:00"))
-    testData.setField(6, Timestamp.valueOf("1990-10-14 00:00:00.0"))
-    testData.setField(7, 12000)
-    testData.setField(8, 1467012213000L)
-    testData.setField(9, 24)
-    testData.setField(10, 12000L)
-    testData
-  }
-
-  def typeInfo = {
-    new RowTypeInfo(
-      Types.DATE,
-      Types.TIME,
-      Types.TIMESTAMP,
-      Types.DATE,
-      Types.DATE,
-      Types.TIME,
-      Types.TIMESTAMP,
-      Types.INT,
-      Types.LONG,
-      Types.INTERVAL_MONTHS,
-      Types.INTERVAL_MILLIS).asInstanceOf[TypeInformation[Any]]
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/UserDefinedScalarFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/UserDefinedScalarFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/UserDefinedScalarFunctionTest.scala
deleted file mode 100644
index 567cca1..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/UserDefinedScalarFunctionTest.scala
+++ /dev/null
@@ -1,228 +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.api.table.expressions
-
-import java.sql.{Date, Time, Timestamp}
-
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.expressions.utils._
-import org.apache.flink.api.table.functions.ScalarFunction
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.api.table.Types
-import org.apache.flink.types.Row
-import org.junit.Test
-
-class UserDefinedScalarFunctionTest extends ExpressionTestBase {
-
-  @Test
-  def testParameters(): Unit = {
-    testAllApis(
-      Func0('f0),
-      "Func0(f0)",
-      "Func0(f0)",
-      "42")
-
-    testAllApis(
-      Func1('f0),
-      "Func1(f0)",
-      "Func1(f0)",
-      "43")
-
-    testAllApis(
-      Func2('f0, 'f1, 'f3),
-      "Func2(f0, f1, f3)",
-      "Func2(f0, f1, f3)",
-      "42 and Test and SimplePojo(Bob,36)")
-
-    testAllApis(
-      Func0(123),
-      "Func0(123)",
-      "Func0(123)",
-      "123")
-
-    testAllApis(
-      Func6('f4, 'f5, 'f6),
-      "Func6(f4, f5, f6)",
-      "Func6(f4, f5, f6)",
-      "(1990-10-14,12:10:10,1990-10-14 12:10:10.0)")
-  }
-
-  @Test
-  def testNullableParameters(): Unit = {
-    testAllApis(
-      Func3(Null(INT_TYPE_INFO), Null(STRING_TYPE_INFO)),
-      "Func3(Null(INT), Null(STRING))",
-      "Func3(NULL, NULL)",
-      "null and null")
-
-    testAllApis(
-      Func3(Null(INT_TYPE_INFO), "Test"),
-      "Func3(Null(INT), 'Test')",
-      "Func3(NULL, 'Test')",
-      "null and Test")
-
-    testAllApis(
-      Func3(42, Null(STRING_TYPE_INFO)),
-      "Func3(42, Null(STRING))",
-      "Func3(42, NULL)",
-      "42 and null")
-
-    testAllApis(
-      Func0(Null(INT_TYPE_INFO)),
-      "Func0(Null(INT))",
-      "Func0(NULL)",
-      "-1")
-  }
-
-  @Test
-  def testResults(): Unit = {
-    testAllApis(
-      Func4(),
-      "Func4()",
-      "Func4()",
-      "null")
-
-    testAllApis(
-      Func5(),
-      "Func5()",
-      "Func5()",
-      "-1")
-  }
-
-  @Test
-  def testNesting(): Unit = {
-    testAllApis(
-      Func0(Func0('f0)),
-      "Func0(Func0(f0))",
-      "Func0(Func0(f0))",
-      "42")
-
-    testAllApis(
-      Func0(Func0('f0)),
-      "Func0(Func0(f0))",
-      "Func0(Func0(f0))",
-      "42")
-
-    testAllApis(
-      Func7(Func7(Func7(1, 1), Func7(1, 1)), Func7(Func7(1, 1), Func7(1, 1))),
-      "Func7(Func7(Func7(1, 1), Func7(1, 1)), Func7(Func7(1, 1), Func7(1, 1)))",
-      "Func7(Func7(Func7(1, 1), Func7(1, 1)), Func7(Func7(1, 1), Func7(1, 1)))",
-      "8")
-  }
-
-  @Test
-  def testOverloadedParameters(): Unit = {
-    testAllApis(
-      Func8(1),
-      "Func8(1)",
-      "Func8(1)",
-      "a")
-
-    testAllApis(
-      Func8(1, 1),
-      "Func8(1, 1)",
-      "Func8(1, 1)",
-      "b")
-
-    testAllApis(
-      Func8("a", "a"),
-      "Func8('a', 'a')",
-      "Func8('a', 'a')",
-      "c")
-  }
-
-  @Test
-  def testTimePointsOnPrimitives(): Unit = {
-    testAllApis(
-      Func9('f4, 'f5, 'f6),
-      "Func9(f4, f5, f6)",
-      "Func9(f4, f5, f6)",
-      "7591 and 43810000 and 655906210000")
-
-    testAllApis(
-      Func10('f6),
-      "Func10(f6)",
-      "Func10(f6)",
-      "1990-10-14 12:10:10.0")
-  }
-
-  @Test
-  def testTimeIntervalsOnPrimitives(): Unit = {
-    testAllApis(
-      Func11('f7, 'f8),
-      "Func11(f7, f8)",
-      "Func11(f7, f8)",
-      "12 and 1000")
-
-    testAllApis(
-      Func12('f8),
-      "Func12(f8)",
-      "Func12(f8)",
-      "+0 00:00:01.000")
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  override def testData: Any = {
-    val testData = new Row(9)
-    testData.setField(0, 42)
-    testData.setField(1, "Test")
-    testData.setField(2, null)
-    testData.setField(3, SimplePojo("Bob", 36))
-    testData.setField(4, Date.valueOf("1990-10-14"))
-    testData.setField(5, Time.valueOf("12:10:10"))
-    testData.setField(6, Timestamp.valueOf("1990-10-14 12:10:10"))
-    testData.setField(7, 12)
-    testData.setField(8, 1000L)
-    testData
-  }
-
-  override def typeInfo: TypeInformation[Any] = {
-    new RowTypeInfo(
-      Types.INT,
-      Types.STRING,
-      Types.BOOLEAN,
-      TypeInformation.of(classOf[SimplePojo]),
-      Types.DATE,
-      Types.TIME,
-      Types.TIMESTAMP,
-      Types.INTERVAL_MONTHS,
-      Types.INTERVAL_MILLIS
-    ).asInstanceOf[TypeInformation[Any]]
-  }
-
-  override def functions: Map[String, ScalarFunction] = Map(
-    "Func0" -> Func0,
-    "Func1" -> Func1,
-    "Func2" -> Func2,
-    "Func3" -> Func3,
-    "Func4" -> Func4,
-    "Func5" -> Func5,
-    "Func6" -> Func6,
-    "Func7" -> Func7,
-    "Func8" -> Func8,
-    "Func9" -> Func9,
-    "Func10" -> Func10,
-    "Func11" -> Func11,
-    "Func12" -> Func12
-  )
-}
-

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionTestBase.scala
deleted file mode 100644
index 3156ba8..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/ExpressionTestBase.scala
+++ /dev/null
@@ -1,217 +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.api.table.expressions.utils
-
-import org.apache.calcite.rex.RexNode
-import org.apache.calcite.sql.`type`.SqlTypeName._
-import org.apache.calcite.sql2rel.RelDecorrelator
-import org.apache.calcite.tools.{Programs, RelBuilder}
-import org.apache.flink.api.common.functions.{Function, MapFunction}
-import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.java.{DataSet => JDataSet}
-import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment}
-import org.apache.flink.api.table._
-import org.apache.flink.api.table.codegen.{CodeGenerator, Compiler, GeneratedFunction}
-import org.apache.flink.api.table.expressions.{Expression, ExpressionParser}
-import org.apache.flink.api.table.functions.ScalarFunction
-import org.apache.flink.api.table.plan.nodes.dataset.{DataSetCalc, DataSetConvention}
-import org.apache.flink.api.table.plan.rules.FlinkRuleSets
-import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.types.Row
-import org.junit.Assert._
-import org.junit.{After, Before}
-import org.mockito.Mockito._
-
-import scala.collection.mutable
-
-/**
-  * Base test class for expression tests.
-  */
-abstract class ExpressionTestBase {
-
-  private val testExprs = mutable.ArrayBuffer[(RexNode, String)]()
-
-  // setup test utils
-  private val tableName = "testTable"
-  private val context = prepareContext(typeInfo)
-  private val planner = new FlinkPlannerImpl(
-    context._2.getFrameworkConfig,
-    context._2.getPlanner,
-    context._2.getTypeFactory)
-  private val optProgram = Programs.ofRules(FlinkRuleSets.DATASET_OPT_RULES)
-
-  private def prepareContext(typeInfo: TypeInformation[Any]): (RelBuilder, TableEnvironment) = {
-    // create DataSetTable
-    val dataSetMock = mock(classOf[DataSet[Any]])
-    val jDataSetMock = mock(classOf[JDataSet[Any]])
-    when(dataSetMock.javaSet).thenReturn(jDataSetMock)
-    when(jDataSetMock.getType).thenReturn(typeInfo)
-
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tEnv = TableEnvironment.getTableEnvironment(env)
-    tEnv.registerDataSet(tableName, dataSetMock)
-    functions.foreach(f => tEnv.registerFunction(f._1, f._2))
-
-    // prepare RelBuilder
-    val relBuilder = tEnv.getRelBuilder
-    relBuilder.scan(tableName)
-
-    (relBuilder, tEnv)
-  }
-
-  def testData: Any
-
-  def typeInfo: TypeInformation[Any]
-
-  def functions: Map[String, ScalarFunction] = Map()
-
-  @Before
-  def resetTestExprs() = {
-    testExprs.clear()
-  }
-
-  @After
-  def evaluateExprs() = {
-    val relBuilder = context._1
-    val config = new TableConfig()
-    val generator = new CodeGenerator(config, false, typeInfo)
-
-    // cast expressions to String
-    val stringTestExprs = testExprs.map(expr => relBuilder.cast(expr._1, VARCHAR)).toSeq
-
-    // generate code
-    val resultType = new RowTypeInfo(Seq.fill(testExprs.size)(STRING_TYPE_INFO): _*)
-    val genExpr = generator.generateResultExpression(
-      resultType,
-      resultType.getFieldNames,
-      stringTestExprs)
-
-    val bodyCode =
-      s"""
-        |${genExpr.code}
-        |return ${genExpr.resultTerm};
-        |""".stripMargin
-
-    val genFunc = generator.generateFunction[MapFunction[Any, String]](
-      "TestFunction",
-      classOf[MapFunction[Any, String]],
-      bodyCode,
-      resultType.asInstanceOf[TypeInformation[Any]])
-
-    // compile and evaluate
-    val clazz = new TestCompiler[MapFunction[Any, String]]().compile(genFunc)
-    val mapper = clazz.newInstance()
-    val result = mapper.map(testData).asInstanceOf[Row]
-
-    // compare
-    testExprs
-      .zipWithIndex
-      .foreach {
-        case ((expr, expected), index) =>
-          val actual = result.getField(index)
-          assertEquals(
-            s"Wrong result for: $expr",
-            expected,
-            if (actual == null) "null" else actual)
-      }
-  }
-
-  private def addSqlTestExpr(sqlExpr: String, expected: String): Unit = {
-    // create RelNode from SQL expression
-    val parsed = planner.parse(s"SELECT $sqlExpr FROM $tableName")
-    val validated = planner.validate(parsed)
-    val converted = planner.rel(validated).rel
-
-    // create DataSetCalc
-    val decorPlan = RelDecorrelator.decorrelateQuery(converted)
-    val flinkOutputProps = converted.getTraitSet.replace(DataSetConvention.INSTANCE).simplify()
-    val dataSetCalc = optProgram.run(context._2.getPlanner, decorPlan, flinkOutputProps)
-
-    // extract RexNode
-    val calcProgram = dataSetCalc
-     .asInstanceOf[DataSetCalc]
-     .calcProgram
-    val expanded = calcProgram.expandLocalRef(calcProgram.getProjectList.get(0))
-
-    testExprs += ((expanded, expected))
-  }
-
-  private def addTableApiTestExpr(tableApiExpr: Expression, expected: String): Unit = {
-    // create RelNode from Table API expression
-    val env = context._2
-    val converted = env
-      .asInstanceOf[BatchTableEnvironment]
-      .scan(tableName)
-      .select(tableApiExpr)
-      .getRelNode
-
-    // create DataSetCalc
-    val decorPlan = RelDecorrelator.decorrelateQuery(converted)
-    val flinkOutputProps = converted.getTraitSet.replace(DataSetConvention.INSTANCE).simplify()
-    val dataSetCalc = optProgram.run(context._2.getPlanner, decorPlan, flinkOutputProps)
-
-    // extract RexNode
-    val calcProgram = dataSetCalc
-     .asInstanceOf[DataSetCalc]
-     .calcProgram
-    val expanded = calcProgram.expandLocalRef(calcProgram.getProjectList.get(0))
-
-    testExprs += ((expanded, expected))
-  }
-
-  private def addTableApiTestExpr(tableApiString: String, expected: String): Unit = {
-    addTableApiTestExpr(ExpressionParser.parseExpression(tableApiString), expected)
-  }
-
-  def testAllApis(
-      expr: Expression,
-      exprString: String,
-      sqlExpr: String,
-      expected: String)
-    : Unit = {
-    addTableApiTestExpr(expr, expected)
-    addTableApiTestExpr(exprString, expected)
-    addSqlTestExpr(sqlExpr, expected)
-  }
-
-  def testTableApi(
-      expr: Expression,
-      exprString: String,
-      expected: String)
-    : Unit = {
-    addTableApiTestExpr(expr, expected)
-    addTableApiTestExpr(exprString, expected)
-  }
-
-  def testSqlApi(
-      sqlExpr: String,
-      expected: String)
-    : Unit = {
-    addSqlTestExpr(sqlExpr, expected)
-  }
-
-  // ----------------------------------------------------------------------------------------------
-
-  // TestCompiler that uses current class loader
-  class TestCompiler[T <: Function] extends Compiler[T] {
-    def compile(genFunc: GeneratedFunction[T]): Class[T] =
-      compile(getClass.getClassLoader, genFunc.name, genFunc.code)
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/UserDefinedScalarFunctions.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/UserDefinedScalarFunctions.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/UserDefinedScalarFunctions.scala
deleted file mode 100644
index d4772cf..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/expressions/utils/UserDefinedScalarFunctions.scala
+++ /dev/null
@@ -1,121 +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.api.table.expressions.utils
-
-import java.sql.{Date, Time, Timestamp}
-
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.table.Types
-import org.apache.flink.api.table.functions.ScalarFunction
-
-case class SimplePojo(name: String, age: Int)
-
-object Func0 extends ScalarFunction {
-  def eval(index: Int): Int = {
-    index
-  }
-}
-
-object Func1 extends ScalarFunction {
-  def eval(index: Integer): Integer = {
-    index + 1
-  }
-}
-
-object Func2 extends ScalarFunction {
-  def eval(index: Integer, str: String, pojo: SimplePojo): String = {
-    s"$index and $str and $pojo"
-  }
-}
-
-object Func3 extends ScalarFunction {
-  def eval(index: Integer, str: String): String = {
-    s"$index and $str"
-  }
-}
-
-object Func4 extends ScalarFunction {
-  def eval(): Integer = {
-    null
-  }
-}
-
-object Func5 extends ScalarFunction {
-  def eval(): Int = {
-    -1
-  }
-}
-
-object Func6 extends ScalarFunction {
-  def eval(date: Date, time: Time, timestamp: Timestamp): (Date, Time, Timestamp) = {
-    (date, time, timestamp)
-  }
-}
-
-object Func7 extends ScalarFunction {
-  def eval(a: Integer, b: Integer): Integer = {
-    a + b
-  }
-}
-
-object Func8 extends ScalarFunction {
-  def eval(a: Int): String = {
-    "a"
-  }
-
-  def eval(a: Int, b: Int): String = {
-    "b"
-  }
-
-  def eval(a: String, b: String): String = {
-    "c"
-  }
-}
-
-object Func9 extends ScalarFunction {
-  def eval(a: Int, b: Int, c: Long): String = {
-    s"$a and $b and $c"
-  }
-}
-
-object Func10 extends ScalarFunction {
-  def eval(c: Long): Long = {
-    c
-  }
-
-  override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = {
-    Types.TIMESTAMP
-  }
-}
-
-object Func11 extends ScalarFunction {
-  def eval(a: Int, b: Long): String = {
-    s"$a and $b"
-  }
-}
-
-object Func12 extends ScalarFunction {
-  def eval(a: Long): Long = {
-    a
-  }
-
-  override def getResultType(signature: Array[Class[_]]): TypeInformation[_] = {
-    Types.INTERVAL_MILLIS
-  }
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/plan/FieldProjectionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/plan/FieldProjectionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/plan/FieldProjectionTest.scala
deleted file mode 100644
index 1cefb8a..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/plan/FieldProjectionTest.scala
+++ /dev/null
@@ -1,317 +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.api.table.plan
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.ValidationException
-import org.apache.flink.api.table.expressions.{RowtimeAttribute, Upper, WindowReference}
-import org.apache.flink.api.table.functions.ScalarFunction
-import org.apache.flink.api.table.plan.FieldProjectionTest._
-import org.apache.flink.api.table.plan.logical.EventTimeTumblingGroupWindow
-import org.apache.flink.api.table.utils.TableTestBase
-import org.apache.flink.api.table.utils.TableTestUtil._
-import org.junit.Test
-
-/**
-  * Tests for all the situations when we can do fields projection. Like selecting few fields
-  * from a large field count source.
-  */
-class FieldProjectionTest extends TableTestBase {
-
-  val util = batchTestUtil()
-
-  val streamUtil = streamTestUtil()
-
-  @Test
-  def testSimpleSelect(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-    val resultTable = sourceTable.select('a, 'b)
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select", "a", "b")
-    )
-
-    util.verifyTable(resultTable, expected)
-  }
-
-  @Test
-  def testSelectAllFields(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-    val resultTable1 = sourceTable.select('*)
-    val resultTable2 = sourceTable.select('a, 'b, 'c, 'd)
-
-    val expected = batchTableNode(0)
-
-    util.verifyTable(resultTable1, expected)
-    util.verifyTable(resultTable2, expected)
-  }
-
-  @Test
-  def testSelectAggregation(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-    val resultTable = sourceTable.select('a.sum, 'b.max)
-
-    val expected = unaryNode(
-      "DataSetAggregate",
-      binaryNode(
-        "DataSetUnion",
-        values(
-          "DataSetValues",
-          tuples(List(null, null)),
-          term("values", "a", "b")
-        ),
-        unaryNode(
-          "DataSetCalc",
-          batchTableNode(0),
-          term("select", "a", "b")
-        ),
-        term("union", "a", "b")
-      ),
-      term("select", "SUM(a) AS TMP_0", "MAX(b) AS TMP_1")
-    )
-
-    util.verifyTable(resultTable, expected)
-  }
-
-  @Test
-  def testSelectFunction(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-
-    util.tEnv.registerFunction("hashCode", MyHashCode)
-
-    val resultTable = sourceTable.select("hashCode(c), b")
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      batchTableNode(0),
-      term("select", s"${MyHashCode.getClass.getCanonicalName}(c) AS _c0", "b")
-    )
-
-    util.verifyTable(resultTable, expected)
-  }
-
-  @Test
-  def testSelectFromGroupedTable(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-    val resultTable = sourceTable.groupBy('a, 'c).select('a)
-
-    val expected = unaryNode(
-      "DataSetCalc",
-      unaryNode(
-        "DataSetAggregate",
-        unaryNode(
-          "DataSetCalc",
-          batchTableNode(0),
-          term("select", "a", "c")
-        ),
-        term("groupBy", "a", "c"),
-        term("select", "a", "c")
-      ),
-      term("select", "a")
-    )
-
-    util.verifyTable(resultTable, expected)
-  }
-
-  @Test
-  def testSelectAllFieldsFromGroupedTable(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-    val resultTable = sourceTable.groupBy('a, 'c).select('a, 'c)
-
-    val expected = unaryNode(
-      "DataSetAggregate",
-      unaryNode(
-        "DataSetCalc",
-        batchTableNode(0),
-        term("select", "a", "c")
-      ),
-      term("groupBy", "a", "c"),
-      term("select", "a", "c")
-    )
-
-    util.verifyTable(resultTable, expected)
-  }
-
-  @Test
-  def testSelectAggregationFromGroupedTable(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-    val resultTable = sourceTable.groupBy('c).select('a.sum)
-
-    val expected =
-      unaryNode(
-        "DataSetCalc",
-        unaryNode(
-          "DataSetAggregate",
-          unaryNode(
-            "DataSetCalc",
-            batchTableNode(0),
-            term("select", "a", "c")
-          ),
-          term("groupBy", "c"),
-          term("select", "c", "SUM(a) AS TMP_0")
-        ),
-        term("select", "TMP_0 AS TMP_1")
-      )
-
-    util.verifyTable(resultTable, expected)
-  }
-
-  @Test
-  def testSelectFromGroupedTableWithNonTrivialKey(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-    val resultTable = sourceTable.groupBy(Upper('c) as 'k).select('a.sum)
-
-    val expected =
-      unaryNode(
-        "DataSetCalc",
-        unaryNode(
-          "DataSetAggregate",
-          unaryNode(
-            "DataSetCalc",
-            batchTableNode(0),
-            term("select", "a", "c", "UPPER(c) AS k")
-          ),
-          term("groupBy", "k"),
-          term("select", "k", "SUM(a) AS TMP_0")
-        ),
-        term("select", "TMP_0 AS TMP_1")
-      )
-
-    util.verifyTable(resultTable, expected)
-  }
-
-  @Test
-  def testSelectFromGroupedTableWithFunctionKey(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-    val resultTable = sourceTable.groupBy(MyHashCode('c) as 'k).select('a.sum)
-
-    val expected =
-      unaryNode(
-        "DataSetCalc",
-        unaryNode(
-          "DataSetAggregate",
-          unaryNode(
-            "DataSetCalc",
-            batchTableNode(0),
-            term("select", "a", "c", s"${MyHashCode.getClass.getCanonicalName}(c) AS k")
-          ),
-          term("groupBy", "k"),
-          term("select", "k", "SUM(a) AS TMP_0")
-        ),
-        term("select", "TMP_0 AS TMP_1")
-      )
-
-    util.verifyTable(resultTable, expected)
-  }
-
-  @Test
-  def testSelectFromStreamingWindow(): Unit = {
-    val sourceTable = streamUtil.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-    val resultTable = sourceTable
-        .window(Tumble over 5.millis on 'rowtime as 'w)
-        .select(Upper('c).count, 'a.sum)
-
-    val expected =
-      unaryNode(
-        "DataStreamAggregate",
-        unaryNode(
-          "DataStreamCalc",
-          streamTableNode(0),
-          term("select", "c", "a", "UPPER(c) AS $f2")
-        ),
-        term("window",
-          EventTimeTumblingGroupWindow(
-            Some(WindowReference("w")),
-            RowtimeAttribute(),
-            5.millis)),
-        term("select", "COUNT($f2) AS TMP_0", "SUM(a) AS TMP_1")
-      )
-
-    streamUtil.verifyTable(resultTable, expected)
-  }
-
-  @Test
-  def testSelectFromStreamingGroupedWindow(): Unit = {
-    val sourceTable = streamUtil.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-    val resultTable = sourceTable
-        .groupBy('b)
-        .window(Tumble over 5.millis on 'rowtime as 'w)
-        .select(Upper('c).count, 'a.sum, 'b)
-
-    val expected = unaryNode(
-        "DataStreamCalc",
-        unaryNode(
-          "DataStreamAggregate",
-          unaryNode(
-            "DataStreamCalc",
-            streamTableNode(0),
-            term("select", "c", "a", "b", "UPPER(c) AS $f3")
-          ),
-          term("groupBy", "b"),
-          term("window",
-            EventTimeTumblingGroupWindow(
-              Some(WindowReference("w")),
-              RowtimeAttribute(),
-              5.millis)),
-          term("select", "b", "COUNT($f3) AS TMP_0", "SUM(a) AS TMP_1")
-        ),
-        term("select", "TMP_0 AS TMP_2", "TMP_1 AS TMP_3", "b")
-    )
-
-    streamUtil.verifyTable(resultTable, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testSelectFromBatchWindow1(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-
-    // time field is selected
-    val resultTable = sourceTable
-        .window(Tumble over 5.millis on 'a as 'w)
-        .select('a.sum, 'c.count)
-
-    val expected = "TODO"
-
-    util.verifyTable(resultTable, expected)
-  }
-
-  @Test(expected = classOf[ValidationException])
-  def testSelectFromBatchWindow2(): Unit = {
-    val sourceTable = util.addTable[(Int, Long, String, Double)]("MyTable", 'a, 'b, 'c, 'd)
-
-    // time field is not selected
-    val resultTable = sourceTable
-        .window(Tumble over 5.millis on 'a as 'w)
-        .select('c.count)
-
-    val expected = "TODO"
-
-    util.verifyTable(resultTable, expected)
-  }
-}
-
-object FieldProjectionTest {
-
-  object MyHashCode extends ScalarFunction {
-    def eval(s: String): Int = s.hashCode()
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/plan/rules/util/RexProgramProjectExtractorTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/plan/rules/util/RexProgramProjectExtractorTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/plan/rules/util/RexProgramProjectExtractorTest.scala
deleted file mode 100644
index 156f281..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/plan/rules/util/RexProgramProjectExtractorTest.scala
+++ /dev/null
@@ -1,120 +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.api.table.plan.rules.util
-
-import java.math.BigDecimal
-
-import org.apache.calcite.adapter.java.JavaTypeFactory
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl
-import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeSystem}
-import org.apache.calcite.sql.`type`.SqlTypeName.{BIGINT, DOUBLE, INTEGER, VARCHAR}
-import org.apache.calcite.rex.{RexBuilder, RexProgram, RexProgramBuilder}
-import org.apache.calcite.sql.fun.SqlStdOperatorTable
-
-import scala.collection.JavaConverters._
-import org.apache.flink.api.table.plan.rules.util.RexProgramProjectExtractor._
-import org.junit.{Assert, Before, Test}
-
-/**
-  * This class is responsible for testing RexProgramProjectExtractor
-  */
-class RexProgramProjectExtractorTest {
-  private var typeFactory: JavaTypeFactory = null
-  private var rexBuilder: RexBuilder = null
-  private var allFieldTypes: Seq[RelDataType] = null
-  private val allFieldNames = List("name", "id", "amount", "price")
-
-  @Before
-  def setUp: Unit = {
-    typeFactory = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT)
-    rexBuilder = new RexBuilder(typeFactory)
-    allFieldTypes = List(VARCHAR, BIGINT, INTEGER, DOUBLE).map(typeFactory.createSqlType(_))
-  }
-
-  @Test
-  def testExtractRefInputFields: Unit = {
-    val usedFields = extractRefInputFields(buildRexProgram)
-    Assert.assertArrayEquals(usedFields, Array(2, 3, 1))
-  }
-
-  @Test
-  def testRewriteRexProgram: Unit = {
-    val originRexProgram = buildRexProgram
-    Assert.assertTrue(extractExprStrList(originRexProgram).sameElements(Array(
-      "$0",
-      "$1",
-      "$2",
-      "$3",
-      "*($t2, $t3)",
-      "100",
-      "<($t4, $t5)",
-      "6",
-      ">($t1, $t7)",
-      "AND($t6, $t8)")))
-    // use amount, id, price fields to create a new RexProgram
-    val usedFields = Array(2, 3, 1)
-    val types = usedFields.map(allFieldTypes(_)).toList.asJava
-    val names = usedFields.map(allFieldNames(_)).toList.asJava
-    val inputRowType = typeFactory.createStructType(types, names)
-    val newRexProgram = rewriteRexProgram(originRexProgram, inputRowType, usedFields, rexBuilder)
-    Assert.assertTrue(extractExprStrList(newRexProgram).sameElements(Array(
-      "$0",
-      "$1",
-      "$2",
-      "*($t0, $t1)",
-      "100",
-      "<($t3, $t4)",
-      "6",
-      ">($t2, $t6)",
-      "AND($t5, $t7)")))
-  }
-
-  private def buildRexProgram: RexProgram = {
-    val types = allFieldTypes.asJava
-    val names = allFieldNames.asJava
-    val inputRowType = typeFactory.createStructType(types, names)
-    val builder = new RexProgramBuilder(inputRowType, rexBuilder)
-    val t0 = rexBuilder.makeInputRef(types.get(2), 2)
-    val t1 = rexBuilder.makeInputRef(types.get(1), 1)
-    val t2 = rexBuilder.makeInputRef(types.get(3), 3)
-    val t3 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.MULTIPLY, t0, t2))
-    val t4 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(100L))
-    val t5 = rexBuilder.makeExactLiteral(BigDecimal.valueOf(6L))
-    // project: amount, amount * price
-    builder.addProject(t0, "amount")
-    builder.addProject(t3, "total")
-    // condition: amount * price < 100 and id > 6
-    val t6 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, t3, t4))
-    val t7 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN, t1, t5))
-    val t8 = builder.addExpr(rexBuilder.makeCall(SqlStdOperatorTable.AND, List(t6, t7).asJava))
-    builder.addCondition(t8)
-    builder.getProgram
-  }
-
-  /**
-    * extract all expression string list from input RexProgram expression lists
-    *
-    * @param rexProgram input RexProgram instance to analyze
-    * @return all expression string list of input RexProgram expression lists
-    */
-  private def extractExprStrList(rexProgram: RexProgram) = {
-    rexProgram.getExprList.asScala.map(_.toString)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala
deleted file mode 100644
index 4e33a61..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AggregateTestBase.scala
+++ /dev/null
@@ -1,110 +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.api.table.runtime.aggregate
-
-import java.math.BigDecimal
-import org.apache.flink.types.Row
-import org.junit.Test
-import org.junit.Assert.assertEquals
-
-abstract class AggregateTestBase[T] {
-
-  private val offset = 2
-  private val rowArity: Int = offset + aggregator.intermediateDataType.length
-
-  def inputValueSets: Seq[Seq[_]]
-
-  def expectedResults: Seq[T]
-
-  def aggregator: Aggregate[T]
-
-  private def createAggregator(): Aggregate[T] = {
-    val agg = aggregator
-    agg.setAggOffsetInRow(offset)
-    agg
-  }
-
-  private def createRow(): Row = {
-    new Row(rowArity)
-  }
-
-  @Test
-  def testAggregate(): Unit = {
-
-    // iterate over input sets
-    for((vals, expected) <- inputValueSets.zip(expectedResults)) {
-
-      // prepare mapper
-      val rows: Seq[Row] = prepare(vals)
-
-      val result = if (aggregator.supportPartial) {
-        // test with combiner
-        val (firstVals, secondVals) = rows.splitAt(rows.length / 2)
-        val combined = partialAgg(firstVals) :: partialAgg(secondVals) :: Nil
-        finalAgg(combined)
-
-      } else {
-        // test without combiner
-        finalAgg(rows)
-      }
-
-      (expected, result) match {
-        case (e: BigDecimal, r: BigDecimal) =>
-          // BigDecimal.equals() value and scale but we are only interested in value.
-          assert(e.compareTo(r) == 0)
-        case _ =>
-          assertEquals(expected, result)
-      }
-    }
-  }
-
-  private def prepare(vals: Seq[_]): Seq[Row] = {
-
-    val agg = createAggregator()
-
-    vals.map { v =>
-      val row = createRow()
-      agg.prepare(v, row)
-      row
-    }
-  }
-
-  private def partialAgg(rows: Seq[Row]): Row = {
-
-    val agg = createAggregator()
-    val aggBuf = createRow()
-
-    agg.initiate(aggBuf)
-    rows.foreach(v => agg.merge(v, aggBuf))
-
-    aggBuf
-  }
-
-  private def finalAgg(rows: Seq[Row]): T = {
-
-    val agg = createAggregator()
-    val aggBuf = createRow()
-
-    agg.initiate(aggBuf)
-    rows.foreach(v => agg.merge(v, aggBuf))
-
-    agg.evaluate(partialAgg(rows))
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregateTest.scala
deleted file mode 100644
index 23b3054..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/AvgAggregateTest.scala
+++ /dev/null
@@ -1,154 +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.api.table.runtime.aggregate
-
-import java.math.BigDecimal
-
-abstract class AvgAggregateTestBase[T: Numeric] extends AggregateTestBase[T] {
-
-  private val numeric: Numeric[T] = implicitly[Numeric[T]]
-
-  def minVal: T
-  def maxVal: T
-
-  override def inputValueSets: Seq[Seq[T]] = Seq(
-    Seq(
-      minVal,
-      minVal,
-      null.asInstanceOf[T],
-      minVal,
-      minVal,
-      null.asInstanceOf[T],
-      minVal,
-      minVal,
-      minVal
-    ),
-    Seq(
-      maxVal,
-      maxVal,
-      null.asInstanceOf[T],
-      maxVal,
-      maxVal,
-      null.asInstanceOf[T],
-      maxVal,
-      maxVal,
-      maxVal
-    ),
-    Seq(
-      minVal,
-      maxVal,
-      null.asInstanceOf[T],
-      numeric.fromInt(0),
-      numeric.negate(maxVal),
-      numeric.negate(minVal),
-      null.asInstanceOf[T]
-    ),
-    Seq(
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T]
-    )
-  )
-
-  override def expectedResults: Seq[T] = Seq(
-    minVal,
-    maxVal,
-    numeric.fromInt(0),
-    null.asInstanceOf[T]
-  )
-}
-
-class ByteAvgAggregateTest extends AvgAggregateTestBase[Byte] {
-
-  override def minVal = (Byte.MinValue + 1).toByte
-  override def maxVal = (Byte.MaxValue - 1).toByte
-
-  override def aggregator = new ByteAvgAggregate()
-}
-
-class ShortAvgAggregateTest extends AvgAggregateTestBase[Short] {
-
-  override def minVal = (Short.MinValue + 1).toShort
-  override def maxVal = (Short.MaxValue - 1).toShort
-
-  override def aggregator = new ShortAvgAggregate()
-}
-
-class IntAvgAggregateTest extends AvgAggregateTestBase[Int] {
-
-  override def minVal = Int.MinValue + 1
-  override def maxVal = Int.MaxValue - 1
-
-  override def aggregator = new IntAvgAggregate()
-}
-
-class LongAvgAggregateTest extends AvgAggregateTestBase[Long] {
-
-  override def minVal = Long.MinValue + 1
-  override def maxVal = Long.MaxValue - 1
-
-  override def aggregator = new LongAvgAggregate()
-}
-
-class FloatAvgAggregateTest extends AvgAggregateTestBase[Float] {
-
-  override def minVal = Float.MinValue
-  override def maxVal = Float.MaxValue
-
-  override def aggregator = new FloatAvgAggregate()
-}
-
-class DoubleAvgAggregateTest extends AvgAggregateTestBase[Double] {
-
-  override def minVal = Float.MinValue
-  override def maxVal = Float.MaxValue
-
-  override def aggregator = new DoubleAvgAggregate()
-}
-
-class DecimalAvgAggregateTest extends AggregateTestBase[BigDecimal] {
-
-  override def inputValueSets: Seq[Seq[_]] = Seq(
-    Seq(
-      new BigDecimal("987654321000000"),
-      new BigDecimal("-0.000000000012345"),
-      null,
-      new BigDecimal("0.000000000012345"),
-      new BigDecimal("-987654321000000"),
-      null,
-      new BigDecimal("0")
-    ),
-    Seq(
-      null,
-      null,
-      null,
-      null
-    )
-  )
-
-  override def expectedResults: Seq[BigDecimal] = Seq(
-    BigDecimal.ZERO,
-    null
-  )
-
-  override def aggregator: Aggregate[BigDecimal] = new DecimalAvgAggregate()
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregateTest.scala
deleted file mode 100644
index 4389a3a..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/CountAggregateTest.scala
+++ /dev/null
@@ -1,31 +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.api.table.runtime.aggregate
-
-class CountAggregateTest extends AggregateTestBase[Long] {
-
-  override def inputValueSets: Seq[Seq[_]] = Seq(
-    Seq("a", "b", null, "c", null, "d", "e", null, "f"),
-    Seq(null, null, null, null, null, null)
-  )
-
-  override def expectedResults: Seq[Long] = Seq(6L, 0L)
-
-  override def aggregator: Aggregate[Long] = new CountAggregate()
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala
deleted file mode 100644
index aea3318..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MaxAggregateTest.scala
+++ /dev/null
@@ -1,177 +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.api.table.runtime.aggregate
-
-import java.math.BigDecimal
-
-abstract class MaxAggregateTestBase[T: Numeric] extends AggregateTestBase[T] {
-
-  private val numeric: Numeric[T] = implicitly[Numeric[T]]
-
-  def minVal: T
-  def maxVal: T
-
-  override def inputValueSets: Seq[Seq[T]] = Seq(
-    Seq(
-      numeric.fromInt(1),
-      null.asInstanceOf[T],
-      maxVal,
-      numeric.fromInt(-99),
-      numeric.fromInt(3),
-      numeric.fromInt(56),
-      numeric.fromInt(0),
-      minVal,
-      numeric.fromInt(-20),
-      numeric.fromInt(17),
-      null.asInstanceOf[T]
-    ),
-    Seq(
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T]
-    )
-  )
-
-  override def expectedResults: Seq[T] = Seq(
-    maxVal,
-    null.asInstanceOf[T]
-  )
-}
-
-class ByteMaxAggregateTest extends MaxAggregateTestBase[Byte] {
-
-  override def minVal = (Byte.MinValue + 1).toByte
-  override def maxVal = (Byte.MaxValue - 1).toByte
-
-  override def aggregator: Aggregate[Byte] = new ByteMaxAggregate()
-}
-
-class ShortMaxAggregateTest extends MaxAggregateTestBase[Short] {
-
-  override def minVal = (Short.MinValue + 1).toShort
-  override def maxVal = (Short.MaxValue - 1).toShort
-
-  override def aggregator: Aggregate[Short] = new ShortMaxAggregate()
-}
-
-class IntMaxAggregateTest extends MaxAggregateTestBase[Int] {
-
-  override def minVal = Int.MinValue + 1
-  override def maxVal = Int.MaxValue - 1
-
-  override def aggregator: Aggregate[Int] = new IntMaxAggregate()
-}
-
-class LongMaxAggregateTest extends MaxAggregateTestBase[Long] {
-
-  override def minVal = Long.MinValue + 1
-  override def maxVal = Long.MaxValue - 1
-
-  override def aggregator: Aggregate[Long] = new LongMaxAggregate()
-}
-
-class FloatMaxAggregateTest extends MaxAggregateTestBase[Float] {
-
-  override def minVal = Float.MinValue / 2
-  override def maxVal = Float.MaxValue / 2
-
-  override def aggregator: Aggregate[Float] = new FloatMaxAggregate()
-}
-
-class DoubleMaxAggregateTest extends MaxAggregateTestBase[Double] {
-
-  override def minVal = Double.MinValue / 2
-  override def maxVal = Double.MaxValue / 2
-
-  override def aggregator: Aggregate[Double] = new DoubleMaxAggregate()
-}
-
-class BooleanMaxAggregateTest extends AggregateTestBase[Boolean] {
-
-  override def inputValueSets: Seq[Seq[Boolean]] = Seq(
-    Seq(
-      false,
-      false,
-      false
-    ),
-    Seq(
-      true,
-      true,
-      true
-    ),
-    Seq(
-      true,
-      false,
-      null.asInstanceOf[Boolean],
-      true,
-      false,
-      true,
-      null.asInstanceOf[Boolean]
-    ),
-    Seq(
-      null.asInstanceOf[Boolean],
-      null.asInstanceOf[Boolean],
-      null.asInstanceOf[Boolean]
-    )
-  )
-
-  override def expectedResults: Seq[Boolean] = Seq(
-    false,
-    true,
-    true,
-    null.asInstanceOf[Boolean]
-  )
-
-  override def aggregator: Aggregate[Boolean] = new BooleanMaxAggregate()
-}
-
-class DecimalMaxAggregateTest extends AggregateTestBase[BigDecimal] {
-
-  override def inputValueSets: Seq[Seq[_]] = Seq(
-    Seq(
-      new BigDecimal("1"),
-      new BigDecimal("1000.000001"),
-      new BigDecimal("-1"),
-      new BigDecimal("-999.998999"),
-      null,
-      new BigDecimal("0"),
-      new BigDecimal("-999.999"),
-      null,
-      new BigDecimal("999.999")
-    ),
-    Seq(
-      null,
-      null,
-      null,
-      null,
-      null
-    )
-  )
-
-  override def expectedResults: Seq[BigDecimal] = Seq(
-    new BigDecimal("1000.000001"),
-    null
-  )
-
-  override def aggregator: Aggregate[BigDecimal] = new DecimalMaxAggregate()
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala
deleted file mode 100644
index f007d02..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/MinAggregateTest.scala
+++ /dev/null
@@ -1,177 +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.api.table.runtime.aggregate
-
-import java.math.BigDecimal
-
-abstract class MinAggregateTestBase[T: Numeric] extends AggregateTestBase[T] {
-
-  private val numeric: Numeric[T] = implicitly[Numeric[T]]
-
-  def minVal: T
-  def maxVal: T
-
-  override def inputValueSets: Seq[Seq[T]] = Seq(
-    Seq(
-      numeric.fromInt(1),
-      null.asInstanceOf[T],
-      maxVal,
-      numeric.fromInt(-99),
-      numeric.fromInt(3),
-      numeric.fromInt(56),
-      numeric.fromInt(0),
-      minVal,
-      numeric.fromInt(-20),
-      numeric.fromInt(17),
-      null.asInstanceOf[T]
-    ),
-    Seq(
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T]
-    )
-  )
-
-  override def expectedResults: Seq[T] = Seq(
-    minVal,
-    null.asInstanceOf[T]
-  )
-}
-
-class ByteMinAggregateTest extends MinAggregateTestBase[Byte] {
-
-  override def minVal = (Byte.MinValue + 1).toByte
-  override def maxVal = (Byte.MaxValue - 1).toByte
-
-  override def aggregator: Aggregate[Byte] = new ByteMinAggregate()
-}
-
-class ShortMinAggregateTest extends MinAggregateTestBase[Short] {
-
-  override def minVal = (Short.MinValue + 1).toShort
-  override def maxVal = (Short.MaxValue - 1).toShort
-
-  override def aggregator: Aggregate[Short] = new ShortMinAggregate()
-}
-
-class IntMinAggregateTest extends MinAggregateTestBase[Int] {
-
-  override def minVal = Int.MinValue + 1
-  override def maxVal = Int.MaxValue - 1
-
-  override def aggregator: Aggregate[Int] = new IntMinAggregate()
-}
-
-class LongMinAggregateTest extends MinAggregateTestBase[Long] {
-
-  override def minVal = Long.MinValue + 1
-  override def maxVal = Long.MaxValue - 1
-
-  override def aggregator: Aggregate[Long] = new LongMinAggregate()
-}
-
-class FloatMinAggregateTest extends MinAggregateTestBase[Float] {
-
-  override def minVal = Float.MinValue / 2
-  override def maxVal = Float.MaxValue / 2
-
-  override def aggregator: Aggregate[Float] = new FloatMinAggregate()
-}
-
-class DoubleMinAggregateTest extends MinAggregateTestBase[Double] {
-
-  override def minVal = Double.MinValue / 2
-  override def maxVal = Double.MaxValue / 2
-
-  override def aggregator: Aggregate[Double] = new DoubleMinAggregate()
-}
-
-class BooleanMinAggregateTest extends AggregateTestBase[Boolean] {
-
-  override def inputValueSets: Seq[Seq[Boolean]] = Seq(
-    Seq(
-      false,
-      false,
-      false
-    ),
-    Seq(
-      true,
-      true,
-      true
-    ),
-    Seq(
-      true,
-      false,
-      null.asInstanceOf[Boolean],
-      true,
-      false,
-      true,
-      null.asInstanceOf[Boolean]
-    ),
-    Seq(
-      null.asInstanceOf[Boolean],
-      null.asInstanceOf[Boolean],
-      null.asInstanceOf[Boolean]
-    )
-  )
-
-  override def expectedResults: Seq[Boolean] = Seq(
-    false,
-    true,
-    false,
-    null.asInstanceOf[Boolean]
-  )
-
-  override def aggregator: Aggregate[Boolean] = new BooleanMinAggregate()
-}
-
-class DecimalMinAggregateTest extends AggregateTestBase[BigDecimal] {
-
-  override def inputValueSets: Seq[Seq[_]] = Seq(
-    Seq(
-      new BigDecimal("1"),
-      new BigDecimal("1000"),
-      new BigDecimal("-1"),
-      new BigDecimal("-999.998999"),
-      null,
-      new BigDecimal("0"),
-      new BigDecimal("-999.999"),
-      null,
-      new BigDecimal("999.999")
-    ),
-    Seq(
-      null,
-      null,
-      null,
-      null,
-      null
-    )
-  )
-
-  override def expectedResults: Seq[BigDecimal] = Seq(
-    new BigDecimal("-999.999"),
-    null
-  )
-
-  override def aggregator: Aggregate[BigDecimal] = new DecimalMinAggregate()
-
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregateTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregateTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregateTest.scala
deleted file mode 100644
index 7e4e47b..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/aggregate/SumAggregateTest.scala
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.api.table.runtime.aggregate
-
-import java.math.BigDecimal
-
-abstract class SumAggregateTestBase[T: Numeric] extends AggregateTestBase[T] {
-
-  private val numeric: Numeric[T] = implicitly[Numeric[T]]
-
-  def maxVal: T
-  private val minVal = numeric.negate(maxVal)
-
-  override def inputValueSets: Seq[Seq[T]] = Seq(
-    Seq(
-      minVal,
-      numeric.fromInt(1),
-      null.asInstanceOf[T],
-      numeric.fromInt(2),
-      numeric.fromInt(3),
-      numeric.fromInt(4),
-      numeric.fromInt(5),
-      numeric.fromInt(-10),
-      numeric.fromInt(-20),
-      numeric.fromInt(17),
-      null.asInstanceOf[T],
-      maxVal
-    ),
-    Seq(
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T],
-      null.asInstanceOf[T]
-    )
-  )
-
-  override def expectedResults: Seq[T] = Seq(
-    numeric.fromInt(2),
-    null.asInstanceOf[T]
-  )
-}
-
-class ByteSumAggregateTest extends SumAggregateTestBase[Byte] {
-
-  override def maxVal = (Byte.MaxValue / 2).toByte
-
-  override def aggregator: Aggregate[Byte] = new ByteSumAggregate
-}
-
-class ShortSumAggregateTest extends SumAggregateTestBase[Short] {
-
-  override def maxVal = (Short.MaxValue / 2).toShort
-
-  override def aggregator: Aggregate[Short] = new ShortSumAggregate
-}
-
-class IntSumAggregateTest extends SumAggregateTestBase[Int] {
-
-  override def maxVal = Int.MaxValue / 2
-
-  override def aggregator: Aggregate[Int] = new IntSumAggregate
-}
-
-class LongSumAggregateTest extends SumAggregateTestBase[Long] {
-
-  override def maxVal = Long.MaxValue / 2
-
-  override def aggregator: Aggregate[Long] = new LongSumAggregate
-}
-
-class FloatSumAggregateTest extends SumAggregateTestBase[Float] {
-
-  override def maxVal = 12345.6789f
-
-  override def aggregator: Aggregate[Float] = new FloatSumAggregate
-}
-
-class DoubleSumAggregateTest extends SumAggregateTestBase[Double] {
-
-  override def maxVal = 12345.6789d
-
-  override def aggregator: Aggregate[Double] = new DoubleSumAggregate
-}
-
-class DecimalSumAggregateTest extends AggregateTestBase[BigDecimal] {
-
-  override def inputValueSets: Seq[Seq[_]] = Seq(
-    Seq(
-      new BigDecimal("1"),
-      new BigDecimal("2"),
-      new BigDecimal("3"),
-      null,
-      new BigDecimal("0"),
-      new BigDecimal("-1000"),
-      new BigDecimal("0.000000000002"),
-      new BigDecimal("1000"),
-      new BigDecimal("-0.000000000001"),
-      new BigDecimal("999.999"),
-      null,
-      new BigDecimal("4"),
-      new BigDecimal("-999.999"),
-      null
-    ),
-    Seq(
-      null,
-      null,
-      null,
-      null,
-      null
-    )
-  )
-
-  override def expectedResults: Seq[BigDecimal] = Seq(
-    new BigDecimal("10.000000000001"),
-    null
-  )
-
-  override def aggregator: Aggregate[BigDecimal] = new DecimalSumAggregate()
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/dataset/DataSetCorrelateITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/dataset/DataSetCorrelateITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/dataset/DataSetCorrelateITCase.scala
deleted file mode 100644
index 993347f..0000000
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/api/table/runtime/dataset/DataSetCorrelateITCase.scala
+++ /dev/null
@@ -1,178 +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.api.table.runtime.dataset
-
-import org.apache.flink.api.scala._
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase
-import org.apache.flink.api.scala.batch.utils.TableProgramsTestBase.TableConfigMode
-import org.apache.flink.api.scala.table._
-import org.apache.flink.api.table.expressions.utils._
-import org.apache.flink.api.table.utils._
-import org.apache.flink.api.table.TableEnvironment
-import org.apache.flink.types.Row
-import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
-import org.apache.flink.test.util.TestBaseUtils
-import org.junit.Test
-import org.junit.runner.RunWith
-import org.junit.runners.Parameterized
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable
-
-@RunWith(classOf[Parameterized])
-class DataSetCorrelateITCase(
-  mode: TestExecutionMode,
-  configMode: TableConfigMode)
-  extends TableProgramsTestBase(mode, configMode) {
-
-  @Test
-  def testCrossJoin(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
-    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
-
-    val func1 = new TableFunc1
-    val result = in.join(func1('c) as 's).select('c, 's).toDataSet[Row]
-    val results = result.collect()
-    val expected = "Jack#22,Jack\n" + "Jack#22,22\n" + "John#19,John\n" + "John#19,19\n" +
-      "Anna#44,Anna\n" + "Anna#44,44\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-
-    // with overloading
-    val result2 = in.join(func1('c, "$") as 's).select('c, 's).toDataSet[Row]
-    val results2 = result2.collect()
-    val expected2 = "Jack#22,$Jack\n" + "Jack#22,$22\n" + "John#19,$John\n" +
-      "John#19,$19\n" + "Anna#44,$Anna\n" + "Anna#44,$44\n"
-    TestBaseUtils.compareResultAsText(results2.asJava, expected2)
-  }
-
-  @Test
-  def testLeftOuterJoin(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
-    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
-
-    val func2 = new TableFunc2
-    val result = in.leftOuterJoin(func2('c) as ('s, 'l)).select('c, 's, 'l).toDataSet[Row]
-    val results = result.collect()
-    val expected = "Jack#22,Jack,4\n" + "Jack#22,22,2\n" + "John#19,John,4\n" +
-      "John#19,19,2\n" + "Anna#44,Anna,4\n" + "Anna#44,44,2\n" + "nosharp,null,null"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testWithFilter(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
-    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
-    val func0 = new TableFunc0
-
-    val result = in
-      .join(func0('c) as ('name, 'age))
-      .select('c, 'name, 'age)
-      .filter('age > 20)
-      .toDataSet[Row]
-
-    val results = result.collect()
-    val expected = "Jack#22,Jack,22\n" + "Anna#44,Anna,44\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testCustomReturnType(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
-    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
-    val func2 = new TableFunc2
-
-    val result = in
-      .join(func2('c) as ('name, 'len))
-      .select('c, 'name, 'len)
-      .toDataSet[Row]
-
-    val results = result.collect()
-    val expected = "Jack#22,Jack,4\n" + "Jack#22,22,2\n" + "John#19,John,4\n" +
-      "John#19,19,2\n" + "Anna#44,Anna,4\n" + "Anna#44,44,2\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testHierarchyType(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
-    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
-
-    val hierarchy = new HierarchyTableFunction
-    val result = in
-      .join(hierarchy('c) as ('name, 'adult, 'len))
-      .select('c, 'name, 'adult, 'len)
-      .toDataSet[Row]
-
-    val results = result.collect()
-    val expected = "Jack#22,Jack,true,22\n" + "John#19,John,false,19\n" +
-      "Anna#44,Anna,true,44\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testPojoType(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
-    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
-
-    val pojo = new PojoTableFunc()
-    val result = in
-      .join(pojo('c))
-      .select('c, 'name, 'age)
-      .toDataSet[Row]
-
-    val results = result.collect()
-    val expected = "Jack#22,Jack,22\n" + "John#19,John,19\n" + "Anna#44,Anna,44\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  @Test
-  def testUDTFWithScalarFunction(): Unit = {
-    val env = ExecutionEnvironment.getExecutionEnvironment
-    val tableEnv = TableEnvironment.getTableEnvironment(env, config)
-    val in = testData(env).toTable(tableEnv).as('a, 'b, 'c)
-    val func1 = new TableFunc1
-
-    val result = in
-      .join(func1('c.substring(2)) as 's)
-      .select('c, 's)
-      .toDataSet[Row]
-
-    val results = result.collect()
-    val expected = "Jack#22,ack\n" + "Jack#22,22\n" + "John#19,ohn\n" + "John#19,19\n" +
-      "Anna#44,nna\n" + "Anna#44,44\n"
-    TestBaseUtils.compareResultAsText(results.asJava, expected)
-  }
-
-  private def testData(
-      env: ExecutionEnvironment)
-    : DataSet[(Int, Long, String)] = {
-
-    val data = new mutable.MutableList[(Int, Long, String)]
-    data.+=((1, 1L, "Jack#22"))
-    data.+=((2, 2L, "John#19"))
-    data.+=((3, 2L, "Anna#44"))
-    data.+=((4, 3L, "nosharp"))
-    env.fromCollection(data)
-  }
-}


[21/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/ProjectionTranslator.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/ProjectionTranslator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/ProjectionTranslator.scala
new file mode 100644
index 0000000..ed6cf7b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/ProjectionTranslator.scala
@@ -0,0 +1,263 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan
+
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.plan.logical.{LogicalNode, Project}
+
+import scala.collection.mutable.ListBuffer
+
+object ProjectionTranslator {
+
+  /**
+    * Extracts and deduplicates all aggregation and window property expressions (zero, one, or more)
+    * from the given expressions.
+    *
+    * @param exprs    a list of expressions to extract
+    * @param tableEnv the TableEnvironment
+    * @return a Tuple2, the first field contains the extracted and deduplicated aggregations,
+    *         and the second field contains the extracted and deduplicated window properties.
+    */
+  def extractAggregationsAndProperties(
+      exprs: Seq[Expression],
+      tableEnv: TableEnvironment): (Map[Expression, String], Map[Expression, String]) = {
+    exprs.foldLeft((Map[Expression, String](), Map[Expression, String]())) {
+      (x, y) => identifyAggregationsAndProperties(y, tableEnv, x._1, x._2)
+    }
+  }
+
+  /** Identifies and deduplicates aggregation functions and window properties. */
+  private def identifyAggregationsAndProperties(
+      exp: Expression,
+      tableEnv: TableEnvironment,
+      aggNames: Map[Expression, String],
+      propNames: Map[Expression, String]) : (Map[Expression, String], Map[Expression, String]) = {
+
+    exp match {
+      case agg: Aggregation =>
+        if (aggNames contains agg) {
+          (aggNames, propNames)
+        } else {
+          (aggNames + (agg -> tableEnv.createUniqueAttributeName()), propNames)
+        }
+      case prop: WindowProperty =>
+        if (propNames contains prop) {
+          (aggNames, propNames)
+        } else {
+          (aggNames, propNames + (prop -> tableEnv.createUniqueAttributeName()))
+        }
+      case l: LeafExpression =>
+        (aggNames, propNames)
+      case u: UnaryExpression =>
+        identifyAggregationsAndProperties(u.child, tableEnv, aggNames, propNames)
+      case b: BinaryExpression =>
+        val l = identifyAggregationsAndProperties(b.left, tableEnv, aggNames, propNames)
+        identifyAggregationsAndProperties(b.right, tableEnv, l._1, l._2)
+
+      // Functions calls
+      case c @ Call(name, args) =>
+        args.foldLeft((aggNames, propNames)){
+          (x, y) => identifyAggregationsAndProperties(y, tableEnv, x._1, x._2)
+        }
+
+      case sfc @ ScalarFunctionCall(clazz, args) =>
+        args.foldLeft((aggNames, propNames)){
+          (x, y) => identifyAggregationsAndProperties(y, tableEnv, x._1, x._2)
+        }
+
+      // General expression
+      case e: Expression =>
+        e.productIterator.foldLeft((aggNames, propNames)){
+          (x, y) => y match {
+            case e: Expression => identifyAggregationsAndProperties(e, tableEnv, x._1, x._2)
+            case _ => (x._1, x._2)
+          }
+        }
+    }
+  }
+
+  /**
+    * Replaces expressions with deduplicated aggregations and properties.
+    *
+    * @param exprs     a list of expressions to replace
+    * @param tableEnv  the TableEnvironment
+    * @param aggNames  the deduplicated aggregations
+    * @param propNames the deduplicated properties
+    * @return a list of replaced expressions
+    */
+  def replaceAggregationsAndProperties(
+      exprs: Seq[Expression],
+      tableEnv: TableEnvironment,
+      aggNames: Map[Expression, String],
+      propNames: Map[Expression, String]): Seq[NamedExpression] = {
+    exprs.map(replaceAggregationsAndProperties(_, tableEnv, aggNames, propNames))
+        .map(UnresolvedAlias)
+  }
+
+  private def replaceAggregationsAndProperties(
+      exp: Expression,
+      tableEnv: TableEnvironment,
+      aggNames: Map[Expression, String],
+      propNames: Map[Expression, String]) : Expression = {
+
+    exp match {
+      case agg: Aggregation =>
+        val name = aggNames(agg)
+        Alias(UnresolvedFieldReference(name), tableEnv.createUniqueAttributeName())
+      case prop: WindowProperty =>
+        val name = propNames(prop)
+        Alias(UnresolvedFieldReference(name), tableEnv.createUniqueAttributeName())
+      case n @ Alias(agg: Aggregation, name, _) =>
+        val aName = aggNames(agg)
+        Alias(UnresolvedFieldReference(aName), name)
+      case n @ Alias(prop: WindowProperty, name, _) =>
+        val pName = propNames(prop)
+        Alias(UnresolvedFieldReference(pName), name)
+      case l: LeafExpression => l
+      case u: UnaryExpression =>
+        val c = replaceAggregationsAndProperties(u.child, tableEnv, aggNames, propNames)
+        u.makeCopy(Array(c))
+      case b: BinaryExpression =>
+        val l = replaceAggregationsAndProperties(b.left, tableEnv, aggNames, propNames)
+        val r = replaceAggregationsAndProperties(b.right, tableEnv, aggNames, propNames)
+        b.makeCopy(Array(l, r))
+
+      // Functions calls
+      case c @ Call(name, args) =>
+        val newArgs = args.map(replaceAggregationsAndProperties(_, tableEnv, aggNames, propNames))
+        c.makeCopy(Array(name, newArgs))
+
+      case sfc @ ScalarFunctionCall(clazz, args) =>
+        val newArgs: Seq[Expression] = args
+          .map(replaceAggregationsAndProperties(_, tableEnv, aggNames, propNames))
+        sfc.makeCopy(Array(clazz, newArgs))
+
+      // array constructor
+      case c @ ArrayConstructor(args) =>
+        val newArgs = c.elements
+          .map(replaceAggregationsAndProperties(_, tableEnv, aggNames, propNames))
+        c.makeCopy(Array(newArgs))
+
+      // General expression
+      case e: Expression =>
+        val newArgs = e.productIterator.map {
+          case arg: Expression =>
+            replaceAggregationsAndProperties(arg, tableEnv, aggNames, propNames)
+        }
+        e.makeCopy(newArgs.toArray)
+    }
+  }
+
+  /**
+    * Expands an UnresolvedFieldReference("*") to parent's full project list.
+    */
+  def expandProjectList(
+      exprs: Seq[Expression],
+      parent: LogicalNode,
+      tableEnv: TableEnvironment)
+    : Seq[Expression] = {
+
+    val projectList = new ListBuffer[Expression]
+
+    exprs.foreach {
+      case n: UnresolvedFieldReference if n.name == "*" =>
+        projectList ++= parent.output.map(a => UnresolvedFieldReference(a.name))
+
+      case Flattening(unresolved) =>
+        // simulate a simple project to resolve fields using current parent
+        val project = Project(Seq(UnresolvedAlias(unresolved)), parent).validate(tableEnv)
+        val resolvedExpr = project
+          .output
+          .headOption
+          .getOrElse(throw new RuntimeException("Could not find resolved composite."))
+        resolvedExpr.validateInput()
+        val newProjects = resolvedExpr.resultType match {
+          case ct: CompositeType[_] =>
+            (0 until ct.getArity).map { idx =>
+              projectList += GetCompositeField(unresolved, ct.getFieldNames()(idx))
+            }
+          case _ =>
+            projectList += unresolved
+        }
+
+      case e: Expression => projectList += e
+    }
+    projectList
+  }
+
+  /**
+    * Extract all field references from the given expressions.
+    *
+    * @param exprs a list of expressions to extract
+    * @return a list of field references extracted from the given expressions
+    */
+  def extractFieldReferences(exprs: Seq[Expression]): Seq[NamedExpression] = {
+    exprs.foldLeft(Set[NamedExpression]()) {
+      (fieldReferences, expr) => identifyFieldReferences(expr, fieldReferences)
+    }.toSeq
+  }
+
+  private def identifyFieldReferences(
+      expr: Expression,
+      fieldReferences: Set[NamedExpression]): Set[NamedExpression] = expr match {
+
+    case f: UnresolvedFieldReference =>
+      fieldReferences + UnresolvedAlias(f)
+
+    case b: BinaryExpression =>
+      val l = identifyFieldReferences(b.left, fieldReferences)
+      identifyFieldReferences(b.right, l)
+
+    // Functions calls
+    case c @ Call(name, args) =>
+      args.foldLeft(fieldReferences) {
+        (fieldReferences, expr) => identifyFieldReferences(expr, fieldReferences)
+      }
+    case sfc @ ScalarFunctionCall(clazz, args) =>
+      args.foldLeft(fieldReferences) {
+        (fieldReferences, expr) => identifyFieldReferences(expr, fieldReferences)
+      }
+
+    // array constructor
+    case c @ ArrayConstructor(args) =>
+      args.foldLeft(fieldReferences) {
+        (fieldReferences, expr) => identifyFieldReferences(expr, fieldReferences)
+      }
+
+    // ignore fields from window property
+    case w : WindowProperty =>
+      fieldReferences
+
+    // keep this case after all unwanted unary expressions
+    case u: UnaryExpression =>
+      identifyFieldReferences(u.child, fieldReferences)
+
+    // General expression
+    case e: Expression =>
+      e.productIterator.foldLeft(fieldReferences) {
+        (fieldReferences, expr) => expr match {
+          case e: Expression => identifyFieldReferences(e, fieldReferences)
+          case _ => fieldReferences
+        }
+      }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/TreeNode.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/TreeNode.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/TreeNode.scala
new file mode 100644
index 0000000..fdf45e7
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/TreeNode.scala
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.plan
+
+import org.apache.commons.lang.ClassUtils
+
+/**
+ * Generic base class for trees that can be transformed and traversed.
+ */
+abstract class TreeNode[A <: TreeNode[A]] extends Product { self: A =>
+
+  /**
+   * List of child nodes that should be considered when doing transformations. Other values
+   * in the Product will not be transformed, only handed through.
+   */
+  private[flink] def children: Seq[A]
+
+  /**
+   * Tests for equality by first testing for reference equality.
+   */
+  private[flink] def fastEquals(other: TreeNode[_]): Boolean = this.eq(other) || this == other
+
+  /**
+    * Do tree transformation in post order.
+    */
+  private[flink] def postOrderTransform(rule: PartialFunction[A, A]): A = {
+    def childrenTransform(rule: PartialFunction[A, A]): A = {
+      var changed = false
+      val newArgs = productIterator.map {
+        case arg: TreeNode[_] if children.contains(arg) =>
+          val newChild = arg.asInstanceOf[A].postOrderTransform(rule)
+          if (!(newChild fastEquals arg)) {
+            changed = true
+            newChild
+          } else {
+            arg
+          }
+        case args: Traversable[_] => args.map {
+          case arg: TreeNode[_] if children.contains(arg) =>
+            val newChild = arg.asInstanceOf[A].postOrderTransform(rule)
+            if (!(newChild fastEquals arg)) {
+              changed = true
+              newChild
+            } else {
+              arg
+            }
+          case other => other
+        }
+        case nonChild: AnyRef => nonChild
+        case null => null
+      }.toArray
+      if (changed) makeCopy(newArgs) else this
+    }
+
+    val afterChildren = childrenTransform(rule)
+    if (afterChildren fastEquals this) {
+      rule.applyOrElse(this, identity[A])
+    } else {
+      rule.applyOrElse(afterChildren, identity[A])
+    }
+  }
+
+  /**
+    * Runs the given function first on the node and then recursively on all its children.
+    */
+  private[flink] def preOrderVisit(f: A => Unit): Unit = {
+    f(this)
+    children.foreach(_.preOrderVisit(f))
+  }
+
+  /**
+   * Creates a new copy of this expression with new children. This is used during transformation
+   * if children change.
+   */
+  private[flink] def makeCopy(newArgs: Array[AnyRef]): A = {
+    val ctors = getClass.getConstructors.filter(_.getParameterTypes.size > 0)
+    if (ctors.isEmpty) {
+      throw new RuntimeException(s"No valid constructor for ${getClass.getSimpleName}")
+    }
+
+    val defaultCtor = ctors.find { ctor =>
+      if (ctor.getParameterTypes.size != newArgs.length) {
+        false
+      } else if (newArgs.contains(null)) {
+        false
+      } else {
+        val argsClasses: Array[Class[_]] = newArgs.map(_.getClass)
+        ClassUtils.isAssignable(argsClasses, ctor.getParameterTypes)
+      }
+    }.getOrElse(ctors.maxBy(_.getParameterTypes.size))
+
+    try {
+      defaultCtor.newInstance(newArgs: _*).asInstanceOf[A]
+    } catch {
+      case e: Throwable =>
+        throw new RuntimeException(
+          s"Fail to copy treeNode ${getClass.getName}: ${e.getStackTraceString}")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/cost/DataSetCost.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/cost/DataSetCost.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/cost/DataSetCost.scala
new file mode 100644
index 0000000..7b439ec
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/cost/DataSetCost.scala
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.cost
+
+import org.apache.calcite.plan.{RelOptUtil, RelOptCostFactory, RelOptCost}
+import org.apache.calcite.util.Util
+
+/**
+  * This class is based on Apache Calcite's `org.apache.calcite.plan.volcano.VolcanoCost` and has
+  * an adapted cost comparison method `isLe(other: RelOptCost)` that takes io and cpu into account.
+  */
+class DataSetCost(val rowCount: Double, val cpu: Double, val io: Double) extends RelOptCost {
+
+  def getCpu: Double = cpu
+
+  def isInfinite: Boolean = {
+    (this eq DataSetCost.Infinity) ||
+      (this.rowCount == Double.PositiveInfinity) ||
+      (this.cpu == Double.PositiveInfinity) ||
+      (this.io == Double.PositiveInfinity)
+  }
+
+  def getIo: Double = io
+
+  def isLe(other: RelOptCost): Boolean = {
+    val that: DataSetCost = other.asInstanceOf[DataSetCost]
+    (this eq that) ||
+      (this.io < that.io) ||
+      (this.io == that.io && this.cpu < that.cpu) ||
+      (this.io == that.io && this.cpu == that.cpu && this.rowCount < that.rowCount)
+  }
+
+  def isLt(other: RelOptCost): Boolean = {
+    isLe(other) && !(this == other)
+  }
+
+  def getRows: Double = rowCount
+
+  override def hashCode: Int = Util.hashCode(rowCount) + Util.hashCode(cpu) + Util.hashCode(io)
+
+  def equals(other: RelOptCost): Boolean = {
+    (this eq other) ||
+      other.isInstanceOf[DataSetCost] &&
+        (this.rowCount == other.asInstanceOf[DataSetCost].rowCount) &&
+        (this.cpu == other.asInstanceOf[DataSetCost].cpu) &&
+        (this.io == other.asInstanceOf[DataSetCost].io)
+  }
+
+  def isEqWithEpsilon(other: RelOptCost): Boolean = {
+    if (!other.isInstanceOf[DataSetCost]) {
+      return false
+    }
+    val that: DataSetCost = other.asInstanceOf[DataSetCost]
+    (this eq that) ||
+      ((Math.abs(this.rowCount - that.rowCount) < RelOptUtil.EPSILON) &&
+        (Math.abs(this.cpu - that.cpu) < RelOptUtil.EPSILON) &&
+        (Math.abs(this.io - that.io) < RelOptUtil.EPSILON))
+  }
+
+  def minus(other: RelOptCost): RelOptCost = {
+    if (this eq DataSetCost.Infinity) {
+      return this
+    }
+    val that: DataSetCost = other.asInstanceOf[DataSetCost]
+    new DataSetCost(this.rowCount - that.rowCount, this.cpu - that.cpu, this.io - that.io)
+  }
+
+  def multiplyBy(factor: Double): RelOptCost = {
+    if (this eq DataSetCost.Infinity) {
+      return this
+    }
+    new DataSetCost(rowCount * factor, cpu * factor, io * factor)
+  }
+
+  def divideBy(cost: RelOptCost): Double = {
+    val that: DataSetCost = cost.asInstanceOf[DataSetCost]
+    var d: Double = 1
+    var n: Double = 0
+    if ((this.rowCount != 0) && !this.rowCount.isInfinite &&
+      (that.rowCount != 0) && !that.rowCount.isInfinite)
+    {
+      d *= this.rowCount / that.rowCount
+      n += 1
+    }
+    if ((this.cpu != 0) && !this.cpu.isInfinite && (that.cpu != 0) && !that.cpu.isInfinite) {
+      d *= this.cpu / that.cpu
+      n += 1
+    }
+    if ((this.io != 0) && !this.io.isInfinite && (that.io != 0) && !that.io.isInfinite) {
+      d *= this.io / that.io
+      n += 1
+    }
+    if (n == 0) {
+      return 1.0
+    }
+    Math.pow(d, 1 / n)
+  }
+
+  def plus(other: RelOptCost): RelOptCost = {
+    val that: DataSetCost = other.asInstanceOf[DataSetCost]
+    if ((this eq DataSetCost.Infinity) || (that eq DataSetCost.Infinity)) {
+      return DataSetCost.Infinity
+    }
+    new DataSetCost(this.rowCount + that.rowCount, this.cpu + that.cpu, this.io + that.io)
+  }
+
+  override def toString: String = s"{$rowCount rows, $cpu cpu, $io io}"
+
+}
+
+object DataSetCost {
+
+  private[flink] val Infinity = new DataSetCost(
+    Double.PositiveInfinity,
+    Double.PositiveInfinity,
+    Double.PositiveInfinity)
+  {
+    override def toString: String = "{inf}"
+  }
+
+  private[flink] val Huge = new DataSetCost(Double.MaxValue, Double.MaxValue, Double.MaxValue) {
+    override def toString: String = "{huge}"
+  }
+
+  private[flink] val Zero = new DataSetCost(0.0, 0.0, 0.0) {
+    override def toString: String = "{0}"
+  }
+
+  private[flink] val Tiny = new DataSetCost(1.0, 1.0, 0.0) {
+    override def toString = "{tiny}"
+  }
+
+  val FACTORY: RelOptCostFactory = new DataSetCostFactory
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/cost/DataSetCostFactory.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/cost/DataSetCostFactory.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/cost/DataSetCostFactory.scala
new file mode 100644
index 0000000..50d3842
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/cost/DataSetCostFactory.scala
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.cost
+
+import org.apache.calcite.plan.{RelOptCost, RelOptCostFactory}
+
+/**
+  * This class is based on Apache Calcite's `org.apache.calcite.plan.volcano.VolcanoCost#Factory`.
+  */
+class DataSetCostFactory extends RelOptCostFactory {
+
+  override def makeCost(dRows: Double, dCpu: Double, dIo: Double): RelOptCost = {
+    new DataSetCost(dRows, dCpu, dIo)
+  }
+
+  override def makeHugeCost: RelOptCost = {
+    DataSetCost.Huge
+  }
+
+  override def makeInfiniteCost: RelOptCost = {
+    DataSetCost.Infinity
+  }
+
+  override def makeTinyCost: RelOptCost = {
+    DataSetCost.Tiny
+  }
+
+  override def makeZeroCost: RelOptCost = {
+    DataSetCost.Zero
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalNode.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalNode.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalNode.scala
new file mode 100644
index 0000000..7a9b08e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalNode.scala
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.plan.logical
+
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.table.plan.TreeNode
+import org.apache.flink.table.api.{TableEnvironment, ValidationException}
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.typeutils.TypeCoercion
+import org.apache.flink.table.validate._
+
+/**
+  * LogicalNode is created and validated as we construct query plan using Table API.
+  *
+  * The main validation procedure is separated into two phases:
+  *
+  * Expressions' resolution and transformation ([[resolveExpressions]]):
+  *
+  * - translate [[UnresolvedFieldReference]] into [[ResolvedFieldReference]]
+  *     using child operator's output
+  * - translate [[Call]](UnresolvedFunction) into solid Expression
+  * - generate alias names for query output
+  * - ....
+  *
+  * LogicalNode validation ([[validate]]):
+  *
+  * - check no [[UnresolvedFieldReference]] exists any more
+  * - check if all expressions have children of needed type
+  * - check each logical operator have desired input
+  *
+  * Once we pass the validation phase, we can safely convert LogicalNode into Calcite's RelNode.
+  */
+abstract class LogicalNode extends TreeNode[LogicalNode] {
+  def output: Seq[Attribute]
+
+  def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = {
+    // resolve references and function calls
+    val exprResolved = expressionPostOrderTransform {
+      case u @ UnresolvedFieldReference(name) =>
+        resolveReference(tableEnv, name).getOrElse(u)
+      case c @ Call(name, children) if c.childrenValid =>
+        tableEnv.getFunctionCatalog.lookupFunction(name, children)
+    }
+
+    exprResolved.expressionPostOrderTransform {
+      case ips: InputTypeSpec if ips.childrenValid =>
+        var changed: Boolean = false
+        val newChildren = ips.expectedTypes.zip(ips.children).map { case (tpe, child) =>
+          val childType = child.resultType
+          if (childType != tpe && TypeCoercion.canSafelyCast(childType, tpe)) {
+            changed = true
+            Cast(child, tpe)
+          } else {
+            child
+          }
+        }.toArray[AnyRef]
+        if (changed) ips.makeCopy(newChildren) else ips
+    }
+  }
+
+  final def toRelNode(relBuilder: RelBuilder): RelNode = construct(relBuilder).build()
+
+  protected[logical] def construct(relBuilder: RelBuilder): RelBuilder
+
+  def validate(tableEnv: TableEnvironment): LogicalNode = {
+    val resolvedNode = resolveExpressions(tableEnv)
+    resolvedNode.expressionPostOrderTransform {
+      case a: Attribute if !a.valid =>
+        val from = children.flatMap(_.output).map(_.name).mkString(", ")
+        failValidation(s"Cannot resolve [${a.name}] given input [$from].")
+
+      case e: Expression if e.validateInput().isFailure =>
+        failValidation(s"Expression $e failed on input check: " +
+          s"${e.validateInput().asInstanceOf[ValidationFailure].message}")
+    }
+  }
+
+  /**
+    * Resolves the given strings to a [[NamedExpression]] using the input from all child
+    * nodes of this LogicalPlan.
+    */
+  def resolveReference(tableEnv: TableEnvironment, name: String): Option[NamedExpression] = {
+    val childrenOutput = children.flatMap(_.output)
+    val candidates = childrenOutput.filter(_.name.equalsIgnoreCase(name))
+    if (candidates.length > 1) {
+      failValidation(s"Reference $name is ambiguous.")
+    } else if (candidates.isEmpty) {
+      None
+    } else {
+      Some(candidates.head.withName(name))
+    }
+  }
+
+  /**
+    * Runs [[postOrderTransform]] with `rule` on all expressions present in this logical node.
+    *
+    * @param rule the rule to be applied to every expression in this logical node.
+    */
+  def expressionPostOrderTransform(rule: PartialFunction[Expression, Expression]): LogicalNode = {
+    var changed = false
+
+    def expressionPostOrderTransform(e: Expression): Expression = {
+      val newExpr = e.postOrderTransform(rule)
+      if (newExpr.fastEquals(e)) {
+        e
+      } else {
+        changed = true
+        newExpr
+      }
+    }
+
+    val newArgs = productIterator.map {
+      case e: Expression => expressionPostOrderTransform(e)
+      case Some(e: Expression) => Some(expressionPostOrderTransform(e))
+      case seq: Traversable[_] => seq.map {
+        case e: Expression => expressionPostOrderTransform(e)
+        case other => other
+      }
+      case r: Resolvable[_] => r.resolveExpressions(e => expressionPostOrderTransform(e))
+      case other: AnyRef => other
+    }.toArray
+
+    if (changed) makeCopy(newArgs) else this
+  }
+
+  protected def failValidation(msg: String): Nothing = {
+    throw new ValidationException(msg)
+  }
+}
+
+abstract class LeafNode extends LogicalNode {
+  override def children: Seq[LogicalNode] = Nil
+}
+
+abstract class UnaryNode extends LogicalNode {
+  def child: LogicalNode
+
+  override def children: Seq[LogicalNode] = child :: Nil
+}
+
+abstract class BinaryNode extends LogicalNode {
+  def left: LogicalNode
+  def right: LogicalNode
+
+  override def children: Seq[LogicalNode] = left :: right :: Nil
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
new file mode 100644
index 0000000..1264566
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/LogicalWindow.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.logical
+
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.expressions.{Expression, WindowReference}
+import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
+
+abstract class LogicalWindow(val alias: Option[Expression]) extends Resolvable[LogicalWindow] {
+
+  def resolveExpressions(resolver: (Expression) => Expression): LogicalWindow = this
+
+  def validate(tableEnv: TableEnvironment): ValidationResult = alias match {
+    case Some(WindowReference(_)) => ValidationSuccess
+    case Some(_) => ValidationFailure("Window reference for window expected.")
+    case None => ValidationSuccess
+  }
+
+  override def toString: String = getClass.getSimpleName
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/Resolvable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/Resolvable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/Resolvable.scala
new file mode 100644
index 0000000..995bac5
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/Resolvable.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.logical
+
+import org.apache.flink.table.expressions.Expression
+
+/**
+  * A class implementing this interface can resolve the expressions of its parameters and
+  * return a new instance with resolved parameters. This is necessary if expression are nested in
+  * a not supported structure. By default, the validation of a logical node can resolve common
+  * structures like `Expression`, `Option[Expression]`, `Traversable[Expression]`.
+  *
+  * See also [[LogicalNode.expressionPostOrderTransform(scala.PartialFunction)]].
+  *
+  * @tparam T class which expression parameters need to be resolved
+  */
+trait Resolvable[T <: AnyRef] {
+
+  /**
+    * An implementing class can resolve its expressions by applying the given resolver
+    * function on its parameters.
+    *
+    * @param resolver function that can resolve an expression
+    * @return class with resolved expression parameters
+    */
+  def resolveExpressions(resolver: (Expression) => Expression): T
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
new file mode 100644
index 0000000..b12e654
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/groupWindows.scala
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.logical
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.table.api.{BatchTableEnvironment, StreamTableEnvironment, TableEnvironment}
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.typeutils.{RowIntervalTypeInfo, TimeIntervalTypeInfo, TypeCoercion}
+import org.apache.flink.table.validate.{ValidationFailure, ValidationResult, ValidationSuccess}
+
+abstract class EventTimeGroupWindow(
+    name: Option[Expression],
+    time: Expression)
+  extends LogicalWindow(name) {
+
+  override def validate(tableEnv: TableEnvironment): ValidationResult = {
+    val valid = super.validate(tableEnv)
+    if (valid.isFailure) {
+        return valid
+    }
+
+    tableEnv match {
+      case _: StreamTableEnvironment =>
+        time match {
+          case RowtimeAttribute() =>
+            ValidationSuccess
+          case _ =>
+            ValidationFailure("Event-time window expects a 'rowtime' time field.")
+      }
+      case _: BatchTableEnvironment =>
+        if (!TypeCoercion.canCast(time.resultType, BasicTypeInfo.LONG_TYPE_INFO)) {
+          ValidationFailure(s"Event-time window expects a time field that can be safely cast " +
+            s"to Long, but is ${time.resultType}")
+        } else {
+          ValidationSuccess
+        }
+    }
+
+  }
+}
+
+abstract class ProcessingTimeGroupWindow(name: Option[Expression]) extends LogicalWindow(name)
+
+// ------------------------------------------------------------------------------------------------
+// Tumbling group windows
+// ------------------------------------------------------------------------------------------------
+
+object TumblingGroupWindow {
+  def validate(tableEnv: TableEnvironment, size: Expression): ValidationResult = size match {
+    case Literal(_, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
+      ValidationSuccess
+    case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
+      ValidationSuccess
+    case _ =>
+      ValidationFailure("Tumbling window expects size literal of type Interval of Milliseconds " +
+        "or Interval of Rows.")
+  }
+}
+
+case class ProcessingTimeTumblingGroupWindow(
+    name: Option[Expression],
+    size: Expression)
+  extends ProcessingTimeGroupWindow(name) {
+
+  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
+    ProcessingTimeTumblingGroupWindow(
+      name.map(resolve),
+      resolve(size))
+
+  override def validate(tableEnv: TableEnvironment): ValidationResult =
+    super.validate(tableEnv).orElse(TumblingGroupWindow.validate(tableEnv, size))
+
+  override def toString: String = s"ProcessingTimeTumblingGroupWindow($name, $size)"
+}
+
+case class EventTimeTumblingGroupWindow(
+    name: Option[Expression],
+    timeField: Expression,
+    size: Expression)
+  extends EventTimeGroupWindow(
+    name,
+    timeField) {
+
+  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
+    EventTimeTumblingGroupWindow(
+      name.map(resolve),
+      resolve(timeField),
+      resolve(size))
+
+  override def validate(tableEnv: TableEnvironment): ValidationResult =
+    super.validate(tableEnv)
+      .orElse(TumblingGroupWindow.validate(tableEnv, size))
+      .orElse(size match {
+        case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
+          ValidationFailure(
+            "Event-time grouping windows on row intervals are currently not supported.")
+        case _ =>
+          ValidationSuccess
+      })
+
+  override def toString: String = s"EventTimeTumblingGroupWindow($name, $timeField, $size)"
+}
+
+// ------------------------------------------------------------------------------------------------
+// Sliding group windows
+// ------------------------------------------------------------------------------------------------
+
+object SlidingGroupWindow {
+  def validate(
+      tableEnv: TableEnvironment,
+      size: Expression,
+      slide: Expression)
+    : ValidationResult = {
+
+    val checkedSize = size match {
+      case Literal(_, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
+        ValidationSuccess
+      case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
+        ValidationSuccess
+      case _ =>
+        ValidationFailure("Sliding window expects size literal of type Interval of " +
+          "Milliseconds or Interval of Rows.")
+    }
+
+    val checkedSlide = slide match {
+      case Literal(_, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
+        ValidationSuccess
+      case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
+        ValidationSuccess
+      case _ =>
+        ValidationFailure("Sliding window expects slide literal of type Interval of " +
+          "Milliseconds or Interval of Rows.")
+    }
+
+    checkedSize
+      .orElse(checkedSlide)
+      .orElse {
+        if (size.resultType != slide.resultType) {
+          ValidationFailure("Sliding window expects same type of size and slide.")
+        } else {
+          ValidationSuccess
+        }
+      }
+  }
+}
+
+case class ProcessingTimeSlidingGroupWindow(
+    name: Option[Expression],
+    size: Expression,
+    slide: Expression)
+  extends ProcessingTimeGroupWindow(name) {
+
+  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
+    ProcessingTimeSlidingGroupWindow(
+      name.map(resolve),
+      resolve(size),
+      resolve(slide))
+
+  override def validate(tableEnv: TableEnvironment): ValidationResult =
+    super.validate(tableEnv).orElse(SlidingGroupWindow.validate(tableEnv, size, slide))
+
+  override def toString: String = s"ProcessingTimeSlidingGroupWindow($name, $size, $slide)"
+}
+
+case class EventTimeSlidingGroupWindow(
+    name: Option[Expression],
+    timeField: Expression,
+    size: Expression,
+    slide: Expression)
+  extends EventTimeGroupWindow(name, timeField) {
+
+  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
+    EventTimeSlidingGroupWindow(
+      name.map(resolve),
+      resolve(timeField),
+      resolve(size),
+      resolve(slide))
+
+  override def validate(tableEnv: TableEnvironment): ValidationResult =
+    super.validate(tableEnv)
+      .orElse(SlidingGroupWindow.validate(tableEnv, size, slide))
+      .orElse(size match {
+        case Literal(_, RowIntervalTypeInfo.INTERVAL_ROWS) =>
+          ValidationFailure(
+            "Event-time grouping windows on row intervals are currently not supported.")
+        case _ =>
+          ValidationSuccess
+      })
+
+  override def toString: String = s"EventTimeSlidingGroupWindow($name, $timeField, $size, $slide)"
+}
+
+// ------------------------------------------------------------------------------------------------
+// Session group windows
+// ------------------------------------------------------------------------------------------------
+
+object SessionGroupWindow {
+
+  def validate(tableEnv: TableEnvironment, gap: Expression): ValidationResult = gap match {
+    case Literal(timeInterval: Long, TimeIntervalTypeInfo.INTERVAL_MILLIS) =>
+      ValidationSuccess
+    case _ =>
+      ValidationFailure(
+        "Session window expects gap literal of type Interval of Milliseconds.")
+  }
+}
+
+case class ProcessingTimeSessionGroupWindow(
+    name: Option[Expression],
+    gap: Expression)
+  extends ProcessingTimeGroupWindow(name) {
+
+  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
+    ProcessingTimeSessionGroupWindow(
+      name.map(resolve),
+      resolve(gap))
+
+  override def validate(tableEnv: TableEnvironment): ValidationResult =
+    super.validate(tableEnv).orElse(SessionGroupWindow.validate(tableEnv, gap))
+
+  override def toString: String = s"ProcessingTimeSessionGroupWindow($name, $gap)"
+}
+
+case class EventTimeSessionGroupWindow(
+    name: Option[Expression],
+    timeField: Expression,
+    gap: Expression)
+  extends EventTimeGroupWindow(
+    name,
+    timeField) {
+
+  override def resolveExpressions(resolve: (Expression) => Expression): LogicalWindow =
+    EventTimeSessionGroupWindow(
+      name.map(resolve),
+      resolve(timeField),
+      resolve(gap))
+
+  override def validate(tableEnv: TableEnvironment): ValidationResult =
+    super.validate(tableEnv).orElse(SessionGroupWindow.validate(tableEnv, gap))
+
+  override def toString: String = s"EventTimeSessionGroupWindow($name, $timeField, $gap)"
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
new file mode 100644
index 0000000..eae42cd
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/operators.scala
@@ -0,0 +1,694 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.plan.logical
+
+import java.lang.reflect.Method
+import java.util
+
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.CorrelationId
+import org.apache.calcite.rel.logical.{LogicalProject, LogicalTableFunctionScan}
+import org.apache.calcite.rex.{RexInputRef, RexNode}
+import org.apache.calcite.tools.RelBuilder
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.operators.join.JoinType
+import org.apache.flink.table._
+import org.apache.flink.table.api.{StreamTableEnvironment, TableEnvironment, UnresolvedException}
+import org.apache.flink.table.calcite.{FlinkRelBuilder, FlinkTypeFactory}
+import org.apache.flink.table.expressions._
+import org.apache.flink.table.functions.TableFunction
+import org.apache.flink.table.functions.utils.TableSqlFunction
+import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils._
+import org.apache.flink.table.plan.schema.FlinkTableFunctionImpl
+import org.apache.flink.table.typeutils.TypeConverter
+import org.apache.flink.table.validate.{ValidationFailure, ValidationSuccess}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+case class Project(projectList: Seq[NamedExpression], child: LogicalNode) extends UnaryNode {
+  override def output: Seq[Attribute] = projectList.map(_.toAttribute)
+
+  override def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = {
+    val afterResolve = super.resolveExpressions(tableEnv).asInstanceOf[Project]
+    val newProjectList =
+      afterResolve.projectList.zipWithIndex.map { case (e, i) =>
+        e match {
+          case u @ UnresolvedAlias(c) => c match {
+            case ne: NamedExpression => ne
+            case expr if !expr.valid => u
+            case c @ Cast(ne: NamedExpression, tp) => Alias(c, s"${ne.name}-$tp")
+            case gcf: GetCompositeField => Alias(gcf, gcf.aliasName().getOrElse(s"_c$i"))
+            case other => Alias(other, s"_c$i")
+          }
+          case _ =>
+            throw new RuntimeException("This should never be called and probably points to a bug.")
+        }
+    }
+    Project(newProjectList, child)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    val resolvedProject = super.validate(tableEnv).asInstanceOf[Project]
+    val names: mutable.Set[String] = mutable.Set()
+
+    def checkName(name: String): Unit = {
+      if (names.contains(name)) {
+        failValidation(s"Duplicate field name $name.")
+      } else if (tableEnv.isInstanceOf[StreamTableEnvironment] && name == "rowtime") {
+        failValidation("'rowtime' cannot be used as field name in a streaming environment.")
+      } else {
+        names.add(name)
+      }
+    }
+
+    resolvedProject.projectList.foreach {
+      case n: Alias =>
+        // explicit name
+        checkName(n.name)
+      case r: ResolvedFieldReference =>
+        // simple field forwarding
+        checkName(r.name)
+      case _ => // Do nothing
+    }
+    resolvedProject
+  }
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    val allAlias = projectList.forall(_.isInstanceOf[Alias])
+    child.construct(relBuilder)
+    if (allAlias) {
+      // Calcite's RelBuilder does not translate identity projects even if they rename fields.
+      //   Add a projection ourselves (will be automatically removed by translation rules).
+      val project = LogicalProject.create(relBuilder.peek(),
+        // avoid AS call
+        projectList.map(_.asInstanceOf[Alias].child.toRexNode(relBuilder)).asJava,
+        projectList.map(_.name).asJava)
+      relBuilder.build()  // pop previous relNode
+      relBuilder.push(project)
+    } else {
+      relBuilder.project(projectList.map(_.toRexNode(relBuilder)): _*)
+    }
+  }
+}
+
+case class AliasNode(aliasList: Seq[Expression], child: LogicalNode) extends UnaryNode {
+  override def output: Seq[Attribute] =
+    throw UnresolvedException("Invalid call to output on AliasNode")
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder =
+    throw UnresolvedException("Invalid call to toRelNode on AliasNode")
+
+  override def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = {
+    if (aliasList.length > child.output.length) {
+      failValidation("Aliasing more fields than we actually have")
+    } else if (!aliasList.forall(_.isInstanceOf[UnresolvedFieldReference])) {
+      failValidation("Alias only accept name expressions as arguments")
+    } else if (!aliasList.forall(_.asInstanceOf[UnresolvedFieldReference].name != "*")) {
+      failValidation("Alias can not accept '*' as name")
+    } else if (tableEnv.isInstanceOf[StreamTableEnvironment] && !aliasList.forall {
+          case UnresolvedFieldReference(name) => name != "rowtime"
+        }) {
+      failValidation("'rowtime' cannot be used as field name in a streaming environment.")
+    } else {
+      val names = aliasList.map(_.asInstanceOf[UnresolvedFieldReference].name)
+      val input = child.output
+      Project(
+        names.zip(input).map { case (name, attr) =>
+          Alias(attr, name)} ++ input.drop(names.length), child)
+    }
+  }
+}
+
+case class Distinct(child: LogicalNode) extends UnaryNode {
+  override def output: Seq[Attribute] = child.output
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    child.construct(relBuilder)
+    relBuilder.distinct()
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
+      failValidation(s"Distinct on stream tables is currently not supported.")
+    }
+    this
+  }
+}
+
+case class Sort(order: Seq[Ordering], child: LogicalNode) extends UnaryNode {
+  override def output: Seq[Attribute] = child.output
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    child.construct(relBuilder)
+    relBuilder.sort(order.map(_.toRexNode(relBuilder)).asJava)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
+      failValidation(s"Sort on stream tables is currently not supported.")
+    }
+    super.validate(tableEnv)
+  }
+}
+
+case class Limit(offset: Int, fetch: Int = -1, child: LogicalNode) extends UnaryNode {
+  override def output: Seq[Attribute] = child.output
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    child.construct(relBuilder)
+    relBuilder.limit(offset, fetch)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
+      failValidation(s"Limit on stream tables is currently not supported.")
+    }
+    if (!child.validate(tableEnv).isInstanceOf[Sort]) {
+      failValidation(s"Limit operator must be preceded by an OrderBy operator.")
+    }
+    if (offset < 0) {
+      failValidation(s"Offset should be greater than or equal to zero.")
+    }
+    super.validate(tableEnv)
+  }
+}
+
+case class Filter(condition: Expression, child: LogicalNode) extends UnaryNode {
+  override def output: Seq[Attribute] = child.output
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    child.construct(relBuilder)
+    relBuilder.filter(condition.toRexNode(relBuilder))
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    val resolvedFilter = super.validate(tableEnv).asInstanceOf[Filter]
+    if (resolvedFilter.condition.resultType != BOOLEAN_TYPE_INFO) {
+      failValidation(s"Filter operator requires a boolean expression as input," +
+        s" but ${resolvedFilter.condition} is of type ${resolvedFilter.condition.resultType}")
+    }
+    resolvedFilter
+  }
+}
+
+case class Aggregate(
+    groupingExpressions: Seq[Expression],
+    aggregateExpressions: Seq[NamedExpression],
+    child: LogicalNode) extends UnaryNode {
+
+  override def output: Seq[Attribute] = {
+    (groupingExpressions ++ aggregateExpressions) map {
+      case ne: NamedExpression => ne.toAttribute
+      case e => Alias(e, e.toString).toAttribute
+    }
+  }
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    child.construct(relBuilder)
+    relBuilder.aggregate(
+      relBuilder.groupKey(groupingExpressions.map(_.toRexNode(relBuilder)).asJava),
+      aggregateExpressions.map {
+        case Alias(agg: Aggregation, name, _) => agg.toAggCall(name)(relBuilder)
+        case _ => throw new RuntimeException("This should never happen.")
+      }.asJava)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
+      failValidation(s"Aggregate on stream tables is currently not supported.")
+    }
+
+    val resolvedAggregate = super.validate(tableEnv).asInstanceOf[Aggregate]
+    val groupingExprs = resolvedAggregate.groupingExpressions
+    val aggregateExprs = resolvedAggregate.aggregateExpressions
+    aggregateExprs.foreach(validateAggregateExpression)
+    groupingExprs.foreach(validateGroupingExpression)
+
+    def validateAggregateExpression(expr: Expression): Unit = expr match {
+      // check no nested aggregation exists.
+      case aggExpr: Aggregation =>
+        aggExpr.children.foreach { child =>
+          child.preOrderVisit {
+            case agg: Aggregation =>
+              failValidation(
+                "It's not allowed to use an aggregate function as " +
+                  "input of another aggregate function")
+            case _ => // OK
+          }
+        }
+      case a: Attribute if !groupingExprs.exists(_.checkEquals(a)) =>
+        failValidation(
+          s"expression '$a' is invalid because it is neither" +
+            " present in group by nor an aggregate function")
+      case e if groupingExprs.exists(_.checkEquals(e)) => // OK
+      case e => e.children.foreach(validateAggregateExpression)
+    }
+
+    def validateGroupingExpression(expr: Expression): Unit = {
+      if (!expr.resultType.isKeyType) {
+        failValidation(
+          s"expression $expr cannot be used as a grouping expression " +
+            "because it's not a valid key type which must be hashable and comparable")
+      }
+    }
+    resolvedAggregate
+  }
+}
+
+case class Minus(left: LogicalNode, right: LogicalNode, all: Boolean) extends BinaryNode {
+  override def output: Seq[Attribute] = left.output
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    left.construct(relBuilder)
+    right.construct(relBuilder)
+    relBuilder.minus(all)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
+      failValidation(s"Minus on stream tables is currently not supported.")
+    }
+
+    val resolvedMinus = super.validate(tableEnv).asInstanceOf[Minus]
+    if (left.output.length != right.output.length) {
+      failValidation(s"Minus two table of different column sizes:" +
+        s" ${left.output.size} and ${right.output.size}")
+    }
+    val sameSchema = left.output.zip(right.output).forall { case (l, r) =>
+      l.resultType == r.resultType
+    }
+    if (!sameSchema) {
+      failValidation(s"Minus two table of different schema:" +
+        s" [${left.output.map(a => (a.name, a.resultType)).mkString(", ")}] and" +
+        s" [${right.output.map(a => (a.name, a.resultType)).mkString(", ")}]")
+    }
+    resolvedMinus
+  }
+}
+
+case class Union(left: LogicalNode, right: LogicalNode, all: Boolean) extends BinaryNode {
+  override def output: Seq[Attribute] = left.output
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    left.construct(relBuilder)
+    right.construct(relBuilder)
+    relBuilder.union(all)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    if (tableEnv.isInstanceOf[StreamTableEnvironment] && !all) {
+      failValidation(s"Union on stream tables is currently not supported.")
+    }
+
+    val resolvedUnion = super.validate(tableEnv).asInstanceOf[Union]
+    if (left.output.length != right.output.length) {
+      failValidation(s"Union two tables of different column sizes:" +
+        s" ${left.output.size} and ${right.output.size}")
+    }
+    val sameSchema = left.output.zip(right.output).forall { case (l, r) =>
+      l.resultType == r.resultType
+    }
+    if (!sameSchema) {
+      failValidation(s"Union two tables of different schema:" +
+        s" [${left.output.map(a => (a.name, a.resultType)).mkString(", ")}] and" +
+        s" [${right.output.map(a => (a.name, a.resultType)).mkString(", ")}]")
+    }
+    resolvedUnion
+  }
+}
+
+case class Intersect(left: LogicalNode, right: LogicalNode, all: Boolean) extends BinaryNode {
+  override def output: Seq[Attribute] = left.output
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    left.construct(relBuilder)
+    right.construct(relBuilder)
+    relBuilder.intersect(all)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    if (tableEnv.isInstanceOf[StreamTableEnvironment]) {
+      failValidation(s"Intersect on stream tables is currently not supported.")
+    }
+
+    val resolvedIntersect = super.validate(tableEnv).asInstanceOf[Intersect]
+    if (left.output.length != right.output.length) {
+      failValidation(s"Intersect two tables of different column sizes:" +
+        s" ${left.output.size} and ${right.output.size}")
+    }
+    // allow different column names between tables
+    val sameSchema = left.output.zip(right.output).forall { case (l, r) =>
+      l.resultType == r.resultType
+    }
+    if (!sameSchema) {
+      failValidation(s"Intersect two tables of different schema:" +
+        s" [${left.output.map(a => (a.name, a.resultType)).mkString(", ")}] and" +
+        s" [${right.output.map(a => (a.name, a.resultType)).mkString(", ")}]")
+    }
+    resolvedIntersect
+  }
+}
+
+case class Join(
+    left: LogicalNode,
+    right: LogicalNode,
+    joinType: JoinType,
+    condition: Option[Expression],
+    correlated: Boolean) extends BinaryNode {
+
+  override def output: Seq[Attribute] = {
+    left.output ++ right.output
+  }
+
+  private case class JoinFieldReference(
+    name: String,
+    resultType: TypeInformation[_],
+    left: LogicalNode,
+    right: LogicalNode) extends Attribute {
+
+    val isFromLeftInput = left.output.map(_.name).contains(name)
+
+    val (indexInInput, indexInJoin) = if (isFromLeftInput) {
+      val indexInLeft = left.output.map(_.name).indexOf(name)
+      (indexInLeft, indexInLeft)
+    } else {
+      val indexInRight = right.output.map(_.name).indexOf(name)
+      (indexInRight, indexInRight + left.output.length)
+    }
+
+    override def toString = s"'$name"
+
+    override def toRexNode(implicit relBuilder: RelBuilder): RexNode = {
+      // look up type of field
+      val fieldType = relBuilder.field(2, if (isFromLeftInput) 0 else 1, name).getType
+      // create a new RexInputRef with index offset
+      new RexInputRef(indexInJoin, fieldType)
+    }
+
+    override def withName(newName: String): Attribute = {
+      if (newName == name) {
+        this
+      } else {
+        JoinFieldReference(newName, resultType, left, right)
+      }
+    }
+  }
+
+  override def resolveExpressions(tableEnv: TableEnvironment): LogicalNode = {
+    val node = super.resolveExpressions(tableEnv).asInstanceOf[Join]
+    val partialFunction: PartialFunction[Expression, Expression] = {
+      case field: ResolvedFieldReference => JoinFieldReference(
+        field.name,
+        field.resultType,
+        left,
+        right)
+    }
+    val resolvedCondition = node.condition.map(_.postOrderTransform(partialFunction))
+    Join(node.left, node.right, node.joinType, resolvedCondition, correlated)
+  }
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    left.construct(relBuilder)
+    right.construct(relBuilder)
+
+    val corSet = mutable.Set[CorrelationId]()
+    if (correlated) {
+      corSet += relBuilder.peek().getCluster.createCorrel()
+    }
+
+    relBuilder.join(
+      TypeConverter.flinkJoinTypeToRelType(joinType),
+      condition.map(_.toRexNode(relBuilder)).getOrElse(relBuilder.literal(true)),
+      corSet.asJava)
+  }
+
+  private def ambiguousName: Set[String] =
+    left.output.map(_.name).toSet.intersect(right.output.map(_.name).toSet)
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    if (tableEnv.isInstanceOf[StreamTableEnvironment]
+      && !right.isInstanceOf[LogicalTableFunctionCall]) {
+      failValidation(s"Join on stream tables is currently not supported.")
+    }
+
+    val resolvedJoin = super.validate(tableEnv).asInstanceOf[Join]
+    if (!resolvedJoin.condition.forall(_.resultType == BOOLEAN_TYPE_INFO)) {
+      failValidation(s"Filter operator requires a boolean expression as input, " + 
+        s"but ${resolvedJoin.condition} is of type ${resolvedJoin.joinType}")
+    } else if (ambiguousName.nonEmpty) {
+      failValidation(s"join relations with ambiguous names: ${ambiguousName.mkString(", ")}")
+    }
+
+    resolvedJoin.condition.foreach(testJoinCondition)
+    resolvedJoin
+  }
+
+  private def testJoinCondition(expression: Expression): Unit = {
+
+    def checkIfJoinCondition(exp : BinaryComparison) = exp.children match {
+        case (x : JoinFieldReference) :: (y : JoinFieldReference) :: Nil
+          if x.isFromLeftInput != y.isFromLeftInput => Unit
+        case x => failValidation(
+          s"Invalid non-join predicate $exp. For non-join predicates use Table#where.")
+      }
+
+    var equiJoinFound = false
+    def validateConditions(exp: Expression, isAndBranch: Boolean): Unit = exp match {
+      case x: And => x.children.foreach(validateConditions(_, isAndBranch))
+      case x: Or => x.children.foreach(validateConditions(_, isAndBranch = false))
+      case x: EqualTo =>
+        if (isAndBranch) {
+          equiJoinFound = true
+        }
+        checkIfJoinCondition(x)
+      case x: BinaryComparison => checkIfJoinCondition(x)
+      case x => failValidation(
+        s"Unsupported condition type: ${x.getClass.getSimpleName}. Condition: $x")
+    }
+
+    validateConditions(expression, isAndBranch = true)
+    if (!equiJoinFound) {
+      failValidation(s"Invalid join condition: $expression. At least one equi-join required.")
+    }
+  }
+}
+
+case class CatalogNode(
+    tableName: String,
+    rowType: RelDataType) extends LeafNode {
+
+  val output: Seq[Attribute] = rowType.getFieldList.asScala.map { field =>
+    ResolvedFieldReference(field.getName, FlinkTypeFactory.toTypeInfo(field.getType))
+  }
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    relBuilder.scan(tableName)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = this
+}
+
+/**
+  * Wrapper for valid logical plans generated from SQL String.
+  */
+case class LogicalRelNode(
+    relNode: RelNode) extends LeafNode {
+
+  val output: Seq[Attribute] = relNode.getRowType.getFieldList.asScala.map { field =>
+    ResolvedFieldReference(field.getName, FlinkTypeFactory.toTypeInfo(field.getType))
+  }
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    relBuilder.push(relNode)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = this
+}
+
+case class WindowAggregate(
+    groupingExpressions: Seq[Expression],
+    window: LogicalWindow,
+    propertyExpressions: Seq[NamedExpression],
+    aggregateExpressions: Seq[NamedExpression],
+    child: LogicalNode)
+  extends UnaryNode {
+
+  override def output: Seq[Attribute] = {
+    (groupingExpressions ++ aggregateExpressions ++ propertyExpressions) map {
+      case ne: NamedExpression => ne.toAttribute
+      case e => Alias(e, e.toString).toAttribute
+    }
+  }
+
+  // resolve references of this operator's parameters
+  override def resolveReference(
+      tableEnv: TableEnvironment,
+      name: String)
+    : Option[NamedExpression] = tableEnv match {
+    // resolve reference to rowtime attribute in a streaming environment
+    case _: StreamTableEnvironment if name == "rowtime" =>
+      Some(RowtimeAttribute())
+    case _ =>
+      window.alias match {
+        // resolve reference to this window's alias
+        case Some(UnresolvedFieldReference(alias)) if name == alias =>
+          // check if reference can already be resolved by input fields
+          val found = super.resolveReference(tableEnv, name)
+          if (found.isDefined) {
+            failValidation(s"Reference $name is ambiguous.")
+          } else {
+            Some(WindowReference(name))
+          }
+        case _ =>
+          // resolve references as usual
+          super.resolveReference(tableEnv, name)
+      }
+  }
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    val flinkRelBuilder = relBuilder.asInstanceOf[FlinkRelBuilder]
+    child.construct(flinkRelBuilder)
+    flinkRelBuilder.aggregate(
+      window,
+      relBuilder.groupKey(groupingExpressions.map(_.toRexNode(relBuilder)).asJava),
+      propertyExpressions.map {
+        case Alias(prop: WindowProperty, name, _) => prop.toNamedWindowProperty(name)(relBuilder)
+        case _ => throw new RuntimeException("This should never happen.")
+      },
+      aggregateExpressions.map {
+        case Alias(agg: Aggregation, name, _) => agg.toAggCall(name)(relBuilder)
+        case _ => throw new RuntimeException("This should never happen.")
+      }.asJava)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    val resolvedWindowAggregate = super.validate(tableEnv).asInstanceOf[WindowAggregate]
+    val groupingExprs = resolvedWindowAggregate.groupingExpressions
+    val aggregateExprs = resolvedWindowAggregate.aggregateExpressions
+    aggregateExprs.foreach(validateAggregateExpression)
+    groupingExprs.foreach(validateGroupingExpression)
+
+    def validateAggregateExpression(expr: Expression): Unit = expr match {
+      // check no nested aggregation exists.
+      case aggExpr: Aggregation =>
+        aggExpr.children.foreach { child =>
+          child.preOrderVisit {
+            case agg: Aggregation =>
+              failValidation(
+                "It's not allowed to use an aggregate function as " +
+                  "input of another aggregate function")
+            case _ => // ok
+          }
+        }
+      case a: Attribute if !groupingExprs.exists(_.checkEquals(a)) =>
+        failValidation(
+          s"Expression '$a' is invalid because it is neither" +
+            " present in group by nor an aggregate function")
+      case e if groupingExprs.exists(_.checkEquals(e)) => // ok
+      case e => e.children.foreach(validateAggregateExpression)
+    }
+
+    def validateGroupingExpression(expr: Expression): Unit = {
+      if (!expr.resultType.isKeyType) {
+        failValidation(
+          s"Expression $expr cannot be used as a grouping expression " +
+            "because it's not a valid key type which must be hashable and comparable")
+      }
+    }
+
+    // validate window
+    resolvedWindowAggregate.window.validate(tableEnv) match {
+      case ValidationFailure(msg) =>
+        failValidation(s"$window is invalid: $msg")
+      case ValidationSuccess => // ok
+    }
+
+    resolvedWindowAggregate
+  }
+}
+
+/**
+  * LogicalNode for calling a user-defined table functions.
+  *
+  * @param functionName function name
+  * @param tableFunction table function to be called (might be overloaded)
+  * @param parameters actual parameters
+  * @param fieldNames output field names
+  * @param child child logical node
+  */
+case class LogicalTableFunctionCall(
+    functionName: String,
+    tableFunction: TableFunction[_],
+    parameters: Seq[Expression],
+    resultType: TypeInformation[_],
+    fieldNames: Array[String],
+    child: LogicalNode)
+  extends UnaryNode {
+
+  private val (_, fieldIndexes, fieldTypes) = getFieldInfo(resultType)
+  private var evalMethod: Method = _
+
+  override def output: Seq[Attribute] = fieldNames.zip(fieldTypes).map {
+    case (n, t) => ResolvedFieldReference(n, t)
+  }
+
+  override def validate(tableEnv: TableEnvironment): LogicalNode = {
+    val node = super.validate(tableEnv).asInstanceOf[LogicalTableFunctionCall]
+    // check if not Scala object
+    checkNotSingleton(tableFunction.getClass)
+    // check if class could be instantiated
+    checkForInstantiation(tableFunction.getClass)
+    // look for a signature that matches the input types
+    val signature = node.parameters.map(_.resultType)
+    val foundMethod = getEvalMethod(tableFunction, signature)
+    if (foundMethod.isEmpty) {
+      failValidation(
+        s"Given parameters of function '$functionName' do not match any signature. \n" +
+          s"Actual: ${signatureToString(signature)} \n" +
+          s"Expected: ${signaturesToString(tableFunction)}")
+    } else {
+      node.evalMethod = foundMethod.get
+    }
+    node
+  }
+
+  override protected[logical] def construct(relBuilder: RelBuilder): RelBuilder = {
+    val fieldIndexes = getFieldInfo(resultType)._2
+    val function = new FlinkTableFunctionImpl(resultType, fieldIndexes, fieldNames, evalMethod)
+    val typeFactory = relBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+    val sqlFunction = TableSqlFunction(
+      tableFunction.toString,
+      tableFunction,
+      resultType,
+      typeFactory,
+      function)
+
+    val scan = LogicalTableFunctionScan.create(
+      relBuilder.peek().getCluster,
+      new util.ArrayList[RelNode](),
+      relBuilder.call(sqlFunction, parameters.map(_.toRexNode(relBuilder)).asJava),
+      function.getElementType(null),
+      function.getRowType(relBuilder.getTypeFactory, null),
+      null)
+
+    relBuilder.push(scan)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala
new file mode 100644
index 0000000..d0d9af4
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/logical/rel/LogicalWindowAggregate.scala
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.logical.rel
+
+import java.util
+
+import org.apache.calcite.plan.{Convention, RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.{Aggregate, AggregateCall}
+import org.apache.calcite.rel.{RelNode, RelShuttle}
+import org.apache.calcite.util.ImmutableBitSet
+import org.apache.flink.table.calcite.{FlinkTypeFactory, FlinkRelBuilder}
+import FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.plan.logical.LogicalWindow
+
+class LogicalWindowAggregate(
+    window: LogicalWindow,
+    namedProperties: Seq[NamedWindowProperty],
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    child: RelNode,
+    indicator: Boolean,
+    groupSet: ImmutableBitSet,
+    groupSets: util.List[ImmutableBitSet],
+    aggCalls: util.List[AggregateCall])
+  extends Aggregate(
+    cluster,
+    traitSet,
+    child,
+    indicator,
+    groupSet,
+    groupSets,
+    aggCalls) {
+
+  def getWindow = window
+
+  def getNamedProperties = namedProperties
+
+  override def copy(
+      traitSet: RelTraitSet,
+      input: RelNode,
+      indicator: Boolean,
+      groupSet: ImmutableBitSet,
+      groupSets: util.List[ImmutableBitSet],
+      aggCalls: util.List[AggregateCall])
+    : Aggregate = {
+
+    new LogicalWindowAggregate(
+      window,
+      namedProperties,
+      cluster,
+      traitSet,
+      input,
+      indicator,
+      groupSet,
+      groupSets,
+      aggCalls)
+  }
+
+  override def accept(shuttle: RelShuttle): RelNode = shuttle.visit(this)
+
+  override def deriveRowType(): RelDataType = {
+    val aggregateRowType = super.deriveRowType()
+    val typeFactory = getCluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+    val builder = typeFactory.builder
+    builder.addAll(aggregateRowType.getFieldList)
+    namedProperties.foreach { namedProp =>
+      builder.add(
+        namedProp.name,
+        typeFactory.createTypeFromTypeInfo(namedProp.property.resultType)
+      )
+    }
+    builder.build()
+  }
+}
+
+object LogicalWindowAggregate {
+
+  def create(
+      window: LogicalWindow,
+      namedProperties: Seq[NamedWindowProperty],
+      aggregate: Aggregate)
+    : LogicalWindowAggregate = {
+
+    val cluster: RelOptCluster = aggregate.getCluster
+    val traitSet: RelTraitSet = cluster.traitSetOf(Convention.NONE)
+    new LogicalWindowAggregate(
+      window,
+      namedProperties,
+      cluster,
+      traitSet,
+      aggregate.getInput,
+      aggregate.indicator,
+      aggregate.getGroupSet,
+      aggregate.getGroupSets,
+      aggregate.getAggCallList)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkAggregate.scala
new file mode 100644
index 0000000..7290594
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkAggregate.scala
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.flink.table.calcite.FlinkRelBuilder
+import FlinkRelBuilder.NamedWindowProperty
+import org.apache.flink.table.runtime.aggregate.AggregateUtil._
+
+import scala.collection.JavaConverters._
+
+trait FlinkAggregate {
+
+  private[flink] def groupingToString(inputType: RelDataType, grouping: Array[Int]): String = {
+
+    val inFields = inputType.getFieldNames.asScala
+    grouping.map( inFields(_) ).mkString(", ")
+  }
+
+  private[flink] def aggregationToString(
+      inputType: RelDataType,
+      grouping: Array[Int],
+      rowType: RelDataType,
+      namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+      namedProperties: Seq[NamedWindowProperty])
+    : String = {
+
+    val inFields = inputType.getFieldNames.asScala
+    val outFields = rowType.getFieldNames.asScala
+
+    val groupStrings = grouping.map( inFields(_) )
+
+    val aggs = namedAggregates.map(_.getKey)
+    val aggStrings = aggs.map( a => s"${a.getAggregation}(${
+      if (a.getArgList.size() > 0) {
+        inFields(a.getArgList.get(0))
+      } else {
+        "*"
+      }
+    })")
+
+    val propStrings = namedProperties.map(_.property.toString)
+
+    (groupStrings ++ aggStrings ++ propStrings).zip(outFields).map {
+      case (f, o) => if (f == o) {
+        f
+      } else {
+        s"$f AS $o"
+      }
+    }.mkString(", ")
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkCalc.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkCalc.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkCalc.scala
new file mode 100644
index 0000000..5ebd3ee
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkCalc.scala
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex.{RexNode, RexProgram}
+import org.apache.flink.api.common.functions.{FlatMapFunction, RichFlatMapFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.codegen.{CodeGenerator, GeneratedFunction}
+import org.apache.flink.table.runtime.FlatMapRunner
+import org.apache.flink.table.typeutils.TypeConverter._
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+trait FlinkCalc {
+
+  private[flink] def functionBody(
+    generator: CodeGenerator,
+    inputType: TypeInformation[Any],
+    rowType: RelDataType,
+    calcProgram: RexProgram,
+    config: TableConfig,
+    expectedType: Option[TypeInformation[Any]]): String = {
+
+    val returnType = determineReturnType(
+      rowType,
+      expectedType,
+      config.getNullCheck,
+      config.getEfficientTypeUsage)
+
+    val condition = calcProgram.getCondition
+    val expandedExpressions = calcProgram.getProjectList.map(
+      expr => calcProgram.expandLocalRef(expr))
+    val projection = generator.generateResultExpression(
+      returnType,
+      rowType.getFieldNames,
+      expandedExpressions)
+
+      // only projection
+      if (condition == null) {
+        s"""
+          |${projection.code}
+          |${generator.collectorTerm}.collect(${projection.resultTerm});
+          |""".stripMargin
+      }
+      else {
+        val filterCondition = generator.generateExpression(
+          calcProgram.expandLocalRef(calcProgram.getCondition))
+        // only filter
+        if (projection == null) {
+          // conversion
+          if (inputType != returnType) {
+            val conversion = generator.generateConverterResultExpression(
+              returnType,
+              rowType.getFieldNames)
+
+            s"""
+              |${filterCondition.code}
+              |if (${filterCondition.resultTerm}) {
+              |  ${conversion.code}
+              |  ${generator.collectorTerm}.collect(${conversion.resultTerm});
+              |}
+              |""".stripMargin
+          }
+          // no conversion
+          else {
+            s"""
+              |${filterCondition.code}
+              |if (${filterCondition.resultTerm}) {
+              |  ${generator.collectorTerm}.collect(${generator.input1Term});
+              |}
+              |""".stripMargin
+          }
+        }
+        // both filter and projection
+        else {
+          s"""
+            |${filterCondition.code}
+            |if (${filterCondition.resultTerm}) {
+            |  ${projection.code}
+            |  ${generator.collectorTerm}.collect(${projection.resultTerm});
+            |}
+            |""".stripMargin
+        }
+      }
+    }
+
+  private[flink] def calcMapFunction(
+      genFunction: GeneratedFunction[FlatMapFunction[Any, Any]]): RichFlatMapFunction[Any, Any] = {
+
+    new FlatMapRunner[Any, Any](
+      genFunction.name,
+      genFunction.code,
+      genFunction.returnType)
+  }
+
+  private[flink] def conditionToString(
+      calcProgram: RexProgram,
+      expression: (RexNode, List[String], Option[List[RexNode]]) => String): String = {
+
+    val cond = calcProgram.getCondition
+    val inFields = calcProgram.getInputRowType.getFieldNames.asScala.toList
+    val localExprs = calcProgram.getExprList.asScala.toList
+
+    if (cond != null) {
+      expression(cond, inFields, Some(localExprs))
+    } else {
+      ""
+    }
+  }
+
+  private[flink] def selectionToString(
+      calcProgram: RexProgram,
+      expression: (RexNode, List[String], Option[List[RexNode]]) => String): String = {
+
+    val proj = calcProgram.getProjectList.asScala.toList
+    val inFields = calcProgram.getInputRowType.getFieldNames.asScala.toList
+    val localExprs = calcProgram.getExprList.asScala.toList
+    val outFields = calcProgram.getOutputRowType.getFieldNames.asScala.toList
+
+    proj
+      .map(expression(_, inFields, Some(localExprs)))
+      .zip(outFields).map { case (e, o) => {
+      if (e != o) {
+        e + " AS " + o
+      } else {
+        e
+      }
+    }
+    }.mkString(", ")
+  }
+
+  private[flink] def calcOpName(
+      calcProgram: RexProgram,
+      expression: (RexNode, List[String], Option[List[RexNode]]) => String) = {
+
+    val conditionStr = conditionToString(calcProgram, expression)
+    val selectionStr = selectionToString(calcProgram, expression)
+
+    s"${if (calcProgram.getCondition != null) {
+      s"where: ($conditionStr), "
+    } else {
+      ""
+    }}select: ($selectionStr)"
+  }
+
+  private[flink] def calcToString(
+      calcProgram: RexProgram,
+      expression: (RexNode, List[String], Option[List[RexNode]]) => String) = {
+
+    val name = calcOpName(calcProgram, expression)
+    s"Calc($name)"
+  }
+}


[04/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/UserDefinedTableFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/UserDefinedTableFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/UserDefinedTableFunctionTest.scala
new file mode 100644
index 0000000..f8d9c92
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/table/UserDefinedTableFunctionTest.scala
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.batch.table
+
+import org.apache.flink.api.java.{DataSet => JDataSet, ExecutionEnvironment => JavaExecutionEnv}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.api.scala.{DataSet, ExecutionEnvironment => ScalaExecutionEnv, _}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils.{PojoTableFunc, TableFunc2, _}
+import org.apache.flink.table.api.{TableEnvironment, Types}
+import org.junit.Test
+import org.mockito.Mockito._
+
+class UserDefinedTableFunctionTest extends TableTestBase {
+
+  @Test
+  def testJavaScalaTableAPIEquality(): Unit = {
+    // mock
+    val ds = mock(classOf[DataSet[Row]])
+    val jDs = mock(classOf[JDataSet[Row]])
+    val typeInfo = new RowTypeInfo(Seq(Types.INT, Types.LONG, Types.STRING): _*)
+    when(ds.javaSet).thenReturn(jDs)
+    when(jDs.getType).thenReturn(typeInfo)
+
+    // Scala environment
+    val env = mock(classOf[ScalaExecutionEnv])
+    val tableEnv = TableEnvironment.getTableEnvironment(env)
+    val in1 = ds.toTable(tableEnv).as('a, 'b, 'c)
+
+    // Java environment
+    val javaEnv = mock(classOf[JavaExecutionEnv])
+    val javaTableEnv = TableEnvironment.getTableEnvironment(javaEnv)
+    val in2 = javaTableEnv.fromDataSet(jDs).as("a, b, c")
+    javaTableEnv.registerTable("MyTable", in2)
+
+    // test cross join
+    val func1 = new TableFunc1
+    javaTableEnv.registerFunction("func1", func1)
+    var scalaTable = in1.join(func1('c) as 's).select('c, 's)
+    var javaTable = in2.join("func1(c).as(s)").select("c, s")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test left outer join
+    scalaTable = in1.leftOuterJoin(func1('c) as 's).select('c, 's)
+    javaTable = in2.leftOuterJoin("as(func1(c), s)").select("c, s")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test overloading
+    scalaTable = in1.join(func1('c, "$") as 's).select('c, 's)
+    javaTable = in2.join("func1(c, '$') as (s)").select("c, s")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test custom result type
+    val func2 = new TableFunc2
+    javaTableEnv.registerFunction("func2", func2)
+    scalaTable = in1.join(func2('c) as ('name, 'len)).select('c, 'name, 'len)
+    javaTable = in2.join("func2(c).as(name, len)").select("c, name, len")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test hierarchy generic type
+    val hierarchy = new HierarchyTableFunction
+    javaTableEnv.registerFunction("hierarchy", hierarchy)
+    scalaTable = in1.join(hierarchy('c) as ('name, 'adult, 'len))
+      .select('c, 'name, 'len, 'adult)
+    javaTable = in2.join("AS(hierarchy(c), name, adult, len)")
+      .select("c, name, len, adult")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test pojo type
+    val pojo = new PojoTableFunc
+    javaTableEnv.registerFunction("pojo", pojo)
+    scalaTable = in1.join(pojo('c))
+      .select('c, 'name, 'age)
+    javaTable = in2.join("pojo(c)")
+      .select("c, name, age")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test with filter
+    scalaTable = in1.join(func2('c) as ('name, 'len))
+      .select('c, 'name, 'len).filter('len > 2)
+    javaTable = in2.join("func2(c) as (name, len)")
+      .select("c, name, len").filter("len > 2")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test with scalar function
+    scalaTable = in1.join(func1('c.substring(2)) as 's)
+      .select('a, 'c, 's)
+    javaTable = in2.join("func1(substring(c, 2)) as (s)")
+      .select("a, c, s")
+    verifyTableEquals(scalaTable, javaTable)
+  }
+
+  @Test
+  def testCrossJoin(): Unit = {
+    val util = batchTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = util.addFunction("func1", new TableFunc1)
+
+    val result1 = table.join(function('c) as 's).select('c, 's)
+
+    val expected1 = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", s"$function($$2)"),
+        term("function", function),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "s")
+    )
+
+    util.verifyTable(result1, expected1)
+
+    // test overloading
+
+    val result2 = table.join(function('c, "$") as 's).select('c, 's)
+
+    val expected2 = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", s"$function($$2, '$$')"),
+        term("function", function),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "s")
+    )
+
+    util.verifyTable(result2, expected2)
+  }
+
+  @Test
+  def testLeftOuterJoin(): Unit = {
+    val util = batchTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = util.addFunction("func1", new TableFunc1)
+
+    val result = table.leftOuterJoin(function('c) as 's).select('c, 's)
+
+    val expected = unaryNode(
+      "DataSetCalc",
+      unaryNode(
+        "DataSetCorrelate",
+        batchTableNode(0),
+        term("invocation", s"$function($$2)"),
+        term("function", function),
+        term("rowType",
+          "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
+        term("joinType", "LEFT")
+      ),
+      term("select", "c", "s")
+    )
+
+    util.verifyTable(result, expected)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/utils/SortTestUtils.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/utils/SortTestUtils.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/utils/SortTestUtils.scala
new file mode 100644
index 0000000..ef425d3
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/utils/SortTestUtils.scala
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.batch.utils
+
+object SortTestUtils {
+
+  val tupleDataSetStrings = List((1, 1L, "Hi")
+    ,(2, 2L, "Hello")
+    ,(3, 2L, "Hello world")
+    ,(4, 3L, "Hello world, how are you?")
+    ,(5, 3L, "I am fine.")
+    ,(6, 3L, "Luke Skywalker")
+    ,(7, 4L, "Comment#1")
+    ,(8, 4L, "Comment#2")
+    ,(9, 4L, "Comment#3")
+    ,(10, 4L, "Comment#4")
+    ,(11, 5L, "Comment#5")
+    ,(12, 5L, "Comment#6")
+    ,(13, 5L, "Comment#7")
+    ,(14, 5L, "Comment#8")
+    ,(15, 5L, "Comment#9")
+    ,(16, 6L, "Comment#10")
+    ,(17, 6L, "Comment#11")
+    ,(18, 6L, "Comment#12")
+    ,(19, 6L, "Comment#13")
+    ,(20, 6L, "Comment#14")
+    ,(21, 6L, "Comment#15"))
+
+  def sortExpectedly(dataSet: List[Product])
+                    (implicit ordering: Ordering[Product]): String = 
+    sortExpectedly(dataSet, 0, dataSet.length)
+
+  def sortExpectedly(dataSet: List[Product], start: Int, end: Int)
+                    (implicit ordering: Ordering[Product]): String = {
+    dataSet
+      .sorted(ordering)
+      .slice(start, end)
+      .mkString("\n")
+      .replaceAll("[\\(\\)]", "")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/utils/TableProgramsTestBase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/utils/TableProgramsTestBase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/utils/TableProgramsTestBase.scala
new file mode 100644
index 0000000..45315d6
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/utils/TableProgramsTestBase.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.api.scala.batch.utils
+
+import java.util
+
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.api.scala.batch.utils.TableProgramsTestBase.{EFFICIENT, NO_NULL, TableConfigMode}
+import org.apache.flink.test.util.MultipleProgramsTestBase
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.junit.runners.Parameterized
+
+import scala.collection.JavaConversions._
+
+class TableProgramsTestBase(
+    mode: TestExecutionMode,
+    tableConfigMode: TableConfigMode)
+  extends MultipleProgramsTestBase(mode) {
+
+  def config: TableConfig = {
+    val conf = new TableConfig
+    tableConfigMode match {
+      case NO_NULL =>
+        conf.setNullCheck(false)
+      case EFFICIENT =>
+        conf.setEfficientTypeUsage(true)
+      case _ => // keep default
+    }
+    conf
+  }
+}
+
+object TableProgramsTestBase {
+  case class TableConfigMode(nullCheck: Boolean, efficientTypes: Boolean)
+
+  val DEFAULT = TableConfigMode(nullCheck = true, efficientTypes = false)
+  val NO_NULL = TableConfigMode(nullCheck = false, efficientTypes = false)
+  val EFFICIENT = TableConfigMode(nullCheck = false, efficientTypes = true)
+
+  @Parameterized.Parameters(name = "Execution mode = {0}, Table config = {1}")
+  def parameters(): util.Collection[Array[java.lang.Object]] = {
+    Seq[Array[AnyRef]](Array(TestExecutionMode.COLLECTION, DEFAULT))
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/ExplainStreamTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/ExplainStreamTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/ExplainStreamTest.scala
new file mode 100644
index 0000000..7b168bb
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/ExplainStreamTest.scala
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.TableEnvironment
+import org.junit.Assert.assertEquals
+import org.junit._
+
+class ExplainStreamTest
+  extends StreamingMultipleProgramsTestBase {
+
+  val testFilePath = ExplainStreamTest.this.getClass.getResource("/").getFile
+
+  @Test
+  def testFilter(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val table = env.fromElements((1, "hello"))
+      .toTable(tEnv, 'a, 'b)
+      .filter("a % 2 = 0")
+
+    val result = replaceString(tEnv.explain(table))
+
+    val source = scala.io.Source.fromFile(testFilePath +
+      "../../src/test/scala/resources/testFilterStream0.out").mkString
+    val expect = replaceString(source)
+    assertEquals(result, expect)
+  }
+
+  @Test
+  def testUnion(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    val table1 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
+    val table2 = env.fromElements((1, "hello")).toTable(tEnv, 'count, 'word)
+    val table = table1.unionAll(table2)
+
+    val result = replaceString(tEnv.explain(table))
+
+    val source = scala.io.Source.fromFile(testFilePath +
+      "../../src/test/scala/resources/testUnionStream0.out").mkString
+    val expect = replaceString(source)
+    assertEquals(result, expect)
+  }
+
+  def replaceString(s: String): String = {
+    /* Stage {id} is ignored, because id keeps incrementing in test class
+     * while StreamExecutionEnvironment is up
+     */
+    s.replaceAll("\\r\\n", "\n").replaceAll("Stage \\d+", "")
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
new file mode 100644
index 0000000..cdc4329
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSinkITCase.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream
+
+import java.io.File
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.StreamTestData
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.sinks.CsvTableSink
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.test.util.TestBaseUtils
+import org.junit.Test
+
+class TableSinkITCase extends StreamingMultipleProgramsTestBase {
+
+  @Test
+  def testStreamTableSink(): Unit = {
+
+    val tmpFile = File.createTempFile("flink-table-sink-test", ".tmp")
+    tmpFile.deleteOnExit()
+    val path = tmpFile.toURI.toString
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    env.setParallelism(4)
+
+    val input = StreamTestData.get3TupleDataStream(env)
+      .map(x => x).setParallelism(4) // increase DOP to 4
+
+    val results = input.toTable(tEnv, 'a, 'b, 'c)
+      .where('a < 5 || 'a > 17)
+      .select('c, 'b)
+      .writeToSink(new CsvTableSink(path))
+
+    env.execute()
+
+    val expected = Seq(
+      "Hi,1", "Hello,2", "Hello world,2", "Hello world, how are you?,3",
+      "Comment#12,6", "Comment#13,6", "Comment#14,6", "Comment#15,6").mkString("\n")
+
+    TestBaseUtils.compareResultsByLinesInMemory(expected, path)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
new file mode 100644
index 0000000..ce910db
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/TableSourceITCase.scala
@@ -0,0 +1,193 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream
+
+import java.io.{File, FileOutputStream, OutputStreamWriter}
+
+import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala.stream.utils.StreamITCase
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.sources.{CsvTableSource, StreamTableSource}
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.streaming.api.environment
+import org.apache.flink.streaming.api.functions.source.SourceFunction
+import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.types.Row
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.mutable
+
+class TableSourceITCase extends StreamingMultipleProgramsTestBase {
+
+  @Test
+  def testStreamTableSourceTableAPI(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    tEnv.registerTableSource("MyTestTable", new TestStreamTableSource(33))
+    tEnv.ingest("MyTestTable")
+      .where('amount < 4)
+      .select('amount * 'id, 'name)
+      .toDataStream[Row]
+      .addSink(new StreamITCase.StringSink)
+
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "0,Record_0", "0,Record_16", "0,Record_32",
+      "1,Record_1", "17,Record_17", "36,Record_18",
+      "4,Record_2", "57,Record_19", "9,Record_3")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testStreamTableSourceSQL(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    tEnv.registerTableSource("MyTestTable", new TestStreamTableSource(33))
+    tEnv.sql(
+      "SELECT amount * id, name FROM MyTestTable WHERE amount < 4")
+      .toDataStream[Row]
+      .addSink(new StreamITCase.StringSink)
+
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "0,Record_0", "0,Record_16", "0,Record_32",
+      "1,Record_1", "17,Record_17", "36,Record_18",
+      "4,Record_2", "57,Record_19", "9,Record_3")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testCsvTableSource(): Unit = {
+
+    val csvRecords = Seq(
+      "First#Id#Score#Last",
+      "Mike#1#12.3#Smith",
+      "Bob#2#45.6#Taylor",
+      "Sam#3#7.89#Miller",
+      "Peter#4#0.12#Smith",
+      "% Just a comment",
+      "Liz#5#34.5#Williams",
+      "Sally#6#6.78#Miller",
+      "Alice#7#90.1#Smith",
+      "Kelly#8#2.34#Williams"
+    )
+
+    val tempFile = File.createTempFile("csv-test", "tmp")
+    tempFile.deleteOnExit()
+    val tmpWriter = new OutputStreamWriter(new FileOutputStream(tempFile), "UTF-8")
+    tmpWriter.write(csvRecords.mkString("$"))
+    tmpWriter.close()
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val csvTable = new CsvTableSource(
+      tempFile.getAbsolutePath,
+      Array("first", "id", "score", "last"),
+      Array(
+        BasicTypeInfo.STRING_TYPE_INFO,
+        BasicTypeInfo.INT_TYPE_INFO,
+        BasicTypeInfo.DOUBLE_TYPE_INFO,
+        BasicTypeInfo.STRING_TYPE_INFO
+      ),
+      fieldDelim = "#",
+      rowDelim = "$",
+      ignoreFirstLine = true,
+      ignoreComments = "%"
+    )
+
+    tEnv.registerTableSource("csvTable", csvTable)
+    tEnv.sql(
+      "SELECT last, score, id FROM csvTable WHERE id < 4 ")
+      .toDataStream[Row]
+      .addSink(new StreamITCase.StringSink)
+
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "Smith,12.3,1",
+      "Taylor,45.6,2",
+      "Miller,7.89,3")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+}
+
+
+class TestStreamTableSource(val numRecords: Int) extends StreamTableSource[Row] {
+
+  val fieldTypes: Array[TypeInformation[_]] = Array(
+    BasicTypeInfo.STRING_TYPE_INFO,
+    BasicTypeInfo.LONG_TYPE_INFO,
+    BasicTypeInfo.INT_TYPE_INFO
+  )
+
+  /** Returns the data of the table as a [[DataStream]]. */
+  override def getDataStream(execEnv: environment.StreamExecutionEnvironment): DataStream[Row] = {
+    execEnv.addSource(new GeneratingSourceFunction(numRecords), getReturnType).setParallelism(1)
+  }
+
+  /** Returns the types of the table fields. */
+  override def getFieldTypes: Array[TypeInformation[_]] = fieldTypes
+
+  /** Returns the names of the table fields. */
+  override def getFieldsNames: Array[String] = Array("name", "id", "amount")
+
+  /** Returns the [[TypeInformation]] for the return type. */
+  override def getReturnType: TypeInformation[Row] = new RowTypeInfo(fieldTypes: _*)
+
+  /** Returns the number of fields of the table. */
+  override def getNumberOfFields: Int = 3
+}
+
+class GeneratingSourceFunction(val num: Long) extends SourceFunction[Row] {
+
+  var running = true
+
+  override def run(ctx: SourceContext[Row]): Unit = {
+    var cnt = 0L
+    while(running && cnt < num) {
+      val out = new Row(3)
+      out.setField(0, s"Record_$cnt")
+      out.setField(1, cnt)
+      out.setField(2, (cnt % 16).toInt)
+
+      ctx.collect(out)
+      cnt += 1
+    }
+  }
+
+  override def cancel(): Unit = {
+    running = false
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
new file mode 100644
index 0000000..97e76fa
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/SqlITCase.scala
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, StreamTestData}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.TableEnvironment
+import org.junit.Assert._
+import org.junit._
+
+import scala.collection.mutable
+
+class SqlITCase extends StreamingMultipleProgramsTestBase {
+
+  /** test selection **/
+  @Test
+  def testSelectExpressionFromTable(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val sqlQuery = "SELECT a * 2, b - 1 FROM MyTable"
+
+    val t = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", t)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("2,0", "4,1", "6,1")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  /** test filtering with registered table **/
+  @Test
+  def testSimpleFilter(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val sqlQuery = "SELECT * FROM MyTable WHERE a = 3"
+
+    val t = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", t)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("3,2,Hello world")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  /** test filtering with registered datastream **/
+  @Test
+  def testDatastreamFilter(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val sqlQuery = "SELECT * FROM MyTable WHERE _1 = 3"
+
+    val t = StreamTestData.getSmall3TupleDataStream(env)
+    tEnv.registerDataStream("MyTable", t)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("3,2,Hello world")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  /** test union with registered tables **/
+  @Test
+  def testUnion(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val sqlQuery = "SELECT * FROM T1 " +
+      "UNION ALL " +
+      "SELECT * FROM T2"
+
+    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("T1", t1)
+    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("T2", t2)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,1,Hi", "1,1,Hi",
+      "2,2,Hello", "2,2,Hello",
+      "3,2,Hello world", "3,2,Hello world")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  /** test union with filter **/
+  @Test
+  def testUnionWithFilter(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val sqlQuery = "SELECT * FROM T1 WHERE a = 3 " +
+      "UNION ALL " +
+      "SELECT * FROM T2 WHERE a = 2"
+
+    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("T1", t1)
+    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("T2", t2)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "2,2,Hello",
+      "3,2,Hello world")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  /** test union of a table and a datastream **/
+  @Test
+  def testUnionTableWithDataSet(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val sqlQuery = "SELECT c FROM T1 WHERE a = 3 " +
+      "UNION ALL " +
+      "SELECT c FROM T2 WHERE a = 2"
+
+    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("T1", t1)
+    val t2 = StreamTestData.get3TupleDataStream(env)
+    tEnv.registerDataStream("T2", t2, 'a, 'b, 'c)
+
+    val result = tEnv.sql(sqlQuery).toDataStream[Row]
+    result.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("Hello", "Hello world")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UserDefinedTableFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UserDefinedTableFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UserDefinedTableFunctionTest.scala
new file mode 100644
index 0000000..58eedd0
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/sql/UserDefinedTableFunctionTest.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.api.scala.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.utils.{HierarchyTableFunction, PojoTableFunc, TableFunc2}
+import org.apache.flink.table.utils._
+import org.apache.flink.table.utils.TableTestUtil._
+import org.junit.Test
+
+class UserDefinedTableFunctionTest extends TableTestBase {
+
+  @Test
+  def testCrossJoin(): Unit = {
+    val util = streamTestUtil()
+    val func1 = new TableFunc1
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    util.addFunction("func1", func1)
+
+    val sqlQuery = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(c)) AS T(s)"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", "func1($cor0.c)"),
+        term("function", func1.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "f0 AS s")
+    )
+
+    util.verifySql(sqlQuery, expected)
+
+    // test overloading
+
+    val sqlQuery2 = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(c, '$')) AS T(s)"
+
+    val expected2 = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", "func1($cor0.c, '$')"),
+        term("function", func1.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "f0 AS s")
+    )
+
+    util.verifySql(sqlQuery2, expected2)
+  }
+
+  @Test
+  def testLeftOuterJoin(): Unit = {
+    val util = streamTestUtil()
+    val func1 = new TableFunc1
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    util.addFunction("func1", func1)
+
+    val sqlQuery = "SELECT c, s FROM MyTable LEFT JOIN LATERAL TABLE(func1(c)) AS T(s) ON TRUE"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", "func1($cor0.c)"),
+        term("function", func1.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
+        term("joinType", "LEFT")
+      ),
+      term("select", "c", "f0 AS s")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testCustomType(): Unit = {
+    val util = streamTestUtil()
+    val func2 = new TableFunc2
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    util.addFunction("func2", func2)
+
+    val sqlQuery = "SELECT c, name, len FROM MyTable, LATERAL TABLE(func2(c)) AS T(name, len)"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", "func2($cor0.c)"),
+        term("function", func2.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
+               "VARCHAR(2147483647) f0, INTEGER f1)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "f0 AS name", "f1 AS len")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testHierarchyType(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = new HierarchyTableFunction
+    util.addFunction("hierarchy", function)
+
+    val sqlQuery = "SELECT c, T.* FROM MyTable, LATERAL TABLE(hierarchy(c)) AS T(name, adult, len)"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", "hierarchy($cor0.c)"),
+        term("function", function.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," +
+               " VARCHAR(2147483647) f0, BOOLEAN f1, INTEGER f2)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "f0 AS name", "f1 AS adult", "f2 AS len")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testPojoType(): Unit = {
+    val util = streamTestUtil()
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = new PojoTableFunc
+    util.addFunction("pojo", function)
+
+    val sqlQuery = "SELECT c, name, age FROM MyTable, LATERAL TABLE(pojo(c))"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", "pojo($cor0.c)"),
+        term("function", function.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," +
+               " INTEGER age, VARCHAR(2147483647) name)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "name", "age")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testFilter(): Unit = {
+    val util = streamTestUtil()
+    val func2 = new TableFunc2
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    util.addFunction("func2", func2)
+
+    val sqlQuery = "SELECT c, name, len FROM MyTable, LATERAL TABLE(func2(c)) AS T(name, len) " +
+      "WHERE len > 2"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", "func2($cor0.c)"),
+        term("function", func2.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
+               "VARCHAR(2147483647) f0, INTEGER f1)"),
+        term("joinType", "INNER"),
+        term("condition", ">($1, 2)")
+      ),
+      term("select", "c", "f0 AS name", "f1 AS len")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+
+  @Test
+  def testScalarFunction(): Unit = {
+    val util = streamTestUtil()
+    val func1 = new TableFunc1
+    util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    util.addFunction("func1", func1)
+
+    val sqlQuery = "SELECT c, s FROM MyTable, LATERAL TABLE(func1(SUBSTRING(c, 2))) AS T(s)"
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", "func1(SUBSTRING($cor0.c, 2))"),
+        term("function", func1.getClass.getCanonicalName),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) f0)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "f0 AS s")
+    )
+
+    util.verifySql(sqlQuery, expected)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala
new file mode 100644
index 0000000..f41cae1
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/AggregationsITCase.scala
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.table
+
+import org.apache.flink.api.scala._
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.scala.stream.table.GroupWindowITCase.TimestampWithEqualWatermark
+import org.apache.flink.table.api.scala.stream.utils.StreamITCase
+import org.apache.flink.table.api.scala._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.functions.AssignerWithPunctuatedWatermarks
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.api.watermark.Watermark
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.TableEnvironment
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.mutable
+
+/**
+  * We only test some aggregations until better testing of constructed DataStream
+  * programs is possible.
+  */
+class AggregationsITCase extends StreamingMultipleProgramsTestBase {
+
+  val data = List(
+    (1L, 1, "Hi"),
+    (2L, 2, "Hello"),
+    (4L, 2, "Hello"),
+    (8L, 3, "Hello world"),
+    (16L, 3, "Hello world"))
+
+  @Test
+  def testProcessingTimeSlidingGroupWindowOverCount(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val stream = env.fromCollection(data)
+    val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Slide over 2.rows every 1.rows)
+      .select('string, 'int.count, 'int.avg)
+
+    val results = windowedTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq("Hello world,1,3", "Hello world,2,3", "Hello,1,2", "Hello,2,2", "Hi,1,1")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testEventTimeSessionGroupWindowOverTime(): 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, 'long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Session withGap 7.milli on 'rowtime)
+      .select('string, 'int.count)
+
+    val results = windowedTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq("Hello world,1", "Hello world,1", "Hello,2", "Hi,1")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testAllProcessingTimeTumblingGroupWindowOverCount(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val stream = env.fromCollection(data)
+    val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+    val windowedTable = table
+      .window(Tumble over 2.rows)
+      .select('int.count)
+
+    val results = windowedTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq("2", "2")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testEventTimeTumblingWindow(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime)
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+
+    val stream = env
+      .fromCollection(data)
+      .assignTimestampsAndWatermarks(new TimestampWithEqualWatermark())
+    val table = stream.toTable(tEnv, 'long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Tumble over 5.milli on 'rowtime as 'w)
+      .select('string, 'int.count, 'int.avg, 'w.start, 'w.end)
+
+    val results = windowedTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq(
+      "Hello world,1,3,1970-01-01 00:00:00.005,1970-01-01 00:00:00.01",
+      "Hello world,1,3,1970-01-01 00:00:00.015,1970-01-01 00:00:00.02",
+      "Hello,2,2,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005",
+      "Hi,1,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.005")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testEventTimeSlidingWindow(): 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, 'long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Slide over 10.milli every 5.milli on 'rowtime as 'w)
+      .select('string, 'int.count, 'w.start, 'w.end, 'w.start)
+
+    val results = windowedTable.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = Seq(
+      "Hello world,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.01,1970-01-01 00:00:00.0",
+      "Hello world,1,1970-01-01 00:00:00.005,1970-01-01 00:00:00.015,1970-01-01 00:00:00.005",
+      "Hello world,1,1970-01-01 00:00:00.01,1970-01-01 00:00:00.02,1970-01-01 00:00:00.01",
+      "Hello world,1,1970-01-01 00:00:00.015,1970-01-01 00:00:00.025,1970-01-01 00:00:00.015",
+      "Hello,2,1969-12-31 23:59:59.995,1970-01-01 00:00:00.005,1969-12-31 23:59:59.995",
+      "Hello,2,1970-01-01 00:00:00.0,1970-01-01 00:00:00.01,1970-01-01 00:00:00.0",
+      "Hi,1,1969-12-31 23:59:59.995,1970-01-01 00:00:00.005,1969-12-31 23:59:59.995",
+      "Hi,1,1970-01-01 00:00:00.0,1970-01-01 00:00:00.01,1970-01-01 00:00:00.0")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+}
+
+object GroupWindowITCase {
+  class TimestampWithEqualWatermark extends AssignerWithPunctuatedWatermarks[(Long, Int, String)] {
+
+    override def checkAndGetNextWatermark(
+        lastElement: (Long, Int, String),
+        extractedTimestamp: Long)
+      : Watermark = {
+      new Watermark(extractedTimestamp)
+    }
+
+    override def extractTimestamp(
+        element: (Long, Int, String),
+        previousElementTimestamp: Long): Long = {
+      element._1
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/CalcITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/CalcITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/CalcITCase.scala
new file mode 100644
index 0000000..f541eb4
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/CalcITCase.scala
@@ -0,0 +1,286 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.table
+
+import org.apache.flink.api.scala._
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, StreamTestData}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.Literal
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.{TableEnvironment, TableException}
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.mutable
+
+class CalcITCase extends StreamingMultipleProgramsTestBase {
+
+  @Test
+  def testSimpleSelectAll(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).select('_1, '_2, '_3)
+
+    val results = ds.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+        "1,1,Hi",
+        "2,2,Hello",
+        "3,2,Hello world")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testSelectFirst(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).select('_1)
+
+    val results = ds.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("1", "2", "3")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testSimpleSelectWithNaming(): Unit = {
+
+    // verify ProjectMergeRule.
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv)
+      .select('_1 as 'a, '_2 as 'b, '_1 as 'c)
+      .select('a, 'b)
+
+    val results = ds.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "1,1", "2,2", "3,2", "4,3", "5,3", "6,3", "7,4",
+      "8,4", "9,4", "10,4", "11,5", "12,5", "13,5", "14,5", "15,5",
+      "16,6", "17,6", "18,6", "19,6", "20,6", "21,6")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testSimpleSelectAllWithAs(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+      .select('a, 'b, 'c)
+
+    val results = ds.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+        "1,1,Hi",
+        "2,2,Hello",
+        "3,2,Hello world")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testAsWithToFewFields(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b)
+
+    val results = ds.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("no")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testAsWithToManyFields(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c, 'd)
+
+    val results = ds.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("no")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test(expected = classOf[TableException])
+  def testAsWithAmbiguousFields(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'b)
+
+    val results = ds.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("no")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+
+  @Test(expected = classOf[TableException])
+  def testOnlyFieldRefInAs(): Unit = {
+
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b as 'c, 'd)
+
+    val results = ds.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("no")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testSimpleFilter(): Unit = {
+    /*
+     * Test simple filter
+     */
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter('a === 3)
+    val results = filterDs.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("3,2,Hello world")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testAllRejectingFilter(): Unit = {
+    /*
+     * Test all-rejecting filter
+     */
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter( Literal(false) )
+    val results = filterDs.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    assertEquals(true, StreamITCase.testResults.isEmpty)
+  }
+
+  @Test
+  def testAllPassingFilter(): Unit = {
+    /*
+     * Test all-passing filter
+     */
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter( Literal(true) )
+    val results = filterDs.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+        "1,1,Hi",
+        "2,2,Hello",
+        "3,2,Hello world")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testFilterOnIntegerTupleField(): Unit = {
+    /*
+     * Test filter on Integer tuple field.
+     */
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter( 'a % 2 === 0 )
+    val results = filterDs.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+      "2,2,Hello", "4,3,Hello world, how are you?",
+      "6,3,Luke Skywalker", "8,4,Comment#2", "10,4,Comment#4",
+      "12,5,Comment#6", "14,5,Comment#8", "16,6,Comment#10",
+      "18,6,Comment#12", "20,6,Comment#14")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testNotEquals(): Unit = {
+    /*
+     * Test filter on Integer tuple field.
+     */
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    StreamITCase.testResults = mutable.MutableList()
+    val ds = StreamTestData.get3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+
+    val filterDs = ds.filter( 'a % 2 !== 0)
+    val results = filterDs.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+    val expected = mutable.MutableList(
+      "1,1,Hi", "3,2,Hello world",
+      "5,3,I am fine.", "7,4,Comment#1", "9,4,Comment#3",
+      "11,5,Comment#5", "13,5,Comment#7", "15,5,Comment#9",
+      "17,6,Comment#11", "19,6,Comment#13", "21,6,Comment#15")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowTest.scala
new file mode 100644
index 0000000..ee24cf7
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/GroupWindowTest.scala
@@ -0,0 +1,735 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.table
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table._
+import org.apache.flink.table.api.{TableException, ValidationException}
+import org.apache.flink.table.expressions.{RowtimeAttribute, WindowReference}
+import org.apache.flink.table.plan.logical._
+import org.apache.flink.table.utils.TableTestBase
+import org.apache.flink.table.utils.TableTestUtil.{streamTableNode, term, unaryNode}
+import org.junit.{Ignore, Test}
+
+class GroupWindowTest extends TableTestBase {
+
+  // batch windows are not supported yet
+  @Test(expected = classOf[ValidationException])
+  def testInvalidBatchWindow(): Unit = {
+    val util = batchTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table
+      .groupBy('string)
+      .window(Session withGap 100.milli as 'string)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidWindowProperty(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table
+      .groupBy('string)
+      .select('string, 'string.start) // property in non windowed table
+  }
+
+  @Test(expected = classOf[TableException])
+  def testInvalidRowtime1(): Unit = {
+    val util = streamTestUtil()
+    // rowtime attribute must not be a field name
+    util.addTable[(Long, Int, String)]('rowtime, 'long, 'int, 'string)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidRowtime2(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table
+      .select('string, 'int as 'rowtime) // rowtime attribute must not be an alias
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidRowtime3(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table.as('rowtime, 'myint, 'mystring) // rowtime attribute must not be an alias
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidRowtime4(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table
+      .groupBy('string)
+      // only rowtime is a valid time attribute in a stream environment
+      .window(Tumble over 50.milli on 'string)
+      .select('string, 'int.count)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidTumblingSize(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table
+      .groupBy('string)
+      .window(Tumble over "WRONG") // string is not a valid interval
+      .select('string, 'int.count)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidSlidingSize(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table
+      .groupBy('string)
+      .window(Slide over "WRONG" every "WRONG") // string is not a valid interval
+      .select('string, 'int.count)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidSlidingSlide(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table
+      .groupBy('string)
+      .window(Slide over 12.rows every 1.minute) // row and time intervals may not be mixed
+      .select('string, 'int.count)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidSessionGap(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table
+      .groupBy('string)
+      .window(Session withGap 10.rows) // row interval is not valid for session windows
+      .select('string, 'int.count)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidWindowAlias1(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table
+      .groupBy('string)
+      .window(Session withGap 100.milli as 1 + 1) // expression instead of a symbol
+      .select('string, 'int.count)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidWindowAlias2(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    table
+      .groupBy('string)
+      .window(Session withGap 100.milli as 'string) // field name "string" is already present
+      .select('string, 'int.count)
+  }
+
+  @Test
+  def testProcessingTimeTumblingGroupWindowOverTime(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Tumble over 50.milli)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int")
+      ),
+      term("groupBy", "string"),
+      term("window", ProcessingTimeTumblingGroupWindow(None, 50.milli)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testProcessingTimeTumblingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Tumble over 2.rows)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int")
+      ),
+      term("groupBy", "string"),
+      term("window", ProcessingTimeTumblingGroupWindow(None, 2.rows)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testEventTimeTumblingGroupWindowOverTime(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Tumble over 5.milli on 'rowtime)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int")
+      ),
+      term("groupBy", "string"),
+      term("window", EventTimeTumblingGroupWindow(None, RowtimeAttribute(), 5.milli)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  @Ignore // see comments in DataStreamAggregate
+  def testEventTimeTumblingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Tumble over 2.rows on 'rowtime)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      streamTableNode(0),
+      term("groupBy", "string"),
+      term("window", EventTimeTumblingGroupWindow(None, RowtimeAttribute(), 2.rows)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testProcessingTimeSlidingGroupWindowOverTime(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Slide over 50.milli every 50.milli)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int")
+      ),
+      term("groupBy", "string"),
+      term("window", ProcessingTimeSlidingGroupWindow(None, 50.milli, 50.milli)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testProcessingTimeSlidingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Slide over 2.rows every 1.rows)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int")
+      ),
+      term("groupBy", "string"),
+      term("window", ProcessingTimeSlidingGroupWindow(None, 2.rows, 1.rows)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testEventTimeSlidingGroupWindowOverTime(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Slide over 8.milli every 10.milli on 'rowtime)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int")
+      ),
+      term("groupBy", "string"),
+      term("window", EventTimeSlidingGroupWindow(None, RowtimeAttribute(), 8.milli, 10.milli)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  @Ignore // see comments in DataStreamAggregate
+  def testEventTimeSlidingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Slide over 2.rows every 1.rows on 'rowtime)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      streamTableNode(0),
+      term("groupBy", "string"),
+      term("window", EventTimeSlidingGroupWindow(None, RowtimeAttribute(), 2.rows, 1.rows)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testEventTimeSessionGroupWindowOverTime(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Session withGap 7.milli on 'rowtime)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int")
+      ),
+      term("groupBy", "string"),
+      term("window", EventTimeSessionGroupWindow(None, RowtimeAttribute(), 7.milli)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testAllProcessingTimeTumblingGroupWindowOverTime(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Tumble over 50.milli)
+      .select('string, 'int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int")
+      ),
+      term("groupBy", "string"),
+      term("window", ProcessingTimeTumblingGroupWindow(None, 50.milli)),
+      term("select", "string", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testAllProcessingTimeTumblingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .window(Tumble over 2.rows)
+      .select('int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "int")
+      ),
+      term("window", ProcessingTimeTumblingGroupWindow(None, 2.rows)),
+      term("select", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testAllEventTimeTumblingGroupWindowOverTime(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .window(Tumble over 5.milli on 'rowtime)
+      .select('int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "int")
+      ),
+      term("window", EventTimeTumblingGroupWindow(None, RowtimeAttribute(), 5.milli)),
+      term("select", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  @Ignore // see comments in DataStreamAggregate
+  def testAllEventTimeTumblingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .window(Tumble over 2.rows on 'rowtime)
+      .select('int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "int")
+      ),
+      term("window", EventTimeTumblingGroupWindow(None, RowtimeAttribute(), 2.rows)),
+      term("select", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+
+  @Test
+  def testAllProcessingTimeSlidingGroupWindowOverTime(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .window(Slide over 50.milli every 50.milli)
+      .select('int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "int")
+      ),
+      term("window", ProcessingTimeSlidingGroupWindow(None, 50.milli, 50.milli)),
+      term("select", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testAllProcessingTimeSlidingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .window(Slide over 2.rows every 1.rows)
+      .select('int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "int")
+      ),
+      term("window", ProcessingTimeSlidingGroupWindow(None, 2.rows, 1.rows)),
+      term("select", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testAllEventTimeSlidingGroupWindowOverTime(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .window(Slide over 8.milli every 10.milli on 'rowtime)
+      .select('int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "int")
+      ),
+      term("window", EventTimeSlidingGroupWindow(None, RowtimeAttribute(), 8.milli, 10.milli)),
+      term("select", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  @Ignore // see comments in DataStreamAggregate
+  def testAllEventTimeSlidingGroupWindowOverCount(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .window(Slide over 2.rows every 1.rows on 'rowtime)
+      .select('int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "int")
+      ),
+      term("window", EventTimeSlidingGroupWindow(None, RowtimeAttribute(), 2.rows, 1.rows)),
+      term("select", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testAllEventTimeSessionGroupWindowOverTime(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .window(Session withGap 7.milli on 'rowtime)
+      .select('int.count)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "int")
+      ),
+      term("window", EventTimeSessionGroupWindow(None, RowtimeAttribute(), 7.milli)),
+      term("select", "COUNT(int) AS TMP_0")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testTumbleWindowStartEnd(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Tumble over 5.milli on 'rowtime as 'w)
+      .select('string, 'int.count, 'w.start, 'w.end)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int")
+      ),
+      term("groupBy", "string"),
+      term("window",
+        EventTimeTumblingGroupWindow(
+          Some(WindowReference("w")),
+          RowtimeAttribute(),
+          5.milli)),
+      term("select",
+        "string",
+        "COUNT(int) AS TMP_0",
+        "start(WindowReference(w)) AS TMP_1",
+        "end(WindowReference(w)) AS TMP_2")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testSlideWindowStartEnd(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Slide over 10.milli every 5.milli on 'rowtime as 'w)
+      .select('string, 'int.count, 'w.start, 'w.end)
+
+    val expected = unaryNode(
+      "DataStreamAggregate",
+      unaryNode(
+        "DataStreamCalc",
+        streamTableNode(0),
+        term("select", "string", "int")
+      ),
+      term("groupBy", "string"),
+      term("window",
+        EventTimeSlidingGroupWindow(
+          Some(WindowReference("w")),
+          RowtimeAttribute(),
+          10.milli,
+          5.milli)),
+      term("select",
+        "string",
+        "COUNT(int) AS TMP_0",
+        "start(WindowReference(w)) AS TMP_1",
+        "end(WindowReference(w)) AS TMP_2")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testSessionWindowStartWithTwoEnd(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Session withGap 3.milli on 'rowtime as 'w)
+      .select('w.end as 'we1, 'string, 'int.count as 'cnt, 'w.start as 'ws, 'w.end as 'we2)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamAggregate",
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(0),
+          term("select", "string", "int")
+        ),
+        term("groupBy", "string"),
+        term("window",
+          EventTimeSessionGroupWindow(
+            Some(WindowReference("w")),
+            RowtimeAttribute(),
+            3.milli)),
+        term("select",
+          "string",
+          "COUNT(int) AS TMP_1",
+          "end(WindowReference(w)) AS TMP_0",
+          "start(WindowReference(w)) AS TMP_2")
+      ),
+      term("select", "TMP_0 AS we1", "string", "TMP_1 AS cnt", "TMP_2 AS ws", "TMP_0 AS we2")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+
+  @Test
+  def testTumbleWindowWithDuplicateAggsAndProps(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Long, Int, String)]('long, 'int, 'string)
+
+    val windowedTable = table
+      .groupBy('string)
+      .window(Tumble over 5.millis on 'rowtime as 'w)
+      .select('string, 'int.sum + 1 as 's1, 'int.sum + 3 as 's2, 'w.start as 'x, 'w.start as 'x2,
+        'w.end as 'x3, 'w.end)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamAggregate",
+        unaryNode(
+          "DataStreamCalc",
+          streamTableNode(0),
+          term("select", "string", "int")
+        ),
+        term("groupBy", "string"),
+        term("window",
+          EventTimeTumblingGroupWindow(
+            Some(WindowReference("w")),
+            RowtimeAttribute(),
+            5.millis)),
+        term("select",
+          "string",
+          "SUM(int) AS TMP_0",
+          "start(WindowReference(w)) AS TMP_1",
+          "end(WindowReference(w)) AS TMP_2")
+      ),
+      term("select",
+        "string",
+        "+(CAST(AS(TMP_0, 'TMP_3')), CAST(1)) AS s1",
+        "+(CAST(AS(TMP_0, 'TMP_4')), CAST(3)) AS s2",
+        "TMP_1 AS x",
+        "TMP_1 AS x2",
+        "TMP_2 AS x3",
+        "TMP_2 AS TMP_5")
+    )
+
+    util.verifyTable(windowedTable, expected)
+  }
+}


[20/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkCorrelate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkCorrelate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkCorrelate.scala
new file mode 100644
index 0000000..fc69493
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkCorrelate.scala
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.plan.nodes
+
+import org.apache.calcite.rel.`type`.RelDataType
+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.table.codegen.{CodeGenerator, GeneratedExpression, GeneratedFunction}
+import org.apache.flink.table.codegen.CodeGenUtils.primitiveDefaultValue
+import org.apache.flink.table.codegen.GeneratedExpression.{ALWAYS_NULL, NO_CODE}
+import org.apache.flink.table.functions.utils.TableSqlFunction
+import org.apache.flink.table.runtime.FlatMapRunner
+import org.apache.flink.table.typeutils.TypeConverter._
+import org.apache.flink.table.api.{TableConfig, TableException}
+
+import scala.collection.JavaConverters._
+
+/**
+  * Join a user-defined table function
+  */
+trait FlinkCorrelate {
+
+  private[flink] def functionBody(
+      generator: CodeGenerator,
+      udtfTypeInfo: TypeInformation[Any],
+      rowType: RelDataType,
+      rexCall: RexCall,
+      condition: Option[RexNode],
+      config: TableConfig,
+      joinType: SemiJoinType,
+      expectedType: Option[TypeInformation[Any]]): String = {
+
+    val returnType = determineReturnType(
+      rowType,
+      expectedType,
+      config.getNullCheck,
+      config.getEfficientTypeUsage)
+
+    val (input1AccessExprs, input2AccessExprs) = generator.generateCorrelateAccessExprs
+
+    val call = generator.generateExpression(rexCall)
+    var body =
+      s"""
+         |${call.code}
+         |java.util.Iterator iter = ${call.resultTerm}.getRowsIterator();
+       """.stripMargin
+
+    if (joinType == SemiJoinType.INNER) {
+      // cross join
+      body +=
+        s"""
+           |if (!iter.hasNext()) {
+           |  return;
+           |}
+        """.stripMargin
+    } else if (joinType == SemiJoinType.LEFT) {
+      // left outer join
+
+      // in case of left outer join and the returned row of table function is empty,
+      // fill all fields of row with null
+      val input2NullExprs = input2AccessExprs.map { x =>
+        GeneratedExpression(
+          primitiveDefaultValue(x.resultType),
+          ALWAYS_NULL,
+          NO_CODE,
+          x.resultType)
+      }
+      val outerResultExpr = generator.generateResultExpression(
+        input1AccessExprs ++ input2NullExprs, returnType, rowType.getFieldNames.asScala)
+      body +=
+        s"""
+           |if (!iter.hasNext()) {
+           |  ${outerResultExpr.code}
+           |  ${generator.collectorTerm}.collect(${outerResultExpr.resultTerm});
+           |  return;
+           |}
+        """.stripMargin
+    } else {
+      throw TableException(s"Unsupported SemiJoinType: $joinType for correlate join.")
+    }
+
+    val crossResultExpr = generator.generateResultExpression(
+      input1AccessExprs ++ input2AccessExprs,
+      returnType,
+      rowType.getFieldNames.asScala)
+
+    val projection = if (condition.isEmpty) {
+      s"""
+         |${crossResultExpr.code}
+         |${generator.collectorTerm}.collect(${crossResultExpr.resultTerm});
+       """.stripMargin
+    } else {
+      val filterGenerator = new CodeGenerator(config, false, udtfTypeInfo)
+      filterGenerator.input1Term = filterGenerator.input2Term
+      val filterCondition = filterGenerator.generateExpression(condition.get)
+      s"""
+         |${filterGenerator.reuseInputUnboxingCode()}
+         |${filterCondition.code}
+         |if (${filterCondition.resultTerm}) {
+         |  ${crossResultExpr.code}
+         |  ${generator.collectorTerm}.collect(${crossResultExpr.resultTerm});
+         |}
+         |""".stripMargin
+    }
+
+    val outputTypeClass = udtfTypeInfo.getTypeClass.getCanonicalName
+    body +=
+      s"""
+         |while (iter.hasNext()) {
+         |  $outputTypeClass ${generator.input2Term} = ($outputTypeClass) iter.next();
+         |  $projection
+         |}
+       """.stripMargin
+    body
+  }
+
+  private[flink] def correlateMapFunction(
+      genFunction: GeneratedFunction[FlatMapFunction[Any, Any]])
+    : FlatMapRunner[Any, Any] = {
+
+    new FlatMapRunner[Any, Any](
+      genFunction.name,
+      genFunction.code,
+      genFunction.returnType)
+  }
+
+  private[flink] def selectToString(rowType: RelDataType): String = {
+    rowType.getFieldNames.asScala.mkString(",")
+  }
+
+  private[flink] def correlateOpName(
+      rexCall: RexCall,
+      sqlFunction: TableSqlFunction,
+      rowType: RelDataType)
+    : String = {
+
+    s"correlate: ${correlateToString(rexCall, sqlFunction)}, select: ${selectToString(rowType)}"
+  }
+
+  private[flink] def correlateToString(rexCall: RexCall, sqlFunction: TableSqlFunction): String = {
+    val udtfName = sqlFunction.getName
+    val operands = rexCall.getOperands.asScala.map(_.toString).mkString(",")
+    s"table($udtfName($operands))"
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkRel.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkRel.scala
new file mode 100644
index 0000000..c8211a2
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/FlinkRel.scala
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes
+
+import org.apache.calcite.rex._
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.runtime.MapRunner
+
+import scala.collection.JavaConversions._
+
+trait FlinkRel {
+
+  private[flink] def getExpressionString(
+    expr: RexNode,
+    inFields: List[String],
+    localExprsTable: Option[List[RexNode]]): String = {
+
+    expr match {
+      case i: RexInputRef =>
+        inFields.get(i.getIndex)
+
+      case l: RexLiteral =>
+        l.toString
+
+      case l: RexLocalRef if localExprsTable.isEmpty =>
+        throw new IllegalArgumentException("Encountered RexLocalRef without " +
+          "local expression table")
+
+      case l: RexLocalRef =>
+        val lExpr = localExprsTable.get(l.getIndex)
+        getExpressionString(lExpr, inFields, localExprsTable)
+
+      case c: RexCall =>
+        val op = c.getOperator.toString
+        val ops = c.getOperands.map(getExpressionString(_, inFields, localExprsTable))
+        s"$op(${ops.mkString(", ")})"
+
+      case fa: RexFieldAccess =>
+        val referenceExpr = getExpressionString(fa.getReferenceExpr, inFields, localExprsTable)
+        val field = fa.getField.getName
+        s"$referenceExpr.$field"
+
+      case _ =>
+        throw new IllegalArgumentException(s"Unknown expression type '${expr.getClass}': $expr")
+    }
+  }
+
+  private[flink] def getConversionMapper(
+      config: TableConfig,
+      nullableInput: Boolean,
+      inputType: TypeInformation[Any],
+      expectedType: TypeInformation[Any],
+      conversionOperatorName: String,
+      fieldNames: Seq[String],
+      inputPojoFieldMapping: Option[Array[Int]] = None)
+    : MapFunction[Any, Any] = {
+
+    val generator = new CodeGenerator(
+      config,
+      nullableInput,
+      inputType,
+      None,
+      inputPojoFieldMapping)
+    val conversion = generator.generateConverterResultExpression(expectedType, fieldNames)
+
+    val body =
+      s"""
+         |${conversion.code}
+         |return ${conversion.resultTerm};
+         |""".stripMargin
+
+    val genFunction = generator.generateFunction(
+      conversionOperatorName,
+      classOf[MapFunction[Any, Any]],
+      body,
+      expectedType)
+
+    new MapRunner[Any, Any](
+      genFunction.name,
+      genFunction.code,
+      genFunction.returnType)
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
new file mode 100644
index 0000000..252bb2e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchScan.scala
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.api.java.typeutils.PojoTypeInfo
+import org.apache.flink.table.api.TableConfig
+import org.apache.flink.table.plan.schema.FlinkTable
+import org.apache.flink.table.typeutils.TypeConverter.determineReturnType
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+abstract class BatchScan(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    table: RelOptTable)
+  extends TableScan(cluster, traitSet, table)
+  with DataSetRel {
+
+  override def toString: String = {
+    s"Source(from: (${getRowType.getFieldNames.asScala.toList.mkString(", ")}))"
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+
+    val rowCnt = metadata.getRowCount(this)
+    planner.getCostFactory.makeCost(rowCnt, rowCnt, 0)
+  }
+
+  protected def convertToExpectedType(
+      input: DataSet[Any],
+      flinkTable: FlinkTable[_],
+      expectedType: Option[TypeInformation[Any]],
+      config: TableConfig): DataSet[Any] = {
+
+    val inputType = input.getType
+
+    expectedType match {
+
+      // special case:
+      // if efficient type usage is enabled and no expected type is set
+      // we can simply forward the DataSet to the next operator.
+      // however, we cannot forward PojoTypes as their fields don't have an order
+      case None if config.getEfficientTypeUsage && !inputType.isInstanceOf[PojoTypeInfo[_]] =>
+        input
+
+      case _ =>
+        val determinedType = determineReturnType(
+          getRowType,
+          expectedType,
+          config.getNullCheck,
+          config.getEfficientTypeUsage)
+
+        // conversion
+        if (determinedType != inputType) {
+
+          val mapFunc = getConversionMapper(
+            config,
+            nullableInput = false,
+            inputType,
+            determinedType,
+            "DataSetSourceConversion",
+            getRowType.getFieldNames,
+            Some(flinkTable.fieldIndexes))
+
+          val opName = s"from: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
+
+          input.map(mapFunc).name(opName)
+        }
+        // no conversion necessary, forward
+        else {
+          input
+        }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala
new file mode 100644
index 0000000..09cb180
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/BatchTableSourceScan.scala
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.calcite.rel.{RelNode, RelWriter}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.api.BatchTableEnvironment
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.schema.TableSourceTable
+import org.apache.flink.table.sources.BatchTableSource
+
+/** Flink RelNode to read data from an external source defined by a [[BatchTableSource]]. */
+class BatchTableSourceScan(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    table: RelOptTable,
+    val tableSource: BatchTableSource[_])
+  extends BatchScan(cluster, traitSet, table) {
+
+  override def deriveRowType() = {
+    val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
+    flinkTypeFactory.buildRowDataType(tableSource.getFieldsNames, tableSource.getFieldTypes)
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+    val rowCnt = metadata.getRowCount(this)
+    planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * estimateRowSize(getRowType))
+  }
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new BatchTableSourceScan(
+      cluster,
+      traitSet,
+      getTable,
+      tableSource
+    )
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .item("fields", tableSource.getFieldsNames.mkString(", "))
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
+
+    val config = tableEnv.getConfig
+    val inputDataSet = tableSource.getDataSet(tableEnv.execEnv).asInstanceOf[DataSet[Any]]
+
+    convertToExpectedType(inputDataSet, new TableSourceTable(tableSource), expectedType, config)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
new file mode 100644
index 0000000..a5c42d9
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.nodes.FlinkAggregate
+import org.apache.flink.table.runtime.aggregate.AggregateUtil
+import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
+import org.apache.flink.table.typeutils.TypeConverter
+import org.apache.flink.table.api.BatchTableEnvironment
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConverters._
+
+/**
+  * Flink RelNode which matches along with a LogicalAggregate.
+  */
+class DataSetAggregate(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputNode: RelNode,
+    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+    rowRelDataType: RelDataType,
+    inputType: RelDataType,
+    grouping: Array[Int])
+  extends SingleRel(cluster, traitSet, inputNode)
+  with FlinkAggregate
+  with DataSetRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataSetAggregate(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      namedAggregates,
+      getRowType,
+      inputType,
+      grouping)
+  }
+
+  override def toString: String = {
+    s"Aggregate(${ if (!grouping.isEmpty) {
+      s"groupBy: (${groupingToString(inputType, grouping)}), "
+    } else {
+      ""
+    }}select: (${aggregationToString(inputType, grouping, getRowType, namedAggregates, Nil)}))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .itemIf("groupBy", groupingToString(inputType, grouping), !grouping.isEmpty)
+      .item("select", aggregationToString(inputType, grouping, getRowType, namedAggregates, Nil))
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+
+    val child = this.getInput
+    val rowCnt = metadata.getRowCount(child)
+    val rowSize = this.estimateRowSize(child.getRowType)
+    val aggCnt = this.namedAggregates.size
+    planner.getCostFactory.makeCost(rowCnt, rowCnt * aggCnt, rowCnt * rowSize)
+  }
+
+  override def translateToPlan(
+    tableEnv: BatchTableEnvironment,
+    expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
+
+    val config = tableEnv.getConfig
+
+    val groupingKeys = grouping.indices.toArray
+
+    val mapFunction = AggregateUtil.createPrepareMapFunction(
+      namedAggregates,
+      grouping,
+      inputType)
+
+    val groupReduceFunction = AggregateUtil.createAggregateGroupReduceFunction(
+      namedAggregates,
+      inputType,
+      rowRelDataType,
+      grouping)
+
+    val inputDS = getInput.asInstanceOf[DataSetRel].translateToPlan(
+      tableEnv,
+      // tell the input operator that this operator currently only supports Rows as input
+      Some(TypeConverter.DEFAULT_ROW_TYPE))
+
+    // get the output types
+    val fieldTypes: Array[TypeInformation[_]] = getRowType.getFieldList.asScala
+    .map(field => FlinkTypeFactory.toTypeInfo(field.getType))
+    .toArray
+
+    val aggString = aggregationToString(inputType, grouping, getRowType, namedAggregates, Nil)
+    val prepareOpName = s"prepare select: ($aggString)"
+    val mappedInput = inputDS
+      .map(mapFunction)
+      .name(prepareOpName)
+
+    val rowTypeInfo = new RowTypeInfo(fieldTypes: _*)
+
+    val result = {
+      if (groupingKeys.length > 0) {
+        // grouped aggregation
+        val aggOpName = s"groupBy: (${groupingToString(inputType, grouping)}), " +
+          s"select: ($aggString)"
+
+        mappedInput.asInstanceOf[DataSet[Row]]
+          .groupBy(groupingKeys: _*)
+          .reduceGroup(groupReduceFunction)
+          .returns(rowTypeInfo)
+          .name(aggOpName)
+          .asInstanceOf[DataSet[Any]]
+      }
+      else {
+        // global aggregation
+        val aggOpName = s"select:($aggString)"
+        mappedInput.asInstanceOf[DataSet[Row]]
+          .reduceGroup(groupReduceFunction)
+          .returns(rowTypeInfo)
+          .name(aggOpName)
+          .asInstanceOf[DataSet[Any]]
+      }
+    }
+
+    // if the expected type is not a Row, inject a mapper to convert to the expected type
+    expectedType match {
+      case Some(typeInfo) if typeInfo.getTypeClass != classOf[Row] =>
+        val mapName = s"convert: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
+        result.map(getConversionMapper(
+          config = config,
+          nullableInput = false,
+          inputType = rowTypeInfo.asInstanceOf[TypeInformation[Any]],
+          expectedType = expectedType.get,
+          conversionOperatorName = "DataSetAggregateConversion",
+          fieldNames = getRowType.getFieldNames.asScala
+        ))
+        .name(mapName)
+      case _ => result
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..03178ad
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCalc.scala
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+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.table.codegen.CodeGenerator
+import org.apache.flink.table.plan.nodes.FlinkCalc
+import org.apache.flink.table.typeutils.TypeConverter
+import TypeConverter._
+import org.apache.calcite.rex._
+import org.apache.flink.table.api.BatchTableEnvironment
+
+import scala.collection.JavaConverters._
+
+/**
+  * Flink RelNode which matches along with LogicalCalc.
+  *
+  */
+class DataSetCalc(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    input: RelNode,
+    rowRelDataType: RelDataType,
+    private[flink] val calcProgram: RexProgram, // for tests
+    ruleDescription: String)
+  extends SingleRel(cluster, traitSet, input)
+  with FlinkCalc
+  with DataSetRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataSetCalc(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      getRowType,
+      calcProgram,
+      ruleDescription)
+  }
+
+  override def toString: String = calcToString(calcProgram, getExpressionString)
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .item("select", selectionToString(calcProgram, getExpressionString))
+      .itemIf("where",
+        conditionToString(calcProgram, getExpressionString),
+        calcProgram.getCondition != null)
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+
+    val child = this.getInput
+    val rowCnt = metadata.getRowCount(child)
+
+    // compute number of expressions that do not access a field or literal, i.e. computations,
+    //   conditions, etc. We only want to account for computations, not for simple projections.
+    val compCnt = calcProgram.getExprList.asScala.toList.count {
+      case i: RexInputRef => false
+      case l: RexLiteral => false
+      case _ => true
+    }
+
+    planner.getCostFactory.makeCost(rowCnt, rowCnt * compCnt, 0)
+  }
+
+  override def estimateRowCount(metadata: RelMetadataQuery): Double = {
+    val child = this.getInput
+    val rowCnt = metadata.getRowCount(child)
+
+    if (calcProgram.getCondition != null) {
+      // we reduce the result card to push filters down
+      (rowCnt * 0.75).min(1.0)
+    } else {
+      rowCnt
+    }
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
+
+    val config = tableEnv.getConfig
+
+    val inputDS = getInput.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+
+    val returnType = determineReturnType(
+      getRowType,
+      expectedType,
+      config.getNullCheck,
+      config.getEfficientTypeUsage)
+
+    val generator = new CodeGenerator(config, false, inputDS.getType)
+
+    val body = functionBody(
+      generator,
+      inputDS.getType,
+      getRowType,
+      calcProgram,
+      config,
+      expectedType)
+
+    val genFunction = generator.generateFunction(
+      ruleDescription,
+      classOf[FlatMapFunction[Any, Any]],
+      body,
+      returnType)
+
+    val mapFunc = calcMapFunction(genFunction)
+    inputDS.flatMap(mapFunc).name(calcOpName(calcProgram, getExpressionString))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetConvention.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetConvention.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetConvention.scala
new file mode 100644
index 0000000..1b7bab5
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetConvention.scala
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan._
+
+class DataSetConvention extends Convention {
+
+  override def toString: String = getName
+
+  override def useAbstractConvertersForConversion(
+    fromTraits: RelTraitSet,
+    toTraits: RelTraitSet): Boolean = false
+
+  override def canConvertConvention(toConvention: Convention): Boolean = false
+
+  def getInterface: Class[_] = classOf[DataSetRel]
+
+  def getName: String = "DATASET"
+
+  def getTraitDef: RelTraitDef[_ <: RelTrait] = ConventionTraitDef.INSTANCE
+
+  def satisfies(`trait`: RelTrait): Boolean = this eq `trait`
+
+  def register(planner: RelOptPlanner): Unit = { }
+}
+
+object DataSetConvention {
+
+  val INSTANCE = new DataSetConvention
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/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
new file mode 100644
index 0000000..fa1afc3
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetCorrelate.scala
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan
+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.table.api.BatchTableEnvironment
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.functions.utils.TableSqlFunction
+import org.apache.flink.table.plan.nodes.FlinkCorrelate
+import org.apache.flink.table.typeutils.TypeConverter._
+
+/**
+  * Flink RelNode which matches along with join a user defined table function.
+  */
+class DataSetCorrelate(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inputNode: RelNode,
+    scan: LogicalTableFunctionScan,
+    condition: Option[RexNode],
+    relRowType: RelDataType,
+    joinRowType: RelDataType,
+    joinType: SemiJoinType,
+    ruleDescription: String)
+  extends SingleRel(cluster, traitSet, inputNode)
+  with FlinkCorrelate
+  with DataSetRel {
+
+  override def deriveRowType() = relRowType
+
+  override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+    val rowCnt = metadata.getRowCount(getInput) * 1.5
+    planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * 0.5)
+  }
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataSetCorrelate(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      scan,
+      condition,
+      relRowType,
+      joinRowType,
+      joinType,
+      ruleDescription)
+  }
+
+  override def toString: String = {
+    val rexCall = scan.getCall.asInstanceOf[RexCall]
+    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
+    correlateToString(rexCall, sqlFunction)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    val rexCall = scan.getCall.asInstanceOf[RexCall]
+    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
+    super.explainTerms(pw)
+      .item("invocation", scan.getCall)
+      .item("function", sqlFunction.getTableFunction.getClass.getCanonicalName)
+      .item("rowType", relRowType)
+      .item("joinType", joinType)
+      .itemIf("condition", condition.orNull, condition.isDefined)
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]])
+    : DataSet[Any] = {
+
+    val config = tableEnv.getConfig
+    val returnType = determineReturnType(
+      getRowType,
+      expectedType,
+      config.getNullCheck,
+      config.getEfficientTypeUsage)
+
+    // we do not need to specify input type
+    val inputDS = inputNode.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+
+    val funcRel = scan.asInstanceOf[LogicalTableFunctionScan]
+    val rexCall = funcRel.getCall.asInstanceOf[RexCall]
+    val sqlFunction = rexCall.getOperator.asInstanceOf[TableSqlFunction]
+    val pojoFieldMapping = sqlFunction.getPojoFieldMapping
+    val udtfTypeInfo = sqlFunction.getRowTypeInfo.asInstanceOf[TypeInformation[Any]]
+
+    val generator = new CodeGenerator(
+      config,
+      false,
+      inputDS.getType,
+      Some(udtfTypeInfo),
+      None,
+      Some(pojoFieldMapping))
+
+    val body = functionBody(
+      generator,
+      udtfTypeInfo,
+      getRowType,
+      rexCall,
+      condition,
+      config,
+      joinType,
+      expectedType)
+
+    val genFunction = generator.generateFunction(
+      ruleDescription,
+      classOf[FlatMapFunction[Any, Any]],
+      body,
+      returnType)
+
+    val mapFunc = correlateMapFunction(genFunction)
+
+    inputDS.flatMap(mapFunc).name(correlateOpName(rexCall, sqlFunction, relRowType))
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetIntersect.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetIntersect.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetIntersect.scala
new file mode 100644
index 0000000..332aa8a
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetIntersect.scala
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.api.BatchTableEnvironment
+import org.apache.flink.table.runtime.IntersectCoGroupFunction
+import org.apache.flink.table.typeutils.TypeConverter._
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+  * Flink RelNode which translates Intersect into CoGroup Operator.
+  *
+  */
+class DataSetIntersect(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    leftNode: RelNode,
+    rightNode: RelNode,
+    rowRelDataType: RelDataType,
+    all: Boolean)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+    with DataSetRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataSetIntersect(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      inputs.get(1),
+      getRowType,
+      all
+    )
+  }
+
+  override def toString: String = {
+    s"Intersect(intersect: ($intersectSelectionToString))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw).item("intersect", intersectSelectionToString)
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+    val children = this.getInputs
+    children.foldLeft(planner.getCostFactory.makeCost(0, 0, 0)) { (cost, child) =>
+      val rowCnt = metadata.getRowCount(child)
+      val rowSize = this.estimateRowSize(child.getRowType)
+      cost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize))
+    }
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
+
+    val leftDataSet: DataSet[Any] = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+    val rightDataSet: DataSet[Any] = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+
+    val coGroupedDs = leftDataSet.coGroup(rightDataSet)
+
+    val coGroupOpName = s"intersect: ($intersectSelectionToString)"
+    val coGroupFunction = new IntersectCoGroupFunction[Any](all)
+
+    val intersectDs = coGroupedDs.where("*").equalTo("*")
+      .`with`(coGroupFunction).name(coGroupOpName)
+
+    val config = tableEnv.getConfig
+    val leftType = leftDataSet.getType
+
+    // here we only care about left type information, because we emit records from left DataSet
+    expectedType match {
+      case None if config.getEfficientTypeUsage =>
+        intersectDs
+
+      case _ =>
+        val determinedType = determineReturnType(
+          getRowType,
+          expectedType,
+          config.getNullCheck,
+          config.getEfficientTypeUsage)
+
+        // conversion
+        if (determinedType != leftType) {
+          val mapFunc = getConversionMapper(
+            config,
+            false,
+            leftType,
+            determinedType,
+            "DataSetIntersectConversion",
+            getRowType.getFieldNames)
+
+          val opName = s"convert: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
+
+          intersectDs.map(mapFunc).name(opName)
+        }
+        // no conversion necessary, forward
+        else {
+          intersectDs
+        }
+    }
+  }
+
+  private def intersectSelectionToString: String = {
+    getRowType.getFieldNames.asScala.toList.mkString(", ")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetJoin.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetJoin.scala
new file mode 100644
index 0000000..324e949
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetJoin.scala
@@ -0,0 +1,229 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.util.mapping.IntPair
+import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.runtime.FlatJoinRunner
+import org.apache.flink.table.typeutils.TypeConverter.determineReturnType
+import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.api.{BatchTableEnvironment, TableException}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+/**
+  * Flink RelNode which matches along with JoinOperator and its related operations.
+  */
+class DataSetJoin(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    leftNode: RelNode,
+    rightNode: RelNode,
+    rowRelDataType: RelDataType,
+    joinCondition: RexNode,
+    joinRowType: RelDataType,
+    joinInfo: JoinInfo,
+    keyPairs: List[IntPair],
+    joinType: JoinRelType,
+    joinHint: JoinHint,
+    ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+  with DataSetRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataSetJoin(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      inputs.get(1),
+      getRowType,
+      joinCondition,
+      joinRowType,
+      joinInfo,
+      keyPairs,
+      joinType,
+      joinHint,
+      ruleDescription)
+  }
+
+  override def toString: String = {
+    s"$joinTypeToString(where: ($joinConditionToString), join: ($joinSelectionToString))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .item("where", joinConditionToString)
+      .item("join", joinSelectionToString)
+      .item("joinType", joinTypeToString)
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+
+    val leftRowCnt = metadata.getRowCount(getLeft)
+    val leftRowSize = estimateRowSize(getLeft.getRowType)
+
+    val rightRowCnt = metadata.getRowCount(getRight)
+    val rightRowSize = estimateRowSize(getRight.getRowType)
+
+    val ioCost = (leftRowCnt * leftRowSize) + (rightRowCnt * rightRowSize)
+    val cpuCost = leftRowCnt + rightRowCnt
+    val rowCnt = leftRowCnt + rightRowCnt
+
+    planner.getCostFactory.makeCost(rowCnt, cpuCost, ioCost)
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
+
+    val config = tableEnv.getConfig
+
+    val returnType = determineReturnType(
+      getRowType,
+      expectedType,
+      config.getNullCheck,
+      config.getEfficientTypeUsage)
+
+    // get the equality keys
+    val leftKeys = ArrayBuffer.empty[Int]
+    val rightKeys = ArrayBuffer.empty[Int]
+    if (keyPairs.isEmpty) {
+      // if no equality keys => not supported
+      throw TableException(
+        "Joins should have at least one equality condition.\n" +
+          s"\tLeft: ${left.toString},\n" +
+          s"\tRight: ${right.toString},\n" +
+          s"\tCondition: ($joinConditionToString)"
+      )
+    }
+    else {
+      // at least one equality expression
+      val leftFields = left.getRowType.getFieldList
+      val rightFields = right.getRowType.getFieldList
+
+      keyPairs.foreach(pair => {
+        val leftKeyType = leftFields.get(pair.source).getType.getSqlTypeName
+        val rightKeyType = rightFields.get(pair.target).getType.getSqlTypeName
+
+        // check if keys are compatible
+        if (leftKeyType == rightKeyType) {
+          // add key pair
+          leftKeys.add(pair.source)
+          rightKeys.add(pair.target)
+        } else {
+          throw TableException(
+            "Equality join predicate on incompatible types.\n" +
+              s"\tLeft: ${left.toString},\n" +
+              s"\tRight: ${right.toString},\n" +
+              s"\tCondition: ($joinConditionToString)"
+          )
+        }
+      })
+    }
+
+    val leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+    val rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+
+    val (joinOperator, nullCheck) = joinType match {
+      case JoinRelType.INNER => (leftDataSet.join(rightDataSet), false)
+      case JoinRelType.LEFT => (leftDataSet.leftOuterJoin(rightDataSet), true)
+      case JoinRelType.RIGHT => (leftDataSet.rightOuterJoin(rightDataSet), true)
+      case JoinRelType.FULL => (leftDataSet.fullOuterJoin(rightDataSet), true)
+    }
+
+    if (nullCheck && !config.getNullCheck) {
+      throw TableException("Null check in TableConfig must be enabled for outer joins.")
+    }
+
+    val generator = new CodeGenerator(
+      config,
+      nullCheck,
+      leftDataSet.getType,
+      Some(rightDataSet.getType))
+    val conversion = generator.generateConverterResultExpression(
+      returnType,
+      joinRowType.getFieldNames)
+
+    var body = ""
+
+    if (joinInfo.isEqui) {
+      // only equality condition
+      body = s"""
+           |${conversion.code}
+           |${generator.collectorTerm}.collect(${conversion.resultTerm});
+           |""".stripMargin
+    }
+    else {
+      val condition = generator.generateExpression(joinCondition)
+      body = s"""
+           |${condition.code}
+           |if (${condition.resultTerm}) {
+           |  ${conversion.code}
+           |  ${generator.collectorTerm}.collect(${conversion.resultTerm});
+           |}
+           |""".stripMargin
+    }
+    val genFunction = generator.generateFunction(
+      ruleDescription,
+      classOf[FlatJoinFunction[Any, Any, Any]],
+      body,
+      returnType)
+
+    val joinFun = new FlatJoinRunner[Any, Any, Any](
+      genFunction.name,
+      genFunction.code,
+      genFunction.returnType)
+
+    val joinOpName = s"where: ($joinConditionToString), join: ($joinSelectionToString)"
+
+    joinOperator.where(leftKeys.toArray: _*).equalTo(rightKeys.toArray: _*)
+      .`with`(joinFun).name(joinOpName).asInstanceOf[DataSet[Any]]
+  }
+
+  private def joinSelectionToString: String = {
+    getRowType.getFieldNames.asScala.toList.mkString(", ")
+  }
+
+  private def joinConditionToString: String = {
+
+    val inFields = joinRowType.getFieldNames.asScala.toList
+    getExpressionString(joinCondition, inFields, None)
+  }
+
+  private def joinTypeToString = joinType match {
+    case JoinRelType.INNER => "InnerJoin"
+    case JoinRelType.LEFT=> "LeftOuterJoin"
+    case JoinRelType.RIGHT => "RightOuterJoin"
+    case JoinRelType.FULL => "FullOuterJoin"
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetMinus.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetMinus.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetMinus.scala
new file mode 100644
index 0000000..672ff9c
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetMinus.scala
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.api.BatchTableEnvironment
+import org.apache.flink.table.runtime.MinusCoGroupFunction
+import org.apache.flink.table.typeutils.TypeConverter._
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+  * Flink RelNode which implements set minus operation.
+  *
+  */
+class DataSetMinus(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    leftNode: RelNode,
+    rightNode: RelNode,
+    rowRelDataType: RelDataType,
+    all: Boolean)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+    with DataSetRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataSetMinus(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      inputs.get(1),
+      getRowType,
+      all
+    )
+  }
+
+  override def toString: String = {
+    s"Minus(minus: ($minusSelectionToString}))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw).item("minus", minusSelectionToString)
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+    val children = this.getInputs
+    children.foldLeft(planner.getCostFactory.makeCost(0, 0, 0)) { (cost, child) =>
+      val rowCnt = metadata.getRowCount(child)
+      val rowSize = this.estimateRowSize(child.getRowType)
+      cost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize))
+    }
+  }
+
+  override def estimateRowCount(mq: RelMetadataQuery): Double = {
+    // from org.apache.calcite.rel.metadata.RelMdUtil.getMinusRowCount
+    val children = this.getInputs
+    var rowCnt = mq.getRowCount(children.head)
+    getInputs.tail.foreach(rowCnt -= 0.5 * mq.getRowCount(_))
+    if (rowCnt < 0) {
+      rowCnt = 0.0
+    }
+    rowCnt
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
+
+    val leftDataSet: DataSet[Any] = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+    val rightDataSet: DataSet[Any] = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+
+    val coGroupedDs = leftDataSet.coGroup(rightDataSet)
+
+    val coGroupOpName = s"minus: ($minusSelectionToString)"
+    val coGroupFunction = new MinusCoGroupFunction[Any](all)
+
+    val minusDs = coGroupedDs.where("*").equalTo("*")
+      .`with`(coGroupFunction).name(coGroupOpName)
+
+    val config = tableEnv.getConfig
+    val leftType = leftDataSet.getType
+
+    // here we only care about left type information, because we emit records from left DataSet
+    expectedType match {
+      case None if config.getEfficientTypeUsage =>
+        minusDs
+
+      case _ =>
+        val determinedType = determineReturnType(
+          getRowType,
+          expectedType,
+          config.getNullCheck,
+          config.getEfficientTypeUsage)
+
+        // conversion
+        if (determinedType != leftType) {
+          val mapFunc = getConversionMapper(
+            config = config,
+            nullableInput = false,
+            inputType = leftType,
+            expectedType = determinedType,
+            conversionOperatorName = "DataSetMinusConversion",
+            fieldNames = getRowType.getFieldNames)
+
+          val opName = s"convert: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
+
+          minusDs.map(mapFunc).name(opName)
+        }
+        // no conversion necessary, forward
+        else {
+          minusDs
+        }
+    }
+  }
+
+  private def minusSelectionToString: String = {
+    getRowType.getFieldNames.asScala.toList.mkString(", ")
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetRel.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetRel.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetRel.scala
new file mode 100644
index 0000000..210ae03
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetRel.scala
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.sql.`type`.SqlTypeName
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.api.{BatchTableEnvironment, TableException}
+import org.apache.flink.table.plan.nodes.FlinkRel
+
+import scala.collection.JavaConversions._
+
+trait DataSetRel extends RelNode with FlinkRel {
+
+  /**
+    * Translates the [[DataSetRel]] node into a [[DataSet]] operator.
+    *
+    * @param tableEnv     [[BatchTableEnvironment]] of the translated Table.
+    * @param expectedType specifies the type the Flink operator should return. The type must
+    *                     have the same arity as the result. For instance, if the
+    *                     expected type is a RowTypeInfo this method will return a DataSet of
+    *                     type Row. If the expected type is Tuple2, the operator will return
+    *                     a Tuple2 if possible. Row otherwise.
+    * @return DataSet of type expectedType or RowTypeInfo
+    */
+  def translateToPlan(
+     tableEnv: BatchTableEnvironment,
+     expectedType: Option[TypeInformation[Any]] = None) : DataSet[Any]
+
+  private[flink] def estimateRowSize(rowType: RelDataType): Double = {
+
+    rowType.getFieldList.map(_.getType.getSqlTypeName).foldLeft(0) { (s, t) =>
+      t match {
+        case SqlTypeName.TINYINT => s + 1
+        case SqlTypeName.SMALLINT => s + 2
+        case SqlTypeName.INTEGER => s + 4
+        case SqlTypeName.BIGINT => s + 8
+        case SqlTypeName.BOOLEAN => s + 1
+        case SqlTypeName.FLOAT => s + 4
+        case SqlTypeName.DOUBLE => s + 8
+        case SqlTypeName.VARCHAR => s + 12
+        case SqlTypeName.CHAR => s + 1
+        case SqlTypeName.DECIMAL => s + 12
+        case typeName if SqlTypeName.YEAR_INTERVAL_TYPES.contains(typeName) => s + 8
+        case typeName if SqlTypeName.DAY_INTERVAL_TYPES.contains(typeName) => s + 4
+        case SqlTypeName.TIME | SqlTypeName.TIMESTAMP | SqlTypeName.DATE => s + 12
+        case _ => throw TableException(s"Unsupported data type encountered: $t")
+      }
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetScan.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetScan.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetScan.scala
new file mode 100644
index 0000000..48bbb74
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetScan.scala
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.api.BatchTableEnvironment
+import org.apache.flink.table.plan.schema.DataSetTable
+
+/**
+  * Flink RelNode which matches along with DataSource.
+  * It ensures that types without deterministic field order (e.g. POJOs) are not part of
+  * the plan translation.
+  */
+class DataSetScan(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    table: RelOptTable,
+    rowRelDataType: RelDataType)
+  extends BatchScan(cluster, traitSet, table) {
+
+  val dataSetTable: DataSetTable[Any] = getTable.unwrap(classOf[DataSetTable[Any]])
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataSetScan(
+      cluster,
+      traitSet,
+      getTable,
+      getRowType
+    )
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
+
+    val config = tableEnv.getConfig
+    val inputDataSet: DataSet[Any] = dataSetTable.dataSet
+
+    convertToExpectedType(inputDataSet, dataSetTable, expectedType, config)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetSingleRowJoin.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetSingleRowJoin.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetSingleRowJoin.scala
new file mode 100644
index 0000000..a70b4ab
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetSingleRowJoin.scala
@@ -0,0 +1,193 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.api.common.functions.{FlatJoinFunction, FlatMapFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.runtime.{MapJoinLeftRunner, MapJoinRightRunner}
+import org.apache.flink.table.typeutils.TypeConverter.determineReturnType
+import org.apache.flink.table.api.{BatchTableEnvironment, TableConfig}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+  * Flink RelNode that executes a Join where one of inputs is a single row.
+  */
+class DataSetSingleRowJoin(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    leftNode: RelNode,
+    rightNode: RelNode,
+    leftIsSingle: Boolean,
+    rowRelDataType: RelDataType,
+    joinCondition: RexNode,
+    joinRowType: RelDataType,
+    ruleDescription: String)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+  with DataSetRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataSetSingleRowJoin(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      inputs.get(1),
+      leftIsSingle,
+      getRowType,
+      joinCondition,
+      joinRowType,
+      ruleDescription)
+  }
+
+  override def toString: String = {
+    s"$joinTypeToString(where: ($joinConditionToString), join: ($joinSelectionToString))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .item("where", joinConditionToString)
+      .item("join", joinSelectionToString)
+      .item("joinType", joinTypeToString)
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+    val child = if (leftIsSingle) {
+      this.getRight
+    } else {
+      this.getLeft
+    }
+    val rowCnt = metadata.getRowCount(child)
+    val rowSize = this.estimateRowSize(child.getRowType)
+    planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize)
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
+
+    val leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+    val rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+    val broadcastSetName = "joinSet"
+    val mapSideJoin = generateMapFunction(
+      tableEnv.getConfig,
+      leftDataSet.getType,
+      rightDataSet.getType,
+      leftIsSingle,
+      joinCondition,
+      broadcastSetName,
+      expectedType)
+
+    val (multiRowDataSet, singleRowDataSet) =
+      if (leftIsSingle) {
+        (rightDataSet, leftDataSet)
+      } else {
+        (leftDataSet, rightDataSet)
+      }
+
+    multiRowDataSet
+      .flatMap(mapSideJoin)
+      .withBroadcastSet(singleRowDataSet, broadcastSetName)
+      .name(getMapOperatorName)
+      .asInstanceOf[DataSet[Any]]
+  }
+
+  private def generateMapFunction(
+      config: TableConfig,
+      inputType1: TypeInformation[Any],
+      inputType2: TypeInformation[Any],
+      firstIsSingle: Boolean,
+      joinCondition: RexNode,
+      broadcastInputSetName: String,
+      expectedType: Option[TypeInformation[Any]]): FlatMapFunction[Any, Any] = {
+
+    val codeGenerator = new CodeGenerator(
+      config,
+      false,
+      inputType1,
+      Some(inputType2))
+
+    val returnType = determineReturnType(
+      getRowType,
+      expectedType,
+      config.getNullCheck,
+      config.getEfficientTypeUsage)
+
+    val conversion = codeGenerator.generateConverterResultExpression(
+      returnType,
+      joinRowType.getFieldNames)
+
+    val condition = codeGenerator.generateExpression(joinCondition)
+
+    val joinMethodBody = s"""
+                  |${condition.code}
+                  |if (${condition.resultTerm}) {
+                  |  ${conversion.code}
+                  |  ${codeGenerator.collectorTerm}.collect(${conversion.resultTerm});
+                  |}
+                  |""".stripMargin
+
+    val genFunction = codeGenerator.generateFunction(
+      ruleDescription,
+      classOf[FlatJoinFunction[Any, Any, Any]],
+      joinMethodBody,
+      returnType)
+
+    if (firstIsSingle) {
+      new MapJoinRightRunner[Any, Any, Any](
+        genFunction.name,
+        genFunction.code,
+        genFunction.returnType,
+        broadcastInputSetName)
+    } else {
+      new MapJoinLeftRunner[Any, Any, Any](
+        genFunction.name,
+        genFunction.code,
+        genFunction.returnType,
+        broadcastInputSetName)
+    }
+  }
+
+  private def getMapOperatorName: String = {
+    s"where: ($joinConditionToString), join: ($joinSelectionToString)"
+  }
+
+  private def joinSelectionToString: String = {
+    getRowType.getFieldNames.asScala.toList.mkString(", ")
+  }
+
+  private def joinConditionToString: String = {
+    val inFields = joinRowType.getFieldNames.asScala.toList
+    getExpressionString(joinCondition, inFields, None)
+  }
+
+  private def joinTypeToString: String = {
+    "NestedLoopJoin"
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetSort.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetSort.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetSort.scala
new file mode 100644
index 0000000..428ea84
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetSort.scala
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import java.util
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.RelFieldCollation.Direction
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.{RelCollation, RelNode, RelWriter, SingleRel}
+import org.apache.calcite.rex.{RexLiteral, RexNode}
+import org.apache.flink.api.common.operators.Order
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.api.{BatchTableEnvironment, TableException}
+import org.apache.flink.table.runtime.{CountPartitionFunction, LimitFilterFunction}
+import org.apache.flink.table.typeutils.TypeConverter._
+
+import scala.collection.JavaConverters._
+
+class DataSetSort(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    inp: RelNode,
+    collations: RelCollation,
+    rowRelDataType: RelDataType,
+    offset: RexNode,
+    fetch: RexNode)
+  extends SingleRel(cluster, traitSet, inp)
+  with DataSetRel {
+
+  private val limitStart: Long = if (offset != null) {
+    RexLiteral.intValue(offset)
+  } else {
+    0L
+  }
+
+  private val limitEnd: Long = if (fetch != null) {
+    RexLiteral.intValue(fetch) + limitStart
+  } else {
+    Long.MaxValue
+  }
+
+  override def deriveRowType(): RelDataType = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = {
+    new DataSetSort(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      collations,
+      getRowType,
+      offset,
+      fetch
+    )
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]] = None)
+    : DataSet[Any] = {
+
+    if (fieldCollations.isEmpty) {
+      throw TableException("Limiting the result without sorting is not allowed " +
+        "as it could lead to arbitrary results.")
+    }
+
+    val config = tableEnv.getConfig
+
+    val inputDs = inp.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+
+    val currentParallelism = inputDs.getExecutionEnvironment.getParallelism
+    var partitionedDs = if (currentParallelism == 1) {
+      inputDs
+    } else {
+      inputDs.partitionByRange(fieldCollations.map(_._1): _*)
+        .withOrders(fieldCollations.map(_._2): _*)
+    }
+
+    fieldCollations.foreach { fieldCollation =>
+      partitionedDs = partitionedDs.sortPartition(fieldCollation._1, fieldCollation._2)
+    }
+
+    val limitedDs = if (offset == null && fetch == null) {
+      partitionedDs
+    } else {
+      val countFunction = new CountPartitionFunction[Any]
+
+      val partitionCountName = s"prepare offset/fetch"
+
+      val partitionCount = partitionedDs
+        .mapPartition(countFunction)
+        .name(partitionCountName)
+
+      val broadcastName = "countPartition"
+
+      val limitFunction = new LimitFilterFunction[Any](
+        limitStart,
+        limitEnd,
+        broadcastName)
+
+      val limitName = s"offset: $offsetToString, fetch: $fetchToString"
+
+      partitionedDs
+        .filter(limitFunction)
+        .name(limitName)
+        .withBroadcastSet(partitionCount, broadcastName)
+    }
+
+    val inputType = partitionedDs.getType
+    expectedType match {
+
+      case None if config.getEfficientTypeUsage =>
+        limitedDs
+
+      case _ =>
+        val determinedType = determineReturnType(
+          getRowType,
+          expectedType,
+          config.getNullCheck,
+          config.getEfficientTypeUsage)
+
+        // conversion
+        if (determinedType != inputType) {
+
+          val mapFunc = getConversionMapper(
+            config = config,
+            nullableInput = false,
+            inputType = partitionedDs.getType,
+            expectedType = determinedType,
+            conversionOperatorName = "DataSetSortConversion",
+            fieldNames = getRowType.getFieldNames.asScala
+          )
+
+          val opName = s"convert: (${getRowType.getFieldNames.asScala.toList.mkString(", ")})"
+
+          limitedDs.map(mapFunc).name(opName)
+        }
+        // no conversion necessary, forward
+        else {
+          limitedDs
+        }
+    }
+  }
+
+  private def directionToOrder(direction: Direction) = {
+    direction match {
+      case Direction.ASCENDING | Direction.STRICTLY_ASCENDING => Order.ASCENDING
+      case Direction.DESCENDING | Direction.STRICTLY_DESCENDING => Order.DESCENDING
+      case _ => throw new IllegalArgumentException("Unsupported direction.")
+    }
+
+  }
+
+  private val fieldCollations = collations.getFieldCollations.asScala
+    .map(c => (c.getFieldIndex, directionToOrder(c.getDirection)))
+
+  private val sortFieldsToString = fieldCollations
+    .map(col => s"${getRowType.getFieldNames.get(col._1)} ${col._2.getShortName}" ).mkString(", ")
+
+  private val offsetToString = s"$offset"
+
+  private val fetchToString = if (limitEnd == Long.MaxValue) {
+    "unlimited"
+  } else {
+    s"$limitEnd"
+  }
+
+  override def toString: String =
+    s"Sort(by: ($sortFieldsToString), offset: $offsetToString, fetch: $fetchToString)"
+
+  override def explainTerms(pw: RelWriter) : RelWriter = {
+    super.explainTerms(pw)
+      .item("orderBy", sortFieldsToString)
+      .item("offset", offsetToString)
+      .item("fetch", fetchToString)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetUnion.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetUnion.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetUnion.scala
new file mode 100644
index 0000000..b0c95b5
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetUnion.scala
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.api.BatchTableEnvironment
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+* Flink RelNode which matches along with UnionOperator.
+*
+*/
+class DataSetUnion(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    leftNode: RelNode,
+    rightNode: RelNode,
+    rowRelDataType: RelDataType)
+  extends BiRel(cluster, traitSet, leftNode, rightNode)
+  with DataSetRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataSetUnion(
+      cluster,
+      traitSet,
+      inputs.get(0),
+      inputs.get(1),
+      rowRelDataType
+    )
+  }
+
+  override def toString: String = {
+    s"Union(union: ($unionSelectionToString))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw).item("union", unionSelectionToString)
+  }
+
+  override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
+
+    val children = this.getInputs
+    val rowCnt = children.foldLeft(0D) { (rows, child) =>
+      rows + metadata.getRowCount(child)
+    }
+
+    planner.getCostFactory.makeCost(rowCnt, 0, 0)
+  }
+
+  override def estimateRowCount(mq: RelMetadataQuery): Double = {
+    // adopted from org.apache.calcite.rel.metadata.RelMdUtil.getUnionAllRowCount
+    getInputs.foldLeft(0.0)(_ + mq.getRowCount(_))
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
+
+    var leftDataSet: DataSet[Any] = null
+    var rightDataSet: DataSet[Any] = null
+
+    expectedType match {
+      case None =>
+        leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
+        rightDataSet =
+          right.asInstanceOf[DataSetRel].translateToPlan(tableEnv, Some(leftDataSet.getType))
+      case _ =>
+        leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv, expectedType)
+        rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv, expectedType)
+    }
+
+    leftDataSet.union(rightDataSet).asInstanceOf[DataSet[Any]]
+  }
+
+  private def unionSelectionToString: String = {
+    rowRelDataType.getFieldNames.asScala.toList.mkString(", ")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetValues.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetValues.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetValues.scala
new file mode 100644
index 0000000..e0282f2
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetValues.scala
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.nodes.dataset
+
+import com.google.common.collect.ImmutableList
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.Values
+import org.apache.calcite.rel.{RelNode, RelWriter}
+import org.apache.calcite.rex.RexLiteral
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.DataSet
+import org.apache.flink.table.api.BatchTableEnvironment
+import org.apache.flink.table.codegen.CodeGenerator
+import org.apache.flink.table.runtime.io.ValuesInputFormat
+import org.apache.flink.table.typeutils.TypeConverter._
+
+import scala.collection.JavaConverters._
+
+/**
+  * DataSet RelNode for a LogicalValues.
+  *
+  */
+class DataSetValues(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    rowRelDataType: RelDataType,
+    tuples: ImmutableList[ImmutableList[RexLiteral]],
+    ruleDescription: String)
+  extends Values(cluster, rowRelDataType, tuples, traitSet)
+  with DataSetRel {
+
+  override def deriveRowType() = rowRelDataType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new DataSetValues(
+      cluster,
+      traitSet,
+      getRowType,
+      getTuples,
+      ruleDescription
+    )
+  }
+
+  override def toString: String = {
+    s"Values(values: (${getRowType.getFieldNames.asScala.toList.mkString(", ")}))"
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw).item("values", valuesFieldsToString)
+  }
+
+  override def translateToPlan(
+      tableEnv: BatchTableEnvironment,
+      expectedType: Option[TypeInformation[Any]]): DataSet[Any] = {
+
+    val config = tableEnv.getConfig
+
+    val returnType = determineReturnType(
+      getRowType,
+      expectedType,
+      config.getNullCheck,
+      config.getEfficientTypeUsage)
+
+    val generator = new CodeGenerator(config)
+
+    // generate code for every record
+    val generatedRecords = getTuples.asScala.map { r =>
+      generator.generateResultExpression(
+        returnType,
+        getRowType.getFieldNames.asScala,
+        r.asScala)
+    }
+
+    // generate input format
+    val generatedFunction = generator.generateValuesInputFormat(
+      ruleDescription,
+      generatedRecords.map(_.code),
+      returnType)
+
+    val inputFormat = new ValuesInputFormat[Any](
+      generatedFunction.name,
+      generatedFunction.code,
+      generatedFunction.returnType)
+
+    tableEnv.execEnv.createInput(inputFormat, returnType).asInstanceOf[DataSet[Any]]
+  }
+
+  private def valuesFieldsToString: String = {
+    getRowType.getFieldNames.asScala.toList.mkString(", ")
+  }
+
+}
+
+


[47/47] flink git commit: [FLINK-5255] [table] Generalize detection of single row inputs for DataSetSingleRowJoinRule.

Posted by fh...@apache.org.
[FLINK-5255] [table] Generalize detection of single row inputs for DataSetSingleRowJoinRule.

- Add support for projections and filters following a global aggregation.

This closes #3009.


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

Branch: refs/heads/master
Commit: d5c7bf6ac4807b718f5eb780520f74e11a794b74
Parents: cc34c14
Author: Alexander Shoshin <Al...@epam.com>
Authored: Wed Dec 14 13:59:08 2016 +0300
Committer: Fabian Hueske <fh...@apache.org>
Committed: Fri Dec 16 16:41:21 2016 +0100

----------------------------------------------------------------------
 .../dataSet/DataSetSingleRowJoinRule.scala      | 29 +++++++------
 .../flink/table/runtime/MapJoinLeftRunner.scala |  8 +++-
 .../table/runtime/MapJoinRightRunner.scala      |  8 +++-
 .../flink/table/runtime/MapSideJoinRunner.scala | 13 +++++-
 .../table/api/scala/batch/sql/JoinITCase.scala  | 14 +++++++
 .../api/scala/batch/sql/SingleRowJoinTest.scala | 43 ++++++++++++++++++++
 6 files changed, 97 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/d5c7bf6a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala
index 1f5c91a..dcd02d9 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSingleRowJoinRule.scala
@@ -23,7 +23,7 @@ import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall}
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.convert.ConverterRule
 import org.apache.calcite.rel.core.JoinRelType
-import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalJoin}
+import org.apache.calcite.rel.logical._
 import org.apache.flink.table.plan.nodes.dataset.{DataSetConvention, DataSetSingleRowJoin}
 
 class DataSetSingleRowJoinRule
@@ -31,14 +31,13 @@ class DataSetSingleRowJoinRule
       classOf[LogicalJoin],
       Convention.NONE,
       DataSetConvention.INSTANCE,
-      "DataSetSingleRowCrossRule") {
+      "DataSetSingleRowJoinRule") {
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val join = call.rel(0).asInstanceOf[LogicalJoin]
 
     if (isInnerJoin(join)) {
-      isGlobalAggregation(join.getRight.asInstanceOf[RelSubset].getOriginal) ||
-        isGlobalAggregation(join.getLeft.asInstanceOf[RelSubset].getOriginal)
+      isSingleRow(join.getRight) || isSingleRow(join.getLeft)
     } else {
       false
     }
@@ -48,13 +47,19 @@ class DataSetSingleRowJoinRule
     join.getJoinType == JoinRelType.INNER
   }
 
-  private def isGlobalAggregation(node: RelNode) = {
-    node.isInstanceOf[LogicalAggregate] &&
-      isSingleRow(node.asInstanceOf[LogicalAggregate])
-  }
-
-  private def isSingleRow(agg: LogicalAggregate) = {
-    agg.getGroupSet.isEmpty
+  /**
+    * Recursively checks if a [[RelNode]] returns at most a single row.
+    * Input must be a global aggregation possibly followed by projections or filters.
+    */
+  private def isSingleRow(node: RelNode): Boolean = {
+    node match {
+      case ss: RelSubset => isSingleRow(ss.getOriginal)
+      case lp: LogicalProject => isSingleRow(lp.getInput)
+      case lf: LogicalFilter => isSingleRow(lf.getInput)
+      case lc: LogicalCalc => isSingleRow(lc.getInput)
+      case la: LogicalAggregate => la.getGroupSet.isEmpty
+      case _ => false
+    }
   }
 
   override def convert(rel: RelNode): RelNode = {
@@ -62,7 +67,7 @@ class DataSetSingleRowJoinRule
     val traitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
     val dataSetLeftNode = RelOptRule.convert(join.getLeft, DataSetConvention.INSTANCE)
     val dataSetRightNode = RelOptRule.convert(join.getRight, DataSetConvention.INSTANCE)
-    val leftIsSingle = isGlobalAggregation(join.getLeft.asInstanceOf[RelSubset].getOriginal)
+    val leftIsSingle = isSingleRow(join.getLeft)
 
     new DataSetSingleRowJoin(
       rel.getCluster,

http://git-wip-us.apache.org/repos/asf/flink/blob/d5c7bf6a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
index cf32404..644e855 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
@@ -28,6 +28,10 @@ class MapJoinLeftRunner[IN1, IN2, OUT](
     broadcastSetName: String)
   extends MapSideJoinRunner[IN1, IN2, IN2, IN1, OUT](name, code, returnType, broadcastSetName) {
 
-  override def flatMap(multiInput: IN1, out: Collector[OUT]): Unit =
-    function.join(multiInput, singleInput, out)
+  override def flatMap(multiInput: IN1, out: Collector[OUT]): Unit = {
+    broadcastSet match {
+      case Some(singleInput) => function.join(multiInput, singleInput, out)
+      case None =>
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d5c7bf6a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
index c4bc0d1..eee38d1 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
@@ -28,6 +28,10 @@ class MapJoinRightRunner[IN1, IN2, OUT](
     broadcastSetName: String)
   extends MapSideJoinRunner[IN1, IN2, IN1, IN2, OUT](name, code, returnType, broadcastSetName) {
 
-  override def flatMap(multiInput: IN2, out: Collector[OUT]): Unit =
-    function.join(singleInput, multiInput, out)
+  override def flatMap(multiInput: IN2, out: Collector[OUT]): Unit = {
+    broadcastSet match {
+      case Some(singleInput) => function.join(singleInput, multiInput, out)
+      case None =>
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d5c7bf6a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
index f12590f..090e184 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
@@ -37,14 +37,23 @@ abstract class MapSideJoinRunner[IN1, IN2, SINGLE_IN, MULTI_IN, OUT](
   val LOG = LoggerFactory.getLogger(this.getClass)
 
   protected var function: FlatJoinFunction[IN1, IN2, OUT] = _
-  protected var singleInput: SINGLE_IN = _
+  protected var broadcastSet: Option[SINGLE_IN] = _
 
   override def open(parameters: Configuration): Unit = {
     LOG.debug(s"Compiling FlatJoinFunction: $name \n\n Code:\n$code")
     val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
     LOG.debug("Instantiating FlatJoinFunction.")
     function = clazz.newInstance()
-    singleInput = getRuntimeContext.getBroadcastVariable(broadcastSetName).get(0)
+    broadcastSet = retrieveBroadcastSet
+  }
+
+  private def retrieveBroadcastSet: Option[SINGLE_IN] = {
+    val broadcastSet = getRuntimeContext.getBroadcastVariable(broadcastSetName)
+    if (!broadcastSet.isEmpty) {
+      Option(broadcastSet.get(0))
+    } else {
+      Option.empty
+    }
   }
 
   override def getProducedType: TypeInformation[OUT] = returnType

http://git-wip-us.apache.org/repos/asf/flink/blob/d5c7bf6a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/JoinITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/JoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/JoinITCase.scala
index 344428b..96beea5 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/JoinITCase.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/JoinITCase.scala
@@ -363,4 +363,18 @@ class JoinITCase(
     val result = tEnv.sql(sqlQuery1).collect()
     TestBaseUtils.compareResultAsText(result.asJava, expected)
   }
+
+  @Test
+  def testCrossJoinWithEmptySingleRowInput(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val table = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a1, 'a2, 'a3)
+    tEnv.registerTable("A", table)
+
+    val sqlQuery1 = "SELECT * FROM A CROSS JOIN (SELECT count(*) FROM A HAVING count(*) < 0)"
+    val result = tEnv.sql(sqlQuery1).count()
+
+    Assert.assertEquals(0, result)
+  }
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/d5c7bf6a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SingleRowJoinTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SingleRowJoinTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SingleRowJoinTest.scala
index ecc685d..27e3853 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SingleRowJoinTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/batch/sql/SingleRowJoinTest.scala
@@ -27,6 +27,49 @@ import org.junit.Test
 class SingleRowJoinTest extends TableTestBase {
 
   @Test
+  def testSingleRowJoinWithCalcInput(): Unit = {
+    val util = batchTestUtil()
+    util.addTable[(Int, Int)]("A", 'a1, 'a2)
+
+    val query =
+      "SELECT a1, asum " +
+      "FROM A, (SELECT sum(a1) + sum(a2) AS asum FROM A)"
+
+    val expected =
+      binaryNode(
+        "DataSetSingleRowJoin",
+        unaryNode(
+          "DataSetCalc",
+          batchTableNode(0),
+          term("select", "a1")
+        ),
+        unaryNode(
+          "DataSetCalc",
+          unaryNode(
+            "DataSetAggregate",
+            unaryNode(
+              "DataSetUnion",
+              unaryNode(
+                "DataSetValues",
+                batchTableNode(0),
+                tuples(List(null, null)),
+                term("values", "a1", "a2")
+              ),
+              term("union","a1","a2")
+            ),
+            term("select", "SUM(a1) AS $f0", "SUM(a2) AS $f1")
+          ),
+          term("select", "+($f0, $f1) AS asum")
+        ),
+        term("where", "true"),
+        term("join", "a1", "asum"),
+        term("joinType", "NestedLoopJoin")
+      )
+
+    util.verifySql(query, expected)
+  }
+
+  @Test
   def testSingleRowEquiJoin(): Unit = {
     val util = batchTestUtil()
     util.addTable[(Int, String)]("A", 'a1, 'a2)


[18/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSortRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSortRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSortRule.scala
new file mode 100644
index 0000000..14b9459
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetSortRule.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.{LogicalJoin, LogicalSort}
+import org.apache.flink.table.plan.nodes.dataset.{DataSetConvention, DataSetSort}
+
+class DataSetSortRule
+  extends ConverterRule(
+    classOf[LogicalSort],
+    Convention.NONE,
+    DataSetConvention.INSTANCE,
+    "DataSetSortRule") {
+
+  override def convert(rel: RelNode): RelNode = {
+
+    val sort: LogicalSort = rel.asInstanceOf[LogicalSort]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+    val convInput: RelNode = RelOptRule.convert(sort.getInput, DataSetConvention.INSTANCE)
+
+    new DataSetSort(
+      rel.getCluster,
+      traitSet,
+      convInput,
+      sort.getCollation,
+      rel.getRowType,
+      sort.offset,
+      sort.fetch
+    )
+  }
+}
+
+object DataSetSortRule {
+  val INSTANCE: RelOptRule = new DataSetSortRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetUnionRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetUnionRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetUnionRule.scala
new file mode 100644
index 0000000..e24f477
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetUnionRule.scala
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.{RelOptRuleCall, Convention, RelOptRule, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalUnion
+import org.apache.calcite.rel.rules.UnionToDistinctRule
+import org.apache.flink.table.plan.nodes.dataset.{DataSetConvention, DataSetUnion}
+
+class DataSetUnionRule
+  extends ConverterRule(
+      classOf[LogicalUnion],
+      Convention.NONE,
+      DataSetConvention.INSTANCE,
+      "DataSetUnionRule")
+  {
+
+  /**
+   * Only translate UNION ALL.
+   * Note: A distinct Union are translated into
+   * an Aggregate on top of a UNION ALL by [[UnionToDistinctRule]]
+   */
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val union: LogicalUnion = call.rel(0).asInstanceOf[LogicalUnion]
+    union.all
+  }
+
+  def convert(rel: RelNode): RelNode = {
+
+    val union: LogicalUnion = rel.asInstanceOf[LogicalUnion]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+    val convLeft: RelNode = RelOptRule.convert(union.getInput(0), DataSetConvention.INSTANCE)
+    val convRight: RelNode = RelOptRule.convert(union.getInput(1), DataSetConvention.INSTANCE)
+
+    new DataSetUnion(
+      rel.getCluster,
+      traitSet,
+      convLeft,
+      convRight,
+      rel.getRowType)
+  }
+}
+
+object DataSetUnionRule {
+  val INSTANCE: RelOptRule = new DataSetUnionRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetValuesRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetValuesRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetValuesRule.scala
new file mode 100644
index 0000000..8ecdc74
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/DataSetValuesRule.scala
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.{RelOptRule, RelTraitSet, Convention}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalValues
+import org.apache.flink.table.plan.nodes.dataset.{DataSetValues, DataSetConvention}
+
+class DataSetValuesRule
+  extends ConverterRule(
+    classOf[LogicalValues],
+    Convention.NONE,
+    DataSetConvention.INSTANCE,
+    "DataSetValuesRule")
+{
+
+  def convert(rel: RelNode): RelNode = {
+
+    val values: LogicalValues = rel.asInstanceOf[LogicalValues]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataSetConvention.INSTANCE)
+
+    new DataSetValues(
+      rel.getCluster,
+      traitSet,
+      rel.getRowType,
+      values.getTuples,
+      description)
+  }
+}
+
+object DataSetValuesRule {
+  val INSTANCE: RelOptRule = new DataSetValuesRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala
new file mode 100644
index 0000000..5d91c62
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/dataSet/PushProjectIntoBatchTableSourceScanRule.scala
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.dataSet
+
+import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
+import org.apache.calcite.plan.RelOptRule.{none, operand}
+import org.apache.flink.table.plan.nodes.dataset.{BatchTableSourceScan, DataSetCalc}
+import org.apache.flink.table.plan.rules.util.RexProgramProjectExtractor._
+import org.apache.flink.table.sources.{BatchTableSource, ProjectableTableSource}
+
+/**
+  * This rule tries to push projections into a BatchTableSourceScan.
+  */
+class PushProjectIntoBatchTableSourceScanRule extends RelOptRule(
+  operand(classOf[DataSetCalc],
+          operand(classOf[BatchTableSourceScan], none)),
+  "PushProjectIntoBatchTableSourceScanRule") {
+
+  override def matches(call: RelOptRuleCall) = {
+    val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
+    scan.tableSource match {
+      case _: ProjectableTableSource[_] => true
+      case _ => false
+    }
+  }
+
+  override def onMatch(call: RelOptRuleCall) {
+    val calc: DataSetCalc = call.rel(0).asInstanceOf[DataSetCalc]
+    val scan: BatchTableSourceScan = call.rel(1).asInstanceOf[BatchTableSourceScan]
+
+    val usedFields: Array[Int] = extractRefInputFields(calc.calcProgram)
+
+    // if no fields can be projected, there is no need to transform subtree
+    if (scan.tableSource.getNumberOfFields != usedFields.length) {
+      val originTableSource = scan.tableSource.asInstanceOf[ProjectableTableSource[_]]
+      val newTableSource = originTableSource.projectFields(usedFields)
+      val newScan = new BatchTableSourceScan(
+        scan.getCluster,
+        scan.getTraitSet,
+        scan.getTable,
+        newTableSource.asInstanceOf[BatchTableSource[_]])
+
+      val newCalcProgram = rewriteRexProgram(
+        calc.calcProgram,
+        newScan.getRowType,
+        usedFields,
+        calc.getCluster.getRexBuilder)
+
+      // if project merely returns its input and doesn't exist filter, remove datasetCalc nodes
+      if (newCalcProgram.isTrivial) {
+        call.transformTo(newScan)
+      } else {
+        val newCalc = new DataSetCalc(
+          calc.getCluster,
+          calc.getTraitSet,
+          newScan,
+          calc.getRowType,
+          newCalcProgram,
+          description)
+        call.transformTo(newCalc)
+      }
+    }
+  }
+}
+
+object PushProjectIntoBatchTableSourceScanRule {
+  val INSTANCE: RelOptRule = new PushProjectIntoBatchTableSourceScanRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala
new file mode 100644
index 0000000..bf8a18e
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamAggregateRule.scala
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.datastream
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.expressions.Alias
+import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate
+import org.apache.flink.table.plan.nodes.datastream.{DataStreamAggregate, DataStreamConvention}
+
+import scala.collection.JavaConversions._
+
+class DataStreamAggregateRule
+  extends ConverterRule(
+      classOf[LogicalWindowAggregate],
+      Convention.NONE,
+      DataStreamConvention.INSTANCE,
+      "DataStreamAggregateRule")
+  {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val agg: LogicalWindowAggregate = call.rel(0).asInstanceOf[LogicalWindowAggregate]
+
+    // check if we have distinct aggregates
+    val distinctAggs = agg.getAggCallList.exists(_.isDistinct)
+    if (distinctAggs) {
+      throw TableException("DISTINCT aggregates are currently not supported.")
+    }
+
+    // check if we have grouping sets
+    val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet
+    if (groupSets || agg.indicator) {
+      throw TableException("GROUPING SETS are currently not supported.")
+    }
+
+    !distinctAggs && !groupSets && !agg.indicator
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+    val agg: LogicalWindowAggregate = rel.asInstanceOf[LogicalWindowAggregate]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
+    val convInput: RelNode = RelOptRule.convert(agg.getInput, DataStreamConvention.INSTANCE)
+
+    new DataStreamAggregate(
+      agg.getWindow,
+      agg.getNamedProperties,
+      rel.getCluster,
+      traitSet,
+      convInput,
+      agg.getNamedAggCalls,
+      rel.getRowType,
+      agg.getInput.getRowType,
+      agg.getGroupSet.toArray)
+    }
+  }
+
+object DataStreamAggregateRule {
+  val INSTANCE: RelOptRule = new DataStreamAggregateRule
+}
+

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala
new file mode 100644
index 0000000..4e620c9
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCalcRule.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.datastream
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalCalc
+import org.apache.flink.table.plan.nodes.datastream.DataStreamCalc
+import org.apache.flink.table.plan.nodes.datastream.DataStreamConvention
+
+class DataStreamCalcRule
+  extends ConverterRule(
+    classOf[LogicalCalc],
+    Convention.NONE,
+    DataStreamConvention.INSTANCE,
+    "DataStreamCalcRule")
+{
+
+  def convert(rel: RelNode): RelNode = {
+    val calc: LogicalCalc = rel.asInstanceOf[LogicalCalc]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
+    val convInput: RelNode = RelOptRule.convert(calc.getInput, DataStreamConvention.INSTANCE)
+
+    new DataStreamCalc(
+      rel.getCluster,
+      traitSet,
+      convInput,
+      rel.getRowType,
+      calc.getProgram,
+      description)
+  }
+}
+
+object DataStreamCalcRule {
+  val INSTANCE: RelOptRule = new DataStreamCalcRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCorrelateRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCorrelateRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCorrelateRule.scala
new file mode 100644
index 0000000..adce9f4
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamCorrelateRule.scala
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.plan.rules.datastream
+
+import org.apache.calcite.plan.volcano.RelSubset
+import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.{LogicalFilter, LogicalCorrelate, LogicalTableFunctionScan}
+import org.apache.calcite.rex.RexNode
+import org.apache.flink.table.plan.nodes.datastream.DataStreamConvention
+import org.apache.flink.table.plan.nodes.datastream.DataStreamCorrelate
+
+/**
+  * Rule to convert a LogicalCorrelate into a DataStreamCorrelate.
+  */
+class DataStreamCorrelateRule
+  extends ConverterRule(
+    classOf[LogicalCorrelate],
+    Convention.NONE,
+    DataStreamConvention.INSTANCE,
+    "DataStreamCorrelateRule") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val join: LogicalCorrelate = call.rel(0).asInstanceOf[LogicalCorrelate]
+    val right = join.getRight.asInstanceOf[RelSubset].getOriginal
+
+    right match {
+      // right node is a table function
+      case scan: LogicalTableFunctionScan => true
+      // a filter is pushed above the table function
+      case filter: LogicalFilter =>
+        filter
+          .getInput.asInstanceOf[RelSubset]
+          .getOriginal
+          .isInstanceOf[LogicalTableFunctionScan]
+      case _ => false
+    }
+  }
+
+  override def convert(rel: RelNode): RelNode = {
+    val join: LogicalCorrelate = rel.asInstanceOf[LogicalCorrelate]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
+    val convInput: RelNode = RelOptRule.convert(join.getInput(0), DataStreamConvention.INSTANCE)
+    val right: RelNode = join.getInput(1)
+
+    def convertToCorrelate(relNode: RelNode, condition: Option[RexNode]): DataStreamCorrelate = {
+      relNode match {
+        case rel: RelSubset =>
+          convertToCorrelate(rel.getRelList.get(0), condition)
+
+        case filter: LogicalFilter =>
+          convertToCorrelate(
+            filter.getInput.asInstanceOf[RelSubset].getOriginal,
+            Some(filter.getCondition))
+
+        case scan: LogicalTableFunctionScan =>
+          new DataStreamCorrelate(
+            rel.getCluster,
+            traitSet,
+            convInput,
+            scan,
+            condition,
+            rel.getRowType,
+            join.getRowType,
+            join.getJoinType,
+            description)
+      }
+    }
+    convertToCorrelate(right, None)
+  }
+
+}
+
+object DataStreamCorrelateRule {
+  val INSTANCE: RelOptRule = new DataStreamCorrelateRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamScanRule.scala
new file mode 100644
index 0000000..91fd6e2
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamScanRule.scala
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.datastream
+
+import org.apache.calcite.plan.{RelOptRuleCall, Convention, RelOptRule, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.logical.LogicalTableScan
+import org.apache.flink.table.plan.nodes.datastream.DataStreamConvention
+import org.apache.flink.table.plan.nodes.datastream.DataStreamScan
+import org.apache.flink.table.plan.schema.DataStreamTable
+
+class DataStreamScanRule
+  extends ConverterRule(
+    classOf[LogicalTableScan],
+    Convention.NONE,
+    DataStreamConvention.INSTANCE,
+    "DataStreamScanRule")
+{
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val scan: TableScan = call.rel(0).asInstanceOf[TableScan]
+    val dataSetTable = scan.getTable.unwrap(classOf[DataStreamTable[Any]])
+    dataSetTable match {
+      case _: DataStreamTable[Any] =>
+        true
+      case _ =>
+        false
+    }
+  }
+
+  def convert(rel: RelNode): RelNode = {
+    val scan: LogicalTableScan = rel.asInstanceOf[LogicalTableScan]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
+
+    new DataStreamScan(
+      rel.getCluster,
+      traitSet,
+      scan.getTable,
+      rel.getRowType
+    )
+  }
+}
+
+object DataStreamScanRule {
+  val INSTANCE: RelOptRule = new DataStreamScanRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala
new file mode 100644
index 0000000..475c050
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamUnionRule.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.datastream
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalUnion
+import org.apache.flink.table.plan.nodes.datastream.DataStreamConvention
+import org.apache.flink.table.plan.nodes.datastream.DataStreamUnion
+
+class DataStreamUnionRule
+  extends ConverterRule(
+    classOf[LogicalUnion],
+    Convention.NONE,
+    DataStreamConvention.INSTANCE,
+    "DataStreamUnionRule")
+{
+
+  def convert(rel: RelNode): RelNode = {
+    val union: LogicalUnion = rel.asInstanceOf[LogicalUnion]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
+    val convLeft: RelNode = RelOptRule.convert(union.getInput(0), DataStreamConvention.INSTANCE)
+    val convRight: RelNode = RelOptRule.convert(union.getInput(1), DataStreamConvention.INSTANCE)
+
+    new DataStreamUnion(
+      rel.getCluster,
+      traitSet,
+      convLeft,
+      convRight,
+      rel.getRowType)
+  }
+}
+
+object DataStreamUnionRule {
+  val INSTANCE: RelOptRule = new DataStreamUnionRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamValuesRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamValuesRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamValuesRule.scala
new file mode 100644
index 0000000..db33842
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/DataStreamValuesRule.scala
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.datastream
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.logical.LogicalValues
+import org.apache.flink.table.plan.nodes.datastream.{DataStreamValues, DataStreamConvention}
+
+class DataStreamValuesRule
+  extends ConverterRule(
+    classOf[LogicalValues],
+    Convention.NONE,
+    DataStreamConvention.INSTANCE,
+    "DataStreamValuesRule")
+{
+
+  def convert(rel: RelNode): RelNode = {
+
+    val values: LogicalValues = rel.asInstanceOf[LogicalValues]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
+
+    new DataStreamValues(
+      rel.getCluster,
+      traitSet,
+      rel.getRowType,
+      values.getTuples,
+      description)
+  }
+}
+
+object DataStreamValuesRule {
+  val INSTANCE: RelOptRule = new DataStreamValuesRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala
new file mode 100644
index 0000000..296c86b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/datastream/StreamTableSourceScanRule.scala
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.datastream
+
+import org.apache.calcite.plan.{Convention, RelOptRule, RelOptRuleCall, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.logical.LogicalTableScan
+import org.apache.flink.table.plan.nodes.datastream.
+  {StreamTableSourceScan, DataStreamConvention}
+import org.apache.flink.table.plan.schema.TableSourceTable
+import org.apache.flink.table.sources.StreamTableSource
+
+/** Rule to convert a [[LogicalTableScan]] into a [[StreamTableSourceScan]]. */
+class StreamTableSourceScanRule
+  extends ConverterRule(
+    classOf[LogicalTableScan],
+    Convention.NONE,
+    DataStreamConvention.INSTANCE,
+    "StreamTableSourceScanRule")
+{
+
+  /** Rule must only match if TableScan targets a [[StreamTableSource]] */
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val scan: TableScan = call.rel(0).asInstanceOf[TableScan]
+    val dataSetTable = scan.getTable.unwrap(classOf[TableSourceTable])
+    dataSetTable match {
+      case tst: TableSourceTable =>
+        tst.tableSource match {
+          case _: StreamTableSource[_] =>
+            true
+          case _ =>
+            false
+        }
+      case _ =>
+        false
+    }
+  }
+
+  def convert(rel: RelNode): RelNode = {
+    val scan: LogicalTableScan = rel.asInstanceOf[LogicalTableScan]
+    val traitSet: RelTraitSet = rel.getTraitSet.replace(DataStreamConvention.INSTANCE)
+
+    // The original registered table source
+    val table: TableSourceTable = scan.getTable.unwrap(classOf[TableSourceTable])
+    val tableSource: StreamTableSource[_] = table.tableSource.asInstanceOf[StreamTableSource[_]]
+
+    new StreamTableSourceScan(
+      rel.getCluster,
+      traitSet,
+      scan.getTable,
+      tableSource
+    )
+  }
+}
+
+object StreamTableSourceScanRule {
+  val INSTANCE: RelOptRule = new StreamTableSourceScanRule
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramProjectExtractor.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramProjectExtractor.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramProjectExtractor.scala
new file mode 100644
index 0000000..129cfd1
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/util/RexProgramProjectExtractor.scala
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.rules.util
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex._
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable
+import scala.collection.JavaConverters._
+
+object RexProgramProjectExtractor {
+
+  /**
+    * Extracts the indexes of input fields accessed by the RexProgram.
+    *
+    * @param rexProgram The RexProgram to analyze
+    * @return The indexes of accessed input fields
+    */
+  def extractRefInputFields(rexProgram: RexProgram): Array[Int] = {
+    val visitor = new RefFieldsVisitor
+    // extract input fields from project expressions
+    rexProgram.getProjectList.foreach(exp => rexProgram.expandLocalRef(exp).accept(visitor))
+    val condition = rexProgram.getCondition
+    // extract input fields from condition expression
+    if (condition != null) {
+      rexProgram.expandLocalRef(condition).accept(visitor)
+    }
+    visitor.getFields
+  }
+
+  /**
+    * Generates a new RexProgram based on mapped input fields.
+    *
+    * @param rexProgram      original RexProgram
+    * @param inputRowType    input row type
+    * @param usedInputFields indexes of used input fields
+    * @param rexBuilder      builder for Rex expressions
+    *
+    * @return A RexProgram with mapped input field expressions.
+    */
+  def rewriteRexProgram(
+      rexProgram: RexProgram,
+      inputRowType: RelDataType,
+      usedInputFields: Array[Int],
+      rexBuilder: RexBuilder): RexProgram = {
+
+    val inputRewriter = new InputRewriter(usedInputFields)
+    val newProjectExpressions = rexProgram.getProjectList.map(
+      exp => rexProgram.expandLocalRef(exp).accept(inputRewriter)
+    ).toList.asJava
+
+    val oldCondition = rexProgram.getCondition
+    val newConditionExpression = {
+      oldCondition match {
+        case ref: RexLocalRef => rexProgram.expandLocalRef(ref).accept(inputRewriter)
+        case _ => null // null does not match any type
+      }
+    }
+    RexProgram.create(
+      inputRowType,
+      newProjectExpressions,
+      newConditionExpression,
+      rexProgram.getOutputRowType,
+      rexBuilder
+    )
+  }
+}
+
+/**
+  * A RexVisitor to extract used input fields
+  */
+class RefFieldsVisitor extends RexVisitorImpl[Unit](true) {
+  private var fields = mutable.LinkedHashSet[Int]()
+
+  def getFields: Array[Int] = fields.toArray
+
+  override def visitInputRef(inputRef: RexInputRef): Unit = fields += inputRef.getIndex
+
+  override def visitCall(call: RexCall): Unit =
+    call.operands.foreach(operand => operand.accept(this))
+}
+
+/**
+  * A RexShuttle to rewrite field accesses of a RexProgram.
+  *
+  * @param fields fields mapping
+  */
+class InputRewriter(fields: Array[Int]) extends RexShuttle {
+
+  /** old input fields ref index -> new input fields ref index mappings */
+  private val fieldMap: Map[Int, Int] =
+    fields.zipWithIndex.toMap
+
+  override def visitInputRef(inputRef: RexInputRef): RexNode =
+    new RexInputRef(relNodeIndex(inputRef), inputRef.getType)
+
+  override def visitLocalRef(localRef: RexLocalRef): RexNode =
+    new RexInputRef(relNodeIndex(localRef), localRef.getType)
+
+  private def relNodeIndex(ref: RexSlot): Int =
+    fieldMap.getOrElse(ref.getIndex,
+      throw new IllegalArgumentException("input field contains invalid index"))
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/ArrayRelDataType.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/ArrayRelDataType.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/ArrayRelDataType.scala
new file mode 100644
index 0000000..f7d9e1d
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/ArrayRelDataType.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.schema
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.sql.`type`.ArraySqlType
+import org.apache.flink.api.common.typeinfo.TypeInformation
+
+/**
+  * Flink distinguishes between primitive arrays (int[], double[], ...) and
+  * object arrays (Integer[], MyPojo[], ...). This custom type supports both cases.
+  */
+class ArrayRelDataType(
+    val typeInfo: TypeInformation[_],
+    elementType: RelDataType,
+    isNullable: Boolean)
+  extends ArraySqlType(
+    elementType,
+    isNullable) {
+
+  override def toString = s"ARRAY($typeInfo)"
+
+  def canEqual(other: Any): Boolean = other.isInstanceOf[ArrayRelDataType]
+
+  override def equals(other: Any): Boolean = other match {
+    case that: ArrayRelDataType =>
+      super.equals(that) &&
+        (that canEqual this) &&
+        typeInfo == that.typeInfo
+    case _ => false
+  }
+
+  override def hashCode(): Int = {
+    typeInfo.hashCode()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/CompositeRelDataType.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/CompositeRelDataType.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/CompositeRelDataType.scala
new file mode 100644
index 0000000..92f9199
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/CompositeRelDataType.scala
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.schema
+
+import java.util
+
+import org.apache.calcite.rel.`type`.{RelDataTypeField, RelDataTypeFieldImpl, RelRecordType}
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import org.apache.flink.table.plan.schema.CompositeRelDataType.createFieldList
+
+import scala.collection.JavaConverters._
+
+/**
+  * Composite type for encapsulating Flink's [[CompositeType]].
+  *
+  * @param compositeType CompositeType to encapsulate
+  * @param typeFactory Flink's type factory
+  */
+class CompositeRelDataType(
+    val compositeType: CompositeType[_],
+    typeFactory: FlinkTypeFactory)
+  extends RelRecordType(createFieldList(compositeType, typeFactory)) {
+
+  override def toString = s"COMPOSITE($compositeType)"
+
+  def canEqual(other: Any): Boolean = other.isInstanceOf[CompositeRelDataType]
+
+  override def equals(other: Any): Boolean = other match {
+    case that: CompositeRelDataType =>
+      super.equals(that) &&
+        (that canEqual this) &&
+        compositeType == that.compositeType
+    case _ => false
+  }
+
+  override def hashCode(): Int = {
+    compositeType.hashCode()
+  }
+
+}
+
+object CompositeRelDataType {
+
+  /**
+    * Converts the fields of a composite type to list of [[RelDataTypeField]].
+    */
+  private def createFieldList(
+      compositeType: CompositeType[_],
+      typeFactory: FlinkTypeFactory)
+    : util.List[RelDataTypeField] = {
+
+    compositeType
+      .getFieldNames
+      .zipWithIndex
+      .map { case (name, index) =>
+        new RelDataTypeFieldImpl(
+          name,
+          index,
+          typeFactory.createTypeFromTypeInfo(compositeType.getTypeAt(index)))
+            .asInstanceOf[RelDataTypeField]
+      }
+      .toList
+      .asJava
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataSetTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataSetTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataSetTable.scala
new file mode 100644
index 0000000..f8c6835
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataSetTable.scala
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.schema
+
+import java.lang.Double
+import java.util
+import java.util.Collections
+
+import org.apache.calcite.rel.{RelCollation, RelDistribution}
+import org.apache.calcite.schema.Statistic
+import org.apache.calcite.util.ImmutableBitSet
+import org.apache.flink.api.java.DataSet
+
+class DataSetTable[T](
+    val dataSet: DataSet[T],
+    override val fieldIndexes: Array[Int],
+    override val fieldNames: Array[String])
+  extends FlinkTable[T](dataSet.getType, fieldIndexes, fieldNames) {
+
+  override def getStatistic: Statistic = {
+    new DefaultDataSetStatistic
+  }
+
+}
+
+class DefaultDataSetStatistic extends Statistic {
+
+  override def getRowCount: Double = 1000d
+
+  override def getCollations: util.List[RelCollation] = Collections.emptyList()
+
+  override def isKey(columns: ImmutableBitSet): Boolean = false
+
+  override def getDistribution: RelDistribution = null
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
new file mode 100644
index 0000000..0355fac
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/DataStreamTable.scala
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.schema
+
+import org.apache.flink.streaming.api.datastream.DataStream
+
+class DataStreamTable[T](
+    val dataStream: DataStream[T],
+    override val fieldIndexes: Array[Int],
+    override val fieldNames: Array[String])
+  extends FlinkTable[T](dataStream.getType, fieldIndexes, fieldNames) {
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
new file mode 100644
index 0000000..8bb5c81
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTable.scala
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.schema
+
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
+import org.apache.calcite.schema.impl.AbstractTable
+import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.calcite.FlinkTypeFactory
+
+abstract class FlinkTable[T](
+    val typeInfo: TypeInformation[T],
+    val fieldIndexes: Array[Int],
+    val fieldNames: Array[String])
+  extends AbstractTable {
+
+  if (fieldIndexes.length != fieldNames.length) {
+    throw new TableException(
+      "Number of field indexes and field names must be equal.")
+  }
+
+  // check uniqueness of field names
+  if (fieldNames.length != fieldNames.toSet.size) {
+    throw new TableException(
+      "Table field names must be unique.")
+  }
+
+  val fieldTypes: Array[TypeInformation[_]] =
+    typeInfo match {
+      case cType: CompositeType[T] =>
+        if (fieldNames.length != cType.getArity) {
+          throw new TableException(
+          s"Arity of type (" + cType.getFieldNames.deep + ") " +
+            "not equal to number of field names " + fieldNames.deep + ".")
+        }
+        fieldIndexes.map(cType.getTypeAt(_).asInstanceOf[TypeInformation[_]])
+      case aType: AtomicType[T] =>
+        if (fieldIndexes.length != 1 || fieldIndexes(0) != 0) {
+          throw new TableException(
+            "Non-composite input type may have only a single field and its index must be 0.")
+        }
+        Array(aType)
+    }
+
+  override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = {
+    val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory]
+    flinkTypeFactory.buildRowDataType(fieldNames, fieldTypes)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTableFunctionImpl.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTableFunctionImpl.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTableFunctionImpl.scala
new file mode 100644
index 0000000..1c05883
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/FlinkTableFunctionImpl.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.plan.schema
+
+import java.lang.reflect.{Method, Type}
+import java.util
+
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
+import org.apache.calcite.schema.TableFunction
+import org.apache.calcite.schema.impl.ReflectiveFunctionBase
+import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
+import org.apache.flink.api.common.typeutils.CompositeType
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.calcite.FlinkTypeFactory
+
+/**
+  * This is heavily inspired by Calcite's [[org.apache.calcite.schema.impl.TableFunctionImpl]].
+  * We need it in order to create a [[org.apache.flink.table.functions.utils.TableSqlFunction]].
+  * The main difference is that we override the [[getRowType()]] and [[getElementType()]].
+  */
+class FlinkTableFunctionImpl[T](
+    val typeInfo: TypeInformation[T],
+    val fieldIndexes: Array[Int],
+    val fieldNames: Array[String],
+    val evalMethod: Method)
+  extends ReflectiveFunctionBase(evalMethod)
+  with TableFunction {
+
+  if (fieldIndexes.length != fieldNames.length) {
+    throw new TableException(
+      "Number of field indexes and field names must be equal.")
+  }
+
+  // check uniqueness of field names
+  if (fieldNames.length != fieldNames.toSet.size) {
+    throw new TableException(
+      "Table field names must be unique.")
+  }
+
+  val fieldTypes: Array[TypeInformation[_]] =
+    typeInfo match {
+      case cType: CompositeType[T] =>
+        if (fieldNames.length != cType.getArity) {
+          throw new TableException(
+            s"Arity of type (" + cType.getFieldNames.deep + ") " +
+              "not equal to number of field names " + fieldNames.deep + ".")
+        }
+        fieldIndexes.map(cType.getTypeAt(_).asInstanceOf[TypeInformation[_]])
+      case aType: AtomicType[T] =>
+        if (fieldIndexes.length != 1 || fieldIndexes(0) != 0) {
+          throw new TableException(
+            "Non-composite input type may have only a single field and its index must be 0.")
+        }
+        Array(aType)
+    }
+
+  override def getElementType(arguments: util.List[AnyRef]): Type = classOf[Array[Object]]
+
+  override def getRowType(typeFactory: RelDataTypeFactory,
+                          arguments: util.List[AnyRef]): RelDataType = {
+    val flinkTypeFactory = typeFactory.asInstanceOf[FlinkTypeFactory]
+    val builder = flinkTypeFactory.builder
+    fieldNames
+      .zip(fieldTypes)
+      .foreach { f =>
+        builder.add(f._1, flinkTypeFactory.createTypeFromTypeInfo(f._2)).nullable(true)
+      }
+    builder.build
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/GenericRelDataType.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/GenericRelDataType.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/GenericRelDataType.scala
new file mode 100644
index 0000000..d93908b
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/GenericRelDataType.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.schema
+
+import org.apache.calcite.sql.`type`.{BasicSqlType, SqlTypeName}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.table.calcite.FlinkTypeSystem
+
+/**
+  * Generic type for encapsulating Flink's [[TypeInformation]].
+  *
+  * @param typeInfo TypeInformation to encapsulate
+  * @param typeSystem Flink's type system
+  */
+class GenericRelDataType(
+    val typeInfo: TypeInformation[_],
+    typeSystem: FlinkTypeSystem)
+  extends BasicSqlType(
+    typeSystem,
+    SqlTypeName.ANY) {
+
+  override def toString = s"ANY($typeInfo)"
+
+  def canEqual(other: Any): Boolean = other.isInstanceOf[GenericRelDataType]
+
+  override def equals(other: Any): Boolean = other match {
+    case that: GenericRelDataType =>
+      super.equals(that) &&
+        (that canEqual this) &&
+        typeInfo == that.typeInfo
+    case _ => false
+  }
+
+  override def hashCode(): Int = {
+    typeInfo.hashCode()
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RelTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RelTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RelTable.scala
new file mode 100644
index 0000000..30052a8
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/RelTable.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.schema
+
+import org.apache.calcite.plan.RelOptTable
+import org.apache.calcite.plan.RelOptTable.ToRelContext
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
+import org.apache.calcite.schema.Schema.TableType
+import org.apache.calcite.schema.impl.AbstractTable
+import org.apache.calcite.schema.TranslatableTable
+
+/**
+ * A [[org.apache.calcite.schema.Table]] implementation for registering
+ * Table API Tables in the Calcite schema to be used by Flink SQL.
+ * It implements [[TranslatableTable]] so that its logical scan
+ * can be converted to a relational expression.
+ *
+ * @see [[DataSetTable]]
+ */
+class RelTable(relNode: RelNode) extends AbstractTable with TranslatableTable {
+
+  override def getJdbcTableType: TableType = ???
+
+  override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = relNode.getRowType
+
+  override def toRel(context: ToRelContext, relOptTable: RelOptTable): RelNode = {
+    relNode
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala
new file mode 100644
index 0000000..0f55daf
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/schema/TableSourceTable.scala
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.plan.schema
+
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.sources.TableSource
+import org.apache.flink.types.Row
+
+/** Table which defines an external table via a [[TableSource]] */
+class TableSourceTable(val tableSource: TableSource[_])
+  extends FlinkTable[Row](
+    typeInfo = new RowTypeInfo(tableSource.getFieldTypes: _*),
+    fieldIndexes = 0.until(tableSource.getNumberOfFields).toArray,
+    fieldNames = tableSource.getFieldsNames)

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CountPartitionFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CountPartitionFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CountPartitionFunction.scala
new file mode 100644
index 0000000..9fe4ec3
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CountPartitionFunction.scala
@@ -0,0 +1,40 @@
+/*
+ * 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 java.lang.Iterable
+
+import org.apache.flink.api.common.functions.RichMapPartitionFunction
+import org.apache.flink.util.Collector
+
+class CountPartitionFunction[IN] extends RichMapPartitionFunction[IN, (Int, Long)] {
+
+  override def mapPartition(value: Iterable[IN], out: Collector[(Int, Long)]): Unit = {
+    val partitionIndex = getRuntimeContext.getIndexOfThisSubtask
+    var elementCount = 0L
+    val iterator = value.iterator()
+    while (iterator.hasNext) {
+      if (elementCount != Long.MaxValue) { // prevent overflow
+        elementCount += 1L
+      }
+      iterator.next()
+    }
+    out.collect(partitionIndex, elementCount)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
new file mode 100644
index 0000000..715848d
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
@@ -0,0 +1,52 @@
+/*
+ * 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.{FlatJoinFunction, RichFlatJoinFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+class FlatJoinRunner[IN1, IN2, OUT](
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[OUT])
+  extends RichFlatJoinFunction[IN1, IN2, OUT]
+  with ResultTypeQueryable[OUT]
+  with Compiler[FlatJoinFunction[IN1, IN2, OUT]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  private var function: FlatJoinFunction[IN1, IN2, OUT] = null
+
+  override def open(parameters: Configuration): Unit = {
+    LOG.debug(s"Compiling FlatJoinFunction: $name \n\n Code:\n$code")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+    LOG.debug("Instantiating FlatJoinFunction.")
+    function = clazz.newInstance()
+  }
+
+  override def join(first: IN1, second: IN2, out: Collector[OUT]): Unit =
+    function.join(first, second, out)
+
+  override def getProducedType: TypeInformation[OUT] = returnType
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
new file mode 100644
index 0000000..a7bd980
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatMapRunner.scala
@@ -0,0 +1,52 @@
+/*
+ * 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.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.util.Collector
+import org.slf4j.LoggerFactory
+
+class FlatMapRunner[IN, OUT](
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[OUT])
+  extends RichFlatMapFunction[IN, OUT]
+  with ResultTypeQueryable[OUT]
+  with Compiler[FlatMapFunction[IN, OUT]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  private var function: FlatMapFunction[IN, OUT] = _
+
+  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()
+  }
+
+  override def flatMap(in: IN, out: Collector[OUT]): Unit =
+    function.flatMap(in, out)
+
+  override def getProducedType: TypeInformation[OUT] = returnType
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/IntersectCoGroupFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/IntersectCoGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/IntersectCoGroupFunction.scala
new file mode 100644
index 0000000..39d2914
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/IntersectCoGroupFunction.scala
@@ -0,0 +1,42 @@
+/*
+ * 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 java.lang.{Iterable => JIterable}
+
+import org.apache.flink.api.common.functions.CoGroupFunction
+import org.apache.flink.util.Collector
+
+class IntersectCoGroupFunction[T](all: Boolean) extends CoGroupFunction[T, T, T]{
+  override def coGroup(first: JIterable[T], second: JIterable[T], out: Collector[T]): Unit = {
+    if (first == null || second == null) return
+    val leftIter = first.iterator()
+    val rightIter = second.iterator()
+    if (all) {
+      while (leftIter.hasNext && rightIter.hasNext) {
+        out.collect(leftIter.next)
+        rightIter.next
+      }
+    } else {
+      if (leftIter.hasNext && rightIter.hasNext) {
+        out.collect(leftIter.next)
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/LimitFilterFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/LimitFilterFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/LimitFilterFunction.scala
new file mode 100644
index 0000000..8441245
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/LimitFilterFunction.scala
@@ -0,0 +1,64 @@
+/*
+ * 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.RichFilterFunction
+import org.apache.flink.configuration.Configuration
+
+import scala.collection.JavaConverters._
+
+
+class LimitFilterFunction[T](
+    limitStart: Long,
+    limitEnd: Long,
+    broadcastName: String)
+  extends RichFilterFunction[T] {
+
+  var partitionIndex: Int = _
+  var elementCount: Long = _
+  var countList: Array[Long] = _
+
+  override def open(config: Configuration) {
+    partitionIndex = getRuntimeContext.getIndexOfThisSubtask
+
+    val countPartitionResult = getRuntimeContext
+      .getBroadcastVariable[(Int, Long)](broadcastName)
+      .asScala
+
+    // sort by partition index, extract number per partition, sum with intermediate results
+    countList = countPartitionResult.sortWith(_._1 < _._1).map(_._2).scanLeft(0L) { case (a, b) =>
+        val sum = a + b
+        if (sum < 0L) { // prevent overflow
+          Long.MaxValue
+        }
+        sum
+    }.toArray
+
+    elementCount = 0
+  }
+
+  override def filter(value: T): Boolean = {
+    if (elementCount != Long.MaxValue) { // prevent overflow
+      elementCount += 1L
+    }
+    // we filter out records that are not within the limit (Long.MaxValue is unlimited)
+    limitStart - countList(partitionIndex) < elementCount &&
+      (limitEnd == Long.MaxValue || limitEnd - countList(partitionIndex) >= elementCount)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
new file mode 100644
index 0000000..cf32404
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinLeftRunner.scala
@@ -0,0 +1,33 @@
+/*
+ * 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.typeinfo.TypeInformation
+import org.apache.flink.util.Collector
+
+class MapJoinLeftRunner[IN1, IN2, OUT](
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[OUT],
+    broadcastSetName: String)
+  extends MapSideJoinRunner[IN1, IN2, IN2, IN1, OUT](name, code, returnType, broadcastSetName) {
+
+  override def flatMap(multiInput: IN1, out: Collector[OUT]): Unit =
+    function.join(multiInput, singleInput, out)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
new file mode 100644
index 0000000..c4bc0d1
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapJoinRightRunner.scala
@@ -0,0 +1,33 @@
+/*
+ * 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.typeinfo.TypeInformation
+import org.apache.flink.util.Collector
+
+class MapJoinRightRunner[IN1, IN2, OUT](
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[OUT],
+    broadcastSetName: String)
+  extends MapSideJoinRunner[IN1, IN2, IN1, IN2, OUT](name, code, returnType, broadcastSetName) {
+
+  override def flatMap(multiInput: IN2, out: Collector[OUT]): Unit =
+    function.join(singleInput, multiInput, out)
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
new file mode 100644
index 0000000..51e2fc5
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.runtime
+
+import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.configuration.Configuration
+import org.slf4j.LoggerFactory
+
+class MapRunner[IN, OUT](
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[OUT])
+  extends RichMapFunction[IN, OUT]
+  with ResultTypeQueryable[OUT]
+  with Compiler[MapFunction[IN, OUT]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  private var function: MapFunction[IN, OUT] = null
+
+  override def open(parameters: Configuration): Unit = {
+    LOG.debug(s"Compiling MapFunction: $name \n\n Code:\n$code")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+    LOG.debug("Instantiating MapFunction.")
+    function = clazz.newInstance()
+  }
+
+  override def map(in: IN): OUT =
+    function.map(in)
+
+  override def getProducedType: TypeInformation[OUT] = returnType
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
new file mode 100644
index 0000000..f12590f
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
@@ -0,0 +1,51 @@
+/*
+ * 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.{FlatJoinFunction, RichFlatMapFunction}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable
+import org.apache.flink.table.codegen.Compiler
+import org.apache.flink.configuration.Configuration
+import org.slf4j.LoggerFactory
+
+abstract class MapSideJoinRunner[IN1, IN2, SINGLE_IN, MULTI_IN, OUT](
+    name: String,
+    code: String,
+    @transient returnType: TypeInformation[OUT],
+    broadcastSetName: String)
+  extends RichFlatMapFunction[MULTI_IN, OUT]
+    with ResultTypeQueryable[OUT]
+    with Compiler[FlatJoinFunction[IN1, IN2, OUT]] {
+
+  val LOG = LoggerFactory.getLogger(this.getClass)
+
+  protected var function: FlatJoinFunction[IN1, IN2, OUT] = _
+  protected var singleInput: SINGLE_IN = _
+
+  override def open(parameters: Configuration): Unit = {
+    LOG.debug(s"Compiling FlatJoinFunction: $name \n\n Code:\n$code")
+    val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
+    LOG.debug("Instantiating FlatJoinFunction.")
+    function = clazz.newInstance()
+    singleInput = getRuntimeContext.getBroadcastVariable(broadcastSetName).get(0)
+  }
+
+  override def getProducedType: TypeInformation[OUT] = returnType
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MinusCoGroupFunction.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MinusCoGroupFunction.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MinusCoGroupFunction.scala
new file mode 100644
index 0000000..41679ee
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MinusCoGroupFunction.scala
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.runtime
+
+import java.lang.Iterable
+
+import org.apache.flink.api.common.functions.CoGroupFunction
+import org.apache.flink.util.Collector
+
+class MinusCoGroupFunction[T](all: Boolean) extends CoGroupFunction[T, T, T] {
+  override def coGroup(first: Iterable[T], second: Iterable[T], out: Collector[T]): Unit = {
+    if (first == null || second == null) return
+    val leftIter = first.iterator
+    val rightIter = second.iterator
+
+    if (all) {
+      while (rightIter.hasNext && leftIter.hasNext) {
+        leftIter.next()
+        rightIter.next()
+      }
+
+      while (leftIter.hasNext) {
+        out.collect(leftIter.next())
+      }
+    } else {
+      if (!rightIter.hasNext && leftIter.hasNext) {
+        out.collect(leftIter.next())
+      }
+    }
+  }
+}
+


[03/47] flink git commit: [FLINK-4704] [table] Refactor package structure of flink-table.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnionITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnionITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnionITCase.scala
new file mode 100644
index 0000000..f35ee76
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnionITCase.scala
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.table
+
+import org.apache.flink.api.scala._
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.scala.stream.utils.{StreamITCase, StreamTestData}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.{TableEnvironment, ValidationException}
+import org.junit.Assert._
+import org.junit.Test
+
+import scala.collection.mutable
+
+class UnionITCase extends StreamingMultipleProgramsTestBase {
+
+  @Test
+  def testUnion(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    StreamITCase.testResults = mutable.MutableList()
+    val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'd, 'e, 'f)
+
+    val unionDs = ds1.unionAll(ds2).select('c)
+
+    val results = unionDs.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList(
+        "Hi", "Hello", "Hello world", "Hi", "Hello", "Hello world")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test
+  def testUnionWithFilter(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    StreamITCase.testResults = mutable.MutableList()
+    val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = StreamTestData.get5TupleDataStream(env).toTable(tEnv, 'a, 'b, 'd, 'c, 'e)
+
+    val unionDs = ds1.unionAll(ds2.select('a, 'b, 'c)).filter('b < 2).select('c)
+
+    val results = unionDs.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    val expected = mutable.MutableList("Hi", "Hallo")
+    assertEquals(expected.sorted, StreamITCase.testResults.sorted)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testUnionFieldsNameNotOverlap1(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    StreamITCase.testResults = mutable.MutableList()
+    val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = StreamTestData.get5TupleDataStream(env).toTable(tEnv, 'a, 'b, 'd, 'c, 'e)
+
+    val unionDs = ds1.unionAll(ds2)
+
+    val results = unionDs.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    assertEquals(true, StreamITCase.testResults.isEmpty)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testUnionFieldsNameNotOverlap2(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+
+    StreamITCase.testResults = mutable.MutableList()
+    val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c)
+    val ds2 = StreamTestData.get5TupleDataStream(env).toTable(tEnv, 'a, 'b, 'c, 'd, 'e)
+      .select('a, 'b, 'c)
+
+    val unionDs = ds1.unionAll(ds2)
+
+    val results = unionDs.toDataStream[Row]
+    results.addSink(new StreamITCase.StringSink)
+    env.execute()
+
+    println(StreamITCase.testResults)
+    assertEquals(true, StreamITCase.testResults.isEmpty)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testUnionTablesFromDifferentEnvs(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv1 = TableEnvironment.getTableEnvironment(env)
+    val tEnv2 = TableEnvironment.getTableEnvironment(env)
+
+    val ds1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv1, 'a, 'b, 'c)
+    val ds2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv2, 'a, 'b, 'c)
+
+    // Must fail. Tables are bound to different TableEnvironments.
+    ds1.unionAll(ds2)
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnsupportedOpsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnsupportedOpsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnsupportedOpsTest.scala
new file mode 100644
index 0000000..b6a6660
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UnsupportedOpsTest.scala
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.table
+
+import org.apache.flink.table.api.scala.stream.utils.StreamTestData
+import org.apache.flink.table.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
+import org.apache.flink.table.api.{TableEnvironment, ValidationException}
+import org.junit.Test
+
+class UnsupportedOpsTest extends StreamingMultipleProgramsTestBase {
+
+  @Test(expected = classOf[ValidationException])
+  def testSelectWithAggregation(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).select('_1.min)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testDistinct(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).distinct()
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testSort(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv).orderBy('_1.desc)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testJoin(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    t1.join(t2)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testUnion(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    t1.union(t2)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testIntersect(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    t1.intersect(t2)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testIntersectAll(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    t1.intersectAll(t2)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testMinus(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    t1.minus(t2)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testMinusAll(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    val t2 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    t1.minusAll(t2)
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testLimit(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env)
+    val t1 = StreamTestData.getSmall3TupleDataStream(env).toTable(tEnv)
+    t1.limit(0,5)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UserDefinedTableFunctionTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UserDefinedTableFunctionTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UserDefinedTableFunctionTest.scala
new file mode 100644
index 0000000..168f9ec
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/table/UserDefinedTableFunctionTest.scala
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.table.api.scala.stream.table
+
+import org.apache.flink.api.scala._
+import org.apache.flink.types.Row
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api._
+import org.apache.flink.table.expressions.utils._
+import org.apache.flink.table.utils.TableTestUtil._
+import org.apache.flink.table.utils._
+import org.apache.flink.streaming.api.datastream.{DataStream => JDataStream}
+import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaExecutionEnv}
+import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment => ScalaExecutionEnv}
+import org.junit.Assert.{assertTrue, fail}
+import org.junit.Test
+import org.mockito.Mockito._
+
+class UserDefinedTableFunctionTest extends TableTestBase {
+
+  @Test
+  def testJavaScalaTableAPIEquality(): Unit = {
+    // mock
+    val ds = mock(classOf[DataStream[Row]])
+    val jDs = mock(classOf[JDataStream[Row]])
+    val typeInfo = new RowTypeInfo(Seq(Types.INT, Types.LONG, Types.STRING): _*)
+    when(ds.javaStream).thenReturn(jDs)
+    when(jDs.getType).thenReturn(typeInfo)
+
+    // Scala environment
+    val env = mock(classOf[ScalaExecutionEnv])
+    val tableEnv = TableEnvironment.getTableEnvironment(env)
+    val in1 = ds.toTable(tableEnv).as('a, 'b, 'c)
+
+    // Java environment
+    val javaEnv = mock(classOf[JavaExecutionEnv])
+    val javaTableEnv = TableEnvironment.getTableEnvironment(javaEnv)
+    val in2 = javaTableEnv.fromDataStream(jDs).as("a, b, c")
+
+    // test cross join
+    val func1 = new TableFunc1
+    javaTableEnv.registerFunction("func1", func1)
+    var scalaTable = in1.join(func1('c) as 's).select('c, 's)
+    var javaTable = in2.join("func1(c).as(s)").select("c, s")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test left outer join
+    scalaTable = in1.leftOuterJoin(func1('c) as 's).select('c, 's)
+    javaTable = in2.leftOuterJoin("as(func1(c), s)").select("c, s")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test overloading
+    scalaTable = in1.join(func1('c, "$") as 's).select('c, 's)
+    javaTable = in2.join("func1(c, '$') as (s)").select("c, s")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test custom result type
+    val func2 = new TableFunc2
+    javaTableEnv.registerFunction("func2", func2)
+    scalaTable = in1.join(func2('c) as ('name, 'len)).select('c, 'name, 'len)
+    javaTable = in2.join("func2(c).as(name, len)").select("c, name, len")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test hierarchy generic type
+    val hierarchy = new HierarchyTableFunction
+    javaTableEnv.registerFunction("hierarchy", hierarchy)
+    scalaTable = in1.join(hierarchy('c) as ('name, 'adult, 'len))
+      .select('c, 'name, 'len, 'adult)
+    javaTable = in2.join("AS(hierarchy(c), name, adult, len)")
+      .select("c, name, len, adult")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test pojo type
+    val pojo = new PojoTableFunc
+    javaTableEnv.registerFunction("pojo", pojo)
+    scalaTable = in1.join(pojo('c))
+      .select('c, 'name, 'age)
+    javaTable = in2.join("pojo(c)")
+      .select("c, name, age")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test with filter
+    scalaTable = in1.join(func2('c) as ('name, 'len))
+      .select('c, 'name, 'len).filter('len > 2)
+    javaTable = in2.join("func2(c) as (name, len)")
+      .select("c, name, len").filter("len > 2")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // test with scalar function
+    scalaTable = in1.join(func1('c.substring(2)) as 's)
+      .select('a, 'c, 's)
+    javaTable = in2.join("func1(substring(c, 2)) as (s)")
+      .select("a, c, s")
+    verifyTableEquals(scalaTable, javaTable)
+
+    // check scala object is forbidden
+    expectExceptionThrown(
+      tableEnv.registerFunction("func3", ObjectTableFunction), "Scala object")
+    expectExceptionThrown(
+      javaTableEnv.registerFunction("func3", ObjectTableFunction), "Scala object")
+    expectExceptionThrown(
+      in1.join(ObjectTableFunction('a, 1)), "Scala object")
+
+  }
+
+  @Test
+  def testInvalidTableFunction(): Unit = {
+    // mock
+    val util = streamTestUtil()
+    val t = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val tEnv = TableEnvironment.getTableEnvironment(mock(classOf[JavaExecutionEnv]))
+
+    //=================== check scala object is forbidden =====================
+    // Scala table environment register
+    expectExceptionThrown(util.addFunction("udtf", ObjectTableFunction), "Scala object")
+    // Java table environment register
+    expectExceptionThrown(tEnv.registerFunction("udtf", ObjectTableFunction), "Scala object")
+    // Scala Table API directly call
+    expectExceptionThrown(t.join(ObjectTableFunction('a, 1)), "Scala object")
+
+
+    //============ throw exception when table function is not registered =========
+    // Java Table API call
+    expectExceptionThrown(t.join("nonexist(a)"), "Undefined function: NONEXIST")
+    // SQL API call
+    expectExceptionThrown(
+      util.tEnv.sql("SELECT * FROM MyTable, LATERAL TABLE(nonexist(a))"),
+      "No match found for function signature nonexist(<NUMERIC>)")
+
+
+    //========= throw exception when the called function is a scalar function ====
+    util.addFunction("func0", Func0)
+    // Java Table API call
+    expectExceptionThrown(
+      t.join("func0(a)"),
+      "only accept expressions that define table functions",
+      classOf[TableException])
+    // SQL API call
+    // NOTE: it doesn't throw an exception but an AssertionError, maybe a Calcite bug
+    expectExceptionThrown(
+      util.tEnv.sql("SELECT * FROM MyTable, LATERAL TABLE(func0(a))"),
+      null,
+      classOf[AssertionError])
+
+    //========== throw exception when the parameters is not correct ===============
+    // Java Table API call
+    util.addFunction("func2", new TableFunc2)
+    expectExceptionThrown(
+      t.join("func2(c, c)"),
+      "Given parameters of function 'FUNC2' do not match any signature")
+    // SQL API call
+    expectExceptionThrown(
+      util.tEnv.sql("SELECT * FROM MyTable, LATERAL TABLE(func2(c, c))"),
+      "No match found for function signature func2(<CHARACTER>, <CHARACTER>)")
+  }
+
+  @Test
+  def testCrossJoin(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = util.addFunction("func1", new TableFunc1)
+
+    val result1 = table.join(function('c) as 's).select('c, 's)
+
+    val expected1 = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", s"$function($$2)"),
+        term("function", function),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "s")
+    )
+
+    util.verifyTable(result1, expected1)
+
+    // test overloading
+
+    val result2 = table.join(function('c, "$") as 's).select('c, 's)
+
+    val expected2 = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", s"$function($$2, '$$')"),
+        term("function", function),
+        term("rowType",
+             "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "s")
+    )
+
+    util.verifyTable(result2, expected2)
+  }
+
+  @Test
+  def testLeftOuterJoin(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = util.addFunction("func1", new TableFunc1)
+
+    val result = table.leftOuterJoin(function('c) as 's).select('c, 's)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", s"$function($$2)"),
+        term("function", function),
+        term("rowType",
+          "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
+        term("joinType", "LEFT")
+      ),
+      term("select", "c", "s")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testCustomType(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = util.addFunction("func2", new TableFunc2)
+
+    val result = table.join(function('c) as ('name, 'len)).select('c, 'name, 'len)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", s"$function($$2)"),
+        term("function", function),
+        term("rowType",
+          "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
+           "VARCHAR(2147483647) name, INTEGER len)"),
+        term("joinType", "INNER")
+      ),
+      term("select", "c", "name", "len")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testHierarchyType(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = util.addFunction("hierarchy", new HierarchyTableFunction)
+
+    val result = table.join(function('c) as ('name, 'adult, 'len))
+
+    val expected = unaryNode(
+      "DataStreamCorrelate",
+      streamTableNode(0),
+      term("invocation", s"$function($$2)"),
+      term("function", function),
+      term("rowType",
+        "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c," +
+        " VARCHAR(2147483647) name, BOOLEAN adult, INTEGER len)"),
+      term("joinType", "INNER")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testPojoType(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = util.addFunction("pojo", new PojoTableFunc)
+
+    val result = table.join(function('c))
+
+    val expected = unaryNode(
+      "DataStreamCorrelate",
+      streamTableNode(0),
+      term("invocation", s"$function($$2)"),
+      term("function", function),
+      term("rowType",
+        "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
+         "INTEGER age, VARCHAR(2147483647) name)"),
+      term("joinType", "INNER")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testFilter(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = util.addFunction("func2", new TableFunc2)
+
+    val result = table
+      .join(function('c) as ('name, 'len))
+      .select('c, 'name, 'len)
+      .filter('len > 2)
+
+    val expected = unaryNode(
+      "DataStreamCalc",
+      unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation", s"$function($$2)"),
+        term("function", function),
+        term("rowType",
+          "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, " +
+          "VARCHAR(2147483647) name, INTEGER len)"),
+        term("joinType", "INNER"),
+        term("condition", ">($1, 2)")
+      ),
+      term("select", "c", "name", "len")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  @Test
+  def testScalarFunction(): Unit = {
+    val util = streamTestUtil()
+    val table = util.addTable[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+    val function = util.addFunction("func1", new TableFunc1)
+
+    val result = table.join(function('c.substring(2)) as 's)
+
+    val expected = unaryNode(
+        "DataStreamCorrelate",
+        streamTableNode(0),
+        term("invocation",  s"$function(SUBSTRING($$2, 2, CHAR_LENGTH($$2)))"),
+        term("function", function),
+        term("rowType",
+          "RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)"),
+        term("joinType", "INNER")
+    )
+
+    util.verifyTable(result, expected)
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  private def expectExceptionThrown(
+      function: => Unit,
+      keywords: String,
+      clazz: Class[_ <: Throwable] = classOf[ValidationException])
+    : Unit = {
+    try {
+      function
+      fail(s"Expected a $clazz, but no exception is thrown.")
+    } catch {
+      case e if e.getClass == clazz =>
+        if (keywords != null) {
+          assertTrue(
+            s"The exception message '${e.getMessage}' doesn't contain keyword '$keywords'",
+            e.getMessage.contains(keywords))
+        }
+      case e: Throwable => fail(s"Expected throw ${clazz.getSimpleName}, but is $e.")
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
new file mode 100644
index 0000000..f826bba
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamITCase.scala
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.utils
+
+import java.util.Collections
+
+import org.apache.flink.types.Row
+import org.junit.Assert._
+
+import scala.collection.mutable
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
+
+import scala.collection.JavaConverters._
+
+object StreamITCase {
+
+  var testResults = mutable.MutableList.empty[String]
+
+  def clear = {
+    StreamITCase.testResults.clear()
+  }
+
+  def compareWithList(expected: java.util.List[String]): Unit = {
+    Collections.sort(expected)
+    assertEquals(expected.asScala, StreamITCase.testResults.sorted)
+  }
+
+  final class StringSink extends RichSinkFunction[Row]() {
+    def invoke(value: Row) {
+      testResults.synchronized {
+        testResults += value.toString 
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamTestData.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamTestData.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamTestData.scala
new file mode 100644
index 0000000..6745039
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/api/scala/stream/utils/StreamTestData.scala
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.api.scala.stream.utils
+
+import org.apache.flink.api.scala._
+import scala.collection.mutable
+import org.apache.flink.streaming.api.scala.DataStream
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+
+object StreamTestData {
+
+  def getSmall3TupleDataStream(env: StreamExecutionEnvironment): DataStream[(Int, Long, String)] = {
+    val data = new mutable.MutableList[(Int, Long, String)]
+    data.+=((1, 1L, "Hi"))
+    data.+=((2, 2L, "Hello"))
+    data.+=((3, 2L, "Hello world"))
+    env.fromCollection(data)
+  }
+
+  def get3TupleDataStream(env: StreamExecutionEnvironment): DataStream[(Int, Long, String)] = {
+    val data = new mutable.MutableList[(Int, Long, String)]
+    data.+=((1, 1L, "Hi"))
+    data.+=((2, 2L, "Hello"))
+    data.+=((3, 2L, "Hello world"))
+    data.+=((4, 3L, "Hello world, how are you?"))
+    data.+=((5, 3L, "I am fine."))
+    data.+=((6, 3L, "Luke Skywalker"))
+    data.+=((7, 4L, "Comment#1"))
+    data.+=((8, 4L, "Comment#2"))
+    data.+=((9, 4L, "Comment#3"))
+    data.+=((10, 4L, "Comment#4"))
+    data.+=((11, 5L, "Comment#5"))
+    data.+=((12, 5L, "Comment#6"))
+    data.+=((13, 5L, "Comment#7"))
+    data.+=((14, 5L, "Comment#8"))
+    data.+=((15, 5L, "Comment#9"))
+    data.+=((16, 6L, "Comment#10"))
+    data.+=((17, 6L, "Comment#11"))
+    data.+=((18, 6L, "Comment#12"))
+    data.+=((19, 6L, "Comment#13"))
+    data.+=((20, 6L, "Comment#14"))
+    data.+=((21, 6L, "Comment#15"))
+    env.fromCollection(data)
+  }
+
+  def get5TupleDataStream(env: StreamExecutionEnvironment):
+      DataStream[(Int, Long, Int, String, Long)] = {
+
+    val data = new mutable.MutableList[(Int, Long, Int, String, Long)]
+    data.+=((1, 1L, 0, "Hallo", 1L))
+    data.+=((2, 2L, 1, "Hallo Welt", 2L))
+    data.+=((2, 3L, 2, "Hallo Welt wie", 1L))
+    data.+=((3, 4L, 3, "Hallo Welt wie gehts?", 2L))
+    data.+=((3, 5L, 4, "ABC", 2L))
+    data.+=((3, 6L, 5, "BCD", 3L))
+    data.+=((4, 7L, 6, "CDE", 2L))
+    data.+=((4, 8L, 7, "DEF", 1L))
+    data.+=((4, 9L, 8, "EFG", 1L))
+    data.+=((4, 10L, 9, "FGH", 2L))
+    data.+=((5, 11L, 10, "GHI", 1L))
+    data.+=((5, 12L, 11, "HIJ", 3L))
+    data.+=((5, 13L, 12, "IJK", 3L))
+    data.+=((5, 14L, 13, "JKL", 2L))
+    data.+=((5, 15L, 14, "KLM", 2L))
+    env.fromCollection(data)
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ArrayTypeTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ArrayTypeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ArrayTypeTest.scala
new file mode 100644
index 0000000..49cf572
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ArrayTypeTest.scala
@@ -0,0 +1,360 @@
+/*
+ * 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.expressions
+
+import java.sql.Date
+
+import org.apache.flink.api.common.typeinfo.{PrimitiveArrayTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.ObjectArrayTypeInfo
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.utils.ExpressionTestBase
+import org.apache.flink.table.api.{Types, ValidationException}
+import org.junit.Test
+
+class ArrayTypeTest extends ExpressionTestBase {
+
+  @Test(expected = classOf[ValidationException])
+  def testObviousInvalidIndexTableApi(): Unit = {
+    testTableApi('f2.at(0), "FAIL", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testEmptyArraySql(): Unit = {
+    testSqlApi("ARRAY[]", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testEmptyArrayTableApi(): Unit = {
+    testTableApi("FAIL", "array()", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testNullArraySql(): Unit = {
+    testSqlApi("ARRAY[NULL]", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testDifferentTypesArraySql(): Unit = {
+    testSqlApi("ARRAY[1, TRUE]", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testDifferentTypesArrayTableApi(): Unit = {
+    testTableApi("FAIL", "array(1, true)", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testUnsupportedComparison(): Unit = {
+    testAllApis(
+      'f2 <= 'f5.at(1),
+      "f2 <= f5.at(1)",
+      "f2 <= f5[1]",
+      "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testElementNonArray(): Unit = {
+    testTableApi(
+      'f0.element(),
+      "FAIL",
+      "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testElementNonArraySql(): Unit = {
+    testSqlApi(
+      "ELEMENT(f0)",
+      "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testCardinalityOnNonArray(): Unit = {
+    testTableApi('f0.cardinality(), "FAIL", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testCardinalityOnNonArraySql(): Unit = {
+    testSqlApi("CARDINALITY(f0)", "FAIL")
+  }
+
+  @Test
+  def testArrayLiterals(): Unit = {
+    // primitive literals
+    testAllApis(array(1, 2, 3), "array(1, 2, 3)", "ARRAY[1, 2, 3]", "[1, 2, 3]")
+
+    testAllApis(
+      array(true, true, true),
+      "array(true, true, true)",
+      "ARRAY[TRUE, TRUE, TRUE]",
+      "[true, true, true]")
+
+    // object literals
+    testTableApi(array(BigDecimal(1), BigDecimal(1)), "array(1p, 1p)", "[1, 1]")
+
+    testAllApis(
+      array(array(array(1), array(1))),
+      "array(array(array(1), array(1)))",
+      "ARRAY[ARRAY[ARRAY[1], ARRAY[1]]]",
+      "[[[1], [1]]]")
+
+    testAllApis(
+      array(1 + 1, 3 * 3),
+      "array(1 + 1, 3 * 3)",
+      "ARRAY[1 + 1, 3 * 3]",
+      "[2, 9]")
+
+    testAllApis(
+      array(Null(Types.INT), 1),
+      "array(Null(INT), 1)",
+      "ARRAY[NULLIF(1,1), 1]",
+      "[null, 1]")
+
+    testAllApis(
+      array(array(Null(Types.INT), 1)),
+      "array(array(Null(INT), 1))",
+      "ARRAY[ARRAY[NULLIF(1,1), 1]]",
+      "[[null, 1]]")
+
+    // implicit conversion
+    testTableApi(
+      Array(1, 2, 3),
+      "array(1, 2, 3)",
+      "[1, 2, 3]")
+
+    testTableApi(
+      Array[Integer](1, 2, 3),
+      "array(1, 2, 3)",
+      "[1, 2, 3]")
+
+    testAllApis(
+      Array(Date.valueOf("1985-04-11")),
+      "array('1985-04-11'.toDate)",
+      "ARRAY[DATE '1985-04-11']",
+      "[1985-04-11]")
+
+    testAllApis(
+      Array(BigDecimal(2.0002), BigDecimal(2.0003)),
+      "Array(2.0002p, 2.0003p)",
+      "ARRAY[CAST(2.0002 AS DECIMAL), CAST(2.0003 AS DECIMAL)]",
+      "[2.0002, 2.0003]")
+
+    testAllApis(
+      Array(Array(x = true)),
+      "Array(Array(true))",
+      "ARRAY[ARRAY[TRUE]]",
+      "[[true]]")
+
+    testAllApis(
+      Array(Array(1, 2, 3), Array(3, 2, 1)),
+      "Array(Array(1, 2, 3), Array(3, 2, 1))",
+      "ARRAY[ARRAY[1, 2, 3], ARRAY[3, 2, 1]]",
+      "[[1, 2, 3], [3, 2, 1]]")
+  }
+
+  @Test
+  def testArrayField(): Unit = {
+    testAllApis(
+      array('f0, 'f1),
+      "array(f0, f1)",
+      "ARRAY[f0, f1]",
+      "[null, 42]")
+
+    testAllApis(
+      array('f0, 'f1),
+      "array(f0, f1)",
+      "ARRAY[f0, f1]",
+      "[null, 42]")
+
+    testAllApis(
+      'f2,
+      "f2",
+      "f2",
+      "[1, 2, 3]")
+
+    testAllApis(
+      'f3,
+      "f3",
+      "f3",
+      "[1984-03-12, 1984-02-10]")
+
+    testAllApis(
+      'f5,
+      "f5",
+      "f5",
+      "[[1, 2, 3], null]")
+
+    testAllApis(
+      'f6,
+      "f6",
+      "f6",
+      "[1, null, null, 4]")
+
+    testAllApis(
+      'f2,
+      "f2",
+      "f2",
+      "[1, 2, 3]")
+
+    testAllApis(
+      'f2.at(1),
+      "f2.at(1)",
+      "f2[1]",
+      "1")
+
+    testAllApis(
+      'f3.at(1),
+      "f3.at(1)",
+      "f3[1]",
+      "1984-03-12")
+
+    testAllApis(
+      'f3.at(2),
+      "f3.at(2)",
+      "f3[2]",
+      "1984-02-10")
+
+    testAllApis(
+      'f5.at(1).at(2),
+      "f5.at(1).at(2)",
+      "f5[1][2]",
+      "2")
+
+    testAllApis(
+      'f5.at(2).at(2),
+      "f5.at(2).at(2)",
+      "f5[2][2]",
+      "null")
+
+    testAllApis(
+      'f4.at(2).at(2),
+      "f4.at(2).at(2)",
+      "f4[2][2]",
+      "null")
+  }
+
+  @Test
+  def testArrayOperations(): Unit = {
+    // cardinality
+    testAllApis(
+      'f2.cardinality(),
+      "f2.cardinality()",
+      "CARDINALITY(f2)",
+      "3")
+
+    testAllApis(
+      'f4.cardinality(),
+      "f4.cardinality()",
+      "CARDINALITY(f4)",
+      "null")
+
+    // element
+    testAllApis(
+      'f9.element(),
+      "f9.element()",
+      "ELEMENT(f9)",
+      "1")
+
+    testAllApis(
+      'f8.element(),
+      "f8.element()",
+      "ELEMENT(f8)",
+      "4.0")
+
+    testAllApis(
+      'f10.element(),
+      "f10.element()",
+      "ELEMENT(f10)",
+      "null")
+
+    testAllApis(
+      'f4.element(),
+      "f4.element()",
+      "ELEMENT(f4)",
+      "null")
+
+    // comparison
+    testAllApis(
+      'f2 === 'f5.at(1),
+      "f2 === f5.at(1)",
+      "f2 = f5[1]",
+      "true")
+
+    testAllApis(
+      'f6 === array(1, 2, 3),
+      "f6 === array(1, 2, 3)",
+      "f6 = ARRAY[1, 2, 3]",
+      "false")
+
+    testAllApis(
+      'f2 !== 'f5.at(1),
+      "f2 !== f5.at(1)",
+      "f2 <> f5[1]",
+      "false")
+
+    testAllApis(
+      'f2 === 'f7,
+      "f2 === f7",
+      "f2 = f7",
+      "false")
+
+    testAllApis(
+      'f2 !== 'f7,
+      "f2 !== f7",
+      "f2 <> f7",
+      "true")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  case class MyCaseClass(string: String, int: Int)
+
+  override def testData: Any = {
+    val testData = new Row(11)
+    testData.setField(0, null)
+    testData.setField(1, 42)
+    testData.setField(2, Array(1, 2, 3))
+    testData.setField(3, Array(Date.valueOf("1984-03-12"), Date.valueOf("1984-02-10")))
+    testData.setField(4, null)
+    testData.setField(5, Array(Array(1, 2, 3), null))
+    testData.setField(6, Array[Integer](1, null, null, 4))
+    testData.setField(7, Array(1, 2, 3, 4))
+    testData.setField(8, Array(4.0))
+    testData.setField(9, Array[Integer](1))
+    testData.setField(10, Array[Integer]())
+    testData
+  }
+
+  override def typeInfo: TypeInformation[Any] = {
+    new RowTypeInfo(
+      Types.INT,
+      Types.INT,
+      PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO,
+      ObjectArrayTypeInfo.getInfoFor(Types.DATE),
+      ObjectArrayTypeInfo.getInfoFor(ObjectArrayTypeInfo.getInfoFor(Types.INT)),
+      ObjectArrayTypeInfo.getInfoFor(PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO),
+      ObjectArrayTypeInfo.getInfoFor(Types.INT),
+      PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO,
+      PrimitiveArrayTypeInfo.DOUBLE_PRIMITIVE_ARRAY_TYPE_INFO,
+      ObjectArrayTypeInfo.getInfoFor(Types.INT),
+      ObjectArrayTypeInfo.getInfoFor(Types.INT)
+    ).asInstanceOf[TypeInformation[Any]]
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/CompositeAccessTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/CompositeAccessTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/CompositeAccessTest.scala
new file mode 100644
index 0000000..2025880
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/CompositeAccessTest.scala
@@ -0,0 +1,193 @@
+/*
+ * 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.expressions
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.{TupleTypeInfo, TypeExtractor}
+import org.apache.flink.api.scala.createTypeInformation
+import org.apache.flink.types.Row
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.{Types, ValidationException}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.CompositeAccessTest.{MyCaseClass, MyCaseClass2, MyPojo}
+import org.apache.flink.table.expressions.utils.ExpressionTestBase
+import org.junit.Test
+
+
+class CompositeAccessTest extends ExpressionTestBase {
+
+  @Test
+  def testGetField(): Unit = {
+
+    // single field by string key
+    testAllApis(
+      'f0.get("intField"),
+      "f0.get('intField')",
+      "testTable.f0.intField",
+      "42")
+
+    testSqlApi("testTable.f0.stringField", "Bob")
+
+    testSqlApi("testTable.f0.booleanField", "true")
+
+    // single field by int key
+    testTableApi(
+      'f0.get(0),
+      "f0.get(0)",
+      "42")
+
+    // nested single field
+    testAllApis(
+      'f1.get("objectField").get("intField"),
+      "f1.get('objectField').get('intField')",
+      "testTable.f1.objectField.intField",
+      "25")
+
+    testSqlApi("testTable.f1.objectField.stringField", "Timo")
+
+    testSqlApi("testTable.f1.objectField.booleanField", "false")
+
+    testAllApis(
+      'f2.get(0),
+      "f2.get(0)",
+      "testTable.f2._1",
+      "a")
+
+    testSqlApi("testTable.f3.f1", "b")
+
+    testSqlApi("testTable.f4.myString", "Hello")
+
+    testSqlApi("testTable.f5", "13")
+
+    testAllApis(
+      'f7.get("_1"),
+      "get(f7, '_1')",
+      "testTable.f7._1",
+      "true")
+
+    // composite field return type
+    testSqlApi("testTable.f6", "MyCaseClass2(null)")
+
+    testAllApis(
+      'f1.get("objectField"),
+      "f1.get('objectField')",
+      "testTable.f1.objectField",
+      "MyCaseClass(25,Timo,false)")
+
+    testAllApis(
+      'f0,
+      "f0",
+      "testTable.f0",
+      "MyCaseClass(42,Bob,true)")
+
+    // flattening (test base only returns first column)
+    testAllApis(
+      'f1.get("objectField").flatten(),
+      "f1.get('objectField').flatten()",
+      "testTable.f1.objectField.*",
+      "25")
+
+    testAllApis(
+      'f0.flatten(),
+      "flatten(f0)",
+      "testTable.f0.*",
+      "42")
+
+    testTableApi(12.flatten(), "12.flatten()", "12")
+
+    testTableApi('f5.flatten(), "f5.flatten()", "13")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testWrongSqlField(): Unit = {
+    testSqlApi("testTable.f5.test", "13")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testWrongIntKeyField(): Unit = {
+    testTableApi('f0.get(555), "'fail'", "fail")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testWrongIntKeyField2(): Unit = {
+    testTableApi("fail", "f0.get(555)", "fail")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testWrongStringKeyField(): Unit = {
+    testTableApi('f0.get("fghj"), "'fail'", "fail")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testWrongStringKeyField2(): Unit = {
+    testTableApi("fail", "f0.get('fghj')", "fail")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  def testData = {
+    val testData = new Row(8)
+    testData.setField(0, MyCaseClass(42, "Bob", booleanField = true))
+    testData.setField(1, MyCaseClass2(MyCaseClass(25, "Timo", booleanField = false)))
+    testData.setField(2, ("a", "b"))
+    testData.setField(3, new org.apache.flink.api.java.tuple.Tuple2[String, String]("a", "b"))
+    testData.setField(4, new MyPojo())
+    testData.setField(5, 13)
+    testData.setField(6, MyCaseClass2(null))
+    testData.setField(7, Tuple1(true))
+    testData
+  }
+
+  def typeInfo = {
+    new RowTypeInfo(
+      createTypeInformation[MyCaseClass],
+      createTypeInformation[MyCaseClass2],
+      createTypeInformation[(String, String)],
+      new TupleTypeInfo(Types.STRING, Types.STRING),
+      TypeExtractor.createTypeInfo(classOf[MyPojo]),
+      Types.INT,
+      createTypeInformation[MyCaseClass2],
+      createTypeInformation[Tuple1[Boolean]]
+      ).asInstanceOf[TypeInformation[Any]]
+  }
+
+}
+
+object CompositeAccessTest {
+  case class MyCaseClass(intField: Int, stringField: String, booleanField: Boolean)
+
+  case class MyCaseClass2(objectField: MyCaseClass)
+
+  class MyPojo {
+    private var myInt: Int = 0
+    private var myString: String = "Hello"
+
+    def getMyInt = myInt
+
+    def setMyInt(value: Int) = {
+      myInt = value
+    }
+
+    def getMyString = myString
+
+    def setMyString(value: String) = {
+      myString = myString
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala
new file mode 100644
index 0000000..bdc239d
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/DecimalTypeTest.scala
@@ -0,0 +1,311 @@
+/*
+ * 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.expressions
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.types.Row
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.utils.ExpressionTestBase
+import org.junit.Test
+
+class DecimalTypeTest extends ExpressionTestBase {
+
+  @Test
+  def testDecimalLiterals(): Unit = {
+    // implicit double
+    testAllApis(
+      11.2,
+      "11.2",
+      "11.2",
+      "11.2")
+
+    // implicit double
+    testAllApis(
+      0.7623533651719233,
+      "0.7623533651719233",
+      "0.7623533651719233",
+      "0.7623533651719233")
+
+    // explicit decimal (with precision of 19)
+    testAllApis(
+      BigDecimal("1234567891234567891"),
+      "1234567891234567891p",
+      "1234567891234567891",
+      "1234567891234567891")
+
+    // explicit decimal (high precision, not SQL compliant)
+    testTableApi(
+      BigDecimal("123456789123456789123456789"),
+      "123456789123456789123456789p",
+      "123456789123456789123456789")
+
+    // explicit decimal (high precision, not SQL compliant)
+    testTableApi(
+      BigDecimal("12.3456789123456789123456789"),
+      "12.3456789123456789123456789p",
+      "12.3456789123456789123456789")
+  }
+
+  @Test
+  def testDecimalBorders(): Unit = {
+    testAllApis(
+      Double.MaxValue,
+      Double.MaxValue.toString,
+      Double.MaxValue.toString,
+      Double.MaxValue.toString)
+
+    testAllApis(
+      Double.MinValue,
+      Double.MinValue.toString,
+      Double.MinValue.toString,
+      Double.MinValue.toString)
+
+    testAllApis(
+      Double.MinValue.cast(Types.FLOAT),
+      s"${Double.MinValue}.cast(FLOAT)",
+      s"CAST(${Double.MinValue} AS FLOAT)",
+      Float.NegativeInfinity.toString)
+
+    testAllApis(
+      Byte.MinValue.cast(Types.BYTE),
+      s"(${Byte.MinValue}).cast(BYTE)",
+      s"CAST(${Byte.MinValue} AS TINYINT)",
+      Byte.MinValue.toString)
+
+    testAllApis(
+      Byte.MinValue.cast(Types.BYTE) - 1.cast(Types.BYTE),
+      s"(${Byte.MinValue}).cast(BYTE) - (1).cast(BYTE)",
+      s"CAST(${Byte.MinValue} AS TINYINT) - CAST(1 AS TINYINT)",
+      Byte.MaxValue.toString)
+
+    testAllApis(
+      Short.MinValue.cast(Types.SHORT),
+      s"(${Short.MinValue}).cast(SHORT)",
+      s"CAST(${Short.MinValue} AS SMALLINT)",
+      Short.MinValue.toString)
+
+    testAllApis(
+      Int.MinValue.cast(Types.INT) - 1,
+      s"(${Int.MinValue}).cast(INT) - 1",
+      s"CAST(${Int.MinValue} AS INT) - 1",
+      Int.MaxValue.toString)
+
+    testAllApis(
+      Long.MinValue.cast(Types.LONG),
+      s"(${Long.MinValue}L).cast(LONG)",
+      s"CAST(${Long.MinValue} AS BIGINT)",
+      Long.MinValue.toString)
+  }
+
+  @Test
+  def testDecimalCasting(): Unit = {
+    // from String
+    testTableApi(
+      "123456789123456789123456789".cast(Types.DECIMAL),
+      "'123456789123456789123456789'.cast(DECIMAL)",
+      "123456789123456789123456789")
+
+    // from double
+    testAllApis(
+      'f3.cast(Types.DECIMAL),
+      "f3.cast(DECIMAL)",
+      "CAST(f3 AS DECIMAL)",
+      "4.2")
+
+    // to double
+    testAllApis(
+      'f0.cast(Types.DOUBLE),
+      "f0.cast(DOUBLE)",
+      "CAST(f0 AS DOUBLE)",
+      "1.2345678912345679E8")
+
+    // to int
+    testAllApis(
+      'f4.cast(Types.INT),
+      "f4.cast(INT)",
+      "CAST(f4 AS INT)",
+      "123456789")
+
+    // to long
+    testAllApis(
+      'f4.cast(Types.LONG),
+      "f4.cast(LONG)",
+      "CAST(f4 AS BIGINT)",
+      "123456789")
+
+    // to boolean (not SQL compliant)
+    testTableApi(
+      'f1.cast(Types.BOOLEAN),
+      "f1.cast(BOOL)",
+      "true")
+
+    testTableApi(
+      'f5.cast(Types.BOOLEAN),
+      "f5.cast(BOOL)",
+      "false")
+
+    testTableApi(
+      BigDecimal("123456789.123456789123456789").cast(Types.DOUBLE),
+      "(123456789.123456789123456789p).cast(DOUBLE)",
+      "1.2345678912345679E8")
+  }
+
+  @Test
+  def testDecimalArithmetic(): Unit = {
+    // implicit cast to decimal
+    testAllApis(
+      'f1 + 12,
+      "f1 + 12",
+      "f1 + 12",
+      "123456789123456789123456801")
+
+    // implicit cast to decimal
+    testAllApis(
+      Literal(12) + 'f1,
+      "12 + f1",
+      "12 + f1",
+      "123456789123456789123456801")
+
+    // implicit cast to decimal
+    testAllApis(
+      'f1 + 12.3,
+      "f1 + 12.3",
+      "f1 + 12.3",
+      "123456789123456789123456801.3")
+
+    // implicit cast to decimal
+    testAllApis(
+      Literal(12.3) + 'f1,
+      "12.3 + f1",
+      "12.3 + f1",
+      "123456789123456789123456801.3")
+
+    testAllApis(
+      'f1 + 'f1,
+      "f1 + f1",
+      "f1 + f1",
+      "246913578246913578246913578")
+
+    testAllApis(
+      'f1 - 'f1,
+      "f1 - f1",
+      "f1 - f1",
+      "0")
+
+    testAllApis(
+      'f1 * 'f1,
+      "f1 * f1",
+      "f1 * f1",
+      "15241578780673678546105778281054720515622620750190521")
+
+    testAllApis(
+      'f1 / 'f1,
+      "f1 / f1",
+      "f1 / f1",
+      "1")
+
+    testAllApis(
+      'f1 % 'f1,
+      "f1 % f1",
+      "MOD(f1, f1)",
+      "0")
+
+    testAllApis(
+      -'f0,
+      "-f0",
+      "-f0",
+      "-123456789.123456789123456789")
+  }
+
+  @Test
+  def testDecimalComparison(): Unit = {
+    testAllApis(
+      'f1 < 12,
+      "f1 < 12",
+      "f1 < 12",
+      "false")
+
+    testAllApis(
+      'f1 > 12,
+      "f1 > 12",
+      "f1 > 12",
+      "true")
+
+    testAllApis(
+      'f1 === 12,
+      "f1 === 12",
+      "f1 = 12",
+      "false")
+
+    testAllApis(
+      'f5 === 0,
+      "f5 === 0",
+      "f5 = 0",
+      "true")
+
+    testAllApis(
+      'f1 === BigDecimal("123456789123456789123456789"),
+      "f1 === 123456789123456789123456789p",
+      "f1 = CAST('123456789123456789123456789' AS DECIMAL)",
+      "true")
+
+    testAllApis(
+      'f1 !== BigDecimal("123456789123456789123456789"),
+      "f1 !== 123456789123456789123456789p",
+      "f1 <> CAST('123456789123456789123456789' AS DECIMAL)",
+      "false")
+
+    testAllApis(
+      'f4 < 'f0,
+      "f4 < f0",
+      "f4 < f0",
+      "true")
+
+    // TODO add all tests if FLINK-4070 is fixed
+    testSqlApi(
+      "12 < f1",
+      "true")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  def testData = {
+    val testData = new Row(6)
+    testData.setField(0, BigDecimal("123456789.123456789123456789").bigDecimal)
+    testData.setField(1, BigDecimal("123456789123456789123456789").bigDecimal)
+    testData.setField(2, 42)
+    testData.setField(3, 4.2)
+    testData.setField(4, BigDecimal("123456789").bigDecimal)
+    testData.setField(5, BigDecimal("0.000").bigDecimal)
+    testData
+  }
+
+  def typeInfo = {
+    new RowTypeInfo(
+      Types.DECIMAL,
+      Types.DECIMAL,
+      Types.INT,
+      Types.DOUBLE,
+      Types.DECIMAL,
+      Types.DECIMAL).asInstanceOf[TypeInformation[Any]]
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/NonDeterministicTests.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/NonDeterministicTests.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/NonDeterministicTests.scala
new file mode 100644
index 0000000..bcc53af
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/NonDeterministicTests.scala
@@ -0,0 +1,89 @@
+/*
+ * 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.expressions
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.types.Row
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.utils.ExpressionTestBase
+import org.junit.{Ignore, Test}
+
+/**
+  * Tests that can only be checked manually as they are non-deterministic.
+  */
+class NonDeterministicTests extends ExpressionTestBase {
+
+  @Ignore
+  @Test
+  def testCurrentDate(): Unit = {
+    testAllApis(
+      currentDate(),
+      "currentDate()",
+      "CURRENT_DATE",
+      "PLEASE CHECK MANUALLY")
+  }
+
+  @Ignore
+  @Test
+  def testCurrentTime(): Unit = {
+    testAllApis(
+      currentTime(),
+      "currentTime()",
+      "CURRENT_TIME",
+      "PLEASE CHECK MANUALLY")
+  }
+
+  @Ignore
+  @Test
+  def testCurrentTimestamp(): Unit = {
+    testAllApis(
+      currentTimestamp(),
+      "currentTimestamp()",
+      "CURRENT_TIMESTAMP",
+      "PLEASE CHECK MANUALLY")
+  }
+
+  @Ignore
+  @Test
+  def testLocalTimestamp(): Unit = {
+    testAllApis(
+      localTimestamp(),
+      "localTimestamp()",
+      "LOCALTIMESTAMP",
+      "PLEASE CHECK MANUALLY")
+  }
+
+  @Ignore
+  @Test
+  def testLocalTime(): Unit = {
+    testAllApis(
+      localTime(),
+      "localTime()",
+      "LOCALTIME",
+      "PLEASE CHECK MANUALLY")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  override def testData: Any = new Row(0)
+
+  override def typeInfo: TypeInformation[Any] =
+    new RowTypeInfo().asInstanceOf[TypeInformation[Any]]
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/ffe9ec8e/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala
new file mode 100644
index 0000000..596907b
--- /dev/null
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala
@@ -0,0 +1,1166 @@
+/*
+ * 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.expressions
+
+import java.sql.{Date, Time, Timestamp}
+
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.types.Row
+import org.apache.flink.table.api.{Types, ValidationException}
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.utils.ExpressionTestBase
+import org.junit.Test
+
+class ScalarFunctionsTest extends ExpressionTestBase {
+
+  // ----------------------------------------------------------------------------------------------
+  // String functions
+  // ----------------------------------------------------------------------------------------------
+
+  @Test
+  def testOverlay(): Unit = {
+    testAllApis(
+      "xxxxxtest".overlay("xxxx", 6),
+      "'xxxxxtest'.overlay('xxxx', 6)",
+      "OVERLAY('xxxxxtest' PLACING 'xxxx' FROM 6)",
+      "xxxxxxxxx")
+
+    testAllApis(
+      "xxxxxtest".overlay("xxxx", 6, 2),
+      "'xxxxxtest'.overlay('xxxx', 6, 2)",
+      "OVERLAY('xxxxxtest' PLACING 'xxxx' FROM 6 FOR 2)",
+      "xxxxxxxxxst")
+  }
+
+  @Test
+  def testPosition(): Unit = {
+    testAllApis(
+      "test".position("xxxtest"),
+      "'test'.position('xxxtest')",
+      "POSITION('test' IN 'xxxtest')",
+      "4")
+
+    testAllApis(
+      "testx".position("xxxtest"),
+      "'testx'.position('xxxtest')",
+      "POSITION('testx' IN 'xxxtest')",
+      "0")
+  }
+
+  @Test
+  def testSubstring(): Unit = {
+    testAllApis(
+      'f0.substring(2),
+      "f0.substring(2)",
+      "SUBSTRING(f0, 2)",
+      "his is a test String.")
+
+    testAllApis(
+      'f0.substring(2, 5),
+      "f0.substring(2, 5)",
+      "SUBSTRING(f0, 2, 5)",
+      "his i")
+
+    testAllApis(
+      'f0.substring(1, 'f7),
+      "f0.substring(1, f7)",
+      "SUBSTRING(f0, 1, f7)",
+      "Thi")
+
+    testAllApis(
+      'f0.substring(1.cast(Types.BYTE), 'f7),
+      "f0.substring(1.cast(BYTE), f7)",
+      "SUBSTRING(f0, CAST(1 AS TINYINT), f7)",
+      "Thi")
+
+    testSqlApi(
+      "SUBSTRING(f0 FROM 2 FOR 1)",
+      "h")
+
+    testSqlApi(
+      "SUBSTRING(f0 FROM 2)",
+      "his is a test String.")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidSubstring1(): Unit = {
+    // Must fail. Parameter of substring must be an Integer not a Double.
+    testTableApi("test".substring(2.0.toExpr), "FAIL", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidSubstring2(): Unit = {
+    // Must fail. Parameter of substring must be an Integer not a String.
+    testTableApi("test".substring("test".toExpr), "FAIL", "FAIL")
+  }
+
+  @Test
+  def testTrim(): Unit = {
+    testAllApis(
+      'f8.trim(),
+      "f8.trim()",
+      "TRIM(f8)",
+      "This is a test String.")
+
+    testAllApis(
+      'f8.trim(removeLeading = true, removeTrailing = true, " "),
+      "trim(f8)",
+      "TRIM(f8)",
+      "This is a test String.")
+
+    testAllApis(
+      'f8.trim(removeLeading = false, removeTrailing = true, " "),
+      "f8.trim(TRAILING, ' ')",
+      "TRIM(TRAILING FROM f8)",
+      " This is a test String.")
+
+    testAllApis(
+      'f0.trim(removeLeading = true, removeTrailing = true, "."),
+      "trim(BOTH, '.', f0)",
+      "TRIM(BOTH '.' FROM f0)",
+      "This is a test String")
+  }
+
+  @Test
+  def testCharLength(): Unit = {
+    testAllApis(
+      'f0.charLength(),
+      "f0.charLength()",
+      "CHAR_LENGTH(f0)",
+      "22")
+
+    testAllApis(
+      'f0.charLength(),
+      "charLength(f0)",
+      "CHARACTER_LENGTH(f0)",
+      "22")
+  }
+
+  @Test
+  def testUpperCase(): Unit = {
+    testAllApis(
+      'f0.upperCase(),
+      "f0.upperCase()",
+      "UPPER(f0)",
+      "THIS IS A TEST STRING.")
+  }
+
+  @Test
+  def testLowerCase(): Unit = {
+    testAllApis(
+      'f0.lowerCase(),
+      "f0.lowerCase()",
+      "LOWER(f0)",
+      "this is a test string.")
+  }
+
+  @Test
+  def testInitCap(): Unit = {
+    testAllApis(
+      'f0.initCap(),
+      "f0.initCap()",
+      "INITCAP(f0)",
+      "This Is A Test String.")
+  }
+
+  @Test
+  def testConcat(): Unit = {
+    testAllApis(
+      'f0 + 'f0,
+      "f0 + f0",
+      "f0||f0",
+      "This is a test String.This is a test String.")
+  }
+
+  @Test
+  def testLike(): Unit = {
+    testAllApis(
+      'f0.like("Th_s%"),
+      "f0.like('Th_s%')",
+      "f0 LIKE 'Th_s%'",
+      "true")
+
+    testAllApis(
+      'f0.like("%is a%"),
+      "f0.like('%is a%')",
+      "f0 LIKE '%is a%'",
+      "true")
+  }
+
+  @Test
+  def testNotLike(): Unit = {
+    testAllApis(
+      !'f0.like("Th_s%"),
+      "!f0.like('Th_s%')",
+      "f0 NOT LIKE 'Th_s%'",
+      "false")
+
+    testAllApis(
+      !'f0.like("%is a%"),
+      "!f0.like('%is a%')",
+      "f0 NOT LIKE '%is a%'",
+      "false")
+  }
+
+  @Test
+  def testLikeWithEscape(): Unit = {
+    testSqlApi(
+      "f23 LIKE '&%Th_s%' ESCAPE '&'",
+      "true")
+
+    testSqlApi(
+      "f23 LIKE '&%%is a%' ESCAPE '&'",
+      "true")
+
+    testSqlApi(
+      "f0 LIKE 'Th_s%' ESCAPE '&'",
+      "true")
+
+    testSqlApi(
+      "f0 LIKE '%is a%' ESCAPE '&'",
+      "true")
+  }
+
+  @Test
+  def testNotLikeWithEscape(): Unit = {
+    testSqlApi(
+      "f23 NOT LIKE '&%Th_s%' ESCAPE '&'",
+      "false")
+
+    testSqlApi(
+      "f23 NOT LIKE '&%%is a%' ESCAPE '&'",
+      "false")
+
+    testSqlApi(
+      "f0 NOT LIKE 'Th_s%' ESCAPE '&'",
+      "false")
+
+    testSqlApi(
+      "f0 NOT LIKE '%is a%' ESCAPE '&'",
+      "false")
+  }
+
+  @Test
+  def testSimilar(): Unit = {
+    testAllApis(
+      'f0.similar("_*"),
+      "f0.similar('_*')",
+      "f0 SIMILAR TO '_*'",
+      "true")
+
+    testAllApis(
+      'f0.similar("This (is)? a (test)+ Strin_*"),
+      "f0.similar('This (is)? a (test)+ Strin_*')",
+      "f0 SIMILAR TO 'This (is)? a (test)+ Strin_*'",
+      "true")
+  }
+
+  @Test
+  def testNotSimilar(): Unit = {
+    testAllApis(
+      !'f0.similar("_*"),
+      "!f0.similar('_*')",
+      "f0 NOT SIMILAR TO '_*'",
+      "false")
+
+    testAllApis(
+      !'f0.similar("This (is)? a (test)+ Strin_*"),
+      "!f0.similar('This (is)? a (test)+ Strin_*')",
+      "f0 NOT SIMILAR TO 'This (is)? a (test)+ Strin_*'",
+      "false")
+  }
+
+  @Test
+  def testSimilarWithEscape(): Unit = {
+    testSqlApi(
+      "f24 SIMILAR TO '&*&__*' ESCAPE '&'",
+      "true")
+
+    testSqlApi(
+      "f0 SIMILAR TO '_*' ESCAPE '&'",
+      "true")
+
+    testSqlApi(
+      "f24 SIMILAR TO '&*&_This (is)? a (test)+ Strin_*' ESCAPE '&'",
+      "true")
+
+    testSqlApi(
+      "f0 SIMILAR TO 'This (is)? a (test)+ Strin_*' ESCAPE '&'",
+      "true")
+  }
+
+  @Test
+  def testNotSimilarWithEscape(): Unit = {
+    testSqlApi(
+      "f24 NOT SIMILAR TO '&*&__*' ESCAPE '&'",
+      "false")
+
+    testSqlApi(
+      "f0 NOT SIMILAR TO '_*' ESCAPE '&'",
+      "false")
+
+    testSqlApi(
+      "f24 NOT SIMILAR TO '&*&_This (is)? a (test)+ Strin_*' ESCAPE '&'",
+      "false")
+
+    testSqlApi(
+      "f0 NOT SIMILAR TO 'This (is)? a (test)+ Strin_*' ESCAPE '&'",
+      "false")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // Math functions
+  // ----------------------------------------------------------------------------------------------
+
+  @Test
+  def testMod(): Unit = {
+    testAllApis(
+      'f4.mod('f7),
+      "f4.mod(f7)",
+      "MOD(f4, f7)",
+      "2")
+
+    testAllApis(
+      'f4.mod(3),
+      "mod(f4, 3)",
+      "MOD(f4, 3)",
+      "2")
+
+    testAllApis(
+      'f4 % 3,
+      "mod(44, 3)",
+      "MOD(44, 3)",
+      "2")
+
+  }
+
+  @Test
+  def testExp(): Unit = {
+    testAllApis(
+      'f2.exp(),
+      "f2.exp()",
+      "EXP(f2)",
+      math.exp(42.toByte).toString)
+
+    testAllApis(
+      'f3.exp(),
+      "f3.exp()",
+      "EXP(f3)",
+      math.exp(43.toShort).toString)
+
+    testAllApis(
+      'f4.exp(),
+      "f4.exp()",
+      "EXP(f4)",
+      math.exp(44.toLong).toString)
+
+    testAllApis(
+      'f5.exp(),
+      "f5.exp()",
+      "EXP(f5)",
+      math.exp(4.5.toFloat).toString)
+
+    testAllApis(
+      'f6.exp(),
+      "f6.exp()",
+      "EXP(f6)",
+      math.exp(4.6).toString)
+
+    testAllApis(
+      'f7.exp(),
+      "exp(3)",
+      "EXP(3)",
+      math.exp(3).toString)
+
+    testAllApis(
+      'f7.exp(),
+      "exp(3)",
+      "EXP(3)",
+      math.exp(3).toString)
+  }
+
+  @Test
+  def testLog10(): Unit = {
+    testAllApis(
+      'f2.log10(),
+      "f2.log10()",
+      "LOG10(f2)",
+      math.log10(42.toByte).toString)
+
+    testAllApis(
+      'f3.log10(),
+      "f3.log10()",
+      "LOG10(f3)",
+      math.log10(43.toShort).toString)
+
+    testAllApis(
+      'f4.log10(),
+      "f4.log10()",
+      "LOG10(f4)",
+      math.log10(44.toLong).toString)
+
+    testAllApis(
+      'f5.log10(),
+      "f5.log10()",
+      "LOG10(f5)",
+      math.log10(4.5.toFloat).toString)
+
+    testAllApis(
+      'f6.log10(),
+      "f6.log10()",
+      "LOG10(f6)",
+      math.log10(4.6).toString)
+  }
+
+  @Test
+  def testPower(): Unit = {
+    // f7: int , f4: long, f6: double
+    testAllApis(
+      'f2.power('f7),
+      "f2.power(f7)",
+      "POWER(f2, f7)",
+      math.pow(42.toByte, 3).toString)
+
+    testAllApis(
+      'f3.power('f6),
+      "f3.power(f6)",
+      "POWER(f3, f6)",
+      math.pow(43.toShort, 4.6D).toString)
+
+    testAllApis(
+      'f4.power('f5),
+      "f4.power(f5)",
+      "POWER(f4, f5)",
+      math.pow(44.toLong, 4.5.toFloat).toString)
+
+    testAllApis(
+      'f4.power('f5),
+      "f4.power(f5)",
+      "POWER(f4, f5)",
+      math.pow(44.toLong, 4.5.toFloat).toString)
+
+    // f5: float
+    testAllApis('f5.power('f5),
+      "f5.power(f5)",
+      "power(f5, f5)",
+      math.pow(4.5F, 4.5F).toString)
+
+    testAllApis('f5.power('f6),
+      "f5.power(f6)",
+      "power(f5, f6)",
+      math.pow(4.5F, 4.6D).toString)
+
+    testAllApis('f5.power('f7),
+      "f5.power(f7)",
+      "power(f5, f7)",
+      math.pow(4.5F, 3).toString)
+
+    testAllApis('f5.power('f4),
+      "f5.power(f4)",
+      "power(f5, f4)",
+      math.pow(4.5F, 44L).toString)
+
+    // f22: bigDecimal
+    // TODO delete casting in SQL when CALCITE-1467 is fixed
+    testAllApis(
+      'f22.cast(Types.DOUBLE).power('f5),
+      "f22.cast(DOUBLE).power(f5)",
+      "power(CAST(f22 AS DOUBLE), f5)",
+      math.pow(2, 4.5F).toString)
+
+    testAllApis(
+      'f22.cast(Types.DOUBLE).power('f6),
+      "f22.cast(DOUBLE).power(f6)",
+      "power(CAST(f22 AS DOUBLE), f6)",
+      math.pow(2, 4.6D).toString)
+
+    testAllApis(
+      'f22.cast(Types.DOUBLE).power('f7),
+      "f22.cast(DOUBLE).power(f7)",
+      "power(CAST(f22 AS DOUBLE), f7)",
+      math.pow(2, 3).toString)
+
+    testAllApis(
+      'f22.cast(Types.DOUBLE).power('f4),
+      "f22.cast(DOUBLE).power(f4)",
+      "power(CAST(f22 AS DOUBLE), f4)",
+      math.pow(2, 44L).toString)
+
+    testAllApis(
+      'f6.power('f22.cast(Types.DOUBLE)),
+      "f6.power(f22.cast(DOUBLE))",
+      "power(f6, f22)",
+      math.pow(4.6D, 2).toString)
+  }
+
+  @Test
+  def testSqrt(): Unit = {
+    testAllApis(
+      'f6.sqrt(),
+      "f6.sqrt",
+      "SQRT(f6)",
+      math.sqrt(4.6D).toString)
+
+    testAllApis(
+      'f7.sqrt(),
+      "f7.sqrt",
+      "SQRT(f7)",
+      math.sqrt(3).toString)
+
+    testAllApis(
+      'f4.sqrt(),
+      "f4.sqrt",
+      "SQRT(f4)",
+      math.sqrt(44L).toString)
+
+    testAllApis(
+      'f22.cast(Types.DOUBLE).sqrt(),
+      "f22.cast(DOUBLE).sqrt",
+      "SQRT(CAST(f22 AS DOUBLE))",
+      math.sqrt(2.0).toString)
+
+    testAllApis(
+      'f5.sqrt(),
+      "f5.sqrt",
+      "SQRT(f5)",
+      math.pow(4.5F, 0.5).toString)
+
+    testAllApis(
+      25.sqrt(),
+      "25.sqrt()",
+      "SQRT(25)",
+      "5.0")
+
+    testAllApis(
+      2.2.sqrt(),
+      "2.2.sqrt()",
+      "POWER(CAST(2.2 AS DOUBLE), CAST(0.5 AS DOUBLE))", // TODO fix FLINK-4621
+      math.sqrt(2.2).toString)
+  }
+
+  @Test
+  def testLn(): Unit = {
+    testAllApis(
+      'f2.ln(),
+      "f2.ln()",
+      "LN(f2)",
+      math.log(42.toByte).toString)
+
+    testAllApis(
+      'f3.ln(),
+      "f3.ln()",
+      "LN(f3)",
+      math.log(43.toShort).toString)
+
+    testAllApis(
+      'f4.ln(),
+      "f4.ln()",
+      "LN(f4)",
+      math.log(44.toLong).toString)
+
+    testAllApis(
+      'f5.ln(),
+      "f5.ln()",
+      "LN(f5)",
+      math.log(4.5.toFloat).toString)
+
+    testAllApis(
+      'f6.ln(),
+      "f6.ln()",
+      "LN(f6)",
+      math.log(4.6).toString)
+  }
+
+  @Test
+  def testAbs(): Unit = {
+    testAllApis(
+      'f2.abs(),
+      "f2.abs()",
+      "ABS(f2)",
+      "42")
+
+    testAllApis(
+      'f3.abs(),
+      "f3.abs()",
+      "ABS(f3)",
+      "43")
+
+    testAllApis(
+      'f4.abs(),
+      "f4.abs()",
+      "ABS(f4)",
+      "44")
+
+    testAllApis(
+      'f5.abs(),
+      "f5.abs()",
+      "ABS(f5)",
+      "4.5")
+
+    testAllApis(
+      'f6.abs(),
+      "f6.abs()",
+      "ABS(f6)",
+      "4.6")
+
+    testAllApis(
+      'f9.abs(),
+      "f9.abs()",
+      "ABS(f9)",
+      "42")
+
+    testAllApis(
+      'f10.abs(),
+      "f10.abs()",
+      "ABS(f10)",
+      "43")
+
+    testAllApis(
+      'f11.abs(),
+      "f11.abs()",
+      "ABS(f11)",
+      "44")
+
+    testAllApis(
+      'f12.abs(),
+      "f12.abs()",
+      "ABS(f12)",
+      "4.5")
+
+    testAllApis(
+      'f13.abs(),
+      "f13.abs()",
+      "ABS(f13)",
+      "4.6")
+
+    testAllApis(
+      'f15.abs(),
+      "f15.abs()",
+      "ABS(f15)",
+      "1231.1231231321321321111")
+  }
+
+  @Test
+  def testArithmeticFloorCeil(): Unit = {
+    testAllApis(
+      'f5.floor(),
+      "f5.floor()",
+      "FLOOR(f5)",
+      "4.0")
+
+    testAllApis(
+     'f5.ceil(),
+      "f5.ceil()",
+      "CEIL(f5)",
+      "5.0")
+
+    testAllApis(
+      'f3.floor(),
+      "f3.floor()",
+      "FLOOR(f3)",
+      "43")
+
+    testAllApis(
+      'f3.ceil(),
+      "f3.ceil()",
+      "CEIL(f3)",
+      "43")
+
+    testAllApis(
+      'f15.floor(),
+      "f15.floor()",
+      "FLOOR(f15)",
+      "-1232")
+
+    testAllApis(
+      'f15.ceil(),
+      "f15.ceil()",
+      "CEIL(f15)",
+      "-1231")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // Temporal functions
+  // ----------------------------------------------------------------------------------------------
+
+  @Test
+  def testExtract(): Unit = {
+    testAllApis(
+      'f16.extract(TimeIntervalUnit.YEAR),
+      "f16.extract(YEAR)",
+      "EXTRACT(YEAR FROM f16)",
+      "1996")
+
+    testAllApis(
+      'f16.extract(TimeIntervalUnit.MONTH),
+      "extract(f16, MONTH)",
+      "EXTRACT(MONTH FROM f16)",
+      "11")
+
+    testAllApis(
+      'f16.extract(TimeIntervalUnit.DAY),
+      "f16.extract(DAY)",
+      "EXTRACT(DAY FROM f16)",
+      "10")
+
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.YEAR),
+      "f18.extract(YEAR)",
+      "EXTRACT(YEAR FROM f18)",
+      "1996")
+
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.MONTH),
+      "f18.extract(MONTH)",
+      "EXTRACT(MONTH FROM f18)",
+      "11")
+
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.DAY),
+      "f18.extract(DAY)",
+      "EXTRACT(DAY FROM f18)",
+      "10")
+
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.HOUR),
+      "f18.extract(HOUR)",
+      "EXTRACT(HOUR FROM f18)",
+      "6")
+
+    testAllApis(
+      'f17.extract(TimeIntervalUnit.HOUR),
+      "f17.extract(HOUR)",
+      "EXTRACT(HOUR FROM f17)",
+      "6")
+
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.MINUTE),
+      "f18.extract(MINUTE)",
+      "EXTRACT(MINUTE FROM f18)",
+      "55")
+
+    testAllApis(
+      'f17.extract(TimeIntervalUnit.MINUTE),
+      "f17.extract(MINUTE)",
+      "EXTRACT(MINUTE FROM f17)",
+      "55")
+
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.SECOND),
+      "f18.extract(SECOND)",
+      "EXTRACT(SECOND FROM f18)",
+      "44")
+
+    testAllApis(
+      'f17.extract(TimeIntervalUnit.SECOND),
+      "f17.extract(SECOND)",
+      "EXTRACT(SECOND FROM f17)",
+      "44")
+
+    testAllApis(
+      'f19.extract(TimeIntervalUnit.DAY),
+      "f19.extract(DAY)",
+      "EXTRACT(DAY FROM f19)",
+      "16979")
+
+    testAllApis(
+      'f19.extract(TimeIntervalUnit.HOUR),
+      "f19.extract(HOUR)",
+      "EXTRACT(HOUR FROM f19)",
+      "7")
+
+    testAllApis(
+      'f19.extract(TimeIntervalUnit.MINUTE),
+      "f19.extract(MINUTE)",
+      "EXTRACT(MINUTE FROM f19)",
+      "23")
+
+    testAllApis(
+      'f19.extract(TimeIntervalUnit.SECOND),
+      "f19.extract(SECOND)",
+      "EXTRACT(SECOND FROM f19)",
+      "33")
+
+    testAllApis(
+      'f20.extract(TimeIntervalUnit.MONTH),
+      "f20.extract(MONTH)",
+      "EXTRACT(MONTH FROM f20)",
+      "1")
+
+    testAllApis(
+      'f20.extract(TimeIntervalUnit.YEAR),
+      "f20.extract(YEAR)",
+      "EXTRACT(YEAR FROM f20)",
+      "2")
+  }
+
+  @Test
+  def testTemporalFloor(): Unit = {
+    testAllApis(
+      'f18.floor(TimeIntervalUnit.YEAR),
+      "f18.floor(YEAR)",
+      "FLOOR(f18 TO YEAR)",
+      "1996-01-01 00:00:00.0")
+
+    testAllApis(
+      'f18.floor(TimeIntervalUnit.MONTH),
+      "f18.floor(MONTH)",
+      "FLOOR(f18 TO MONTH)",
+      "1996-11-01 00:00:00.0")
+
+    testAllApis(
+      'f18.floor(TimeIntervalUnit.DAY),
+      "f18.floor(DAY)",
+      "FLOOR(f18 TO DAY)",
+      "1996-11-10 00:00:00.0")
+
+    testAllApis(
+      'f18.floor(TimeIntervalUnit.MINUTE),
+      "f18.floor(MINUTE)",
+      "FLOOR(f18 TO MINUTE)",
+      "1996-11-10 06:55:00.0")
+
+    testAllApis(
+      'f18.floor(TimeIntervalUnit.SECOND),
+      "f18.floor(SECOND)",
+      "FLOOR(f18 TO SECOND)",
+      "1996-11-10 06:55:44.0")
+
+    testAllApis(
+      'f17.floor(TimeIntervalUnit.HOUR),
+      "f17.floor(HOUR)",
+      "FLOOR(f17 TO HOUR)",
+      "06:00:00")
+
+    testAllApis(
+      'f17.floor(TimeIntervalUnit.MINUTE),
+      "f17.floor(MINUTE)",
+      "FLOOR(f17 TO MINUTE)",
+      "06:55:00")
+
+    testAllApis(
+      'f17.floor(TimeIntervalUnit.SECOND),
+      "f17.floor(SECOND)",
+      "FLOOR(f17 TO SECOND)",
+      "06:55:44")
+
+    testAllApis(
+      'f16.floor(TimeIntervalUnit.YEAR),
+      "f16.floor(YEAR)",
+      "FLOOR(f16 TO YEAR)",
+      "1996-01-01")
+
+    testAllApis(
+      'f16.floor(TimeIntervalUnit.MONTH),
+      "f16.floor(MONTH)",
+      "FLOOR(f16 TO MONTH)",
+      "1996-11-01")
+
+    testAllApis(
+      'f18.ceil(TimeIntervalUnit.YEAR),
+      "f18.ceil(YEAR)",
+      "CEIL(f18 TO YEAR)",
+      "1997-01-01 00:00:00.0")
+
+    testAllApis(
+      'f18.ceil(TimeIntervalUnit.MONTH),
+      "f18.ceil(MONTH)",
+      "CEIL(f18 TO MONTH)",
+      "1996-12-01 00:00:00.0")
+
+    testAllApis(
+      'f18.ceil(TimeIntervalUnit.DAY),
+      "f18.ceil(DAY)",
+      "CEIL(f18 TO DAY)",
+      "1996-11-11 00:00:00.0")
+
+    testAllApis(
+      'f18.ceil(TimeIntervalUnit.MINUTE),
+      "f18.ceil(MINUTE)",
+      "CEIL(f18 TO MINUTE)",
+      "1996-11-10 06:56:00.0")
+
+    testAllApis(
+      'f18.ceil(TimeIntervalUnit.SECOND),
+      "f18.ceil(SECOND)",
+      "CEIL(f18 TO SECOND)",
+      "1996-11-10 06:55:45.0")
+
+    testAllApis(
+      'f17.ceil(TimeIntervalUnit.HOUR),
+      "f17.ceil(HOUR)",
+      "CEIL(f17 TO HOUR)",
+      "07:00:00")
+
+    testAllApis(
+      'f17.ceil(TimeIntervalUnit.MINUTE),
+      "f17.ceil(MINUTE)",
+      "CEIL(f17 TO MINUTE)",
+      "06:56:00")
+
+    testAllApis(
+      'f17.ceil(TimeIntervalUnit.SECOND),
+      "f17.ceil(SECOND)",
+      "CEIL(f17 TO SECOND)",
+      "06:55:44")
+
+    testAllApis(
+      'f16.ceil(TimeIntervalUnit.YEAR),
+      "f16.ceil(YEAR)",
+      "CEIL(f16 TO YEAR)",
+      "1996-01-01")
+
+    testAllApis(
+      'f16.ceil(TimeIntervalUnit.MONTH),
+      "f16.ceil(MONTH)",
+      "CEIL(f16 TO MONTH)",
+      "1996-11-01")
+  }
+
+  @Test
+  def testCurrentTimePoint(): Unit = {
+
+    // current time points are non-deterministic
+    // we just test the format of the output
+    // manual test can be found in NonDeterministicTests
+
+    testAllApis(
+      currentDate().cast(Types.STRING).charLength() >= 5,
+      "currentDate().cast(STRING).charLength() >= 5",
+      "CHAR_LENGTH(CAST(CURRENT_DATE AS VARCHAR)) >= 5",
+      "true")
+
+    testAllApis(
+      currentTime().cast(Types.STRING).charLength() >= 5,
+      "currentTime().cast(STRING).charLength() >= 5",
+      "CHAR_LENGTH(CAST(CURRENT_TIME AS VARCHAR)) >= 5",
+      "true")
+
+    testAllApis(
+      currentTimestamp().cast(Types.STRING).charLength() >= 12,
+      "currentTimestamp().cast(STRING).charLength() >= 12",
+      "CHAR_LENGTH(CAST(CURRENT_TIMESTAMP AS VARCHAR)) >= 12",
+      "true")
+
+    testAllApis(
+      localTimestamp().cast(Types.STRING).charLength() >= 12,
+      "localTimestamp().cast(STRING).charLength() >= 12",
+      "CHAR_LENGTH(CAST(LOCALTIMESTAMP AS VARCHAR)) >= 12",
+      "true")
+
+    testAllApis(
+      localTime().cast(Types.STRING).charLength() >= 5,
+      "localTime().cast(STRING).charLength() >= 5",
+      "CHAR_LENGTH(CAST(LOCALTIME AS VARCHAR)) >= 5",
+      "true")
+
+    // comparisons are deterministic
+    testAllApis(
+      localTimestamp() === localTimestamp(),
+      "localTimestamp() === localTimestamp()",
+      "LOCALTIMESTAMP = LOCALTIMESTAMP",
+      "true")
+  }
+
+  @Test
+  def testOverlaps(): Unit = {
+    testAllApis(
+      temporalOverlaps("2:55:00".toTime, 1.hour, "3:30:00".toTime, 2.hours),
+      "temporalOverlaps('2:55:00'.toTime, 1.hour, '3:30:00'.toTime, 2.hours)",
+      "(TIME '2:55:00', INTERVAL '1' HOUR) OVERLAPS (TIME '3:30:00', INTERVAL '2' HOUR)",
+      "true")
+
+    testAllApis(
+      temporalOverlaps("9:00:00".toTime, "9:30:00".toTime, "9:29:00".toTime, "9:31:00".toTime),
+      "temporalOverlaps('9:00:00'.toTime, '9:30:00'.toTime, '9:29:00'.toTime, '9:31:00'.toTime)",
+      "(TIME '9:00:00', TIME '9:30:00') OVERLAPS (TIME '9:29:00', TIME '9:31:00')",
+      "true")
+
+    testAllApis(
+      temporalOverlaps("9:00:00".toTime, "10:00:00".toTime, "10:15:00".toTime, 3.hours),
+      "temporalOverlaps('9:00:00'.toTime, '10:00:00'.toTime, '10:15:00'.toTime, 3.hours)",
+      "(TIME '9:00:00', TIME '10:00:00') OVERLAPS (TIME '10:15:00', INTERVAL '3' HOUR)",
+      "false")
+
+    testAllApis(
+      temporalOverlaps("2011-03-10".toDate, 10.days, "2011-03-19".toDate, 10.days),
+      "temporalOverlaps('2011-03-10'.toDate, 10.days, '2011-03-19'.toDate, 10.days)",
+      "(DATE '2011-03-10', INTERVAL '10' DAY) OVERLAPS (DATE '2011-03-19', INTERVAL '10' DAY)",
+      "true")
+
+    testAllApis(
+      temporalOverlaps("2011-03-10 05:02:02".toTimestamp, 0.milli,
+        "2011-03-10 05:02:02".toTimestamp, "2011-03-10 05:02:01".toTimestamp),
+      "temporalOverlaps('2011-03-10 05:02:02'.toTimestamp, 0.milli, " +
+        "'2011-03-10 05:02:02'.toTimestamp, '2011-03-10 05:02:01'.toTimestamp)",
+      "(TIMESTAMP '2011-03-10 05:02:02', INTERVAL '0' SECOND) OVERLAPS " +
+        "(TIMESTAMP '2011-03-10 05:02:02', TIMESTAMP '2011-03-10 05:02:01')",
+      "false")
+
+    // TODO enable once CALCITE-1435 is fixed
+    // comparison of timestamps based on milliseconds is buggy
+    //testAllApis(
+    //  temporalOverlaps("2011-03-10 02:02:02.001".toTimestamp, 0.milli,
+    //    "2011-03-10 02:02:02.002".toTimestamp, "2011-03-10 02:02:02.002".toTimestamp),
+    //  "temporalOverlaps('2011-03-10 02:02:02.001'.toTimestamp, 0.milli, " +
+    //    "'2011-03-10 02:02:02.002'.toTimestamp, '2011-03-10 02:02:02.002'.toTimestamp)",
+    //  "(TIMESTAMP '2011-03-10 02:02:02.001', INTERVAL '0' SECOND) OVERLAPS " +
+    //    "(TIMESTAMP '2011-03-10 02:02:02.002', TIMESTAMP '2011-03-10 02:02:02.002')",
+    //  "false")
+  }
+
+  @Test
+  def testQuarter(): Unit = {
+    testAllApis(
+      "1997-01-27".toDate.quarter(),
+      "'1997-01-27'.toDate.quarter()",
+      "QUARTER(DATE '1997-01-27')",
+      "1")
+
+    testAllApis(
+      "1997-04-27".toDate.quarter(),
+      "'1997-04-27'.toDate.quarter()",
+      "QUARTER(DATE '1997-04-27')",
+      "2")
+
+    testAllApis(
+      "1997-12-31".toDate.quarter(),
+      "'1997-12-31'.toDate.quarter()",
+      "QUARTER(DATE '1997-12-31')",
+      "4")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // Other functions
+  // ----------------------------------------------------------------------------------------------
+
+  @Test
+  def testIsTrueIsFalse(): Unit = {
+    testAllApis(
+      'f1.isTrue,
+      "f1.isTrue",
+      "f1 IS TRUE",
+      "true")
+
+    testAllApis(
+      'f21.isTrue,
+      "f21.isTrue",
+      "f21 IS TRUE",
+      "false")
+
+    testAllApis(
+      false.isFalse,
+      "false.isFalse",
+      "FALSE IS FALSE",
+      "true")
+
+    testAllApis(
+      'f21.isFalse,
+      "f21.isFalse",
+      "f21 IS FALSE",
+      "false")
+
+    testAllApis(
+      'f1.isNotTrue,
+      "f1.isNotTrue",
+      "f1 IS NOT TRUE",
+      "false")
+
+    testAllApis(
+      'f21.isNotTrue,
+      "f21.isNotTrue",
+      "f21 IS NOT TRUE",
+      "true")
+
+    testAllApis(
+      false.isNotFalse,
+      "false.isNotFalse",
+      "FALSE IS NOT FALSE",
+      "false")
+
+    testAllApis(
+      'f21.isNotFalse,
+      "f21.isNotFalse",
+      "f21 IS NOT FALSE",
+      "true")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  def testData = {
+    val testData = new Row(25)
+    testData.setField(0, "This is a test String.")
+    testData.setField(1, true)
+    testData.setField(2, 42.toByte)
+    testData.setField(3, 43.toShort)
+    testData.setField(4, 44.toLong)
+    testData.setField(5, 4.5.toFloat)
+    testData.setField(6, 4.6)
+    testData.setField(7, 3)
+    testData.setField(8, " This is a test String. ")
+    testData.setField(9, -42.toByte)
+    testData.setField(10, -43.toShort)
+    testData.setField(11, -44.toLong)
+    testData.setField(12, -4.5.toFloat)
+    testData.setField(13, -4.6)
+    testData.setField(14, -3)
+    testData.setField(15, BigDecimal("-1231.1231231321321321111").bigDecimal)
+    testData.setField(16, Date.valueOf("1996-11-10"))
+    testData.setField(17, Time.valueOf("06:55:44"))
+    testData.setField(18, Timestamp.valueOf("1996-11-10 06:55:44.333"))
+    testData.setField(19, 1467012213000L) // +16979 07:23:33.000
+    testData.setField(20, 25) // +2-01
+    testData.setField(21, null)
+    testData.setField(22, BigDecimal("2").bigDecimal)
+    testData.setField(23, "%This is a test String.")
+    testData.setField(24, "*_This is a test String.")
+    testData
+  }
+
+  def typeInfo = {
+    new RowTypeInfo(
+      Types.STRING,
+      Types.BOOLEAN,
+      Types.BYTE,
+      Types.SHORT,
+      Types.LONG,
+      Types.FLOAT,
+      Types.DOUBLE,
+      Types.INT,
+      Types.STRING,
+      Types.BYTE,
+      Types.SHORT,
+      Types.LONG,
+      Types.FLOAT,
+      Types.DOUBLE,
+      Types.INT,
+      Types.DECIMAL,
+      Types.DATE,
+      Types.TIME,
+      Types.TIMESTAMP,
+      Types.INTERVAL_MILLIS,
+      Types.INTERVAL_MONTHS,
+      Types.BOOLEAN,
+      Types.DECIMAL,
+      Types.STRING,
+      Types.STRING).asInstanceOf[TypeInformation[Any]]
+
+  }
+}