You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ja...@apache.org on 2019/08/05 14:50:48 UTC

[flink] branch release-1.9 updated (2a6fb9a -> afb0fbc)

This is an automated email from the ASF dual-hosted git repository.

jark pushed a change to branch release-1.9
in repository https://gitbox.apache.org/repos/asf/flink.git.


    from 2a6fb9a  [FLINK-12576][Network,Metrics] Take LocalInputChannel into account when computing inputQueueLength
     new 4f0cc08  [FLINK-13237][table-planner-blink] Fix regexpReplace and regexpExtract to same of flink-planner
     new 93c056a  [FLINK-13237][table-planner-blink] TypeInfoDataTypeConverter should support convert from DayTimeIntervalType
     new 23254d6  [FLINK-13237][table-planner-blink] Fix sqrt conversion in RexNodeConverter
     new f6605e4  [FLINK-13237][table-planner-blink] Add LocalTime and LocalDateTime support to planner expressions
     new afb0fbc  [FLINK-13237][table-planner-blink] Add expression table api tests in blink planner

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../planner/expressions/RexNodeConverter.java      |    6 +-
 .../expressions/PlannerExpressionConverter.scala   |    3 +-
 .../table/planner/expressions/comparison.scala     |   10 +
 .../flink/table/planner/expressions/time.scala     |   21 +-
 .../planner/typeutils/TypeInfoCheckUtils.scala     |    7 +-
 .../table/planner/expressions/ArrayTypeTest.scala  |  202 ++-
 .../planner/expressions/CompositeAccessTest.scala  |   85 +-
 .../planner/expressions/DecimalTypeTest.scala      |  199 ++-
 .../table/planner/expressions/LiteralTest.scala    |   82 +-
 .../table/planner/expressions/MapTypeTest.scala    |  165 +-
 .../planner/expressions/MathFunctionsTest.scala    |  694 --------
 .../expressions/NonDeterministicTests.scala        |   32 +-
 .../table/planner/expressions/RowTypeTest.scala    |   63 +-
 .../planner/expressions/ScalarFunctionsTest.scala  | 1879 +++++++++++++++-----
 .../planner/expressions/TemporalTypesTest.scala    |  357 +++-
 .../UserDefinedScalarFunctionTest.scala            |  101 +-
 .../expressions/utils/ExpressionTestBase.scala     |   71 +-
 .../utils/ScalarOperatorsTestBase.scala            |    5 +
 .../utils/userDefinedScalarFunctions.scala         |   28 +
 .../validation/ArrayTypeValidationTest.scala       |   43 +
 .../validation/CompositeAccessValidationTest.scala |   21 +
 .../validation/MapTypeValidationTest.scala         |   10 +-
 .../validation/RowTypeValidationTest.scala         |    5 +-
 .../validation/ScalarFunctionsValidationTest.scala |  112 +-
 .../validation/ScalarOperatorsValidationTest.scala |    5 +-
 .../table/runtime/functions/SqlFunctionUtils.java  |   28 +-
 .../runtime/types/TypeInfoDataTypeConverter.java   |    5 +
 27 files changed, 2739 insertions(+), 1500 deletions(-)
 delete mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MathFunctionsTest.scala
 copy flink-table/{flink-table-planner/src/test/scala/org/apache/flink/table => flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner}/expressions/UserDefinedScalarFunctionTest.scala (83%)
 copy flink-table/{flink-table-planner/src/test/scala/org/apache/flink/table => flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner}/expressions/validation/ScalarOperatorsValidationTest.scala (94%)


[flink] 03/05: [FLINK-13237][table-planner-blink] Fix sqrt conversion in RexNodeConverter

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jark pushed a commit to branch release-1.9
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 23254d6f6606f6eaf62146661498cfd28b1488cb
Author: JingsongLi <lz...@aliyun.com>
AuthorDate: Thu Jul 25 16:27:59 2019 +0800

    [FLINK-13237][table-planner-blink] Fix sqrt conversion in RexNodeConverter
---
 .../apache/flink/table/planner/expressions/RexNodeConverter.java    | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)

diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/RexNodeConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/RexNodeConverter.java
index a3d85e7..68b7b7f 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/RexNodeConverter.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/RexNodeConverter.java
@@ -34,6 +34,7 @@ import org.apache.flink.table.expressions.TypeLiteralExpression;
 import org.apache.flink.table.expressions.UnresolvedCallExpression;
 import org.apache.flink.table.expressions.UnresolvedReferenceExpression;
 import org.apache.flink.table.expressions.ValueLiteralExpression;
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils;
 import org.apache.flink.table.functions.AggregateFunction;
 import org.apache.flink.table.functions.AggregateFunctionDefinition;
 import org.apache.flink.table.functions.BuiltInFunctionDefinitions;
@@ -97,6 +98,7 @@ import org.apache.calcite.util.TimestampWithTimeZoneString;
 
 import java.math.BigDecimal;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Calendar;
 import java.util.Date;
 import java.util.HashSet;
@@ -241,7 +243,9 @@ public class RexNodeConverter implements ExpressionVisitor<RexNode> {
 		conversionsOfBuiltInFunc
 				.put(BuiltInFunctionDefinitions.POWER, exprs -> convert(FlinkSqlOperatorTable.POWER, exprs));
 		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.MOD, exprs -> convert(FlinkSqlOperatorTable.MOD, exprs));
-		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.SQRT, exprs -> convert(FlinkSqlOperatorTable.SQRT, exprs));
+		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.SQRT, exprs ->
+				relBuilder.call(FlinkSqlOperatorTable.POWER,
+						convertCallChildren(Arrays.asList(exprs.get(0), ApiExpressionUtils.valueLiteral(0.5)))));
 		conversionsOfBuiltInFunc
 				.put(BuiltInFunctionDefinitions.MINUS_PREFIX, exprs -> convert(FlinkSqlOperatorTable.UNARY_MINUS, exprs));
 		conversionsOfBuiltInFunc.put(BuiltInFunctionDefinitions.SIN, exprs -> convert(FlinkSqlOperatorTable.SIN, exprs));


[flink] 04/05: [FLINK-13237][table-planner-blink] Add LocalTime and LocalDateTime support to planner expressions

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jark pushed a commit to branch release-1.9
in repository https://gitbox.apache.org/repos/asf/flink.git

commit f6605e4c832fec4f88164b416c839f1de25c1a1c
Author: JingsongLi <lz...@aliyun.com>
AuthorDate: Thu Jul 25 16:29:29 2019 +0800

    [FLINK-13237][table-planner-blink] Add LocalTime and LocalDateTime support to planner expressions
---
 .../expressions/PlannerExpressionConverter.scala    |  3 +--
 .../table/planner/expressions/comparison.scala      | 10 ++++++++++
 .../flink/table/planner/expressions/time.scala      | 21 ++++++++++++++++-----
 .../planner/typeutils/TypeInfoCheckUtils.scala      |  7 ++++---
 4 files changed, 31 insertions(+), 10 deletions(-)

diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala
index 9b9fd36..90a8282 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/PlannerExpressionConverter.scala
@@ -28,7 +28,6 @@ import org.apache.flink.table.planner.functions.InternalFunctionDefinitions.THRO
 import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter.fromDataTypeToTypeInfo
 import org.apache.flink.table.types.logical.LogicalTypeRoot.{CHAR, DECIMAL, SYMBOL, TIMESTAMP_WITHOUT_TIME_ZONE}
 import org.apache.flink.table.types.logical.utils.LogicalTypeChecks._
-import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo
 
 import _root_.scala.collection.JavaConverters._
 
@@ -91,7 +90,7 @@ class PlannerExpressionConverter private extends ApiExpressionVisitor[PlannerExp
         assert(children.size == 2)
         return ThrowException(
           children.head.accept(this),
-          fromDataTypeToLegacyInfo(
+          fromDataTypeToTypeInfo(
             children(1).asInstanceOf[TypeLiteralExpression].getOutputDataType))
 
       case _ =>
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/comparison.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/comparison.scala
index b0684df..b8769f2 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/comparison.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/comparison.scala
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
 import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable
 import org.apache.flink.table.planner.typeutils.TypeInfoCheckUtils.{isArray, isComparable, isNumeric}
 import org.apache.flink.table.planner.validate._
+import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 
 import org.apache.calcite.sql.SqlOperator
 
@@ -34,6 +35,9 @@ abstract class BinaryComparison extends BinaryExpression {
     (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) if isComparable(lType) &&
+          fromTypeInfoToLogicalType(lType) == fromTypeInfoToLogicalType(rType) =>
+        ValidationSuccess
       case (lType, rType) =>
         ValidationFailure(
           s"Comparison is only supported for numeric types and " +
@@ -50,6 +54,9 @@ case class EqualTo(left: PlannerExpression, right: PlannerExpression) extends Bi
     (left.resultType, right.resultType) match {
       case (lType, rType) if isNumeric(lType) && isNumeric(rType) => ValidationSuccess
       case (lType, rType) if lType == rType => ValidationSuccess
+      case (lType, rType)
+        if fromTypeInfoToLogicalType(lType) == fromTypeInfoToLogicalType(rType) =>
+        ValidationSuccess
       case (lType, rType) if isArray(lType) && lType.getTypeClass == rType.getTypeClass =>
         ValidationSuccess
       case (lType, rType) =>
@@ -66,6 +73,9 @@ case class NotEqualTo(left: PlannerExpression, right: PlannerExpression) extends
     (left.resultType, right.resultType) match {
       case (lType, rType) if isNumeric(lType) && isNumeric(rType) => ValidationSuccess
       case (lType, rType) if lType == rType => ValidationSuccess
+      case (lType, rType)
+        if fromTypeInfoToLogicalType(lType) == fromTypeInfoToLogicalType(rType) =>
+        ValidationSuccess
       case (lType, rType) if isArray(lType) && lType.getTypeClass == rType.getTypeClass =>
         ValidationSuccess
       case (lType, rType) =>
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/time.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/time.scala
index 2429881..8817f5c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/time.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/expressions/time.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.planner.expressions
 
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo._
-import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation}
+import org.apache.flink.api.common.typeinfo.{LocalTimeTypeInfo, SqlTimeTypeInfo, TypeInformation}
 import org.apache.flink.table.planner.calcite.FlinkRelBuilder
 import org.apache.flink.table.planner.expressions.PlannerTimeIntervalUnit.PlannerTimeIntervalUnit
 import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable
@@ -51,6 +51,8 @@ case class Extract(timeIntervalUnit: PlannerExpression, temporal: PlannerExpress
            | SymbolPlannerExpression(PlannerTimeIntervalUnit.DAY)
         if temporal.resultType == SqlTimeTypeInfo.DATE
           || temporal.resultType == SqlTimeTypeInfo.TIMESTAMP
+          || temporal.resultType == LocalTimeTypeInfo.LOCAL_DATE
+          || temporal.resultType == LocalTimeTypeInfo.LOCAL_DATE_TIME
           || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MILLIS
           || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MONTHS =>
         ValidationSuccess
@@ -60,6 +62,8 @@ case class Extract(timeIntervalUnit: PlannerExpression, temporal: PlannerExpress
            | SymbolPlannerExpression(PlannerTimeIntervalUnit.SECOND)
         if temporal.resultType == SqlTimeTypeInfo.TIME
           || temporal.resultType == SqlTimeTypeInfo.TIMESTAMP
+          || temporal.resultType == LocalTimeTypeInfo.LOCAL_TIME
+          || temporal.resultType == LocalTimeTypeInfo.LOCAL_DATE_TIME
           || temporal.resultType == TimeIntervalTypeInfo.INTERVAL_MILLIS =>
         ValidationSuccess
 
@@ -97,12 +101,15 @@ abstract class TemporalCeilFloor(
 
     (unit.get, temporal.resultType) match {
       case (PlannerTimeIntervalUnit.YEAR | PlannerTimeIntervalUnit.MONTH,
-          SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIMESTAMP) =>
+          SqlTimeTypeInfo.DATE | SqlTimeTypeInfo.TIMESTAMP |
+          LocalTimeTypeInfo.LOCAL_DATE | LocalTimeTypeInfo.LOCAL_DATE_TIME) =>
         ValidationSuccess
-      case (PlannerTimeIntervalUnit.DAY, SqlTimeTypeInfo.TIMESTAMP) =>
+      case (PlannerTimeIntervalUnit.DAY, SqlTimeTypeInfo.TIMESTAMP |
+            LocalTimeTypeInfo.LOCAL_DATE_TIME) =>
         ValidationSuccess
       case (PlannerTimeIntervalUnit.HOUR | PlannerTimeIntervalUnit.MINUTE |
-          PlannerTimeIntervalUnit.SECOND, SqlTimeTypeInfo.TIME | SqlTimeTypeInfo.TIMESTAMP) =>
+          PlannerTimeIntervalUnit.SECOND, SqlTimeTypeInfo.TIME | SqlTimeTypeInfo.TIMESTAMP |
+          LocalTimeTypeInfo.LOCAL_TIME | LocalTimeTypeInfo.LOCAL_DATE_TIME) =>
         ValidationSuccess
       case _ =>
         ValidationFailure(s"Temporal ceil/floor operator does not support " +
@@ -308,7 +315,11 @@ case class TimestampDiff(
         if timePoint1.resultType == SqlTimeTypeInfo.DATE
           || timePoint1.resultType == SqlTimeTypeInfo.TIMESTAMP
           || timePoint2.resultType == SqlTimeTypeInfo.DATE
-          || timePoint2.resultType == SqlTimeTypeInfo.TIMESTAMP =>
+          || timePoint2.resultType == SqlTimeTypeInfo.TIMESTAMP
+          || timePoint1.resultType == LocalTimeTypeInfo.LOCAL_DATE
+          || timePoint1.resultType == LocalTimeTypeInfo.LOCAL_DATE_TIME
+          || timePoint2.resultType == LocalTimeTypeInfo.LOCAL_DATE
+          || timePoint2.resultType == LocalTimeTypeInfo.LOCAL_DATE_TIME =>
         ValidationSuccess
 
       case _ =>
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/typeutils/TypeInfoCheckUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/typeutils/TypeInfoCheckUtils.scala
index 80f4a57..33108dd 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/typeutils/TypeInfoCheckUtils.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/typeutils/TypeInfoCheckUtils.scala
@@ -23,6 +23,7 @@ import org.apache.flink.api.common.typeutils.CompositeType
 import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, PojoTypeInfo}
 import org.apache.flink.table.api.ValidationException
 import org.apache.flink.table.planner.validate._
+import org.apache.flink.table.runtime.typeutils.{BigDecimalTypeInfo, DecimalTypeInfo}
 import org.apache.flink.table.typeutils.TimeIntervalTypeInfo.{INTERVAL_MILLIS, INTERVAL_MONTHS}
 import org.apache.flink.table.typeutils.{TimeIndicatorTypeInfo, TimeIntervalTypeInfo}
 
@@ -48,7 +49,7 @@ object TypeInfoCheckUtils {
 
   def isNumeric(dataType: TypeInformation[_]): Boolean = dataType match {
     case _: NumericTypeInfo[_] => true
-    case BIG_DEC_TYPE_INFO => true
+    case BIG_DEC_TYPE_INFO | _: BigDecimalTypeInfo | _: DecimalTypeInfo => true
     case _ => false
   }
 
@@ -56,7 +57,7 @@ object TypeInfoCheckUtils {
     isTimePoint(dataType) || isTimeInterval(dataType)
 
   def isTimePoint(dataType: TypeInformation[_]): Boolean =
-    dataType.isInstanceOf[SqlTimeTypeInfo[_]]
+    dataType.isInstanceOf[SqlTimeTypeInfo[_]] || dataType.isInstanceOf[LocalTimeTypeInfo[_]]
 
   def isTimeInterval(dataType: TypeInformation[_]): Boolean =
     dataType.isInstanceOf[TimeIntervalTypeInfo[_]]
@@ -103,7 +104,7 @@ object TypeInfoCheckUtils {
   : ValidationResult = dataType match {
     case _: NumericTypeInfo[_] =>
       ValidationSuccess
-    case BIG_DEC_TYPE_INFO =>
+    case BIG_DEC_TYPE_INFO | _: BigDecimalTypeInfo | _: DecimalTypeInfo =>
       ValidationSuccess
     case _ =>
       ValidationFailure(s"$caller requires numeric types, get $dataType here")


[flink] 01/05: [FLINK-13237][table-planner-blink] Fix regexpReplace and regexpExtract to same of flink-planner

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jark pushed a commit to branch release-1.9
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 4f0cc08c5fc9f6a4d4668afdd02791fd45c59339
Author: JingsongLi <lz...@aliyun.com>
AuthorDate: Thu Jul 25 16:14:07 2019 +0800

    [FLINK-13237][table-planner-blink] Fix regexpReplace and regexpExtract to same of flink-planner
---
 .../table/runtime/functions/SqlFunctionUtils.java  | 28 ++++++++--------------
 1 file changed, 10 insertions(+), 18 deletions(-)

diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlFunctionUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlFunctionUtils.java
index 6ae9b8b..f904fd5 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlFunctionUtils.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/functions/SqlFunctionUtils.java
@@ -342,22 +342,15 @@ public class SqlFunctionUtils {
 	}
 
 	/**
-	 * Returns a string resulting from replacing all substrings that match the regular
-	 * expression with replacement.
+	 * Returns a string resulting from replacing all substrings
+	 * that match the regular expression with replacement.
 	 */
 	public static String regexpReplace(String str, String regex, String replacement) {
-		if (regex.isEmpty()) {
-			return str;
+		if (str == null || regex == null || replacement == null) {
+			return null;
 		}
 		try {
-			// we should use StringBuffer here because Matcher only accept it
-			StringBuffer sb = new StringBuffer();
-			Matcher m = REGEXP_PATTERN_CACHE.get(regex).matcher(str);
-			while (m.find()) {
-				m.appendReplacement(sb, replacement);
-			}
-			m.appendTail(sb);
-			return sb.toString();
+			return str.replaceAll(regex, Matcher.quoteReplacement(replacement));
 		} catch (Exception e) {
 			LOG.error(
 				String.format("Exception in regexpReplace('%s', '%s', '%s')", str, regex, replacement),
@@ -368,27 +361,26 @@ public class SqlFunctionUtils {
 	}
 
 	/**
-	 * Returns a string extracted with a specified regular expression and a regex
-	 * match group index.
+	 * Returns a string extracted with a specified regular expression and a regex match group index.
 	 */
 	public static String regexpExtract(String str, String regex, int extractIndex) {
-		if (extractIndex < 0) {
+		if (str == null || regex == null) {
 			return null;
 		}
 
 		try {
-			Matcher m = REGEXP_PATTERN_CACHE.get(regex).matcher(str);
+			Matcher m = Pattern.compile(regex).matcher(str);
 			if (m.find()) {
 				MatchResult mr = m.toMatchResult();
 				return mr.group(extractIndex);
 			}
-			return null;
 		} catch (Exception e) {
 			LOG.error(
 				String.format("Exception in regexpExtract('%s', '%s', '%d')", str, regex, extractIndex),
 				e);
-			return null;
 		}
+
+		return null;
 	}
 
 	public static String regexpExtract(String str, String regex, long extractIndex) {


[flink] 02/05: [FLINK-13237][table-planner-blink] TypeInfoDataTypeConverter should support convert from DayTimeIntervalType

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jark pushed a commit to branch release-1.9
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 93c056a5020a649ddf2a21e69b4ea9735f632c49
Author: JingsongLi <lz...@aliyun.com>
AuthorDate: Thu Jul 25 16:25:33 2019 +0800

    [FLINK-13237][table-planner-blink] TypeInfoDataTypeConverter should support convert from DayTimeIntervalType
---
 .../apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java  | 5 +++++
 1 file changed, 5 insertions(+)

diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java
index 044356c..b08f72d 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/TypeInfoDataTypeConverter.java
@@ -44,6 +44,7 @@ import org.apache.flink.table.types.logical.DecimalType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.table.typeutils.TimeIntervalTypeInfo;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.Preconditions;
 
@@ -110,6 +111,10 @@ public class TypeInfoDataTypeConverter {
 			case BINARY:
 			case VARBINARY: // ignore precision
 				return PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO;
+			case INTERVAL_YEAR_MONTH:
+				return TimeIntervalTypeInfo.INTERVAL_MONTHS;
+			case INTERVAL_DAY_TIME:
+				return TimeIntervalTypeInfo.INTERVAL_MILLIS;
 			case ARRAY:
 				if (dataType instanceof CollectionDataType &&
 						!isPrimitive(((CollectionDataType) dataType).getElementDataType().getLogicalType())) {


[flink] 05/05: [FLINK-13237][table-planner-blink] Add expression table api tests in blink planner

Posted by ja...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jark pushed a commit to branch release-1.9
in repository https://gitbox.apache.org/repos/asf/flink.git

commit afb0fbc5c0952377395ff3c7322214720a7a7ccd
Author: JingsongLi <lz...@aliyun.com>
AuthorDate: Fri Jul 12 11:53:12 2019 +0800

    [FLINK-13237][table-planner-blink] Add expression table api tests in blink planner
    
    This closes #9099
---
 .../table/planner/expressions/ArrayTypeTest.scala  |  202 ++-
 .../planner/expressions/CompositeAccessTest.scala  |   85 +-
 .../planner/expressions/DecimalTypeTest.scala      |  199 ++-
 .../table/planner/expressions/LiteralTest.scala    |   82 +-
 .../table/planner/expressions/MapTypeTest.scala    |  165 +-
 .../planner/expressions/MathFunctionsTest.scala    |  694 --------
 .../expressions/NonDeterministicTests.scala        |   32 +-
 .../table/planner/expressions/RowTypeTest.scala    |   63 +-
 .../planner/expressions/ScalarFunctionsTest.scala  | 1879 +++++++++++++++-----
 .../planner/expressions/TemporalTypesTest.scala    |  357 +++-
 .../UserDefinedScalarFunctionTest.scala            |  485 +++++
 .../expressions/utils/ExpressionTestBase.scala     |   71 +-
 .../utils/ScalarOperatorsTestBase.scala            |    5 +
 .../utils/userDefinedScalarFunctions.scala         |   28 +
 .../validation/ArrayTypeValidationTest.scala       |   43 +
 .../validation/CompositeAccessValidationTest.scala |   21 +
 .../validation/MapTypeValidationTest.scala         |   10 +-
 .../validation/RowTypeValidationTest.scala         |    5 +-
 .../validation/ScalarFunctionsValidationTest.scala |  112 +-
 .../validation/ScalarOperatorsValidationTest.scala |   88 +
 20 files changed, 3202 insertions(+), 1424 deletions(-)

diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
index 82c9229..4e3acd7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ArrayTypeTest.scala
@@ -18,7 +18,10 @@
 
 package org.apache.flink.table.planner.expressions
 
+import org.apache.flink.table.api.DataTypes
+import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.expressions.utils.ArrayTypeTestBase
+import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, localDateTime, localTime => gLocalTime}
 
 import org.junit.Test
 
@@ -27,115 +30,175 @@ class ArrayTypeTest extends ArrayTypeTestBase {
   @Test
   def testArrayLiterals(): Unit = {
     // primitive literals
-    testSqlApi(
-      "ARRAY[1, 2, 3]",
-      "[1, 2, 3]")
+    testAllApis(array(1, 2, 3), "array(1, 2, 3)", "ARRAY[1, 2, 3]", "[1, 2, 3]")
 
-    testSqlApi(
+    testAllApis(
+      array(true, true, true),
+      "array(true, true, true)",
       "ARRAY[TRUE, TRUE, TRUE]",
       "[true, true, true]")
 
-    testSqlApi(
+    // 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]]]")
 
-    testSqlApi(
+    testAllApis(
+      array(1 + 1, 3 * 3),
+      "array(1 + 1, 3 * 3)",
       "ARRAY[1 + 1, 3 * 3]",
       "[2, 9]")
 
-    testSqlApi(
+    testAllApis(
+      array(nullOf(DataTypes.INT), 1),
+      "array(Null(INT), 1)",
       "ARRAY[NULLIF(1,1), 1]",
       "[null, 1]")
 
-    testSqlApi(
+    testAllApis(
+      array(array(nullOf(DataTypes.INT), 1)),
+      "array(array(Null(INT), 1))",
       "ARRAY[ARRAY[NULLIF(1,1), 1]]",
       "[[null, 1]]")
 
-    testSqlApi(
+    // 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(localDate("1985-04-11"), localDate("2018-07-26")),
+      "array('1985-04-11'.toDate, '2018-07-26'.toDate)",
       "ARRAY[DATE '1985-04-11', DATE '2018-07-26']",
       "[1985-04-11, 2018-07-26]")
 
-    testSqlApi(
+    testAllApis(
+      Array(gLocalTime("14:15:16"), gLocalTime("17:18:19")),
+      "array('14:15:16'.toTime, '17:18:19'.toTime)",
       "ARRAY[TIME '14:15:16', TIME '17:18:19']",
       "[14:15:16, 17:18:19]")
 
-    testSqlApi(
+    testAllApis(
+      Array(localDateTime("1985-04-11 14:15:16"), localDateTime("2018-07-26 17:18:19")),
+      "array('1985-04-11 14:15:16'.toTimestamp, '2018-07-26 17:18:19'.toTimestamp)",
       "ARRAY[TIMESTAMP '1985-04-11 14:15:16', TIMESTAMP '2018-07-26 17:18:19']",
       "[1985-04-11 14:15:16.000, 2018-07-26 17:18:19.000]")
 
-    testSqlApi(
+    testAllApis(
+      Array(BigDecimal(2.0002), BigDecimal(2.0003)),
+      "Array(2.0002p, 2.0003p)",
       "ARRAY[CAST(2.0002 AS DECIMAL(10,4)), CAST(2.0003 AS DECIMAL(10,4))]",
       "[2.0002, 2.0003]")
 
-    testSqlApi(
+    testAllApis(
+      Array(Array(x = true)),
+      "Array(Array(true))",
       "ARRAY[ARRAY[TRUE]]",
       "[[true]]")
 
-    testSqlApi(
+    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]]")
 
     // implicit type cast only works on SQL APIs.
-    testSqlApi(
-      "ARRAY[CAST(1 AS DOUBLE), CAST(2 AS FLOAT)]",
-      "[1.0, 2.0]")
+    testSqlApi("ARRAY[CAST(1 AS DOUBLE), CAST(2 AS FLOAT)]", "[1.0, 2.0]")
   }
 
   @Test
   def testArrayField(): Unit = {
-    testSqlApi(
+    testAllApis(
+      array('f0, 'f1),
+      "array(f0, f1)",
       "ARRAY[f0, f1]",
       "[null, 42]")
 
-    testSqlApi(
+    testAllApis(
+      array('f0, 'f1),
+      "array(f0, f1)",
       "ARRAY[f0, f1]",
       "[null, 42]")
 
-    testSqlApi(
+    testAllApis(
+      'f2,
+      "f2",
       "f2",
       "[1, 2, 3]")
 
-    testSqlApi(
+    testAllApis(
+      'f3,
+      "f3",
       "f3",
       "[1984-03-12, 1984-02-10]")
 
-    testSqlApi(
+    testAllApis(
+      'f5,
+      "f5",
       "f5",
       "[[1, 2, 3], null]")
 
-    testSqlApi(
+    testAllApis(
+      'f6,
+      "f6",
       "f6",
       "[1, null, null, 4]")
 
-    testSqlApi(
+    testAllApis(
+      'f2,
+      "f2",
       "f2",
       "[1, 2, 3]")
 
-    testSqlApi(
+    testAllApis(
+      'f2.at(1),
+      "f2.at(1)",
       "f2[1]",
       "1")
 
-    testSqlApi(
+    testAllApis(
+      'f3.at(1),
+      "f3.at(1)",
       "f3[1]",
       "1984-03-12")
 
-    testSqlApi(
+    testAllApis(
+      'f3.at(2),
+      "f3.at(2)",
       "f3[2]",
       "1984-02-10")
 
-    testSqlApi(
+    testAllApis(
+      'f5.at(1).at(2),
+      "f5.at(1).at(2)",
       "f5[1][2]",
       "2")
 
-    testSqlApi(
+    testAllApis(
+      'f5.at(2).at(2),
+      "f5.at(2).at(2)",
       "f5[2][2]",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f4.at(2).at(2),
+      "f4.at(2).at(2)",
       "f4[2][2]",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f11.at(1),
+      "f11.at(1)",
       "f11[1]",
       "1")
   }
@@ -143,74 +206,117 @@ class ArrayTypeTest extends ArrayTypeTestBase {
   @Test
   def testArrayOperations(): Unit = {
     // cardinality
-    testSqlApi(
+    testAllApis(
+      'f2.cardinality(),
+      "f2.cardinality()",
       "CARDINALITY(f2)",
       "3")
 
-    testSqlApi(
+    testAllApis(
+      'f4.cardinality(),
+      "f4.cardinality()",
       "CARDINALITY(f4)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f11.cardinality(),
+      "f11.cardinality()",
       "CARDINALITY(f11)",
       "1")
 
     // element
-    testSqlApi(
+    testAllApis(
+      'f9.element(),
+      "f9.element()",
       "ELEMENT(f9)",
       "1")
 
-    testSqlApi(
+    testAllApis(
+      'f8.element(),
+      "f8.element()",
       "ELEMENT(f8)",
       "4.0")
 
-    testSqlApi(
+    testAllApis(
+      'f10.element(),
+      "f10.element()",
       "ELEMENT(f10)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f4.element(),
+      "f4.element()",
       "ELEMENT(f4)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f11.element(),
+      "f11.element()",
       "ELEMENT(f11)",
       "1")
 
     // comparison
-    testSqlApi(
+    testAllApis(
+      'f2 === 'f5.at(1),
+      "f2 === f5.at(1)",
       "f2 = f5[1]",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f6 === array(1, 2, 3),
+      "f6 === array(1, 2, 3)",
       "f6 = ARRAY[1, 2, 3]",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f2 !== 'f5.at(1),
+      "f2 !== f5.at(1)",
       "f2 <> f5[1]",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f2 === 'f7,
+      "f2 === f7",
       "f2 = f7",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f2 !== 'f7,
+      "f2 !== f7",
       "f2 <> f7",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f11 === 'f11,
+      "f11 === f11",
       "f11 = f11",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f11 === 'f9,
+      "f11 === f9",
       "f11 = f9",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f11 !== 'f11,
+      "f11 !== f11",
       "f11 <> f11",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f11 !== 'f9,
+      "f11 !== f9",
       "f11 <> f9",
       "false")
   }
+
+  @Test
+  def testArrayTypeCasting(): Unit = {
+    testTableApi(
+      'f3.cast(DataTypes.ARRAY(DataTypes.DATE)),
+      "f3.cast(OBJECT_ARRAY(SQL_DATE))",
+      "[1984-03-12, 1984-02-10]"
+    )
+  }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala
index 20e34a9..43b8160 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/CompositeAccessTest.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.planner.expressions
 
+import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.expressions.utils.CompositeTypeTestBase
 
 import org.junit.Test
@@ -28,7 +29,9 @@ class CompositeAccessTest extends CompositeTypeTestBase {
   def testGetField(): Unit = {
 
     // single field by string key
-    testSqlApi(
+    testAllApis(
+      'f0.get("intField"),
+      "f0.get('intField')",
       "testTable.f0.intField",
       "42")
     testSqlApi("f0.intField", "42")
@@ -39,8 +42,16 @@ class CompositeAccessTest extends CompositeTypeTestBase {
     testSqlApi("testTable.f0.booleanField", "true")
     testSqlApi("f0.booleanField", "true")
 
+    // single field by int key
+    testTableApi(
+      'f0.get(0),
+      "f0.get(0)",
+      "42")
+
     // nested single field
-    testSqlApi(
+    testAllApis(
+      'f1.get("objectField").get("intField"),
+      "f1.get('objectField').get('intField')",
       "testTable.f1.objectField.intField",
       "25")
     testSqlApi("f1.objectField.intField", "25")
@@ -51,7 +62,9 @@ class CompositeAccessTest extends CompositeTypeTestBase {
     testSqlApi("testTable.f1.objectField.booleanField", "false")
     testSqlApi("f1.objectField.booleanField", "false")
 
-    testSqlApi(
+    testAllApis(
+      'f2.get(0),
+      "f2.get(0)",
       "testTable.f2._1",
       "a")
     testSqlApi("f2._1", "a")
@@ -65,7 +78,9 @@ class CompositeAccessTest extends CompositeTypeTestBase {
     testSqlApi("testTable.f5", "13")
     testSqlApi("f5", "13")
 
-    testSqlApi(
+    testAllApis(
+      'f7.get("_1"),
+      "get(f7, '_1')",
       "testTable.f7._1",
       "true")
 
@@ -73,70 +88,82 @@ class CompositeAccessTest extends CompositeTypeTestBase {
     testSqlApi("testTable.f6", "MyCaseClass2(null)")
     testSqlApi("f6", "MyCaseClass2(null)")
 
-    // MyCaseClass is converted to BaseRow
-    // so the result of "toString" does'nt contain MyCaseClass prefix
-    testSqlApi(
+    testAllApis(
+      'f1.get("objectField"),
+      "f1.get('objectField')",
       "testTable.f1.objectField",
       "(25,Timo,false)")
     testSqlApi("f1.objectField", "(25,Timo,false)")
 
-    testSqlApi(
+    testAllApis(
+      'f0,
+      "f0",
       "testTable.f0",
       "(42,Bob,true)")
     testSqlApi("f0", "(42,Bob,true)")
 
     // flattening (test base only returns first column)
-    testSqlApi(
+    testAllApis(
+      'f1.get("objectField").flatten(),
+      "f1.get('objectField').flatten()",
       "testTable.f1.objectField.*",
       "25")
     testSqlApi("f1.objectField.*", "25")
 
-    testSqlApi(
+    testAllApis(
+      'f0.flatten(),
+      "flatten(f0)",
       "testTable.f0.*",
       "42")
     testSqlApi("f0.*", "42")
 
+    testTableApi(12.flatten(), "12.flatten()", "12")
+
+    testTableApi('f5.flatten(), "f5.flatten()", "13")
+
     // array of composites
-    testSqlApi(
+    testAllApis(
+      'f8.at(1).get("_1"),
+      "f8.at(1).get('_1')",
       "f8[1]._1",
       "true"
     )
-
-    testSqlApi(
+    testAllApis(
+      'f8.at(1).get("_2"),
+      "f8.at(1).get('_2')",
       "f8[1]._2",
       "23"
     )
-
-    testSqlApi(
+    testAllApis(
+      'f9.at(2).get("_1"),
+      "f9.at(2).get('_1')",
       "f9[2]._1",
       "null"
     )
-
-    testSqlApi(
+    testAllApis(
+      'f10.at(1).get("stringField"),
+      "f10.at(1).get('stringField')",
       "f10[1].stringField",
       "Bob"
     )
-
-    testSqlApi(
+    testAllApis(
+      'f11.at(1).get("myString"),
+      "f11.at(1).get('myString')",
       "f11[1].myString",
       "Hello"
     )
-
-    testSqlApi(
-      "f11[2]",
-      "null"
-    )
-
-    testSqlApi(
+    testAllApis(
+      'f12.at(1).get("arrayField").at(1).get("stringField"),
+      "f12.at(1).get('arrayField').at(1).get('stringField')",
       "f12[1].arrayField[1].stringField",
       "Alice"
     )
 
-    testSqlApi(
+    testAllApis(
+      'f13.at(1).get("objectField").get("stringField"),
+      "f13.at(1).get('objectField').get('stringField')",
       "f13[1].objectField.stringField",
       "Bob"
     )
   }
 }
-
-
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
index b19c4d2..e243930 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/DecimalTypeTest.scala
@@ -18,101 +18,165 @@
 
 package org.apache.flink.table.planner.expressions
 
-import org.apache.flink.api.common.typeinfo.Types
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.dataformat.Decimal
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.{DataTypes, Types}
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral
 import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase
-import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
+import org.apache.flink.table.runtime.typeutils.BigDecimalTypeInfo
 import org.apache.flink.types.Row
 
-import org.junit.Test
+import org.junit.{Ignore, Test}
 
 class DecimalTypeTest extends ExpressionTestBase {
 
   @Test
   def testDecimalLiterals(): Unit = {
     // implicit double
-    testSqlApi(
+    testAllApis(
+      11.2,
+      "11.2",
       "11.2",
       "11.2")
 
     // implicit double
-    testSqlApi(
+    testAllApis(
+      0.7623533651719233,
+      "0.7623533651719233",
       "0.7623533651719233",
       "0.7623533651719233")
 
     // explicit decimal (with precision of 19)
-    testSqlApi(
+    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 = {
-    testSqlApi(
+    testAllApis(
+      Double.MaxValue,
+      Double.MaxValue.toString,
       Double.MaxValue.toString,
       Double.MaxValue.toString)
 
-    testSqlApi(
+    testAllApis(
+      Double.MinValue,
+      Double.MinValue.toString,
       Double.MinValue.toString,
       Double.MinValue.toString)
 
-    testSqlApi(
+    testAllApis(
+      Double.MinValue.cast(DataTypes.FLOAT),
+      s"${Double.MinValue}.cast(FLOAT)",
       s"CAST(${Double.MinValue} AS FLOAT)",
       Float.NegativeInfinity.toString)
 
-    testSqlApi(
+    testAllApis(
+      Byte.MinValue.cast(DataTypes.TINYINT),
+      s"(${Byte.MinValue}).cast(BYTE)",
       s"CAST(${Byte.MinValue} AS TINYINT)",
       Byte.MinValue.toString)
 
-    testSqlApi(
+    testAllApis(
+      Byte.MinValue.cast(DataTypes.TINYINT) - 1.cast(DataTypes.TINYINT),
+      s"(${Byte.MinValue}).cast(BYTE) - (1).cast(BYTE)",
       s"CAST(${Byte.MinValue} AS TINYINT) - CAST(1 AS TINYINT)",
       Byte.MaxValue.toString)
 
-    testSqlApi(
+    testAllApis(
+      Short.MinValue.cast(DataTypes.SMALLINT),
+      s"(${Short.MinValue}).cast(SHORT)",
       s"CAST(${Short.MinValue} AS SMALLINT)",
       Short.MinValue.toString)
 
-    testSqlApi(
+    testAllApis(
+      Int.MinValue.cast(DataTypes.INT) - 1,
+      s"(${Int.MinValue}).cast(INT) - 1",
       s"CAST(${Int.MinValue} AS INT) - 1",
       Int.MaxValue.toString)
 
-    testSqlApi(
+    testAllApis(
+      Long.MinValue.cast(DataTypes.BIGINT()),
+      s"(${Long.MinValue}L).cast(LONG)",
       s"CAST(${Long.MinValue} AS BIGINT)",
       Long.MinValue.toString)
   }
 
+  @Ignore
   @Test
-  def testDecimalCasting(): Unit = {
+  def testDefaultDecimalCasting(): Unit = {
     // from String
-    testSqlApi(
-      "CAST('123456789123456789123456789' AS DECIMAL(27, 0))",
+    testTableApi(
+      "123456789123456789123456789".cast(DataTypes.DECIMAL(38, 0)),
+      "'123456789123456789123456789'.cast(DECIMAL)",
       "123456789123456789123456789")
 
     // from double
-    testSqlApi(
+    testAllApis(
+      'f3.cast(DataTypes.DECIMAL(38, 0)),
+      "f3.cast(DECIMAL)",
       "CAST(f3 AS DECIMAL)",
       "4")
+  }
 
+  @Test
+  def testDecimalCasting(): Unit = {
     testSqlApi(
       "CAST(f3 AS DECIMAL(10,2))",
       "4.20"
     )
 
     // to double
-    testSqlApi(
+    testAllApis(
+      'f0.cast(DataTypes.DOUBLE),
+      "f0.cast(DOUBLE)",
       "CAST(f0 AS DOUBLE)",
       "1.2345678912345679E8")
 
     // to int
-    testSqlApi(
+    testAllApis(
+      'f4.cast(DataTypes.INT),
+      "f4.cast(INT)",
       "CAST(f4 AS INT)",
       "123456789")
 
     // to long
-    testSqlApi(
+    testAllApis(
+      'f4.cast(DataTypes.BIGINT()),
+      "f4.cast(LONG)",
       "CAST(f4 AS BIGINT)",
       "123456789")
+
+    // to boolean (not SQL compliant)
+    testTableApi(
+      'f1.cast(DataTypes.BOOLEAN),
+      "f1.cast(BOOLEAN)",
+      "true")
+
+    testTableApi(
+      'f5.cast(DataTypes.BOOLEAN),
+      "f5.cast(BOOLEAN)",
+      "false")
+
+    testTableApi(
+      BigDecimal("123456789.123456789123456789").cast(DataTypes.DOUBLE),
+      "(123456789.123456789123456789p).cast(DOUBLE)",
+      "1.2345678912345679E8")
   }
 
   @Test
@@ -123,72 +187,111 @@ class DecimalTypeTest extends ExpressionTestBase {
     // Decimal+Double => Double.
 
     // implicit cast to decimal
-    testSqlApi(
+    testAllApis(
+      'f1 + 12,
+      "f1 + 12",
       "f1 + 12",
       "123456789123456789123456801")
 
     // implicit cast to decimal
-    testSqlApi(
+    testAllApis(
+      valueLiteral(12) + 'f1,
+      "12 + f1",
       "12 + f1",
       "123456789123456789123456801")
 
-    testSqlApi(
+    testAllApis(
+      'f1 + BigDecimal("12.3"),
+      "f1 + 12.3p",
       "f1 + 12.3",
       "123456789123456789123456801.3"
     )
 
-    testSqlApi(
+    testAllApis(
+      valueLiteral(BigDecimal("12.3").bigDecimal) + 'f1,
+      "12.3p + f1",
       "12.3 + f1",
       "123456789123456789123456801.3")
 
-    testSqlApi(
+    testAllApis(
+      'f1 + 'f1,
+      "f1 + f1",
       "f1 + f1",
       "246913578246913578246913578")
 
-    testSqlApi(
+    testAllApis(
+      'f1 - 'f1,
+      "f1 - f1",
       "f1 - f1",
       "0")
 
-    testSqlApi(
+    // exceeds max precision 38.
+    //      'f1 * 'f1,
+    //      "f1 * f1",
+    //      "f1 * f1",
+    //      "15241578780673678546105778281054720515622620750190521")
+
+    testAllApis(
+      'f1 / 'f1,
+      "f1 / f1",
       "f1 / f1",
       "1.00000000")
+    // Decimal(30,0) / Decimal(30, 0) => Decimal(61,31) => Decimal(38,8)
 
-    testSqlApi(
+    testAllApis(
+      'f1 % 'f1,
+      "f1 % f1",
       "MOD(f1, f1)",
       "0")
 
-    testSqlApi(
+    testAllApis(
+      -'f0,
+      "-f0",
       "-f0",
       "-123456789.123456789123456789")
   }
 
   @Test
   def testDecimalComparison(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f1 < 12,
+      "f1 < 12",
       "f1 < 12",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f1 > 12,
+      "f1 > 12",
       "f1 > 12",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f1 === 12,
+      "f1 === 12",
       "f1 = 12",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f5 === 0,
+      "f5 === 0",
       "f5 = 0",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f1 === BigDecimal("123456789123456789123456789"),
+      "f1 === 123456789123456789123456789p",
       "f1 = CAST('123456789123456789123456789' AS DECIMAL(30, 0))",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f1 !== BigDecimal("123456789123456789123456789"),
+      "f1 !== 123456789123456789123456789p",
       "f1 <> CAST('123456789123456789123456789' AS DECIMAL(30, 0))",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f4 < 'f0,
+      "f4 < f0",
       "f4 < f0",
       "true")
 
@@ -202,22 +305,22 @@ class DecimalTypeTest extends ExpressionTestBase {
 
   override def testData: Row = {
     val testData = new Row(6)
-    testData.setField(0, Decimal.castFrom("123456789.123456789123456789", 30, 18))
-    testData.setField(1, Decimal.castFrom("123456789123456789123456789", 30, 0))
+    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, Decimal.castFrom("123456789", 10, 0))
-    testData.setField(5, Decimal.castFrom("0.000", 10, 3))
+    testData.setField(4, BigDecimal("123456789").bigDecimal)
+    testData.setField(5, BigDecimal("0.000").bigDecimal)
     testData
   }
 
   override def typeInfo: RowTypeInfo = {
     new RowTypeInfo(
-      /* 0 */ DecimalTypeInfo.of(30, 18),
-      /* 1 */ DecimalTypeInfo.of(30, 0),
-      /* 2 */ Types.INT,
-      /* 3 */ Types.DOUBLE,
-      /* 4 */ DecimalTypeInfo.of(10, 0),
-      /* 5 */ DecimalTypeInfo.of(10, 3))
+      /* 1 */ BigDecimalTypeInfo.of(30, 18),
+      /* 2 */ BigDecimalTypeInfo.of(30, 0),
+      /* 3 */ Types.INT,
+      /* 4 */ Types.DOUBLE,
+      /* 5 */ BigDecimalTypeInfo.of(10, 0),
+      /* 6 */ BigDecimalTypeInfo.of(10, 3))
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/LiteralTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/LiteralTest.scala
index 4857648..5158b25 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/LiteralTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/LiteralTest.scala
@@ -20,7 +20,9 @@ package org.apache.flink.table.planner.expressions
 
 import org.apache.flink.api.common.typeinfo.{TypeInformation, Types}
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.planner.expressions.utils.{ExpressionTestBase, Func3}
 import org.apache.flink.types.Row
 
 import org.junit.Test
@@ -30,27 +32,37 @@ class LiteralTest extends ExpressionTestBase {
   @Test
   def testFieldWithBooleanPrefix(): Unit = {
 
-    testSqlApi(
+    testAllApis(
+      'trUeX,
+      "trUeX",
       "trUeX",
       "trUeX_value"
     )
 
-    testSqlApi(
+    testAllApis(
+      'FALSE_A,
+      "FALSE_A",
       "FALSE_A",
       "FALSE_A_value"
     )
 
-    testSqlApi(
+    testAllApis(
+      'FALSE_AB,
+      "FALSE_AB",
       "FALSE_AB",
       "FALSE_AB_value"
     )
 
-    testSqlApi(
+    testAllApis(
+      true,
+      "trUe",
       "trUe",
       "true"
     )
 
-    testSqlApi(
+    testAllApis(
+      false,
+      "FALSE",
       "FALSE",
       "false"
     )
@@ -58,11 +70,15 @@ class LiteralTest extends ExpressionTestBase {
 
   @Test
   def testNonAsciiLiteral(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f4.like("%测试%"),
+      "f4.like('%测试%')",
       "f4 LIKE '%测试%'",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      "Абвгде" + "谢谢",
+      "'Абвгде' + '谢谢'",
       "'Абвгде' || '谢谢'",
       "Абвгде谢谢")
   }
@@ -70,9 +86,11 @@ class LiteralTest extends ExpressionTestBase {
   @Test
   def testDoubleQuote(): Unit = {
     val hello = "\"<hello>\""
-    testSqlApi(
-      s"concat('a', ' ', '$hello')",
-      s"a $hello")
+    testAllApis(
+      Func3(42, hello),
+      s"Func3(42, '$hello')",
+      s"Func3(42, '$hello')",
+      s"42 and $hello")
   }
 
   @Test
@@ -80,43 +98,57 @@ class LiteralTest extends ExpressionTestBase {
 
     // these tests use Java/Scala escaping for non-quoting unicode characters
 
-    testSqlApi(
+    testAllApis(
+      ">\n<",
+      "'>\n<'",
       "'>\n<'",
       ">\n<")
 
-    testSqlApi(
+    testAllApis(
+      ">\u263A<",
       "'>\u263A<'",
-      ">\u263A<")
-
-    testSqlApi(
       "'>\u263A<'",
       ">\u263A<")
 
-    testSqlApi(
+    testAllApis(
+      ">\\<",
+      "'>\\<'",
       "'>\\<'",
       ">\\<")
 
-    testSqlApi(
+    testAllApis(
+      ">'<",
+      "'>''<'",
       "'>''<'",
       ">'<")
 
-    testSqlApi(
+    testAllApis(
+      " ",
+      "' '",
       "' '",
       " ")
 
-    testSqlApi(
+    testAllApis(
+      "",
+      "''",
       "''",
       "")
 
-    testSqlApi(
+    testAllApis(
+      ">foo([\\w]+)<",
+      "'>foo([\\w]+)<'",
       "'>foo([\\w]+)<'",
       ">foo([\\w]+)<")
 
-    testSqlApi(
+    testAllApis(
+      ">\\'\n<",
+      "\">\\'\n<\"",
       "'>\\''\n<'",
       ">\\'\n<")
 
-    testSqlApi(
+    testAllApis(
+      "It's me.",
+      "'It''s me.'",
       "'It''s me.'",
       "It's me.")
 
@@ -135,6 +167,10 @@ class LiteralTest extends ExpressionTestBase {
       ">\\\\<")
   }
 
+  override def functions: Map[String, ScalarFunction] = Map(
+    "Func3" -> Func3
+  )
+
   override def testData: Row = {
     val testData = new Row(4)
     testData.setField(0, "trUeX_value")
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala
index 3f45a58..5379e35 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MapTypeTest.scala
@@ -18,7 +18,11 @@
 
 package org.apache.flink.table.planner.expressions
 
+import org.apache.flink.table.api.DataTypes
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.utils.ApiExpressionUtils.valueLiteral
 import org.apache.flink.table.planner.expressions.utils.MapTypeTestBase
+import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, localDateTime, localTime => gLocalTime}
 
 import org.junit.Test
 
@@ -36,85 +40,121 @@ class MapTypeTest extends MapTypeTestBase {
   @Test
   def testMapLiteral(): Unit = {
     // primitive literals
-    testSqlApi(
-      "MAP[1, 1]",
-      "{1=1}")
+    testAllApis(map(1, 1), "map(1, 1)", "MAP[1, 1]", "{1=1}")
 
-    testSqlApi(
+    testAllApis(
+      map(true, true),
+      "map(true, true)",
       "map[TRUE, TRUE]",
       "{true=true}")
 
-    testSqlApi(
+    // object literals
+    testTableApi(map(BigDecimal(1), BigDecimal(1)), "map(1p, 1p)", "{1=1}")
+
+    testAllApis(
+      map(map(1, 2), map(3, 4)),
+      "map(map(1, 2), map(3, 4))",
       "MAP[MAP[1, 2], MAP[3, 4]]",
       "{{1=2}={3=4}}")
 
-    testSqlApi(
+    testAllApis(
+      map(1 + 2, 3 * 3, 3 - 6, 4 - 2),
+      "map(1 + 2, 3 * 3, 3 - 6, 4 - 2)",
       "map[1 + 2, 3 * 3, 3 - 6, 4 - 2]",
       "{3=9, -3=2}")
 
-    testSqlApi(
+    testAllApis(
+      map(1, nullOf(DataTypes.INT)),
+      "map(1, Null(INT))",
       "map[1, NULLIF(1,1)]",
       "{1=null}")
 
     // explicit conversion
-    testSqlApi(
+    testAllApis(
+      map(1, 2L , 3, 4L),
+      "map(1, 2L, 3, 4L)",
       "MAP[1, CAST(2 AS BIGINT), 3, CAST(4 AS BIGINT)]",
       "{1=2, 3=4}")
 
-    testSqlApi(
+    testAllApis(
+      map(valueLiteral(localDate("1985-04-11")), valueLiteral(gLocalTime("14:15:16")),
+        valueLiteral(localDate("2018-07-26")), valueLiteral(gLocalTime("17:18:19"))),
+      "map('1985-04-11'.toDate, '14:15:16'.toTime, '2018-07-26'.toDate, '17:18:19'.toTime)",
       "MAP[DATE '1985-04-11', TIME '14:15:16', DATE '2018-07-26', TIME '17:18:19']",
       "{1985-04-11=14:15:16, 2018-07-26=17:18:19}")
 
-    testSqlApi(
+    testAllApis(
+      map(valueLiteral(gLocalTime("14:15:16")), valueLiteral(localDateTime("1985-04-11 14:15:16")),
+        valueLiteral(gLocalTime("17:18:19")), valueLiteral(localDateTime("2018-07-26 17:18:19"))),
+      "map('14:15:16'.toTime, '1985-04-11 14:15:16'.toTimestamp, " +
+          "'17:18:19'.toTime, '2018-07-26 17:18:19'.toTimestamp)",
       "MAP[TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16', " +
-        "TIME '17:18:19', TIMESTAMP '2018-07-26 17:18:19']",
+          "TIME '17:18:19', TIMESTAMP '2018-07-26 17:18:19']",
       "{14:15:16=1985-04-11 14:15:16.000, 17:18:19=2018-07-26 17:18:19.000}")
 
-    testSqlApi(
+    testAllApis(
+      map(BigDecimal(2.0002), BigDecimal(2.0003)),
+      "map(2.0002p, 2.0003p)",
       "MAP[CAST(2.0002 AS DECIMAL(5, 4)), CAST(2.0003 AS DECIMAL(5, 4))]",
       "{2.0002=2.0003}")
 
     // implicit type cast only works on SQL API
-    testSqlApi(
-      "MAP['k1', CAST(1 AS DOUBLE), 'k2', CAST(2 AS FLOAT)]",
-      "{k1=1.0, k2=2.0}")
+    testSqlApi("MAP['k1', CAST(1 AS DOUBLE), 'k2', CAST(2 AS FLOAT)]", "{k1=1.0, k2=2.0}")
   }
 
   @Test
   def testMapField(): Unit = {
-    testSqlApi(
+    testAllApis(
+      map('f4, 'f5),
+      "map(f4, f5)",
       "MAP[f4, f5]",
       "{foo=12}")
 
-    testSqlApi(
+    testAllApis(
+      map('f4, 'f1),
+      "map(f4, f1)",
       "MAP[f4, f1]",
       "{foo={}}")
 
-    testSqlApi(
+    testAllApis(
+      map('f2, 'f3),
+      "map(f2, f3)",
       "MAP[f2, f3]",
       "{{a=12, b=13}={12=a, 13=b}}")
 
-    testSqlApi(
+    testAllApis(
+      map('f1.at("a"), 'f5),
+      "map(f1.at('a'), f5)",
       "MAP[f1['a'], f5]",
       "{null=12}")
 
-    testSqlApi(
+    testAllApis(
+      'f1,
+      "f1",
       "f1",
       "{}")
 
-    testSqlApi(
+    testAllApis(
+      'f2,
+      "f2",
       "f2",
       "{a=12, b=13}")
 
-    testSqlApi(
+    testAllApis(
+      'f2.at("a"),
+      "f2.at('a')",
       "f2['a']",
       "12")
 
-    testSqlApi(
+    testAllApis(
+      'f3.at(12),
+      "f3.at(12)",
       "f3[12]",
       "a")
 
-    testSqlApi(
+    testAllApis(
+      map('f4, 'f3).at("foo").at(13),
+      "map(f4, f3).at('foo').at(13)",
       "MAP[f4, f3]['foo'][13]",
       "b")
   }
@@ -123,72 +163,115 @@ class MapTypeTest extends MapTypeTestBase {
   def testMapOperations(): Unit = {
 
     // comparison
-    testSqlApi(
+    testAllApis(
+      'f1 === 'f2,
+      "f1 === f2",
       "f1 = f2",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f3 === 'f7,
+      "f3 === f7",
       "f3 = f7",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f5 === 'f2.at("a"),
+      "f5 === f2.at('a')",
       "f5 = f2['a']",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f8 === 'f9,
+      "f8 === f9",
       "f8 = f9",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f10 === 'f11,
+      "f10 === f11",
       "f10 = f11",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f8 !== 'f9,
+      "f8 !== f9",
       "f8 <> f9",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f10 !== 'f11,
+      "f10 !== f11",
       "f10 <> f11",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f0.at("map is null"),
+      "f0.at('map is null')",
       "f0['map is null']",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f1.at("map is empty"),
+      "f1.at('map is empty')",
       "f1['map is empty']",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f2.at("b"),
+      "f2.at('b')",
       "f2['b']",
       "13")
 
-    testSqlApi(
+    testAllApis(
+      'f3.at(1),
+      "f3.at(1)",
       "f3[1]",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f3.at(12),
+      "f3.at(12)",
       "f3[12]",
       "a")
 
-    testSqlApi(
+    testAllApis(
+      'f3.cardinality(),
+      "f3.cardinality()",
       "CARDINALITY(f3)",
       "2")
 
-    testSqlApi(
+    testAllApis(
+      'f2.at("a").isNotNull,
+      "f2.at('a').isNotNull",
       "f2['a'] IS NOT NULL",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f2.at("a").isNull,
+      "f2.at('a').isNull",
       "f2['a'] IS NULL",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f2.at("c").isNotNull,
+      "f2.at('c').isNotNull",
       "f2['c'] IS NOT NULL",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f2.at("c").isNull,
+      "f2.at('c').isNull",
       "f2['c'] IS NULL",
       "true")
   }
+
+  @Test
+  def testMapTypeCasting(): Unit = {
+    testTableApi(
+      'f2.cast(DataTypes.MAP(DataTypes.STRING, DataTypes.INT)),
+      "f2.cast(MAP(STRING, INT))",
+      "{a=12, b=13}"
+    )
+  }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MathFunctionsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MathFunctionsTest.scala
deleted file mode 100644
index 01dcf63..0000000
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/MathFunctionsTest.scala
+++ /dev/null
@@ -1,694 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.flink.table.planner.expressions
-
-import org.apache.flink.table.planner.expressions.utils.ScalarTypesTestBase
-
-import org.junit.Test
-
-class MathFunctionsTest extends ScalarTypesTestBase {
-
-  // ----------------------------------------------------------------------------------------------
-  // Math functions
-  // ----------------------------------------------------------------------------------------------
-
-  @Test
-  def testMod(): Unit = {
-    testSqlApi(
-      "MOD(f4, f7)",
-      "2")
-
-    testSqlApi(
-      "MOD(f4, 3)",
-      "2")
-
-    testSqlApi(
-      "MOD(44, 3)",
-      "2")
-  }
-
-  @Test
-  def testExp(): Unit = {
-    testSqlApi(
-      "EXP(f2)",
-      math.exp(42.toByte).toString)
-
-    testSqlApi(
-      "EXP(f3)",
-      math.exp(43.toShort).toString)
-
-    testSqlApi(
-      "EXP(f4)",
-      math.exp(44.toLong).toString)
-
-    testSqlApi(
-      "EXP(f5)",
-      math.exp(4.5.toFloat).toString)
-
-    testSqlApi(
-      "EXP(f6)",
-      math.exp(4.6).toString)
-
-    testSqlApi(
-      "EXP(f7)",
-      math.exp(3).toString)
-
-    testSqlApi(
-      "EXP(3)",
-      math.exp(3).toString)
-  }
-
-  @Test
-  def testLog10(): Unit = {
-    testSqlApi(
-      "LOG10(f2)",
-      math.log10(42.toByte).toString)
-
-    testSqlApi(
-      "LOG10(f3)",
-      math.log10(43.toShort).toString)
-
-    testSqlApi(
-      "LOG10(f4)",
-      math.log10(44.toLong).toString)
-
-    testSqlApi(
-      "LOG10(f5)",
-      math.log10(4.5.toFloat).toString)
-
-    testSqlApi(
-      "LOG10(f6)",
-      math.log10(4.6).toString)
-
-    testSqlApi(
-      "LOG10(f32)",
-      math.log10(-1).toString)
-
-    testSqlApi(
-      "LOG10(f27)",
-      math.log10(0).toString)
-  }
-
-  @Test
-  def testPower(): Unit = {
-    // f7: int , f4: long, f6: double
-    testSqlApi(
-      "POWER(f2, f7)",
-      math.pow(42.toByte, 3).toString)
-
-    testSqlApi(
-      "POWER(f3, f6)",
-      math.pow(43.toShort, 4.6D).toString)
-
-    testSqlApi(
-      "POWER(f4, f5)",
-      math.pow(44.toLong, 4.5.toFloat).toString)
-
-    testSqlApi(
-      "POWER(f4, f5)",
-      math.pow(44.toLong, 4.5.toFloat).toString)
-
-    // f5: float
-    testSqlApi(
-      "power(f5, f5)",
-      math.pow(4.5F, 4.5F).toString)
-
-    testSqlApi(
-      "power(f5, f6)",
-      math.pow(4.5F, 4.6D).toString)
-
-    testSqlApi(
-      "power(f5, f7)",
-      math.pow(4.5F, 3).toString)
-
-    testSqlApi(
-      "power(f5, f4)",
-      math.pow(4.5F, 44L).toString)
-
-    // f22: bigDecimal
-    // TODO delete casting in SQL when CALCITE-1467 is fixed
-    testSqlApi(
-      "power(CAST(f22 AS DOUBLE), f5)",
-      math.pow(2, 4.5F).toString)
-
-    testSqlApi(
-      "power(CAST(f22 AS DOUBLE), f6)",
-      math.pow(2, 4.6D).toString)
-
-    testSqlApi(
-      "power(CAST(f22 AS DOUBLE), f7)",
-      math.pow(2, 3).toString)
-
-    testSqlApi(
-      "power(CAST(f22 AS DOUBLE), f4)",
-      math.pow(2, 44L).toString)
-
-    testSqlApi(
-      "power(f6, f22)",
-      math.pow(4.6D, 2).toString)
-  }
-
-  @Test
-  def testSqrt(): Unit = {
-    testSqlApi(
-      "SQRT(f6)",
-      math.sqrt(4.6D).toString)
-
-    testSqlApi(
-      "SQRT(f7)",
-      math.sqrt(3).toString)
-
-    testSqlApi(
-      "SQRT(f4)",
-      math.sqrt(44L).toString)
-
-    testSqlApi(
-      "SQRT(CAST(f22 AS DOUBLE))",
-      math.sqrt(2.0).toString)
-
-    testSqlApi(
-      "SQRT(f5)",
-      math.pow(4.5F, 0.5).toString)
-
-    testSqlApi(
-      "SQRT(25)",
-      "5.0")
-
-    testSqlApi(
-      "POWER(CAST(2.2 AS DOUBLE), CAST(0.5 AS DOUBLE))", // TODO fix FLINK-4621
-      math.sqrt(2.2).toString)
-  }
-
-  @Test
-  def testLn(): Unit = {
-    testSqlApi(
-      "LN(f2)",
-      math.log(42.toByte).toString)
-
-    testSqlApi(
-      "LN(f3)",
-      math.log(43.toShort).toString)
-
-    testSqlApi(
-      "LN(f4)",
-      math.log(44.toLong).toString)
-
-    testSqlApi(
-      "LN(f5)",
-      math.log(4.5.toFloat).toString)
-
-    testSqlApi(
-      "LN(f6)",
-      math.log(4.6).toString)
-
-    testSqlApi(
-      "LN(f32)",
-      math.log(-1).toString)
-
-    testSqlApi(
-      "LN(f27)",
-      math.log(0).toString)
-  }
-
-  @Test
-  def testAbs(): Unit = {
-    testSqlApi(
-      "ABS(f2)",
-      "42")
-
-    testSqlApi(
-      "ABS(f3)",
-      "43")
-
-    testSqlApi(
-      "ABS(f4)",
-      "44")
-
-    testSqlApi(
-      "ABS(f5)",
-      "4.5")
-
-    testSqlApi(
-      "ABS(f6)",
-      "4.6")
-
-    testSqlApi(
-      "ABS(f9)",
-      "42")
-
-    testSqlApi(
-      "ABS(f10)",
-      "43")
-
-    testSqlApi(
-      "ABS(f11)",
-      "44")
-
-    testSqlApi(
-      "ABS(f12)",
-      "4.5")
-
-    testSqlApi(
-      "ABS(f13)",
-      "4.6")
-
-    testSqlApi(
-      "ABS(f15)",
-      "1231.1231231321321321111")
-  }
-
-  @Test
-  def testArithmeticFloorCeil(): Unit = {
-    testSqlApi(
-      "FLOOR(f5)",
-      "4.0")
-
-    testSqlApi(
-      "CEIL(f5)",
-      "5.0")
-
-    testSqlApi(
-      "FLOOR(f3)",
-      "43")
-
-    testSqlApi(
-      "CEIL(f3)",
-      "43")
-
-    testSqlApi(
-      "FLOOR(f15)",
-      "-1232")
-
-    testSqlApi(
-      "CEIL(f15)",
-      "-1231")
-  }
-
-  @Test
-  def testSin(): Unit = {
-    testSqlApi(
-      "SIN(f2)",
-      math.sin(42.toByte).toString)
-
-    testSqlApi(
-      "SIN(f3)",
-      math.sin(43.toShort).toString)
-
-    testSqlApi(
-      "SIN(f4)",
-      math.sin(44.toLong).toString)
-
-    testSqlApi(
-      "SIN(f5)",
-      math.sin(4.5.toFloat).toString)
-
-    testSqlApi(
-      "SIN(f6)",
-      math.sin(4.6).toString)
-
-    testSqlApi(
-      "SIN(f15)",
-      math.sin(-1231.1231231321321321111).toString)
-  }
-
-  @Test
-  def testCos(): Unit = {
-    testSqlApi(
-      "COS(f2)",
-      math.cos(42.toByte).toString)
-
-    testSqlApi(
-      "COS(f3)",
-      math.cos(43.toShort).toString)
-
-    testSqlApi(
-      "COS(f4)",
-      math.cos(44.toLong).toString)
-
-    testSqlApi(
-      "COS(f5)",
-      math.cos(4.5.toFloat).toString)
-
-    testSqlApi(
-      "COS(f6)",
-      math.cos(4.6).toString)
-
-    testSqlApi(
-      "COS(f15)",
-      math.cos(-1231.1231231321321321111).toString)
-  }
-
-  @Test
-  def testTan(): Unit = {
-    testSqlApi(
-      "TAN(f2)",
-      math.tan(42.toByte).toString)
-
-    testSqlApi(
-      "TAN(f3)",
-      math.tan(43.toShort).toString)
-
-    testSqlApi(
-      "TAN(f4)",
-      math.tan(44.toLong).toString)
-
-    testSqlApi(
-      "TAN(f5)",
-      math.tan(4.5.toFloat).toString)
-
-    testSqlApi(
-      "TAN(f6)",
-      math.tan(4.6).toString)
-
-    testSqlApi(
-      "TAN(f15)",
-      math.tan(-1231.1231231321321321111).toString)
-  }
-
-  @Test
-  def testCot(): Unit = {
-    testSqlApi(
-      "COT(f2)",
-      (1.0d / math.tan(42.toByte)).toString)
-
-    testSqlApi(
-      "COT(f3)",
-      (1.0d / math.tan(43.toShort)).toString)
-
-    testSqlApi(
-      "COT(f4)",
-      (1.0d / math.tan(44.toLong)).toString)
-
-    testSqlApi(
-      "COT(f5)",
-      (1.0d / math.tan(4.5.toFloat)).toString)
-
-    testSqlApi(
-      "COT(f6)",
-      (1.0d / math.tan(4.6)).toString)
-
-    testSqlApi(
-      "COT(f15)",
-      (1.0d / math.tan(-1231.1231231321321321111)).toString)
-  }
-
-  @Test
-  def testAsin(): Unit = {
-    testSqlApi(
-      "ASIN(f25)",
-      math.asin(0.42.toByte).toString)
-
-    testSqlApi(
-      "ASIN(f26)",
-      math.asin(0.toShort).toString)
-
-    testSqlApi(
-      "ASIN(f27)",
-      math.asin(0.toLong).toString)
-
-    testSqlApi(
-      "ASIN(f28)",
-      math.asin(0.45.toFloat).toString)
-
-    testSqlApi(
-      "ASIN(f29)",
-      math.asin(0.46).toString)
-
-    testSqlApi(
-      "ASIN(f30)",
-      math.asin(1).toString)
-
-    testSqlApi(
-      "ASIN(f31)",
-      math.asin(-0.1231231321321321111).toString)
-  }
-
-  @Test
-  def testAcos(): Unit = {
-    testSqlApi(
-      "ACOS(f25)",
-      math.acos(0.42.toByte).toString)
-
-    testSqlApi(
-      "ACOS(f26)",
-      math.acos(0.toShort).toString)
-
-    testSqlApi(
-      "ACOS(f27)",
-      math.acos(0.toLong).toString)
-
-    testSqlApi(
-      "ACOS(f28)",
-      math.acos(0.45.toFloat).toString)
-
-    testSqlApi(
-      "ACOS(f29)",
-      math.acos(0.46).toString)
-
-    testSqlApi(
-      "ACOS(f30)",
-      math.acos(1).toString)
-
-    testSqlApi(
-      "ACOS(f31)",
-      math.acos(-0.1231231321321321111).toString)
-  }
-
-  @Test
-  def testAtan(): Unit = {
-    testSqlApi(
-      "ATAN(f25)",
-      math.atan(0.42.toByte).toString)
-
-    testSqlApi(
-      "ATAN(f26)",
-      math.atan(0.toShort).toString)
-
-    testSqlApi(
-      "ATAN(f27)",
-      math.atan(0.toLong).toString)
-
-    testSqlApi(
-      "ATAN(f28)",
-      math.atan(0.45.toFloat).toString)
-
-    testSqlApi(
-      "ATAN(f29)",
-      math.atan(0.46).toString)
-
-    testSqlApi(
-      "ATAN(f30)",
-      math.atan(1).toString)
-
-    testSqlApi(
-      "ATAN(f31)",
-      math.atan(-0.1231231321321321111).toString)
-  }
-
-  @Test
-  def testDegrees(): Unit = {
-    testSqlApi(
-      "DEGREES(f2)",
-      math.toDegrees(42.toByte).toString)
-
-    testSqlApi(
-      "DEGREES(f3)",
-      math.toDegrees(43.toShort).toString)
-
-    testSqlApi(
-      "DEGREES(f4)",
-      math.toDegrees(44.toLong).toString)
-
-    testSqlApi(
-      "DEGREES(f5)",
-      math.toDegrees(4.5.toFloat).toString)
-
-    testSqlApi(
-      "DEGREES(f6)",
-      math.toDegrees(4.6).toString)
-
-    testSqlApi(
-      "DEGREES(f15)",
-      math.toDegrees(-1231.1231231321321321111).toString)
-  }
-
-  @Test
-  def testRadians(): Unit = {
-    testSqlApi(
-      "RADIANS(f2)",
-      math.toRadians(42.toByte).toString)
-
-    testSqlApi(
-      "RADIANS(f3)",
-      math.toRadians(43.toShort).toString)
-
-    testSqlApi(
-      "RADIANS(f4)",
-      math.toRadians(44.toLong).toString)
-
-    testSqlApi(
-      "RADIANS(f5)",
-      math.toRadians(4.5.toFloat).toString)
-
-    testSqlApi(
-      "RADIANS(f6)",
-      math.toRadians(4.6).toString)
-
-    testSqlApi(
-      "RADIANS(f15)",
-      math.toRadians(-1231.1231231321321321111).toString)
-  }
-
-  @Test
-  def testSign(): Unit = {
-    testSqlApi(
-      "SIGN(f4)",
-      1.toString)
-
-    testSqlApi(
-      "SIGN(f6)",
-      1.0.toString)
-
-    testSqlApi(
-      "SIGN(f15)",
-      "-1.0000000000000000000") // calcite: SIGN(Decimal(p,s)) => Decimal(p,s)
-  }
-
-  @Test
-  def testRound(): Unit = {
-    testSqlApi(
-      "ROUND(f29, f30)",
-      0.5.toString)
-
-    testSqlApi(
-      "ROUND(f31, f7)",
-      "-0.123")
-
-    testSqlApi(
-      "ROUND(f4, f32)",
-      40.toString)
-  }
-
-  @Test
-  def testPi(): Unit = {
-    testSqlApi(
-      "pi()",
-      math.Pi.toString)
-  }
-
-  @Test
-  def testRandAndRandInteger(): Unit = {
-    val random1 = new java.util.Random(1)
-    testSqlApi(
-      "RAND(1)",
-      random1.nextDouble().toString)
-
-    val random2 = new java.util.Random(3)
-    testSqlApi(
-      "RAND(f7)",
-      random2.nextDouble().toString)
-
-    val random3 = new java.util.Random(1)
-    testSqlApi(
-      "RAND_INTEGER(1, 10)",
-      random3.nextInt(10).toString)
-
-    val random4 = new java.util.Random(3)
-    testSqlApi(
-      "RAND_INTEGER(f7, CAST(f4 AS INT))",
-      random4.nextInt(44).toString)
-  }
-
-  @Test
-  def testE(): Unit = {
-    testSqlApi(
-      "E()",
-      math.E.toString)
-
-    testSqlApi(
-      "e()",
-      math.E.toString)
-  }
-
-  @Test
-  def testLog(): Unit = {
-    testSqlApi(
-      "LOG(f6)",
-      "1.5260563034950492"
-    )
-
-    testSqlApi(
-      "LOG(f6-f6 + 10, f6-f6+100)",
-      "2.0"
-    )
-
-    testSqlApi(
-      "LOG(f6+20)",
-      "3.202746442938317"
-    )
-
-    testSqlApi(
-      "LOG(10)",
-      "2.302585092994046"
-    )
-
-    testSqlApi(
-      "LOG(10, 100)",
-      "2.0"
-    )
-
-    testSqlApi(
-      "log(f32, f32)",
-      (math.log(-1)/math.log(-1)).toString)
-
-    testSqlApi(
-      "log(f27, f32)",
-      (math.log(0)/math.log(0)).toString)
-  }
-
-  @Test
-  def testLog2(): Unit = {
-    testSqlApi(
-      "log2(f2)",
-      (math.log(42.toByte)/math.log(2.toByte)).toString)
-
-    testSqlApi(
-      "log2(f3)",
-      (math.log(43.toShort)/math.log(2.toShort)).toString)
-
-    testSqlApi(
-      "log2(f4)",
-      (math.log(44.toLong)/math.log(2.toLong)).toString)
-
-    testSqlApi(
-      "log2(f5)",
-      (math.log(4.5.toFloat)/math.log(2.toFloat)).toString)
-
-    testSqlApi(
-      "log2(f6)",
-      (math.log(4.6)/math.log(2)).toString)
-
-    testSqlApi(
-      "log2(f32)",
-      (math.log(-1)/math.log(2)).toString)
-
-    testSqlApi(
-      "log2(f27)",
-      (math.log(0)/math.log(2)).toString)
-  }
-}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/NonDeterministicTests.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/NonDeterministicTests.scala
index 98c64ad..2565383 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/NonDeterministicTests.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/NonDeterministicTests.scala
@@ -19,6 +19,7 @@
 package org.apache.flink.table.planner.expressions
 
 import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase
 import org.apache.flink.types.Row
 
@@ -27,47 +28,64 @@ import org.junit.{Ignore, Test}
 /**
   * Tests that can only be checked manually as they are non-deterministic.
   */
-@Ignore
 class NonDeterministicTests extends ExpressionTestBase {
 
+  @Ignore
   @Test
   def testCurrentDate(): Unit = {
-    testSqlApi(
+    testAllApis(
+      currentDate(),
+      "currentDate()",
       "CURRENT_DATE",
       "PLEASE CHECK MANUALLY")
   }
 
+  @Ignore
   @Test
   def testCurrentTime(): Unit = {
-    testSqlApi(
+    testAllApis(
+      currentTime(),
+      "currentTime()",
       "CURRENT_TIME",
       "PLEASE CHECK MANUALLY")
   }
 
+  @Ignore
   @Test
   def testCurrentTimestamp(): Unit = {
-    testSqlApi(
+    testAllApis(
+      currentTimestamp(),
+      "currentTimestamp()",
       "CURRENT_TIMESTAMP",
       "PLEASE CHECK MANUALLY")
   }
 
+  @Ignore
   @Test
   def testLocalTimestamp(): Unit = {
-    testSqlApi(
+    testAllApis(
+      localTimestamp(),
+      "localTimestamp()",
       "LOCALTIMESTAMP",
       "PLEASE CHECK MANUALLY")
   }
 
+  @Ignore
   @Test
   def testLocalTime(): Unit = {
-    testSqlApi(
+    testAllApis(
+      localTime(),
+      "localTime()",
       "LOCALTIME",
       "PLEASE CHECK MANUALLY")
   }
 
+  @Ignore
   @Test
   def testUUID(): Unit = {
-    testSqlApi(
+    testAllApis(
+      uuid(),
+      "uuid()",
       "UUID()",
       "PLEASE CHECK MANUALLY")
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala
index 2c6f924..ff7a2a2 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/RowTypeTest.scala
@@ -18,7 +18,10 @@
 
 package org.apache.flink.table.planner.expressions
 
+import org.apache.flink.table.api.DataTypes
+import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.expressions.utils.RowTypeTestBase
+import org.apache.flink.table.planner.utils.DateTimeTestUtil.{localDate, localDateTime, localTime => gLocalTime}
 
 import org.junit.Test
 
@@ -28,17 +31,36 @@ class RowTypeTest extends RowTypeTestBase {
   def testRowLiteral(): Unit = {
 
     // primitive literal
-    testSqlApi(
+    testAllApis(
+      row(1, "foo", true),
+      "row(1, 'foo', true)",
       "ROW(1, 'foo', true)",
       "(1,foo,true)")
 
     // special literal
-    testSqlApi(
+    testAllApis(
+      row(
+        localDate("1985-04-11"),
+        gLocalTime("14:15:16"),
+        localDateTime("1985-04-11 14:15:16"),
+        BigDecimal("0.1").bigDecimal,
+        array(1, 2, 3),
+        map("foo", "bar"),
+        row(1, true)),
+      "row('1985-04-11'.toDate, '14:15:16'.toTime, '1985-04-11 14:15:16'.toTimestamp, " +
+          "0.1p, Array(1, 2, 3), Map('foo', 'bar'), row(1, true))",
       "ROW(DATE '1985-04-11', TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16', " +
-        "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))",
+          "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))",
       "(1985-04-11,14:15:16,1985-04-11 14:15:16.000,0.1,[1, 2, 3],{foo=bar},(1,true))")
 
     testSqlApi(
+      "ROW(DATE '1985-04-11', TIME '14:15:16', TIMESTAMP '1985-04-11 14:15:16', " +
+          "CAST(0.1 AS DECIMAL(2, 1)), ARRAY[1, 2, 3], MAP['foo', 'bar'], row(1, true))",
+      "(1985-04-11,14:15:16,1985-04-11 14:15:16.000,0.1,[1, 2, 3],{foo=bar},(1,true))")
+
+    testAllApis(
+      row(1 + 1, 2 * 3, nullOf(DataTypes.STRING())),
+      "row(1 + 1, 2 * 3, Null(STRING))",
       "ROW(1 + 1, 2 * 3, NULLIF(1, 1))",
       "(2,6,null)"
     )
@@ -48,27 +70,37 @@ class RowTypeTest extends RowTypeTestBase {
 
   @Test
   def testRowField(): Unit = {
-    testSqlApi(
+    testAllApis(
+      row('f0, 'f1),
+      "row(f0, f1)",
       "(f0, f1)",
       "(null,1)"
     )
 
-    testSqlApi(
+    testAllApis(
+      'f2,
+      "f2",
       "f2",
       "(2,foo,true)"
     )
 
-    testSqlApi(
+    testAllApis(
+      row('f2, 'f5),
+      "row(f2, f5)",
       "(f2, f5)",
       "((2,foo,true),(foo,null))"
     )
 
-    testSqlApi(
+    testAllApis(
+      'f4,
+      "f4",
       "f4",
       "(1984-03-12,0.00000000,[1, 2, 3])"
     )
 
-    testSqlApi(
+    testAllApis(
+      row('f1, "foo", true),
+      "row(f1, 'foo', true)",
       "(f1, 'foo',true)",
       "(1,foo,true)"
     )
@@ -76,14 +108,25 @@ class RowTypeTest extends RowTypeTestBase {
 
   @Test
   def testRowOperations(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f5.get("f0"),
+      "f5.get('f0')",
       "f5.f0",
       "foo"
     )
 
-    testSqlApi(
+    testAllApis(
+      'f3.get("f1").get("f2"),
+      "f3.get('f1').get('f2')",
       "f3.f1.f2",
       "true"
     )
+
+    // SQL API for row value constructor follow by field access is not supported
+    testTableApi(
+      row('f1, 'f6, 'f2).get("f1").get("f1"),
+      "row(f1, f6, f2).get('f1').get('f1')",
+      "null"
+    )
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
index 5c3b0f9..244812a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala
@@ -18,7 +18,9 @@
 
 package org.apache.flink.table.planner.expressions
 
-import org.apache.flink.table.expressions.TimePointUnit
+import org.apache.flink.table.api.scala.{currentDate, currentTime, currentTimestamp, localTime, localTimestamp, nullOf, temporalOverlaps, _}
+import org.apache.flink.table.api.{DataTypes, Types}
+import org.apache.flink.table.expressions.{Expression, ExpressionParser, TimeIntervalUnit, TimePointUnit}
 import org.apache.flink.table.planner.expressions.utils.ScalarTypesTestBase
 
 import org.junit.Test
@@ -110,8 +112,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
     )
 
     cases.foreach(x => {
-      // TODO: ignore Table API currently
-      testSqlApi(
+      testAllApis(
+        ExpressionParser.parseExpression(x._1),
+        x._1,
         x._2,
         x._3
       )
@@ -121,17 +124,23 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testPosition(): Unit = {
-    testSqlApi(
+    testAllApis(
+      "test".position("xxxtest"),
+      "'test'.position('xxxtest')",
       "POSITION('test' IN 'xxxtest')",
       "4")
 
-    testSqlApi(
+    testAllApis(
+      "testx".position("xxxtest"),
+      "'testx'.position('xxxtest')",
       "POSITION('testx' IN 'xxxtest')",
       "0")
 
     testSqlApi(
       "POSITION('aa' IN 'aaads')",
       "1")
+
+    testSqlApi("position('aa' in 'aaads')", "1")
   }
 
   @Test
@@ -289,19 +298,27 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testSubstring(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0.substring(2),
+      "f0.substring(2)",
       "SUBSTRING(f0, 2)",
       "his is a test String.")
 
-    testSqlApi(
+    testAllApis(
+      'f0.substring(2, 5),
+      "f0.substring(2, 5)",
       "SUBSTRING(f0, 2, 5)",
       "his i")
 
-    testSqlApi(
+    testAllApis(
+      'f0.substring(1, 'f7),
+      "f0.substring(1, f7)",
       "SUBSTRING(f0, 1, f7)",
       "Thi")
 
-    testSqlApi(
+    testAllApis(
+      'f0.substring(1.cast(DataTypes.TINYINT), 'f7),
+      "f0.substring(1.cast(BYTE), f7)",
       "SUBSTRING(f0, CAST(1 AS TINYINT), f7)",
       "Thi")
 
@@ -327,20 +344,61 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
   }
 
   @Test
+  def testReplace(): Unit = {
+    testAllApis(
+      'f0.replace(" ", "_"),
+      "f0.replace(' ', '_')",
+      "REPLACE(f0, ' ', '_')",
+      "This_is_a_test_String.")
+
+    testAllApis(
+      'f0.replace("i", ""),
+      "f0.replace('i', '')",
+      "REPLACE(f0, 'i', '')",
+      "Ths s a test Strng.")
+
+    testAllApis(
+      'f33.replace("i", ""),
+      "f33.replace('i', '')",
+      "REPLACE(f33, 'i', '')",
+      "null")
+
+    testAllApis(
+      'f0.replace(nullOf(DataTypes.STRING), ""),
+      "f0.replace(Null(STRING), '')",
+      "REPLACE(f0, NULLIF('', ''), '')",
+      "null")
+
+    testAllApis(
+      'f0.replace(" ", nullOf(DataTypes.STRING)),
+      "f0.replace(' ', Null(STRING))",
+      "REPLACE(f0, ' ', NULLIF('', ''))",
+      "null")
+  }
+
+  @Test
   def testTrim(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f8.trim(),
+      "f8.trim()",
       "TRIM(f8)",
       "This is a test String.")
 
-    testSqlApi(
+    testAllApis(
+      'f8.trim(removeLeading = true, removeTrailing = true, " "),
+      "trim(f8)",
       "TRIM(f8)",
       "This is a test String.")
 
-    testSqlApi(
+    testAllApis(
+      'f8.trim(removeLeading = false, removeTrailing = true, " "),
+      "f8.trim(TRAILING, ' ')",
       "TRIM(TRAILING FROM f8)",
       " This is a test String.")
 
-    testSqlApi(
+    testAllApis(
+      'f0.trim(removeLeading = true, removeTrailing = true, "."),
+      "trim(BOTH, '.', f0)",
       "TRIM(BOTH '.' FROM f0)",
       "This is a test String")
 
@@ -373,7 +431,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testLTrim(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f8.ltrim(),
+      "f8.ltrim()",
       "LTRIM(f8)",
       "This is a test String. ")
 
@@ -408,7 +468,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testRTrim(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f8.rtrim(),
+      "f8.rtrim()",
       "rtrim(f8)",
       " This is a test String.")
 
@@ -443,11 +505,15 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testCharLength(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0.charLength(),
+      "f0.charLength()",
       "CHAR_LENGTH(f0)",
       "22")
 
-    testSqlApi(
+    testAllApis(
+      'f0.charLength(),
+      "charLength(f0)",
       "CHARACTER_LENGTH(f0)",
       "22")
   }
@@ -469,21 +535,27 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testUpperCase(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0.upperCase(),
+      "f0.upperCase()",
       "UPPER(f0)",
       "THIS IS A TEST STRING.")
   }
 
   @Test
   def testLowerCase(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0.lowerCase(),
+      "f0.lowerCase()",
       "LOWER(f0)",
       "this is a test string.")
   }
 
   @Test
   def testInitCap(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0.initCap(),
+      "f0.initCap()",
       "INITCAP(f0)",
       "This Is A Test String.")
 
@@ -494,18 +566,24 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testConcat(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0 + 'f0,
+      "f0 + f0",
       "f0||f0",
       "This is a test String.This is a test String.")
   }
 
   @Test
   def testLike(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0.like("Th_s%"),
+      "f0.like('Th_s%')",
       "f0 LIKE 'Th_s%'",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f0.like("%is a%"),
+      "f0.like('%is a%')",
       "f0 LIKE '%is a%'",
       "true")
 
@@ -529,11 +607,15 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testNotLike(): Unit = {
-    testSqlApi(
+    testAllApis(
+      !'f0.like("Th_s%"),
+      "!f0.like('Th_s%')",
       "f0 NOT LIKE 'Th_s%'",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      !'f0.like("%is a%"),
+      "!f0.like('%is a%')",
       "f0 NOT LIKE '%is a%'",
       "false")
   }
@@ -578,22 +660,30 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testSimilar(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0.similar("_*"),
+      "f0.similar('_*')",
       "f0 SIMILAR TO '_*'",
       "true")
 
-    testSqlApi(
+    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 = {
-    testSqlApi(
+    testAllApis(
+      !'f0.similar("_*"),
+      "!f0.similar('_*')",
       "f0 NOT SIMILAR TO '_*'",
       "false")
 
-    testSqlApi(
+    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")
   }
@@ -638,13 +728,19 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testMultiConcat(): Unit = {
-    testSqlApi(
+    testAllApis(
+      concat("xx", 'f33),
+      "concat('xx', f33)",
       "CONCAT('xx', f33)",
       "xx")
-    testSqlApi(
+    testAllApis(
+      concat("AA", "BB", "CC", "---"),
+      "concat('AA','BB','CC','---')",
       "CONCAT('AA','BB','CC','---')",
       "AABBCC---")
-    testSqlApi(
+    testAllApis(
+      concat("x~x", "b~b", "c~~~~c", "---"),
+      "concat('x~x','b~b','c~~~~c','---')",
       "CONCAT('x~x','b~b','c~~~~c','---')",
       "x~xb~bc~~~~c---")
 
@@ -655,124 +751,156 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testConcatWs(): Unit = {
-    testSqlApi(
+    testAllApis(
+      concat_ws('f33, "AA"),
+      "concat_ws(f33, 'AA')",
       "CONCAT_WS(f33, 'AA')",
       "AA")
-    testSqlApi(
+    testAllApis(
+      concat_ws("~~~~", "AA"),
+      "concat_ws('~~~~','AA')",
       "concat_ws('~~~~','AA')",
       "AA")
-    testSqlApi(
+    testAllApis(
+      concat_ws("~", "AA", "BB"),
+      "concat_ws('~','AA','BB')",
       "concat_ws('~','AA','BB')",
       "AA~BB")
-    testSqlApi(
+    testAllApis(
+      concat_ws("~", 'f33, "AA", "BB", "", 'f33, "CC"),
+      "concat_ws('~',f33, 'AA','BB','',f33, 'CC')",
       "concat_ws('~',f33, 'AA','BB','',f33, 'CC')",
       "AA~BB~~CC")
-    testSqlApi(
+    testAllApis(
+      concat_ws("~~~~", "Flink", 'f33, "xx", 'f33, 'f33),
+      "concat_ws('~~~~','Flink', f33, 'xx', f33, f33)",
       "CONCAT_WS('~~~~','Flink', f33, 'xx', f33, f33)",
       "Flink~~~~xx")
 
     testSqlApi("concat_ws('||', f35, f36, f33)", "a||b")
   }
 
+  @Test
   def testRegexpReplace(): Unit = {
 
-    testSqlApi(
+    testAllApis(
+      "foobar".regexpReplace("oo|ar", "abc"),
+      "'foobar'.regexpReplace('oo|ar', 'abc')",
       "regexp_replace('foobar', 'oo|ar', 'abc')",
       "fabcbabc")
 
-    testSqlApi(
+    testAllApis(
+      "foofar".regexpReplace("^f", ""),
+      "'foofar'.regexpReplace('^f', '')",
       "regexp_replace('foofar', '^f', '')",
       "oofar")
 
-    testSqlApi(
+    testAllApis(
+      "foobar".regexpReplace("^f*.*r$", ""),
+      "'foobar'.regexpReplace('^f*.*r$', '')",
       "regexp_replace('foobar', '^f*.*r$', '')",
       "")
 
-    testSqlApi(
+    testAllApis(
+      "foo1bar2".regexpReplace("\\d", ""),
+      "'foo1bar2'.regexpReplace('\\d', '')",
       "regexp_replace('foobar', '\\d', '')",
       "foobar")
 
-    testSqlApi(
+    testAllApis(
+      "foobar".regexpReplace("\\w", ""),
+      "'foobar'.regexpReplace('\\w', '')",
       "regexp_replace('foobar', '\\w', '')",
       "")
 
-    testSqlApi(
+    testAllApis(
+      "fooobar".regexpReplace("oo", "$"),
+      "'fooobar'.regexpReplace('oo', '$')",
       "regexp_replace('fooobar', 'oo', '$')",
       "f$obar")
 
-    testSqlApi(
+    testAllApis(
+      "foobar".regexpReplace("oo", "\\"),
+      "'foobar'.regexpReplace('oo', '\\')",
       "regexp_replace('foobar', 'oo', '\\')",
       "f\\bar")
 
-    testSqlApi(
+    testAllApis(
+      'f33.regexpReplace("oo|ar", ""),
+      "f33.regexpReplace('oo|ar', '')",
       "REGEXP_REPLACE(f33, 'oo|ar', '')",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      "foobar".regexpReplace('f33, ""),
+      "'foobar'.regexpReplace(f33, '')",
       "REGEXP_REPLACE('foobar', f33, '')",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      "foobar".regexpReplace("oo|ar", 'f33),
+      "'foobar'.regexpReplace('oo|ar', f33)",
       "REGEXP_REPLACE('foobar', 'oo|ar', f33)",
       "null")
 
     // This test was added for the null literal problem in string expression parsing (FLINK-10463).
-    testSqlApi(
+    testAllApis(
+      nullOf(Types.STRING).regexpReplace("oo|ar", 'f33),
+      "nullOf(STRING).regexpReplace('oo|ar', f33)",
       "REGEXP_REPLACE(CAST(NULL AS VARCHAR), 'oo|ar', f33)",
       "null")
-
-    testSqlApi("regexp_replace('100-200', '(\\d+)', 'num')", "num-num")
-    testSqlApi("regexp_replace('100-200', '(\\d+)-(\\d+)', '400')", "400")
-    testSqlApi("regexp_replace('100-200', '(\\d+)', '400')", "400-400")
-    testSqlApi("regexp_replace('100-200', '', '400')", "100-200")
-    testSqlApi("regexp_replace(f40, '(\\d+)', '400')", "null")
-    testSqlApi("regexp_replace(CAST(null as VARCHAR), '(\\d+)', 'num')", "null")
-    testSqlApi("regexp_replace('100-200', CAST(null as VARCHAR), '400')", "null")
-    testSqlApi("regexp_replace('100-200', '(\\d+)', CAST(null as VARCHAR))", "null")
   }
 
   @Test
   def testRegexpExtract(): Unit = {
-    testSqlApi(
+    testAllApis(
+      "foothebar".regexpExtract("foo(.*?)(bar)", 2),
+      "'foothebar'.regexpExtract('foo(.*?)(bar)', 2)",
       "REGEXP_EXTRACT('foothebar', 'foo(.*?)(bar)', 2)",
       "bar")
 
-    testSqlApi(
+    testAllApis(
+      "foothebar".regexpExtract("foo(.*?)(bar)", 0),
+      "'foothebar'.regexpExtract('foo(.*?)(bar)', 0)",
       "REGEXP_EXTRACT('foothebar', 'foo(.*?)(bar)', 0)",
       "foothebar")
 
-    testSqlApi(
+    testAllApis(
+      "foothebar".regexpExtract("foo(.*?)(bar)", 1),
+      "'foothebar'.regexpExtract('foo(.*?)(bar)', 1)",
       "REGEXP_EXTRACT('foothebar', 'foo(.*?)(bar)', 1)",
       "the")
 
-    testSqlApi(
+    testAllApis(
+      "foothebar".regexpExtract("foo([\\w]+)", 1),
+      "'foothebar'.regexpExtract('foo([\\w]+)', 1)",
       "REGEXP_EXTRACT('foothebar', 'foo([\\w]+)', 1)",
       "thebar")
 
-    testSqlApi(
+    testAllApis(
+      "foothebar".regexpExtract("foo([\\d]+)", 1),
+      "'foothebar'.regexpExtract('foo([\\d]+)', 1)",
       "REGEXP_EXTRACT('foothebar', 'foo([\\d]+)', 1)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f33.regexpExtract("foo(.*?)(bar)", 2),
+      "f33.regexpExtract('foo(.*?)(bar)', 2)",
       "REGEXP_EXTRACT(f33, 'foo(.*?)(bar)', 2)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      "foothebar".regexpExtract('f33, 2),
+      "'foothebar'.regexpExtract(f33, 2)",
       "REGEXP_EXTRACT('foothebar', f33, 2)",
       "null")
 
     //test for optional group index
-    testSqlApi(
+    testAllApis(
+      "foothebar".regexpExtract("foo(.*?)(bar)"),
+      "'foothebar'.regexpExtract('foo(.*?)(bar)')",
       "REGEXP_EXTRACT('foothebar', 'foo(.*?)(bar)')",
       "foothebar")
-
-    testSqlApi("regexp_extract('100-200', '(\\d+)-(\\d+)', 1)", "100")
-    testSqlApi("regexp_extract('100-200', '', 1)", "null")
-    testSqlApi("regexp_extract('100-200', '(\\d+)-(\\d+)', -1)", "null")
-    testSqlApi("regexp_extract(f40, '(\\d+)-(\\d+)', 1)", "null")
-    testSqlApi("regexp_extract(CAST(null as VARCHAR), '(\\d+)-(\\d+)', 1)", "null")
-    testSqlApi("regexp_extract('100-200', CAST(null as VARCHAR), 1)", "null")
-    testSqlApi("regexp_extract('100-200', '(\\d+)-(\\d+)', CAST(null as BIGINT))", "null")
   }
 
   @Test
@@ -825,35 +953,6 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
   }
 
   @Test
-  def testUUID(): Unit = {
-    testSqlApi(
-      "CHARACTER_LENGTH(UUID())",
-      "36")
-
-    testSqlApi(
-      "SUBSTRING(UUID(), 9, 1)",
-      "-")
-
-    testSqlApi(
-      "SUBSTRING(UUID(), 14, 1)",
-      "-")
-
-    testSqlApi(
-      "SUBSTRING(UUID(), 19, 1)",
-      "-")
-
-    testSqlApi(
-      "SUBSTRING(UUID(), 24, 1)",
-      "-")
-
-    // test uuid with bytes
-    testSqlApi(
-      "UUID(f53)",
-      "5eb63bbb-e01e-3ed0-93cb-22bb8f5acdc3"
-    )
-  }
-
-  @Test
   def testSubString(): Unit = {
     Array("substring", "substr").foreach {
       substr =>
@@ -894,7 +993,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
     testSqlApi("lpad('Hello', -1, 'x') IS NOT NULL", "false")
     testSqlApi("lpad('ab', 5, '')", "null")
 
-    testSqlApi(
+    testAllApis(
+      "äää".lpad(13, "12345"),
+      "'äää'.lpad(13, '12345')",
       "lpad('äää',13,'12345')",
       "1234512345äää")
   }
@@ -918,7 +1019,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
     testSqlApi("rpad('\u0061\u0062',1,'??')", "a") // the unicode of ab is \u0061\u0062
     testSqlApi("rpad('üö',1,'??')", "ü")
     testSqlApi("rpad('abcd', 5, '')", "null")
-    testSqlApi(
+    testAllApis(
+      "äää".rpad(13, "12345"),
+      "'äää'.rpad(13, '12345')",
       "rpad('äää',13,'12345')",
       "äää1234512345")
   }
@@ -993,13 +1096,41 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testRepeat(): Unit = {
-    testSqlApi("repeat(f35, 2)", "aa")
-    testSqlApi("repeat(f35, 0)", "")
-    testSqlApi("repeat(f40, 2)", "null")
-    testSqlApi("repeat('hi', 2)", "hihi")
-    testSqlApi("repeat('hi', 0)", "")
-    testSqlApi("repeat('hi', CAST(null as INT))", "null")
-    testSqlApi("repeat(CAST(null as VARCHAR), 2)", "null")
+    testAllApis(
+      'f0.repeat(1),
+      "f0.repeat(1)",
+      "REPEAT(f0, 1)",
+      "This is a test String.")
+
+    testAllApis(
+      'f0.repeat(2),
+      "f0.repeat(2)",
+      "REPEAT(f0, 2)",
+      "This is a test String.This is a test String.")
+
+    testAllApis(
+      'f0.repeat(0),
+      "f0.repeat(0)",
+      "REPEAT(f0, 0)",
+      "")
+
+    testAllApis(
+      'f0.repeat(-1),
+      "f0.repeat(-1)",
+      "REPEAT(f0, -1)",
+      "")
+
+    testAllApis(
+      'f33.repeat(2),
+      "f33.repeat(2)",
+      "REPEAT(f33, 2)",
+      "null")
+
+    testAllApis(
+      "".repeat(1),
+      "''.repeat(1)",
+      "REPEAT('', 2)",
+      "")
   }
 
   @Test
@@ -1012,19 +1143,6 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
   }
 
   @Test
-  def testReplace(): Unit = {
-    testSqlApi("replace(f38, 'A', 'a')", "aQIDBa==")
-    testSqlApi("replace(f38, 'Z', 'a')", "AQIDBA==")
-    testSqlApi("replace(f38, CAST(null as VARCHAR), 'a')", "null")
-    testSqlApi("replace(f38, 'A', CAST(null as VARCHAR))", "null")
-    testSqlApi("replace(f40, 'A', 'a')", "null")
-    testSqlApi("replace('Test', 'T', 't')", "test")
-    testSqlApi("replace(CAST(null as VARCHAR), 'T', 't')", "null")
-    testSqlApi("replace('Test', CAST(null as VARCHAR), 't')", "null")
-    testSqlApi("replace('Test', 'T', CAST(null as VARCHAR))", "null")
-  }
-
-  @Test
   def testSplitIndex(): Unit = {
     testSqlApi("split_index(f38, 'I', 0)", "AQ")
     testSqlApi("split_index(f38, 'I', 2)", "null")
@@ -1096,67 +1214,209 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
   }
 
   @Test
+  def testHex(): Unit = {
+    testAllApis(
+      100.hex(),
+      "100.hex()",
+      "HEX(100)",
+      "64")
+
+    testAllApis(
+      'f2.hex(),
+      "f2.hex()",
+      "HEX(f2)",
+      "2A")
+
+    testAllApis(
+      nullOf(DataTypes.TINYINT).hex(),
+      "hex(Null(BYTE))",
+      "HEX(CAST(NULL AS TINYINT))",
+      "null")
+
+    testAllApis(
+      'f3.hex(),
+      "f3.hex()",
+      "HEX(f3)",
+      "2B")
+
+    testAllApis(
+      'f4.hex(),
+      "f4.hex()",
+      "HEX(f4)",
+      "2C")
+
+    testAllApis(
+      'f7.hex(),
+      "f7.hex()",
+      "HEX(f7)",
+      "3")
+
+    testAllApis(
+      12.hex(),
+      "12.hex()",
+      "HEX(12)",
+      "C")
+
+    testAllApis(
+      10.hex(),
+      "10.hex()",
+      "HEX(10)",
+      "A")
+
+    testAllApis(
+      0.hex(),
+      "0.hex()",
+      "HEX(0)",
+      "0")
+
+    testAllApis(
+      "ö".hex(),
+      "'ö'.hex()",
+      "HEX('ö')",
+      "C3B6")
+
+    testAllApis(
+      'f32.hex(),
+      "f32.hex()",
+      "HEX(f32)",
+      "FFFFFFFFFFFFFFFF")
+
+    testAllApis(
+      'f0.hex(),
+      "f0.hex()",
+      "HEX(f0)",
+      "546869732069732061207465737420537472696E672E")
+
+    testAllApis(
+      'f8.hex(),
+      "f8.hex()",
+      "HEX(f8)",
+      "20546869732069732061207465737420537472696E672E20")
+
+    testAllApis(
+      'f23.hex(),
+      "f23.hex()",
+      "HEX(f23)",
+      "25546869732069732061207465737420537472696E672E")
+
+    testAllApis(
+      'f24.hex(),
+      "f24.hex()",
+      "HEX(f24)",
+      "2A5F546869732069732061207465737420537472696E672E")
+  }
+
+  @Test
   def testBin(): Unit = {
-    testSqlApi(
+    testAllApis(
+      nullOf(DataTypes.TINYINT).bin(),
+      "bin(Null(BYTE))",
       "BIN((CAST(NULL AS TINYINT)))",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f2.bin(),
+      "f2.bin()",
       "BIN(f2)",
       "101010")
 
-    testSqlApi(
+    testAllApis(
+      'f3.bin(),
+      "f3.bin()",
       "BIN(f3)",
       "101011")
 
-    testSqlApi(
+    testAllApis(
+      'f4.bin(),
+      "f4.bin()",
       "BIN(f4)",
       "101100")
 
-    testSqlApi(
+    testAllApis(
+      'f7.bin(),
+      "f7.bin()",
       "BIN(f7)",
       "11")
 
-    testSqlApi(
+    testAllApis(
+      12.bin(),
+      "12.bin()",
       "BIN(12)",
       "1100")
 
-    testSqlApi(
+    testAllApis(
+      10.bin(),
+      "10.bin()",
       "BIN(10)",
       "1010")
 
-    testSqlApi(
+    testAllApis(
+      0.bin(),
+      "0.bin()",
       "BIN(0)",
       "0")
 
-    testSqlApi(
-      "BIN(-7)",
-      "1111111111111111111111111111111111111111111111111111111111111001")
-
-    testSqlApi(
-      "BIN(-1)",
-      "1111111111111111111111111111111111111111111111111111111111111111")
-
-    testSqlApi(
+    testAllApis(
+      'f32.bin(),
+      "f32.bin()",
       "BIN(f32)",
       "1111111111111111111111111111111111111111111111111111111111111111")
   }
 
+  @Test
+  def testUUID(): Unit = {
+    testAllApis(
+      uuid().charLength(),
+      "uuid().charLength",
+      "CHARACTER_LENGTH(UUID())",
+      "36")
+
+    testAllApis(
+      uuid().substring(9, 1),
+      "uuid().substring(9, 1)",
+      "SUBSTRING(UUID(), 9, 1)",
+      "-")
+
+    testAllApis(
+      uuid().substring(14, 1),
+      "uuid().substring(14, 1)",
+      "SUBSTRING(UUID(), 14, 1)",
+      "-")
+
+    testAllApis(
+      uuid().substring(19, 1),
+      "uuid().substring(19, 1)",
+      "SUBSTRING(UUID(), 19, 1)",
+      "-")
+
+    testAllApis(
+      uuid().substring(24, 1),
+      "uuid().substring(24, 1)",
+      "SUBSTRING(UUID(), 24, 1)",
+      "-")
+  }
+
   // ----------------------------------------------------------------------------------------------
   // Math functions
   // ----------------------------------------------------------------------------------------------
   @Test
   def testAdd(): Unit = {
 
-    testSqlApi(
+    testAllApis(
+      1514356320000L + 6000,
+      "1514356320000L + 6000",
       "1514356320000 + 6000",
       "1514356326000")
 
-    testSqlApi(
+    testAllApis(
+      'f34 + 6,
+      "f34 + 6",
       "f34 + 6",
       "1514356320006")
 
-    testSqlApi(
+    testAllApis(
+      'f34 + 'f34,
+      "f34 + f34",
       "f34 + f34",
       "3028712640000")
   }
@@ -1164,15 +1424,21 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
   @Test
   def testSubtract(): Unit = {
 
-    testSqlApi(
+    testAllApis(
+      1514356320000L - 6000,
+      "1514356320000L - 6000",
       "1514356320000 - 6000",
       "1514356314000")
 
-    testSqlApi(
+    testAllApis(
+      'f34 - 6,
+      "f34 - 6",
       "f34 - 6",
       "1514356319994")
 
-    testSqlApi(
+    testAllApis(
+      'f34 - 'f34,
+      "f34 - f34",
       "f34 - f34",
       "0")
   }
@@ -1180,16 +1446,22 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
   @Test
   def testMultiply(): Unit = {
 
-    testSqlApi(
+    testAllApis(
+      1514356320000L * 60000,
+      "1514356320000L * 60000",
       "1514356320000 * 60000",
       "90861379200000000")
 
-    testSqlApi(
+    testAllApis(
+      'f34 * 6,
+      "f34 * 6",
       "f34 * 6",
       "9086137920000")
 
 
-    testSqlApi(
+    testAllApis(
+      'f34 * 'f34,
+      "f34 * f34",
       "f34 * f34",
       "2293275063923942400000000")
 
@@ -1198,28 +1470,31 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
   @Test
   def testDivide(): Unit = {
 
-    testSqlApi(
+    testAllApis(
+      1514356320000L / 60000.0, // the `/` is Scala operator, not Flink TableApi operator
+      "1514356320000L / 60000",
       "1514356320000 / 60000",
       "2.5239272E7")
 
-    // DIV return decimal
-    testSqlApi(
-      "DIV(1514356320000, 60000)",
-      "25239272")
-
-    testSqlApi(
+    testAllApis(
+      'f7 / 2,
+      "f7 / 2",
       "f7 / 2",
       "1.5")
 
     // f34 => Decimal(19,0)
     // 6 => Integer => Decimal(10,0)
     // Decimal(19,0) / Decimal(10,0) => Decimal(30,11)
-    testSqlApi(
+    testAllApis(
+      'f34 / 6,
+      "f34 / 6",
       "f34 / 6",
       "252392720000.00000000000")
 
     // Decimal(19,0) / Decimal(19,0) => Decimal(39,20) => Decimal(38,19)
-    testSqlApi(
+    testAllApis(
+      'f34 / 'f34,
+      "f34 / f34",
       "f34 / f34",
       "1.0000000000000000000")
   }
@@ -1227,81 +1502,117 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
   @Test
   def testMod(): Unit = {
 
-    testSqlApi(
+    testAllApis(
+      1514356320000L % 60000,
+      "1514356320000L % 60000",
       "mod(1514356320000,60000)",
       "0")
 
-    testSqlApi(
+    testAllApis(
+      'f34.mod('f34),
+      "f34.mod(f34)",
       "mod(f34,f34)",
       "0")
 
-    testSqlApi(
+    testAllApis(
+      'f34.mod(6),
+      "f34.mod(6)",
       "mod(f34,6)",
       "0")
 
-    testSqlApi(
+    testAllApis(
+      'f4.mod('f7),
+      "f4.mod(f7)",
       "MOD(f4, f7)",
       "2")
 
-    testSqlApi(
+    testAllApis(
+      'f4.mod(3),
+      "mod(f4, 3)",
       "MOD(f4, 3)",
       "2")
 
-    testSqlApi(
+    testAllApis(
+      'f4 % 3,
+      "mod(44, 3)",
       "MOD(44, 3)",
       "2")
   }
 
   @Test
   def testExp(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f2.exp(),
+      "f2.exp()",
       "EXP(f2)",
       math.exp(42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f3.exp(),
+      "f3.exp()",
       "EXP(f3)",
       math.exp(43.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.exp(),
+      "f4.exp()",
       "EXP(f4)",
       math.exp(44.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f5.exp(),
+      "f5.exp()",
       "EXP(f5)",
       math.exp(4.5.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.exp(),
+      "f6.exp()",
       "EXP(f6)",
       math.exp(4.6).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f7.exp(),
+      "exp(f7)",
       "EXP(f7)",
       math.exp(3).toString)
 
-    testSqlApi(
+    testAllApis(
+      3.exp(),
+      "exp(3)",
       "EXP(3)",
       math.exp(3).toString)
   }
 
   @Test
   def testLog10(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f2.log10(),
+      "f2.log10()",
       "LOG10(f2)",
       math.log10(42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f3.log10(),
+      "f3.log10()",
       "LOG10(f3)",
       math.log10(43.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.log10(),
+      "f4.log10()",
       "LOG10(f4)",
       math.log10(44.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f5.log10(),
+      "f5.log10()",
       "LOG10(f5)",
       math.log10(4.5.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.log10(),
+      "f6.log10()",
       "LOG10(f6)",
       math.log10(4.6).toString)
   }
@@ -1309,472 +1620,835 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
   @Test
   def testPower(): Unit = {
     // f7: int , f4: long, f6: double
-    testSqlApi(
+    testAllApis(
+      'f2.power('f7),
+      "f2.power(f7)",
       "POWER(f2, f7)",
       math.pow(42.toByte, 3).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f3.power('f6),
+      "f3.power(f6)",
       "POWER(f3, f6)",
       math.pow(43.toShort, 4.6D).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.power('f5),
+      "f4.power(f5)",
       "POWER(f4, f5)",
       math.pow(44.toLong, 4.5.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.power('f5),
+      "f4.power(f5)",
       "POWER(f4, f5)",
       math.pow(44.toLong, 4.5.toFloat).toString)
 
     // f5: float
-    testSqlApi(
+    testAllApis('f5.power('f5),
+      "f5.power(f5)",
       "power(f5, f5)",
       math.pow(4.5F, 4.5F).toString)
 
-    testSqlApi(
+    testAllApis('f5.power('f6),
+      "f5.power(f6)",
       "power(f5, f6)",
       math.pow(4.5F, 4.6D).toString)
 
-    testSqlApi(
+    testAllApis('f5.power('f7),
+      "f5.power(f7)",
       "power(f5, f7)",
       math.pow(4.5F, 3).toString)
 
-    testSqlApi(
+    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
-    testSqlApi(
+    testAllApis(
+      'f22.cast(DataTypes.DOUBLE).power('f5),
+      "f22.cast(DOUBLE).power(f5)",
       "power(CAST(f22 AS DOUBLE), f5)",
       math.pow(2, 4.5F).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f22.cast(DataTypes.DOUBLE).power('f6),
+      "f22.cast(DOUBLE).power(f6)",
       "power(CAST(f22 AS DOUBLE), f6)",
       math.pow(2, 4.6D).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f22.cast(DataTypes.DOUBLE).power('f7),
+      "f22.cast(DOUBLE).power(f7)",
       "power(CAST(f22 AS DOUBLE), f7)",
       math.pow(2, 3).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f22.cast(DataTypes.DOUBLE).power('f4),
+      "f22.cast(DOUBLE).power(f4)",
       "power(CAST(f22 AS DOUBLE), f4)",
       math.pow(2, 44L).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.power('f22.cast(DataTypes.DOUBLE)),
+      "f6.power(f22.cast(DOUBLE))",
       "power(f6, f22)",
       math.pow(4.6D, 2).toString)
   }
 
   @Test
   def testSqrt(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f6.sqrt(),
+      "f6.sqrt",
       "SQRT(f6)",
       math.sqrt(4.6D).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f7.sqrt(),
+      "f7.sqrt",
       "SQRT(f7)",
       math.sqrt(3).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.sqrt(),
+      "f4.sqrt",
       "SQRT(f4)",
       math.sqrt(44L).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f22.cast(DataTypes.DOUBLE).sqrt(),
+      "f22.cast(DOUBLE).sqrt",
       "SQRT(CAST(f22 AS DOUBLE))",
       math.sqrt(2.0).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f5.sqrt(),
+      "f5.sqrt",
       "SQRT(f5)",
       math.pow(4.5F, 0.5).toString)
 
-    testSqlApi(
+    testAllApis(
+      25.sqrt(),
+      "25.sqrt()",
       "SQRT(25)",
       "5.0")
 
-    testSqlApi(
-      // TODO fix FLINK-4621
-      "POWER(CAST(2.2 AS DOUBLE), CAST(0.5 AS DOUBLE))",
+    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 testCosh(): Unit = {
+    testAllApis(
+      0.cosh(),
+      "0.cosh()",
+      "COSH(0)",
+      math.cosh(0).toString
+    )
+
+    testAllApis(
+      -1.cosh(),
+      "-1.cosh()",
+      "COSH(-1)",
+      math.cosh(-1).toString
+    )
+
+    testAllApis(
+      'f4.cosh(),
+      "f4.cosh",
+      "COSH(f4)",
+      math.cosh(44L).toString)
+
+    testAllApis(
+      'f6.cosh(),
+      "f6.cosh",
+      "COSH(f6)",
+      math.cosh(4.6D).toString)
+
+    testAllApis(
+      'f7.cosh(),
+      "f7.cosh",
+      "COSH(f7)",
+      math.cosh(3).toString)
+
+    testAllApis(
+      'f22.cosh(),
+      "f22.cosh",
+      "COSH(f22)",
+      math.cosh(2.0).toString)
+  }
+
+  @Test
   def testLn(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f2.ln(),
+      "f2.ln()",
       "LN(f2)",
       math.log(42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f3.ln(),
+      "f3.ln()",
       "LN(f3)",
       math.log(43.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.ln(),
+      "f4.ln()",
       "LN(f4)",
       math.log(44.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f5.ln(),
+      "f5.ln()",
       "LN(f5)",
       math.log(4.5.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.ln(),
+      "f6.ln()",
       "LN(f6)",
       math.log(4.6).toString)
   }
 
   @Test
   def testAbs(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f2.abs(),
+      "f2.abs()",
       "ABS(f2)",
       "42")
 
-    testSqlApi(
+    testAllApis(
+      'f3.abs(),
+      "f3.abs()",
       "ABS(f3)",
       "43")
 
-    testSqlApi(
+    testAllApis(
+      'f4.abs(),
+      "f4.abs()",
       "ABS(f4)",
       "44")
 
-    testSqlApi(
+    testAllApis(
+      'f5.abs(),
+      "f5.abs()",
       "ABS(f5)",
       "4.5")
 
-    testSqlApi(
+    testAllApis(
+      'f6.abs(),
+      "f6.abs()",
       "ABS(f6)",
       "4.6")
 
-    testSqlApi(
+    testAllApis(
+      'f9.abs(),
+      "f9.abs()",
       "ABS(f9)",
       "42")
 
-    testSqlApi(
+    testAllApis(
+      'f10.abs(),
+      "f10.abs()",
       "ABS(f10)",
       "43")
 
-    testSqlApi(
+    testAllApis(
+      'f11.abs(),
+      "f11.abs()",
       "ABS(f11)",
       "44")
 
-    testSqlApi(
+    testAllApis(
+      'f12.abs(),
+      "f12.abs()",
       "ABS(f12)",
       "4.5")
 
-    testSqlApi(
+    testAllApis(
+      'f13.abs(),
+      "f13.abs()",
       "ABS(f13)",
       "4.6")
 
-    testSqlApi(
+    testAllApis(
+      'f15.abs(),
+      "f15.abs()",
       "ABS(f15)",
       "1231.1231231321321321111")
   }
 
   @Test
   def testArithmeticFloorCeil(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f5.floor(),
+      "f5.floor()",
       "FLOOR(f5)",
       "4.0")
 
-    testSqlApi(
+    testAllApis(
+      'f5.ceil(),
+      "f5.ceil()",
       "CEIL(f5)",
       "5.0")
 
-    testSqlApi(
+    testAllApis(
+      'f3.floor(),
+      "f3.floor()",
       "FLOOR(f3)",
       "43")
 
-    testSqlApi(
+    testAllApis(
+      'f3.ceil(),
+      "f3.ceil()",
       "CEIL(f3)",
       "43")
 
-    testSqlApi(
+    testAllApis(
+      'f15.floor(),
+      "f15.floor()",
       "FLOOR(f15)",
       "-1232")
 
-    testSqlApi(
+    testAllApis(
+      'f15.ceil(),
+      "f15.ceil()",
       "CEIL(f15)",
       "-1231")
   }
 
   @Test
   def testSin(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f2.sin(),
+      "f2.sin()",
       "SIN(f2)",
       math.sin(42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f3.sin(),
+      "f3.sin()",
       "SIN(f3)",
       math.sin(43.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.sin(),
+      "f4.sin()",
       "SIN(f4)",
       math.sin(44.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f5.sin(),
+      "f5.sin()",
       "SIN(f5)",
       math.sin(4.5.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.sin(),
+      "f6.sin()",
       "SIN(f6)",
       math.sin(4.6).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f15.sin(),
+      "sin(f15)",
       "SIN(f15)",
       math.sin(-1231.1231231321321321111).toString)
   }
 
   @Test
   def testCos(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f2.cos(),
+      "f2.cos()",
       "COS(f2)",
       math.cos(42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f3.cos(),
+      "f3.cos()",
       "COS(f3)",
       math.cos(43.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.cos(),
+      "f4.cos()",
       "COS(f4)",
       math.cos(44.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f5.cos(),
+      "f5.cos()",
       "COS(f5)",
       math.cos(4.5.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.cos(),
+      "f6.cos()",
       "COS(f6)",
       math.cos(4.6).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f15.cos(),
+      "cos(f15)",
       "COS(f15)",
       math.cos(-1231.1231231321321321111).toString)
   }
 
   @Test
+  def testSinh(): Unit = {
+    testAllApis(
+      0.sinh(),
+      "0.sinh()",
+      "SINH(0)",
+      math.sinh(0).toString)
+
+    testAllApis(
+      -1.sinh(),
+      "-1.sinh()",
+      "SINH(-1)",
+      math.sinh(-1).toString)
+
+    testAllApis(
+      'f4.sinh(),
+      "f4.sinh",
+      "SINH(f4)",
+      math.sinh(44L).toString)
+
+    testAllApis(
+      'f6.sinh(),
+      "f6.sinh",
+      "SINH(f6)",
+      math.sinh(4.6D).toString)
+
+    testAllApis(
+      'f7.sinh(),
+      "f7.sinh",
+      "SINH(f7)",
+      math.sinh(3).toString)
+
+    testAllApis(
+      'f22.sinh(),
+      "f22.sinh",
+      "SINH(f22)",
+      math.sinh(2.0).toString)
+  }
+
+  @Test
   def testTan(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f2.tan(),
+      "f2.tan()",
       "TAN(f2)",
       math.tan(42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f3.tan(),
+      "f3.tan()",
       "TAN(f3)",
       math.tan(43.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.tan(),
+      "f4.tan()",
       "TAN(f4)",
       math.tan(44.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f5.tan(),
+      "f5.tan()",
       "TAN(f5)",
       math.tan(4.5.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.tan(),
+      "f6.tan()",
       "TAN(f6)",
       math.tan(4.6).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f15.tan(),
+      "tan(f15)",
       "TAN(f15)",
       math.tan(-1231.1231231321321321111).toString)
   }
 
   @Test
+  def testTanh(): Unit = {
+    testAllApis(
+      0.tanh(),
+      "0.tanh()",
+      "TANH(0)",
+      math.tanh(0).toString)
+
+    testAllApis(
+      -1.tanh(),
+      "-1.tanh()",
+      "TANH(-1)",
+      math.tanh(-1).toString)
+
+    testAllApis(
+      'f4.tanh(),
+      "f4.tanh",
+      "TANH(f4)",
+      math.tanh(44L).toString)
+
+    testAllApis(
+      'f6.tanh(),
+      "f6.tanh",
+      "TANH(f6)",
+      math.tanh(4.6D).toString)
+
+    testAllApis(
+      'f7.tanh(),
+      "f7.tanh",
+      "TANH(f7)",
+      math.tanh(3).toString)
+
+    testAllApis(
+      'f22.tanh(),
+      "f22.tanh",
+      "TANH(f22)",
+      math.tanh(2.0).toString)
+  }
+
+  @Test
   def testCot(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f2.cot(),
+      "f2.cot()",
       "COT(f2)",
       (1.0d / math.tan(42.toByte)).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f3.cot(),
+      "f3.cot()",
       "COT(f3)",
       (1.0d / math.tan(43.toShort)).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.cot(),
+      "f4.cot()",
       "COT(f4)",
       (1.0d / math.tan(44.toLong)).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f5.cot(),
+      "f5.cot()",
       "COT(f5)",
       (1.0d / math.tan(4.5.toFloat)).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.cot(),
+      "f6.cot()",
       "COT(f6)",
       (1.0d / math.tan(4.6)).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f15.cot(),
+      "cot(f15)",
       "COT(f15)",
       (1.0d / math.tan(-1231.1231231321321321111)).toString)
   }
 
   @Test
   def testAsin(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f25.asin(),
+      "f25.asin()",
       "ASIN(f25)",
       math.asin(0.42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f26.asin(),
+      "f26.asin()",
       "ASIN(f26)",
       math.asin(0.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f27.asin(),
+      "f27.asin()",
       "ASIN(f27)",
       math.asin(0.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f28.asin(),
+      "f28.asin()",
       "ASIN(f28)",
       math.asin(0.45.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f29.asin(),
+      "f29.asin()",
       "ASIN(f29)",
       math.asin(0.46).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f30.asin(),
+      "f30.asin()",
       "ASIN(f30)",
       math.asin(1).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f31.asin(),
+      "f31.asin()",
       "ASIN(f31)",
       math.asin(-0.1231231321321321111).toString)
   }
 
   @Test
   def testAcos(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f25.acos(),
+      "f25.acos()",
       "ACOS(f25)",
       math.acos(0.42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f26.acos(),
+      "f26.acos()",
       "ACOS(f26)",
       math.acos(0.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f27.acos(),
+      "f27.acos()",
       "ACOS(f27)",
       math.acos(0.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f28.acos(),
+      "f28.acos()",
       "ACOS(f28)",
       math.acos(0.45.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f29.acos(),
+      "f29.acos()",
       "ACOS(f29)",
       math.acos(0.46).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f30.acos(),
+      "f30.acos()",
       "ACOS(f30)",
       math.acos(1).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f31.acos(),
+      "f31.acos()",
       "ACOS(f31)",
       math.acos(-0.1231231321321321111).toString)
   }
 
   @Test
   def testAtan(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f25.atan(),
+      "f25.atan()",
       "ATAN(f25)",
       math.atan(0.42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f26.atan(),
+      "f26.atan()",
       "ATAN(f26)",
       math.atan(0.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f27.atan(),
+      "f27.atan()",
       "ATAN(f27)",
       math.atan(0.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f28.atan(),
+      "f28.atan()",
       "ATAN(f28)",
       math.atan(0.45.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f29.atan(),
+      "f29.atan()",
       "ATAN(f29)",
       math.atan(0.46).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f30.atan(),
+      "f30.atan()",
       "ATAN(f30)",
       math.atan(1).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f31.atan(),
+      "f31.atan()",
       "ATAN(f31)",
       math.atan(-0.1231231321321321111).toString)
   }
 
   @Test
+  def testAtan2(): Unit = {
+    testAllApis(
+      atan2('f25, 'f26),
+      "atan2(f25, f26)",
+      "ATAN2(f25, f26)",
+      math.atan2(0.42.toByte, 0.toByte).toString)
+
+    testAllApis(
+      atan2('f26, 'f25),
+      "atan2(f26, f25)",
+      "ATAN2(f26, f25)",
+      math.atan2(0.toShort, 0.toShort).toString)
+
+    testAllApis(
+      atan2('f27, 'f27),
+      "atan2(f27, f27)",
+      "ATAN2(f27, f27)",
+      math.atan2(0.toLong, 0.toLong).toString)
+
+    testAllApis(
+      atan2('f28, 'f28),
+      "atan2(f28, f28)",
+      "ATAN2(f28, f28)",
+      math.atan2(0.45.toFloat, 0.45.toFloat).toString)
+
+    testAllApis(
+      atan2('f29, 'f29),
+      "atan2(f29, f29)",
+      "ATAN2(f29, f29)",
+      math.atan2(0.46, 0.46).toString)
+
+    testAllApis(
+      atan2('f30, 'f30),
+      "atan2(f30, f30)",
+      "ATAN2(f30, f30)",
+      math.atan2(1, 1).toString)
+
+    testAllApis(
+      atan2('f31, 'f31),
+      "atan2(f31, f31)",
+      "ATAN2(f31, f31)",
+      math.atan2(-0.1231231321321321111, -0.1231231321321321111).toString)
+  }
+
+  @Test
   def testDegrees(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f2.degrees(),
+      "f2.degrees()",
       "DEGREES(f2)",
       math.toDegrees(42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f3.degrees(),
+      "f3.degrees()",
       "DEGREES(f3)",
       math.toDegrees(43.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.degrees(),
+      "f4.degrees()",
       "DEGREES(f4)",
       math.toDegrees(44.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f5.degrees(),
+      "f5.degrees()",
       "DEGREES(f5)",
       math.toDegrees(4.5.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.degrees(),
+      "f6.degrees()",
       "DEGREES(f6)",
       math.toDegrees(4.6).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f15.degrees(),
+      "degrees(f15)",
       "DEGREES(f15)",
       math.toDegrees(-1231.1231231321321321111).toString)
   }
 
   @Test
   def testRadians(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f2.radians(),
+      "f2.radians()",
       "RADIANS(f2)",
       math.toRadians(42.toByte).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f3.radians(),
+      "f3.radians()",
       "RADIANS(f3)",
       math.toRadians(43.toShort).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f4.radians(),
+      "f4.radians()",
       "RADIANS(f4)",
       math.toRadians(44.toLong).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f5.radians(),
+      "f5.radians()",
       "RADIANS(f5)",
       math.toRadians(4.5.toFloat).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.radians(),
+      "f6.radians()",
       "RADIANS(f6)",
       math.toRadians(4.6).toString)
 
-    testSqlApi(
+    testAllApis(
+      'f15.radians(),
+      "radians(f15)",
       "RADIANS(f15)",
       math.toRadians(-1231.1231231321321321111).toString)
   }
 
   @Test
   def testSign(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f4.sign(),
+      "f4.sign()",
       "SIGN(f4)",
       1.toString)
 
-    testSqlApi(
+    testAllApis(
+      'f6.sign(),
+      "f6.sign()",
       "SIGN(f6)",
       1.0.toString)
 
-    testSqlApi(
+    testAllApis(
+      'f15.sign(),
+      "sign(f15)",
       "SIGN(f15)",
       "-1.0000000000000000000") // calcite: SIGN(Decimal(p,s)) => Decimal(p,s)
   }
 
   @Test
   def testRound(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f29.round('f30),
+      "f29.round(f30)",
       "ROUND(f29, f30)",
       0.5.toString)
 
-    testSqlApi(
-      "ROUND(f31, f7)",
-      "-0.123")
-
-    testSqlApi(
+    testAllApis(
+      'f4.round('f32),
+      "f4.round(f32)",
       "ROUND(f4, f32)",
       40.toString)
 
@@ -1797,74 +2471,92 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testPi(): Unit = {
-    // PI function
-    testSqlApi(
-      "PI()",
-      math.Pi.toString)
-
-    // PI operator
-    testSqlApi(
-      "PI",
+    testAllApis(
+      pi(),
+      "pi()",
+      "pi()",
       math.Pi.toString)
   }
 
   @Test
   def testRandAndRandInteger(): Unit = {
     val random1 = new java.util.Random(1)
-    testSqlApi(
+    testAllApis(
+      rand(1),
+      "rand(1)",
       "RAND(1)",
       random1.nextDouble().toString)
 
     val random2 = new java.util.Random(3)
-    testSqlApi(
+    testAllApis(
+      rand('f7),
+      "rand(f7)",
       "RAND(f7)",
       random2.nextDouble().toString)
 
     val random3 = new java.util.Random(1)
-    testSqlApi(
+    testAllApis(
+      randInteger(1, 10),
+      "randInteger(1, 10)",
       "RAND_INTEGER(1, 10)",
       random3.nextInt(10).toString)
 
     val random4 = new java.util.Random(3)
-    testSqlApi(
+    testAllApis(
+      randInteger('f7, 'f4.cast(DataTypes.INT)),
+      "randInteger(f7, f4.cast(INT))",
       "RAND_INTEGER(f7, CAST(f4 AS INT))",
       random4.nextInt(44).toString)
   }
 
   @Test
   def testE(): Unit = {
-    testSqlApi(
+    testAllApis(
+      e(),
+      "E()",
       "E()",
       math.E.toString)
 
-    testSqlApi(
+    testAllApis(
+      e(),
+      "e()",
       "e()",
       math.E.toString)
   }
 
   @Test
   def testLog(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f6.log(),
+      "f6.log",
       "LOG(f6)",
       "1.5260563034950492"
     )
 
-    testSqlApi(
+    testAllApis(
+      ('f6 - 'f6 + 100).log('f6 - 'f6 + 10),
+      "(f6 - f6 + 100).log(f6 - f6 + 10)",
       "LOG(f6 - f6 + 10, f6 - f6 + 100)",
       "2.0"
     )
 
-    testSqlApi(
+    testAllApis(
+      ('f6 + 20).log(),
+      "(f6+20).log",
       "LOG(f6+20)",
       "3.202746442938317"
     )
 
-    testSqlApi(
+    testAllApis(
+      10.log(),
+      "10.log",
       "LOG(10)",
       "2.302585092994046"
     )
 
-    testSqlApi(
+    testAllApis(
+      100.log(10),
+      "100.log(10)",
       "LOG(10, 100)",
       "2.0"
     )
@@ -1891,17 +2583,28 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testLog2(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f6.log2(),
+      "f6.log2",
       "LOG2(f6)",
-      "2.2016338611696504"
-    )
+      "2.2016338611696504")
 
-    testSqlApi(
+    testAllApis(
+      ('f6 - 'f6 + 100).log2(),
+      "(f6 - f6 + 100).log2()",
+      "LOG2(f6 - f6 + 100)",
+      "6.643856189774725")
+
+    testAllApis(
+      ('f6 + 20).log2(),
+      "(f6+20).log2",
       "LOG2(f6+20)",
       "4.620586410451877"
     )
 
-    testSqlApi(
+    testAllApis(
+      10.log2(),
+      "10.log2",
       "LOG2(10)",
       "3.3219280948873626"
     )
@@ -1981,75 +2684,147 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testExtract(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f16.extract(TimeIntervalUnit.YEAR),
+      "f16.extract(YEAR)",
       "EXTRACT(YEAR FROM f16)",
       "1996")
 
-    testSqlApi(
+    testAllApis(
+      'f16.extract(TimeIntervalUnit.QUARTER),
+      "f16.extract(QUARTER)",
+      "EXTRACT(QUARTER FROM f16)",
+      "4")
+
+    testAllApis(
+      'f16.extract(TimeIntervalUnit.MONTH),
+      "extract(f16, MONTH)",
       "EXTRACT(MONTH FROM f16)",
       "11")
 
-    testSqlApi(
+    testAllApis(
+      'f16.extract(TimeIntervalUnit.WEEK),
+      "extract(f16, WEEK)",
+      "EXTRACT(WEEK FROM f16)",
+      "45")
+
+    testAllApis(
+      'f16.extract(TimeIntervalUnit.DAY),
+      "f16.extract(DAY)",
       "EXTRACT(DAY FROM f16)",
       "10")
 
-    testSqlApi(
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.YEAR),
+      "f18.extract(YEAR)",
       "EXTRACT(YEAR FROM f18)",
       "1996")
 
-    testSqlApi(
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.QUARTER),
+      "f18.extract(QUARTER)",
+      "EXTRACT(QUARTER FROM f18)",
+      "4")
+
+    testAllApis(
+      'f16.extract(TimeIntervalUnit.QUARTER),
+      "f16.extract(QUARTER)",
+      "EXTRACT(QUARTER FROM f16)",
+      "4")
+
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.MONTH),
+      "f18.extract(MONTH)",
       "EXTRACT(MONTH FROM f18)",
       "11")
 
-    testSqlApi(
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.WEEK),
+      "f18.extract(WEEK)",
+      "EXTRACT(WEEK FROM f18)",
+      "45")
+
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.DAY),
+      "f18.extract(DAY)",
       "EXTRACT(DAY FROM f18)",
       "10")
 
-    testSqlApi(
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.HOUR),
+      "f18.extract(HOUR)",
       "EXTRACT(HOUR FROM f18)",
       "6")
 
-    testSqlApi(
+    testAllApis(
+      'f17.extract(TimeIntervalUnit.HOUR),
+      "f17.extract(HOUR)",
       "EXTRACT(HOUR FROM f17)",
       "6")
 
-    testSqlApi(
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.MINUTE),
+      "f18.extract(MINUTE)",
       "EXTRACT(MINUTE FROM f18)",
       "55")
 
-    testSqlApi(
+    testAllApis(
+      'f17.extract(TimeIntervalUnit.MINUTE),
+      "f17.extract(MINUTE)",
       "EXTRACT(MINUTE FROM f17)",
       "55")
 
-    testSqlApi(
+    testAllApis(
+      'f18.extract(TimeIntervalUnit.SECOND),
+      "f18.extract(SECOND)",
       "EXTRACT(SECOND FROM f18)",
       "44")
 
-    testSqlApi(
+    testAllApis(
+      'f17.extract(TimeIntervalUnit.SECOND),
+      "f17.extract(SECOND)",
       "EXTRACT(SECOND FROM f17)",
       "44")
 
-    testSqlApi(
+    testAllApis(
+      'f19.extract(TimeIntervalUnit.DAY),
+      "f19.extract(DAY)",
       "EXTRACT(DAY FROM f19)",
       "16979")
 
-    testSqlApi(
+    testAllApis(
+      'f19.extract(TimeIntervalUnit.HOUR),
+      "f19.extract(HOUR)",
       "EXTRACT(HOUR FROM f19)",
       "7")
 
-    testSqlApi(
+    testAllApis(
+      'f19.extract(TimeIntervalUnit.MINUTE),
+      "f19.extract(MINUTE)",
       "EXTRACT(MINUTE FROM f19)",
       "23")
 
-    testSqlApi(
+    testAllApis(
+      'f19.extract(TimeIntervalUnit.SECOND),
+      "f19.extract(SECOND)",
       "EXTRACT(SECOND FROM f19)",
       "33")
 
-    testSqlApi(
+    testAllApis(
+      'f20.extract(TimeIntervalUnit.MONTH),
+      "f20.extract(MONTH)",
       "EXTRACT(MONTH FROM f20)",
       "1")
 
-    testSqlApi(
+    testAllApis(
+      'f20.extract(TimeIntervalUnit.QUARTER),
+      "f20.extract(QUARTER)",
+      "EXTRACT(QUARTER FROM f20)",
+      "1")
+
+    testAllApis(
+      'f20.extract(TimeIntervalUnit.YEAR),
+      "f20.extract(YEAR)",
       "EXTRACT(YEAR FROM f20)",
       "2")
 
@@ -2185,83 +2960,123 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testTemporalFloor(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f18.floor(TimeIntervalUnit.YEAR),
+      "f18.floor(YEAR)",
       "FLOOR(f18 TO YEAR)",
       "1996-01-01 00:00:00.000")
 
-    testSqlApi(
+    testAllApis(
+      'f18.floor(TimeIntervalUnit.MONTH),
+      "f18.floor(MONTH)",
       "FLOOR(f18 TO MONTH)",
       "1996-11-01 00:00:00.000")
 
-    testSqlApi(
+    testAllApis(
+      'f18.floor(TimeIntervalUnit.DAY),
+      "f18.floor(DAY)",
       "FLOOR(f18 TO DAY)",
       "1996-11-10 00:00:00.000")
 
-    testSqlApi(
+    testAllApis(
+      'f18.floor(TimeIntervalUnit.MINUTE),
+      "f18.floor(MINUTE)",
       "FLOOR(f18 TO MINUTE)",
       "1996-11-10 06:55:00.000")
 
-    testSqlApi(
+    testAllApis(
+      'f18.floor(TimeIntervalUnit.SECOND),
+      "f18.floor(SECOND)",
       "FLOOR(f18 TO SECOND)",
       "1996-11-10 06:55:44.000")
 
-    testSqlApi(
+    testAllApis(
+      'f17.floor(TimeIntervalUnit.HOUR),
+      "f17.floor(HOUR)",
       "FLOOR(f17 TO HOUR)",
       "06:00:00")
 
-    testSqlApi(
+    testAllApis(
+      'f17.floor(TimeIntervalUnit.MINUTE),
+      "f17.floor(MINUTE)",
       "FLOOR(f17 TO MINUTE)",
       "06:55:00")
 
-    testSqlApi(
+    testAllApis(
+      'f17.floor(TimeIntervalUnit.SECOND),
+      "f17.floor(SECOND)",
       "FLOOR(f17 TO SECOND)",
       "06:55:44")
 
-    testSqlApi(
+    testAllApis(
+      'f16.floor(TimeIntervalUnit.YEAR),
+      "f16.floor(YEAR)",
       "FLOOR(f16 TO YEAR)",
       "1996-01-01")
 
-    testSqlApi(
+    testAllApis(
+      'f16.floor(TimeIntervalUnit.MONTH),
+      "f16.floor(MONTH)",
       "FLOOR(f16 TO MONTH)",
       "1996-11-01")
 
-    testSqlApi(
+    testAllApis(
+      'f18.ceil(TimeIntervalUnit.YEAR),
+      "f18.ceil(YEAR)",
       "CEIL(f18 TO YEAR)",
       "1997-01-01 00:00:00.000")
 
-    testSqlApi(
+    testAllApis(
+      'f18.ceil(TimeIntervalUnit.MONTH),
+      "f18.ceil(MONTH)",
       "CEIL(f18 TO MONTH)",
       "1996-12-01 00:00:00.000")
 
-    testSqlApi(
+    testAllApis(
+      'f18.ceil(TimeIntervalUnit.DAY),
+      "f18.ceil(DAY)",
       "CEIL(f18 TO DAY)",
       "1996-11-11 00:00:00.000")
 
-    testSqlApi(
+    testAllApis(
+      'f18.ceil(TimeIntervalUnit.MINUTE),
+      "f18.ceil(MINUTE)",
       "CEIL(f18 TO MINUTE)",
       "1996-11-10 06:56:00.000")
 
-    testSqlApi(
+    testAllApis(
+      'f18.ceil(TimeIntervalUnit.SECOND),
+      "f18.ceil(SECOND)",
       "CEIL(f18 TO SECOND)",
       "1996-11-10 06:55:45.000")
 
-    testSqlApi(
+    testAllApis(
+      'f17.ceil(TimeIntervalUnit.HOUR),
+      "f17.ceil(HOUR)",
       "CEIL(f17 TO HOUR)",
       "07:00:00")
 
-    testSqlApi(
+    testAllApis(
+      'f17.ceil(TimeIntervalUnit.MINUTE),
+      "f17.ceil(MINUTE)",
       "CEIL(f17 TO MINUTE)",
       "06:56:00")
 
-    testSqlApi(
+    testAllApis(
+      'f17.ceil(TimeIntervalUnit.SECOND),
+      "f17.ceil(SECOND)",
       "CEIL(f17 TO SECOND)",
       "06:55:44")
 
-    testSqlApi(
+    testAllApis(
+      'f16.ceil(TimeIntervalUnit.YEAR),
+      "f16.ceil(YEAR)",
       "CEIL(f16 TO YEAR)",
       "1997-01-01")
 
-    testSqlApi(
+    testAllApis(
+      'f16.ceil(TimeIntervalUnit.MONTH),
+      "f16.ceil(MONTH)",
       "CEIL(f16 TO MONTH)",
       "1996-12-01")
   }
@@ -2273,58 +3088,86 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
     // we just test the format of the output
     // manual test can be found in NonDeterministicTests
 
-    testSqlApi(
+    testAllApis(
+      currentDate().cast(DataTypes.STRING).charLength() >= 5,
+      "currentDate().cast(STRING).charLength() >= 5",
       "CHAR_LENGTH(CAST(CURRENT_DATE AS VARCHAR)) >= 5",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      currentTime().cast(DataTypes.STRING).charLength() >= 5,
+      "currentTime().cast(STRING).charLength() >= 5",
       "CHAR_LENGTH(CAST(CURRENT_TIME AS VARCHAR)) >= 5",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      currentTimestamp().cast(DataTypes.STRING).charLength() >= 12,
+      "currentTimestamp().cast(STRING).charLength() >= 12",
       "CHAR_LENGTH(CAST(CURRENT_TIMESTAMP AS VARCHAR)) >= 12",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      localTimestamp().cast(DataTypes.STRING).charLength() >= 12,
+      "localTimestamp().cast(STRING).charLength() >= 12",
       "CHAR_LENGTH(CAST(LOCALTIMESTAMP AS VARCHAR)) >= 12",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      localTime().cast(DataTypes.STRING).charLength() >= 5,
+      "localTime().cast(STRING).charLength() >= 5",
       "CHAR_LENGTH(CAST(LOCALTIME AS VARCHAR)) >= 5",
       "true")
 
     // comparisons are deterministic
-    testSqlApi(
+    testAllApis(
+      localTimestamp() === localTimestamp(),
+      "localTimestamp() === localTimestamp()",
       "LOCALTIMESTAMP = LOCALTIMESTAMP",
       "true")
   }
 
   @Test
   def testOverlaps(): Unit = {
-    testSqlApi(
+    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")
 
-    testSqlApi(
+    testAllApis(
+      temporalOverlaps("9:00:00".toTime, "9:30:00".toTime, "9:29:00".toTime, "9:31:00".toTime),
+      "temporalOverlaps(toTime('9:00:00'), '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")
 
-    testSqlApi(
+    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")
 
-    testSqlApi(
+    testAllApis(
+      temporalOverlaps("2011-03-10".toDate, 10.days, "2011-03-19".toDate, 10.days),
+      "temporalOverlaps(toDate('2011-03-10'), 10.days, '2011-03-19'.toDate, 10.days)",
       "(DATE '2011-03-10', INTERVAL '10' DAY) OVERLAPS (DATE '2011-03-19', INTERVAL '10' DAY)",
       "true")
 
-    testSqlApi(
+    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(toTimestamp('2011-03-10 05:02:02'), 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')",
+          "(TIMESTAMP '2011-03-10 05:02:02', TIMESTAMP '2011-03-10 05:02:01')",
       "true")
 
-    testSqlApi(
+    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')",
+          "(TIMESTAMP '2011-03-10 02:02:02.002', TIMESTAMP '2011-03-10 02:02:02.002')",
       "false")
   }
 
@@ -2394,7 +3237,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
       for ((data,index) <- dataParts.zipWithIndex) {
         index match {
           case 0 => // timestamp, timestamp
-            testSqlApi(
+            testAllApis(
+              timestampDiff(unitParts._2, data._1.toTimestamp, data._2.toTimestamp),
+              s"timestampDiff(${unitParts._1}, '${data._1}'.toTimestamp, '${data._2}'.toTimestamp)",
               s"TIMESTAMPDIFF(${unitParts._1}, TIMESTAMP '${data._1}', TIMESTAMP '${data._2}')",
               data._3
             )
@@ -2403,7 +3248,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
               data._3
             )
           case 1 => // date, timestamp
-            testSqlApi(
+            testAllApis(
+              timestampDiff(unitParts._2, data._1.toDate, data._2.toTimestamp),
+              s"timestampDiff(${unitParts._1}, '${data._1}'.toDate, '${data._2}'.toTimestamp)",
               s"TIMESTAMPDIFF(${unitParts._1}, DATE '${data._1}', TIMESTAMP '${data._2}')",
               data._3
             )
@@ -2412,7 +3259,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
               data._3
             )
           case 2 => // timestamp, date
-            testSqlApi(
+            testAllApis(
+              timestampDiff(unitParts._2, data._1.toTimestamp, data._2.toDate),
+              s"timestampDiff(${unitParts._1}, '${data._1}'.toTimestamp, '${data._2}'.toDate)",
               s"TIMESTAMPDIFF(${unitParts._1}, TIMESTAMP '${data._1}', DATE '${data._2}')",
               data._3
             )
@@ -2421,7 +3270,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
               data._3
             )
           case 3 => // date, date
-            testSqlApi(
+            testAllApis(
+              timestampDiff(unitParts._2, data._1.toDate, data._2.toDate),
+              s"timestampDiff(${unitParts._1}, '${data._1}'.toDate, '${data._2}'.toDate)",
               s"TIMESTAMPDIFF(${unitParts._1}, DATE '${data._1}', DATE '${data._2}')",
               data._3
             )
@@ -2433,12 +3284,18 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
       }
     }
 
-    testSqlApi(
+    testAllApis(
+      timestampDiff(TimePointUnit.DAY, nullOf(DataTypes.TIMESTAMP(3)),
+        "2016-02-24 12:42:25".toTimestamp),
+      "timestampDiff(DAY, Null(SQL_TIMESTAMP), '2016-02-24 12:42:25'.toTimestamp)",
       "TIMESTAMPDIFF(DAY, CAST(NULL AS TIMESTAMP), TIMESTAMP '2016-02-24 12:42:25')",
       "null"
     )
 
-    testSqlApi(
+    testAllApis(
+      timestampDiff(TimePointUnit.DAY, "2016-02-24 12:42:25".toTimestamp,
+        nullOf(DataTypes.TIMESTAMP(3))),
+      "timestampDiff(DAY, '2016-02-24 12:42:25'.toTimestamp,  Null(SQL_TIMESTAMP))",
       "TIMESTAMPDIFF(DAY, TIMESTAMP '2016-02-24 12:42:25',  CAST(NULL AS TIMESTAMP))",
       "null"
     )
@@ -2448,11 +3305,11 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
   def testTimestampAdd(): Unit = {
 
     val data = Seq(
-      (1, "TIMESTAMP '2017-11-29 22:58:58.998'"),
-      (3, "TIMESTAMP '2017-11-29 22:58:58.998'"),
-      (-1, "TIMESTAMP '2017-11-29 22:58:58.998'"),
-      (-61, "TIMESTAMP '2017-11-29 22:58:58.998'"),
-      (-1000, "TIMESTAMP '2017-11-29 22:58:58.998'")
+      (1, "2017-11-29 22:58:58.998"),
+      (3, "2017-11-29 22:58:58.998"),
+      (-1, "2017-11-29 22:58:58.998"),
+      (-61, "2017-11-29 22:58:58.998"),
+      (-1000, "2017-11-29 22:58:58.998")
     )
 
     val YEAR = Seq(
@@ -2531,27 +3388,141 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
       "SQL_TSI_SECOND" -> SECOND
     )
 
+    def intervalCount(interval: String, count: Int): (Expression, String) = interval match {
+      case "YEAR" => (count.years, s"$count.years")
+      case "SQL_TSI_YEAR" => (count.years, s"$count.years")
+      case "QUARTER" => (count.quarters, s"$count.quarters")
+      case "SQL_TSI_QUARTER" => (count.quarters, s"$count.quarters")
+      case "MONTH" => (count.months, s"$count.months")
+      case "SQL_TSI_MONTH" => (count.months, s"$count.months")
+      case "WEEK" => (count.weeks, s"$count.weeks")
+      case "SQL_TSI_WEEK" => (count.weeks, s"$count.weeks")
+      case "DAY" => (count.days, s"$count.days")
+      case "SQL_TSI_DAY" => (count.days, s"$count.days")
+      case "HOUR" => (count.hours, s"$count.hours")
+      case "SQL_TSI_HOUR" => (count.hours, s"$count.hours")
+      case "MINUTE" => (count.minutes, s"$count.minutes")
+      case "SQL_TSI_MINUTE" => (count.minutes, s"$count.minutes")
+      case "SECOND" => (count.seconds, s"$count.seconds")
+      case "SQL_TSI_SECOND" => (count.seconds, s"$count.seconds")
+    }
+
     for ((interval, result) <- intervalMapResults) {
-      testSqlApi(
-        s"TIMESTAMPADD($interval, ${data.head._1}, ${data.head._2})", result.head)
-      testSqlApi(
-        s"TIMESTAMPADD($interval, ${data(1)._1}, ${data(1)._2})", result(1))
-      testSqlApi(
-        s"TIMESTAMPADD($interval, ${data(2)._1}, ${data(2)._2})", result(2))
-      testSqlApi(
-        s"TIMESTAMPADD($interval, ${data(3)._1}, ${data(3)._2})", result(3))
-      testSqlApi(
-        s"TIMESTAMPADD($interval, ${data(4)._1}, ${data(4)._2})", result(4))
+      for (i <- 0 to 4) {
+        val (offset, ts) = data(i)
+        val timeInterval = intervalCount(interval, offset)
+        testAllApis(
+          timeInterval._1 + ts.toTimestamp,
+          s"${timeInterval._2} + '$ts'.toTimestamp",
+          s"TIMESTAMPADD($interval, $offset, TIMESTAMP '$ts')",
+          result(i))
+      }
     }
 
-    testSqlApi("TIMESTAMPADD(HOUR, CAST(NULL AS INTEGER), TIMESTAMP '2016-02-24 12:42:25')", "null")
+    testAllApis(
+      "2016-02-24 12:42:25".toTimestamp + nullOf(DataTypes.INTERVAL(DataTypes.MINUTE())),
+      "'2016-02-24 12:42:25'.toTimestamp + Null(INTERVAL_MILLIS)",
+      "TIMESTAMPADD(HOUR, CAST(NULL AS INTEGER), TIMESTAMP '2016-02-24 12:42:25')",
+      "null")
 
-    testSqlApi("TIMESTAMPADD(HOUR, -200, CAST(NULL AS TIMESTAMP))", "null")
+    testAllApis(
+      nullOf(DataTypes.TIMESTAMP(3)) + -200.hours,
+      "Null(SQL_TIMESTAMP) + -200.hours",
+      "TIMESTAMPADD(HOUR, -200, CAST(NULL AS TIMESTAMP))",
+      "null")
 
-    testSqlApi("TIMESTAMPADD(DAY, 1, DATE '2016-06-15')", "2016-06-16")
+    testAllApis(
+      "2016-06-15".toDate + 1.day,
+      "'2016-06-15'.toDate + 1.day",
+      "TIMESTAMPADD(DAY, 1, DATE '2016-06-15')",
+      "2016-06-16")
 
-    testSqlApi("TIMESTAMPADD(MONTH, 3, CAST(NULL AS TIMESTAMP))", "null")
+    testAllApis(
+      nullOf(DataTypes.TIMESTAMP(3)) + 3.months,
+      "Null(SQL_TIMESTAMP) + 3.months",
+      "TIMESTAMPADD(MONTH, 3, CAST(NULL AS TIMESTAMP))",
+      "null")
+
+    testAllApis(
+      "2016-02-24 12:42:25".toTimestamp + nullOf(Types.INTERVAL_MILLIS),
+      "'2016-02-24 12:42:25'.toTimestamp + nullOf(INTERVAL_MILLIS)",
+      "TIMESTAMPADD(HOUR, CAST(NULL AS INTEGER), TIMESTAMP '2016-02-24 12:42:25')",
+      "null")
+
+    testAllApis(
+      nullOf(Types.SQL_TIMESTAMP) + -200.hours,
+      "nullOf(SQL_TIMESTAMP) + -200.hours",
+      "TIMESTAMPADD(HOUR, -200, CAST(NULL AS TIMESTAMP))",
+      "null")
+
+    testAllApis(
+      nullOf(Types.SQL_TIMESTAMP) + 3.months,
+      "nullOf(SQL_TIMESTAMP) + 3.months",
+      "TIMESTAMPADD(MONTH, 3, CAST(NULL AS TIMESTAMP))",
+      "null")
 
+    // TIMESTAMPADD with DATE returns a TIMESTAMP value for sub-day intervals.
+    testAllApis("2016-06-15".toDate + 1.month,
+      "'2016-06-15'.toDate + 1.month",
+      "timestampadd(MONTH, 1, date '2016-06-15')",
+      "2016-07-15")
+
+    testAllApis("2016-06-15".toDate + 1.day,
+      "'2016-06-15'.toDate + 1.day",
+      "timestampadd(DAY, 1, date '2016-06-15')",
+      "2016-06-16")
+
+    // TODO support '2016-06-15'.toTimestamp
+//    testAllApis("2016-06-15".toTimestamp - 1.hour,
+//      "'2016-06-15'.toTimestamp - 1.hour",
+//      "timestampadd(HOUR, -1, date '2016-06-15')",
+//      "2016-06-14 23:00:00.0")
+
+//    testAllApis("2016-06-15".toTimestamp + 1.minute,
+//      "'2016-06-15'.toTimestamp + 1.minute",
+//      "timestampadd(MINUTE, 1, date '2016-06-15')",
+//      "2016-06-15 00:01:00.0")
+
+//    testAllApis("2016-06-15".toTimestamp - 1.second,
+//      "'2016-06-15'.toTimestamp - 1.second",
+//      "timestampadd(SQL_TSI_SECOND, -1, date '2016-06-15')",
+//      "2016-06-14 23:59:59.0")
+
+//    testAllApis("2016-06-15".toTimestamp + 1.second,
+//      "'2016-06-15'.toTimestamp + 1.second",
+//      "timestampadd(SECOND, 1, date '2016-06-15')",
+//      "2016-06-15 00:00:01.0")
+
+    testAllApis(nullOf(Types.SQL_TIMESTAMP) + 1.second,
+      "nullOf(SQL_TIMESTAMP) + 1.second",
+      "timestampadd(SECOND, 1, cast(null as date))",
+      "null")
+
+    testAllApis(nullOf(Types.SQL_TIMESTAMP) + 1.day,
+      "nullOf(SQL_TIMESTAMP) + 1.day",
+      "timestampadd(DAY, 1, cast(null as date))",
+      "null")
+
+    // Round to the last day of previous month
+    testAllApis("2016-05-31".toDate + 1.month,
+      "'2016-05-31'.toDate + 1.month",
+      "timestampadd(MONTH, 1, date '2016-05-31')",
+      "2016-06-30")
+
+    testAllApis("2016-01-31".toDate + 5.month,
+      "'2016-01-31'.toDate + 5.month",
+      "timestampadd(MONTH, 5, date '2016-01-31')",
+      "2016-06-30")
+
+    testAllApis("2016-03-31".toDate - 1.month,
+      "'2016-03-31'.toDate - 1.month",
+      "timestampadd(MONTH, -1, date '2016-03-31')",
+      "2016-02-29")
+
+    testAllApis("2016-03-31".toDate - 1.week,
+      "'2016-03-31'.toDate - 1.week",
+      "timestampadd(WEEK, -1, date '2016-03-31')",
+      "2016-03-24")
   }
 
   @Test
@@ -2602,93 +3573,131 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
     val expectedSha224 = "90a3ed9e32b2aaf4c61c410eb925426119e1a9dc53d4286ade99a809"
     val expectedSha256 = "9f86d081884c7d659a2feaa0c55ad015a3bf4f1b2b0b822cd15d6c15b0f00a08"
     val expectedSha384 = "768412320f7b0aa5812fce428dc4706b3cae50e02a64caa16a7" +
-      "82249bfe8efc4b7ef1ccb126255d196047dfedf17a0a9"
+        "82249bfe8efc4b7ef1ccb126255d196047dfedf17a0a9"
     val expectedSha512 = "ee26b0dd4af7e749aa1a8ee3c10ae9923f618980772e473f8819a" +
-      "5d4940e0db27ac185f8a0e1d5f84f88bc887fd67b143732c304cc5fa9ad8e6f57f50028a8ff"
+        "5d4940e0db27ac185f8a0e1d5f84f88bc887fd67b143732c304cc5fa9ad8e6f57f50028a8ff"
 
-    testSqlApi(
+    testAllApis(
+      "test".md5(),
+      "md5('test')",
       "MD5('test')",
       expectedMd5)
 
-    testSqlApi(
+    testAllApis(
+      "test".sha1(),
+      "sha1('test')",
       "SHA1('test')",
       expectedSha1)
 
     // sha224
-    testSqlApi(
+    testAllApis(
+      "test".sha224(),
+      "sha224('test')",
       "SHA224('test')",
       expectedSha224)
 
     // sha-2 224
-    testSqlApi(
+    testAllApis(
+      "test".sha2(224),
+      "sha2('test', 224)",
       "SHA2('test', 224)",
       expectedSha224)
 
     // sha256
-    testSqlApi(
+    testAllApis(
+      "test".sha256(),
+      "sha256('test')",
       "SHA256('test')",
       expectedSha256)
 
     // sha-2 256
-    testSqlApi(
+    testAllApis(
+      "test".sha2(256),
+      "sha2('test', 256)",
       "SHA2('test', 256)",
       expectedSha256)
 
     // sha384
-    testSqlApi(
+    testAllApis(
+      "test".sha384(),
+      "sha384('test')",
       "SHA384('test')",
       expectedSha384)
 
     // sha-2 384
-    testSqlApi(
+    testAllApis(
+      "test".sha2(384),
+      "sha2('test', 384)",
       "SHA2('test', 384)",
       expectedSha384)
 
     // sha512
-    testSqlApi(
+    testAllApis(
+      "test".sha512(),
+      "sha512('test')",
       "SHA512('test')",
       expectedSha512)
 
     // sha-2 512
-    testSqlApi(
+    testAllApis(
+      "test".sha2(512),
+      "sha2('test', 512)",
       "SHA2('test', 512)",
       expectedSha512)
 
     // null tests
-    testSqlApi(
+    testAllApis(
+      'f33.md5(),
+      "md5(f33)",
       "MD5(f33)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f33.sha1(),
+      "sha1(f33)",
       "SHA1(f33)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f33.sha224(),
+      "sha224(f33)",
       "SHA2(f33, 224)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f33.sha2(224),
+      "sha2(f33, 224)",
       "SHA2(f33, 224)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f33.sha256(),
+      "sha256(f33)",
       "SHA2(f33, 256)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f33.sha384(),
+      "sha384(f33)",
       "SHA2(f33, 384)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      'f33.sha512(),
+      "sha512(f33)",
       "SHA2(f33, 512)",
       "null")
 
-    testSqlApi(
+    testAllApis(
+      "test".sha2(nullOf(DataTypes.INT)),
+      "sha2('test', Null(INT))",
       "SHA2('test', CAST(NULL AS INT))",
       "null")
 
     // non-constant bit length
-    testSqlApi(
+    testAllApis(
+      "test".sha2('f44),
+      "sha2('test', f44)",
       "SHA2('test', f44)",
       expectedSha256)
   }
@@ -2699,35 +3708,51 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testIsTrueIsFalse(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f1.isTrue,
+      "f1.isTrue",
       "f1 IS TRUE",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f21.isTrue,
+      "f21.isTrue",
       "f21 IS TRUE",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      false.isFalse,
+      "false.isFalse",
       "FALSE IS FALSE",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f21.isFalse,
+      "f21.isFalse",
       "f21 IS FALSE",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f1.isNotTrue,
+      "f1.isNotTrue",
       "f1 IS NOT TRUE",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f21.isNotTrue,
+      "f21.isNotTrue",
       "f21 IS NOT TRUE",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      false.isNotFalse,
+      "false.isNotFalse",
       "FALSE IS NOT FALSE",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f21.isNotFalse,
+      "f21.isNotFalse",
       "f21 IS NOT FALSE",
       "true")
   }
@@ -2750,8 +3775,9 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
     functions.foreach ( x => {
       val tableApiString = x._1.format(field)
       val sqlApiString = x._2.format(field)
-      testSqlApi(
-        // TODO: ignore Table API currently
+      testAllApis(
+        ExpressionParser.parseExpression(tableApiString),
+        tableApiString,
         sqlApiString,
         "null"
       )
@@ -2772,11 +3798,12 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
           |.upperCase()
         """.stripMargin.format(x)
       val sqlApiString = "UPPER(%s)"
-        .format("SUBSTRING(%s, 1, -1)")
-        .format(x)
+          .format("SUBSTRING(%s, 1, -1)")
+          .format(x)
 
-      testSqlApi(
-        // TODO: ignore Table API currently
+      testAllApis(
+        ExpressionParser.parseExpression(tableApiString),
+        tableApiString,
         sqlApiString,
         "null"
       )
@@ -2785,10 +3812,11 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
 
   @Test
   def testNullBigDecimal(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f41.sign(),
+      "f41.sign()",
       "SIGN(f41)",
       "null")
-    testSqlApi("SIGN(f41)", "null")
   }
 
   @Test
@@ -3177,5 +4205,4 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
       "BITNOT(-3)",
       "2")
   }
-
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala
index db80979..b2d72b1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/TemporalTypesTest.scala
@@ -20,7 +20,11 @@ package org.apache.flink.table.planner.expressions
 
 import org.apache.flink.api.common.typeinfo.Types
 import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.DataTypes
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.expressions.TimeIntervalUnit
 import org.apache.flink.table.planner.expressions.utils.ExpressionTestBase
+import org.apache.flink.table.planner.utils.DateTimeTestUtil
 import org.apache.flink.table.planner.utils.DateTimeTestUtil._
 import org.apache.flink.table.typeutils.TimeIntervalTypeInfo
 import org.apache.flink.types.Row
@@ -36,131 +40,249 @@ class TemporalTypesTest extends ExpressionTestBase {
 
   @Test
   def testTimePointLiterals(): Unit = {
-    testSqlApi(
+    testAllApis(
+      "1990-10-14".toDate,
+      "'1990-10-14'.toDate",
       "DATE '1990-10-14'",
       "1990-10-14")
 
-    testSqlApi(
+    testTableApi(
+      localDate2Literal(localDate("2040-09-11")),
+      "'2040-09-11'.toDate",
+      "2040-09-11")
+
+    testAllApis(
+      "1500-04-30".cast(DataTypes.DATE),
+      "'1500-04-30'.cast(SQL_DATE)",
       "CAST('1500-04-30' AS DATE)",
       "1500-04-30")
 
-    testSqlApi(
+    testAllApis(
+      "15:45:59".toTime,
+      "'15:45:59'.toTime",
       "TIME '15:45:59'",
       "15:45:59")
 
-    testSqlApi(
+    testTableApi(
+      localTime2Literal(DateTimeTestUtil.localTime("00:00:00")),
+      "'00:00:00'.toTime",
+      "00:00:00")
+
+    testAllApis(
+      "1:30:00".cast(DataTypes.TIME),
+      "'1:30:00'.cast(SQL_TIME)",
       "CAST('1:30:00' AS TIME)",
       "01:30:00")
 
-    testSqlApi(
+    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")
 
-    testSqlApi(
+    testTableApi(
+      localDateTime2Literal(localDateTime("2040-09-11 00:00:00.000")),
+      "'2040-09-11 00:00:00.000'.toTimestamp",
+      "2040-09-11 00:00:00.000")
+
+    testAllApis(
+      "1500-04-30 12:00:00".cast(DataTypes.TIMESTAMP(3)),
+      "'1500-04-30 12:00:00'.cast(SQL_TIMESTAMP)",
       "CAST('1500-04-30 12:00:00' AS TIMESTAMP)",
       "1500-04-30 12:00:00.000")
   }
 
   @Test
   def testTimeIntervalLiterals(): Unit = {
-    testSqlApi(
+    testAllApis(
+      1.year,
+      "1.year",
       "INTERVAL '1' YEAR",
       "+1-00")
 
-    testSqlApi(
+    testAllApis(
+      1.month,
+      "1.month",
       "INTERVAL '1' MONTH",
       "+0-01")
 
-    testSqlApi(
+    testAllApis(
+      12.days,
+      "12.days",
       "INTERVAL '12' DAY",
       "+12 00:00:00.000")
 
-    testSqlApi(
+    testAllApis(
+      1.hour,
+      "1.hour",
       "INTERVAL '1' HOUR",
       "+0 01:00:00.000")
 
-    testSqlApi(
+    testAllApis(
+      3.minutes,
+      "3.minutes",
       "INTERVAL '3' MINUTE",
       "+0 00:03:00.000")
 
-    testSqlApi(
+    testAllApis(
+      3.seconds,
+      "3.seconds",
       "INTERVAL '3' SECOND",
       "+0 00:00:03.000")
 
-    testSqlApi(
+    testAllApis(
+      3.millis,
+      "3.millis",
       "INTERVAL '0.003' SECOND",
       "+0 00:00:00.003")
   }
 
   @Test
   def testTimePointInput(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0,
+      "f0",
       "f0",
       "1990-10-14")
 
-    testSqlApi(
+    testAllApis(
+      'f1,
+      "f1",
       "f1",
       "10:20:45")
 
-    testSqlApi(
+    testAllApis(
+      'f2,
+      "f2",
       "f2",
       "1990-10-14 10:20:45.123")
   }
 
   @Test
   def testTimeIntervalInput(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f9,
+      "f9",
       "f9",
       "+2-00")
 
-    testSqlApi(
+    testAllApis(
+      'f10,
+      "f10",
       "f10",
       "+0 00:00:12.000")
   }
 
   @Test
   def testTimePointCasting(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0.cast(DataTypes.TIMESTAMP(3)),
+      "f0.cast(SQL_TIMESTAMP)",
       "CAST(f0 AS TIMESTAMP)",
       "1990-10-14 00:00:00.000")
 
-    testSqlApi(
+    testAllApis(
+      'f1.cast(DataTypes.TIMESTAMP(3)),
+      "f1.cast(SQL_TIMESTAMP)",
       "CAST(f1 AS TIMESTAMP)",
       "1970-01-01 10:20:45.000")
 
-    testSqlApi(
+    testAllApis(
+      'f2.cast(DataTypes.DATE),
+      "f2.cast(SQL_DATE)",
       "CAST(f2 AS DATE)",
       "1990-10-14")
 
-    testSqlApi(
+    testAllApis(
+      'f2.cast(DataTypes.TIME),
+      "f2.cast(SQL_TIME)",
       "CAST(f2 AS TIME)",
       "10:20:45")
 
-    testSqlApi(
+    testAllApis(
+      'f2.cast(DataTypes.TIME),
+      "f2.cast(SQL_TIME)",
       "CAST(f2 AS TIME)",
       "10:20:45")
 
+    testTableApi(
+      'f7.cast(DataTypes.DATE),
+      "f7.cast(SQL_DATE)",
+      "2002-11-09")
+
+    testTableApi(
+      'f7.cast(DataTypes.DATE).cast(DataTypes.INT),
+      "f7.cast(SQL_DATE).cast(INT)",
+      "12000")
+
+    testTableApi(
+      'f7.cast(DataTypes.TIME),
+      "f7.cast(SQL_TIME)",
+      "00:00:12")
+
+    testTableApi(
+      'f7.cast(DataTypes.TIME).cast(DataTypes.INT),
+      "f7.cast(SQL_TIME).cast(INT)",
+      "12000")
+
+    testTableApi(
+      'f15.cast(DataTypes.TIMESTAMP(3)),
+      "f15.cast(SQL_TIMESTAMP)",
+      "2016-06-27 07:23:33.000")
+
+    testTableApi(
+      'f15.toTimestamp,
+      "f15.toTimestamp",
+      "2016-06-27 07:23:33.000")
+
+    testTableApi(
+      'f8.cast(DataTypes.TIMESTAMP(3)).cast(DataTypes.BIGINT()),
+      "f8.cast(SQL_TIMESTAMP).cast(LONG)",
+      "1467012213000")
+  }
+
+  @Test
+  def testTimeIntervalCasting(): Unit = {
+    testTableApi(
+      'f7.cast(DataTypes.INTERVAL(DataTypes.MONTH)),
+      "f7.cast(INTERVAL_MONTHS)",
+      "+1000-00")
+
+    testTableApi(
+      'f8.cast(DataTypes.INTERVAL(DataTypes.MINUTE())),
+      "f8.cast(INTERVAL_MILLIS)",
+      "+16979 07:23:33.000")
   }
 
   @Test
   def testTimePointComparison(): Unit = {
-    testSqlApi(
+    testAllApis(
+      'f0 < 'f3,
+      "f0 < f3",
       "f0 < f3",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f0 < 'f4,
+      "f0 < f4",
       "f0 < f4",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f1 < 'f5,
+      "f1 < f5",
       "f1 < f5",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      'f0.cast(DataTypes.TIMESTAMP(3)) !== 'f2,
+      "f0.cast(SQL_TIMESTAMP) !== f2",
       "CAST(f0 AS TIMESTAMP) <> f2",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      'f0.cast(DataTypes.TIMESTAMP(3)) === 'f6,
+      "f0.cast(SQL_TIMESTAMP) === f6",
       "CAST(f0 AS TIMESTAMP) = f6",
       "true")
   }
@@ -170,204 +292,282 @@ class TemporalTypesTest extends ExpressionTestBase {
 
     // interval months comparison
 
-    testSqlApi(
+    testAllApis(
+      12.months < 24.months,
+      "12.months < 24.months",
       "INTERVAL '12' MONTH < INTERVAL '24' MONTH",
       "true")
 
-    testSqlApi(
+    testAllApis(
+      8.years === 8.years,
+      "8.years === 8.years",
       "INTERVAL '8' YEAR = INTERVAL '8' YEAR",
       "true")
 
     // interval millis comparison
 
-    testSqlApi(
+    testAllApis(
+      8.millis > 10.millis,
+      "8.millis > 10.millis",
       "INTERVAL '0.008' SECOND > INTERVAL '0.010' SECOND",
       "false")
 
-    testSqlApi(
+    testAllApis(
+      8.millis === 8.millis,
+      "8.millis === 8.millis",
       "INTERVAL '0.008' SECOND = INTERVAL '0.008' SECOND",
       "true")
 
     // interval months addition/subtraction
 
-    testSqlApi(
+    testAllApis(
+      8.years + 10.months,
+      "8.years + 10.months",
       "INTERVAL '8' YEAR + INTERVAL '10' MONTH",
       "+8-10")
 
-    testSqlApi(
+    testAllApis(
+      2.years - 12.months,
+      "2.years - 12.months",
       "INTERVAL '2' YEAR - INTERVAL '12' MONTH",
       "+1-00")
 
-    testSqlApi(
+    testAllApis(
+      -2.years,
+      "-2.years",
       "-INTERVAL '2' YEAR",
       "-2-00")
 
     // interval millis addition/subtraction
 
-    testSqlApi(
+    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")
 
-    testSqlApi(
+    testAllApis(
+      1.minute - 10.seconds,
+      "1.minute - 10.seconds",
       "INTERVAL '1' MINUTE - INTERVAL '10' SECOND",
       "+0 00:00:50.000")
 
-    testSqlApi(
+    testAllApis(
+      -10.seconds,
+      "-10.seconds",
       "-INTERVAL '10' SECOND",
       "-0 00:00:10.000")
 
     // addition to date
 
     // interval millis
-    testSqlApi(
+    testAllApis(
+      'f0 + 2.days,
+      "f0 + 2.days",
       "f0 + INTERVAL '2' DAY",
       "1990-10-16")
 
     // interval millis
-    testSqlApi(
+    testAllApis(
+      30.days + 'f0,
+      "30.days + f0",
       "INTERVAL '30' DAY + f0",
       "1990-11-13")
 
     // interval months
-    testSqlApi(
+    testAllApis(
+      'f0 + 2.months,
+      "f0 + 2.months",
       "f0 + INTERVAL '2' MONTH",
       "1990-12-14")
 
     // interval months
-    testSqlApi(
+    testAllApis(
+      2.months + 'f0,
+      "2.months + f0",
       "INTERVAL '2' MONTH + f0",
       "1990-12-14")
 
     // addition to time
 
     // interval millis
-    testSqlApi(
+    testAllApis(
+      'f1 + 12.hours,
+      "f1 + 12.hours",
       "f1 + INTERVAL '12' HOUR",
       "22:20:45")
 
     // interval millis
-    testSqlApi(
+    testAllApis(
+      12.hours + 'f1,
+      "12.hours + f1",
       "INTERVAL '12' HOUR + f1",
       "22:20:45")
 
     // addition to timestamp
 
     // interval millis
-    testSqlApi(
+    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
-    testSqlApi(
+    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
-    testSqlApi(
+    testAllApis(
+      'f2 + 10.years,
+      "f2 + 10.years",
       "f2 + INTERVAL '10' YEAR",
       "2000-10-14 10:20:45.123")
 
     // interval months
-    testSqlApi(
+    testAllApis(
+      10.years + 'f2,
+      "10.years + f2",
       "INTERVAL '10' YEAR + f2",
       "2000-10-14 10:20:45.123")
 
     // subtraction from date
 
     // interval millis
-    testSqlApi(
+    testAllApis(
+      'f0 - 2.days,
+      "f0 - 2.days",
       "f0 - INTERVAL '2' DAY",
       "1990-10-12")
 
     // interval millis
-    testSqlApi(
+    testAllApis(
+      -30.days + 'f0,
+      "-30.days + f0",
       "INTERVAL '-30' DAY + f0",
       "1990-09-14")
 
     // interval months
-    testSqlApi(
+    testAllApis(
+      'f0 - 2.months,
+      "f0 - 2.months",
       "f0 - INTERVAL '2' MONTH",
       "1990-08-14")
 
     // interval months
-    testSqlApi(
+    testAllApis(
+      -2.months + 'f0,
+      "-2.months + f0",
       "-INTERVAL '2' MONTH + f0",
       "1990-08-14")
 
     // subtraction from time
 
     // interval millis
-    testSqlApi(
+    testAllApis(
+      'f1 - 12.hours,
+      "f1 - 12.hours",
       "f1 - INTERVAL '12' HOUR",
       "22:20:45")
 
     // interval millis
-    testSqlApi(
+    testAllApis(
+      -12.hours + 'f1,
+      "-12.hours + f1",
       "INTERVAL '-12' HOUR + f1",
       "22:20:45")
 
     // subtraction from timestamp
 
     // interval millis
-    testSqlApi(
+    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
-    testSqlApi(
+    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
-    testSqlApi(
+    testAllApis(
+      'f2 - 10.years,
+      "f2 - 10.years",
       "f2 - INTERVAL '10' YEAR",
       "1980-10-14 10:20:45.123")
 
     // interval months
-    testSqlApi(
+    testAllApis(
+      -10.years + 'f2,
+      "-10.years + f2",
       "INTERVAL '-10' YEAR + f2",
       "1980-10-14 10:20:45.123")
 
     // casting
 
-    testSqlApi(
+    testAllApis(
+      -'f9.cast(DataTypes.INTERVAL(DataTypes.MONTH)),
+      "-f9.cast(INTERVAL_MONTHS)",
       "-CAST(f9 AS INTERVAL YEAR)",
       "-2-00")
 
-    testSqlApi(
+    testAllApis(
+      -'f10.cast(DataTypes.INTERVAL(DataTypes.MINUTE())),
+      "-f10.cast(INTERVAL_MILLIS)",
       "-CAST(f10 AS INTERVAL SECOND)",
       "-0 00:00:12.000")
 
     // addition/subtraction of interval millis and interval months
 
-    testSqlApi(
+    testAllApis(
+      'f0 + 2.days + 1.month,
+      "f0 + 2.days + 1.month",
       "f0 + INTERVAL '2' DAY + INTERVAL '1' MONTH",
       "1990-11-16")
 
-    testSqlApi(
+    testAllApis(
+      'f0 - 2.days - 1.month,
+      "f0 - 2.days - 1.month",
       "f0 - INTERVAL '2' DAY - INTERVAL '1' MONTH",
       "1990-09-12")
 
-    testSqlApi(
+    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")
 
-    testSqlApi(
+    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")
   }
 
   @Test
   def testSelectNullValues(): Unit ={
-    testSqlApi(
+    testAllApis(
+      'f11,
+      "f11",
       "f11",
       "null"
     )
-    testSqlApi(
+    testAllApis(
+      'f12,
+      "f12",
       "f12",
       "null"
     )
-    testSqlApi(
+    testAllApis(
+      'f13,
+      "f13",
       "f13",
       "null"
     )
@@ -375,12 +575,16 @@ class TemporalTypesTest extends ExpressionTestBase {
 
   @Test
   def testTemporalNullValues() = {
-    testSqlApi(
+    testAllApis(
+      'f13.extract(TimeIntervalUnit.HOUR),
+      "f13.extract(HOUR)",
       "extract(HOUR FROM f13)",
       "null"
     )
 
-    testSqlApi(
+    testAllApis(
+      'f13.floor(TimeIntervalUnit.HOUR),
+      "f13.floor(HOUR)",
       "FLOOR(f13 TO HOUR)",
       "null"
     )
@@ -402,6 +606,14 @@ class TemporalTypesTest extends ExpressionTestBase {
   }
 
   @Test
+  def testdebug() = {
+    testSqlApi("DATE_FORMAT('2018-03-14 01:02:03', 'yyyy/MM/dd HH:mm:ss')",
+      "2018/03/14 01:02:03")
+    testSqlApi("DATE_FORMAT('2018-03-14 01:02:03', 'yyyy-MM-dd HH:mm:ss', " +
+        "'yyyy/MM/dd HH:mm:ss')", "2018/03/14 01:02:03")
+  }
+
+  @Test
   def testDateAndTime(): Unit = {
     testSqlApi(
       "DATE '2018-03-14'",
@@ -675,8 +887,7 @@ class TemporalTypesTest extends ExpressionTestBase {
 
     cases.foreach {
       caseExpr =>
-        testSqlApi(
-          s"CASE WHEN ($caseExpr) is null THEN '$nullable' ELSE '$notNullable' END", nullable)
+        testSqlNullable(caseExpr)
     }
   }
 
@@ -702,11 +913,11 @@ class TemporalTypesTest extends ExpressionTestBase {
   override def testData: Row = {
     val testData = new Row(21)
     testData.setField(0, localDate("1990-10-14"))
-    testData.setField(1, localTime("10:20:45"))
+    testData.setField(1, DateTimeTestUtil.localTime("10:20:45"))
     testData.setField(2, localDateTime("1990-10-14 10:20:45.123"))
     testData.setField(3, localDate("1990-10-13"))
     testData.setField(4, localDate("1990-10-15"))
-    testData.setField(5, localTime("00:00:00"))
+    testData.setField(5, DateTimeTestUtil.localTime("00:00:00"))
     testData.setField(6, localDateTime("1990-10-14 00:00:00.0"))
     testData.setField(7, 12000)
     testData.setField(8, 1467012213000L)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala
new file mode 100644
index 0000000..01585ef
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/UserDefinedScalarFunctionTest.scala
@@ -0,0 +1,485 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.expressions
+
+import org.apache.flink.api.common.typeinfo.{BasicArrayTypeInfo, TypeInformation}
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.{DataTypes, Types, ValidationException}
+import org.apache.flink.table.functions.ScalarFunction
+import org.apache.flink.table.planner.expressions.utils.{ExpressionTestBase, _}
+import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions._
+import org.apache.flink.table.planner.utils.DateTimeTestUtil
+import org.apache.flink.types.Row
+
+import org.junit.Test
+
+import java.lang.{Boolean => JBoolean}
+
+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(
+      Func1('f11),
+      "Func1(f11)",
+      "Func1(f11)",
+      "4")
+
+    testAllApis(
+      Func1('f12),
+      "Func1(f12)",
+      "Func1(f12)",
+      "4")
+
+    testAllApis(
+      Func1('f13),
+      "Func1(f13)",
+      "Func1(f13)",
+      "4.0")
+
+    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")
+
+    // TODO: GenericType with Date/Time/Timestamp -> String would call toString implicitl
+    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)")
+
+    // function names containing keywords
+    testAllApis(
+      Func0('f0),
+      "getFunc0(f0)",
+      "getFunc0(f0)",
+      "42")
+
+    testAllApis(
+      Func0('f0),
+      "asAlways(f0)",
+      "asAlways(f0)",
+      "42")
+
+    testAllApis(
+      Func0('f0),
+      "toWhatever(f0)",
+      "toWhatever(f0)",
+      "42")
+
+    testAllApis(
+      Func0('f0),
+      "Nullable(f0)",
+      "Nullable(f0)",
+      "42")
+
+    // test row type input
+    testAllApis(
+      Func20('f14),
+      "Func20(f14)",
+      "Func20(f14)",
+      "(12,true,(1,2,3))"
+    )
+  }
+
+  @Test
+  def testNullableParameters(): Unit = {
+    testAllApis(
+      Func3(nullOf(DataTypes.INT), nullOf(DataTypes.STRING)),
+      "Func3(Null(INT), Null(STRING))",
+      "Func3(NULL, NULL)",
+      "null and null")
+
+    testAllApis(
+      Func3(nullOf(DataTypes.INT), "Test"),
+      "Func3(Null(INT), 'Test')",
+      "Func3(NULL, 'Test')",
+      "null and Test")
+
+    testAllApis(
+      Func3(42, nullOf(DataTypes.STRING)),
+      "Func3(42, Null(STRING))",
+      "Func3(42, NULL)",
+      "42 and null")
+
+    testAllApis(
+      Func0(nullOf(DataTypes.INT)),
+      "Func0(Null(INT))",
+      "Func0(NULL)",
+      "-1")
+  }
+
+  @Test
+  def testDoubleQuoteParameters(): Unit = {
+    val hello = "\"<hello>\""
+    testAllApis(
+      Func3(42, hello),
+      s"Func3(42, '$hello')",
+      s"Func3(42, '$hello')",
+      s"42 and $hello")
+  }
+
+  @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")
+
+    // TODO fix FLINK-13580 to blink-planner
+//    testAllApis(
+//      Func21('f15),
+//      "Func21(f15)",
+//      "Func21(f15)",
+//      "student#Bob")
+//
+//    testAllApis(
+//      Func22('f16),
+//      "Func22(f16)",
+//      "Func22(f16)",
+//      "student#Bob")
+  }
+
+  @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.000")
+  }
+
+  @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")
+  }
+  
+  @Test
+  def testVariableArgs(): Unit = {
+    testAllApis(
+      Func14(1, 2, 3, 4),
+      "Func14(1, 2, 3, 4)",
+      "Func14(1, 2, 3, 4)",
+      "10")
+
+    // Test for empty arguments
+    testAllApis(
+      Func14(),
+      "Func14()",
+      "Func14()",
+      "0")
+
+    // Test for override
+    testAllApis(
+      Func15("Hello"),
+      "Func15('Hello')",
+      "Func15('Hello')",
+      "Hello"
+    )
+
+    testAllApis(
+      Func15('f1),
+      "Func15(f1)",
+      "Func15(f1)",
+      "Test"
+    )
+
+    testAllApis(
+      Func15("Hello", 1, 2, 3),
+      "Func15('Hello', 1, 2, 3)",
+      "Func15('Hello', 1, 2, 3)",
+      "Hello3"
+    )
+
+    testAllApis(
+      Func16('f9),
+      "Func16(f9)",
+      "Func16(f9)",
+      "Hello, World"
+    )
+
+    try {
+      testAllApis(
+        Func17("Hello", "World"),
+        "Func17('Hello', 'World')",
+        "Func17('Hello', 'World')",
+        "Hello, World"
+      )
+      throw new RuntimeException("Shouldn't be reached here!")
+    } catch {
+      case ex: ValidationException =>
+        // ok
+    }
+
+    val JavaFunc2 = new JavaFunc2
+    testAllApis(
+      JavaFunc2("Hi", 1, 3, 5, 7),
+      "JavaFunc2('Hi', 1, 3, 5, 7)",
+      "JavaFunc2('Hi', 1, 3, 5, 7)",
+      "Hi105")
+
+    // test overloading
+    val JavaFunc3 = new JavaFunc3
+    testAllApis(
+      JavaFunc3("Hi"),
+      "JavaFunc3('Hi')",
+      "JavaFunc3('Hi')",
+      "Hi")
+
+    testAllApis(
+      JavaFunc3('f1),
+      "JavaFunc3(f1)",
+      "JavaFunc3(f1)",
+      "Test")
+  }
+
+  @Test
+  def testJavaBoxedPrimitives(): Unit = {
+    val JavaFunc0 = new JavaFunc0()
+    val JavaFunc1 = new JavaFunc1()
+    val JavaFunc4 = new JavaFunc4()
+
+    testAllApis(
+      JavaFunc0('f8),
+      "JavaFunc0(f8)",
+      "JavaFunc0(f8)",
+      "1001"
+    )
+
+    testTableApi(
+      JavaFunc0(1000L),
+      "JavaFunc0(1000L)",
+      "1001"
+    )
+
+    testAllApis(
+      JavaFunc1('f4, 'f5, 'f6),
+      "JavaFunc1(f4, f5, f6)",
+      "JavaFunc1(f4, f5, f6)",
+      "7591 and 43810000 and 655906210000")
+
+    testAllApis(
+      JavaFunc1(nullOf(DataTypes.TIME), 15, nullOf(DataTypes.TIMESTAMP(3))),
+      "JavaFunc1(Null(SQL_TIME), 15, Null(SQL_TIMESTAMP))",
+      "JavaFunc1(NULL, 15, NULL)",
+      "null and 15 and null")
+
+    testAllApis(
+      JavaFunc4('f10, array("a", "b", "c")),
+      "JavaFunc4(f10, array('a', 'b', 'c'))",
+      "JavaFunc4(f10, array['a', 'b', 'c'])",
+      "[1, 2, null] and [a, b, c]"
+    )
+  }
+
+  @Test
+  def testRichFunctions(): Unit = {
+    val richFunc0 = new RichFunc0
+    val richFunc1 = new RichFunc1
+    val richFunc2 = new RichFunc2
+    testAllApis(
+      richFunc0('f0),
+      "RichFunc0(f0)",
+      "RichFunc0(f0)",
+      "43")
+
+    testAllApis(
+      richFunc1('f0),
+      "RichFunc1(f0)",
+      "RichFunc1(f0)",
+      "42")
+
+    testAllApis(
+      richFunc2('f1),
+      "RichFunc2(f1)",
+      "RichFunc2(f1)",
+      "#Test")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+
+  override def testData: Row = {
+    val testData = new Row(17)
+    testData.setField(0, 42)
+    testData.setField(1, "Test")
+    testData.setField(2, null)
+    testData.setField(3, SimplePojo("Bob", 36))
+    testData.setField(4, DateTimeTestUtil.localDate("1990-10-14"))
+    testData.setField(5, DateTimeTestUtil.localTime("12:10:10"))
+    testData.setField(6, DateTimeTestUtil.localDateTime("1990-10-14 12:10:10"))
+    testData.setField(7, 12)
+    testData.setField(8, 1000L)
+    testData.setField(9, Seq("Hello", "World"))
+    testData.setField(10, Array[Integer](1, 2, null))
+    testData.setField(11, 3.toByte)
+    testData.setField(12, 3.toShort)
+    testData.setField(13, 3.toFloat)
+    testData.setField(14, Row.of(
+      12.asInstanceOf[Integer],
+      true.asInstanceOf[JBoolean],
+      Row.of(1.asInstanceOf[Integer], 2.asInstanceOf[Integer], 3.asInstanceOf[Integer]))
+    )
+    testData.setField(15, new GraduatedStudent("Bob"))
+    testData.setField(16, Array(new GraduatedStudent("Bob")))
+    testData
+  }
+
+  override def typeInfo: RowTypeInfo = {
+    new RowTypeInfo(
+      Types.INT,
+      Types.STRING,
+      Types.BOOLEAN,
+      TypeInformation.of(classOf[SimplePojo]),
+      Types.LOCAL_DATE,
+      Types.LOCAL_TIME,
+      Types.LOCAL_DATE_TIME,
+      Types.INTERVAL_MONTHS,
+      Types.INTERVAL_MILLIS,
+      TypeInformation.of(classOf[Seq[String]]),
+      BasicArrayTypeInfo.INT_ARRAY_TYPE_INFO,
+      Types.BYTE,
+      Types.SHORT,
+      Types.FLOAT,
+      Types.ROW(Types.INT, Types.BOOLEAN, Types.ROW(Types.INT, Types.INT, Types.INT)),
+      TypeInformation.of(classOf[GraduatedStudent]),
+      TypeInformation.of(classOf[Array[GraduatedStudent]])
+    )
+  }
+
+  override def functions: Map[String, ScalarFunction] = Map(
+    "Func0" -> Func0,
+    "getFunc0" -> Func0,
+    "asAlways" -> Func0,
+    "toWhatever" -> Func0,
+    "Nullable" -> 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,
+    "Func14" -> Func14,
+    "Func15" -> Func15,
+    "Func16" -> Func16,
+    "Func17" -> Func17,
+    "Func19" -> Func19,
+    "Func20" -> Func20,
+    "JavaFunc0" -> new JavaFunc0,
+    "JavaFunc1" -> new JavaFunc1,
+    "JavaFunc2" -> new JavaFunc2,
+    "JavaFunc3" -> new JavaFunc3,
+    "JavaFunc4" -> new JavaFunc4,
+    "RichFunc0" -> new RichFunc0,
+    "RichFunc1" -> new RichFunc1,
+    "RichFunc2" -> new RichFunc2
+  )
+}
+
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala
index 9b7bd4a..02cc054 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala
@@ -26,10 +26,14 @@ import org.apache.flink.configuration.Configuration
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
 import org.apache.flink.table.api.internal.TableEnvironmentImpl
 import org.apache.flink.table.api.java.internal.StreamTableEnvironmentImpl
+import org.apache.flink.table.api.scala._
 import org.apache.flink.table.api.{EnvironmentSettings, TableConfig}
 import org.apache.flink.table.dataformat.{BaseRow, BinaryRow, DataFormatConverters}
+import org.apache.flink.table.expressions.{Expression, ExpressionParser}
+import org.apache.flink.table.functions.ScalarFunction
 import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, ExprCodeGenerator, FunctionCodeGenerator}
 import org.apache.flink.table.planner.delegation.PlannerBase
+import org.apache.flink.table.planner.expressions.ExpressionBuilder
 import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType
 import org.apache.flink.table.types.DataType
 import org.apache.flink.table.types.logical.{RowType, VarCharType}
@@ -77,10 +81,13 @@ abstract class ExpressionTestBase {
   @Rule
   def thrown: ExpectedException = expectedException
 
+  def functions: Map[String, ScalarFunction] = Map()
+
   @Before
   def prepare(): Unit = {
     val ds = env.fromCollection(Collections.emptyList[Row](), typeInfo)
     tEnv.registerDataStream(tableName, ds)
+    functions.foreach(f => tEnv.registerFunction(f._1, f._2))
 
     // prepare RelBuilder
     relBuilder.scan(tableName)
@@ -161,12 +168,13 @@ abstract class ExpressionTestBase {
             null
           }
 
+          val original = if (originalExpr == null) "" else s"for: [$originalExpr]"
+
           assertEquals(
-            s"Wrong result for: [$originalExpr] optimized to: [$optimizedExpr]",
+            s"Wrong result $original optimized to: [$optimizedExpr]",
             expected,
             if (actual == null) "null" else actual)
       }
-
   }
 
   private def addSqlTestExpr(sqlExpr: String, expected: String): Unit = {
@@ -174,11 +182,14 @@ abstract class ExpressionTestBase {
     val parsed = calcitePlanner.parse(s"SELECT $sqlExpr FROM $tableName")
     val validated = calcitePlanner.validate(parsed)
     val converted = calcitePlanner.rel(validated).rel
+    addTestExpr(converted, expected, sqlExpr)
+  }
 
+  private def addTestExpr(relNode: RelNode, expected: String, summaryString: String): Unit = {
     val builder = new HepProgramBuilder()
     builder.addRuleInstance(ProjectToCalcRule.INSTANCE)
     val hep = new HepPlanner(builder.build())
-    hep.setRoot(converted)
+    hep.setRoot(relNode)
     val optimized = hep.findBestExp()
 
     // throw exception if plan contains more than a calc
@@ -186,7 +197,7 @@ abstract class ExpressionTestBase {
       fail("Expression is converted into more than a Calc operation. Use a different test method.")
     }
 
-    testExprs += ((sqlExpr, extractRexNode(optimized), expected))
+    testExprs += ((summaryString, extractRexNode(optimized), expected))
   }
 
   private def extractRexNode(node: RelNode): RexNode = {
@@ -196,10 +207,62 @@ abstract class ExpressionTestBase {
     calcProgram.expandLocalRef(calcProgram.getProjectList.get(0))
   }
 
+  def testAllApis(
+      expr: Expression,
+      exprString: String,
+      sqlExpr: String,
+      expected: String): Unit = {
+    addTableApiTestExpr(expr, expected)
+    addTableApiTestExpr(exprString, expected)
+    addSqlTestExpr(sqlExpr, expected)
+    if (expected == nullable) {
+      testTableNullable(expr, exprString)
+      testSqlNullable(sqlExpr)
+    }
+  }
+
+  def testTableApi(
+      expr: Expression,
+      exprString: String,
+      expected: String): Unit = {
+    addTableApiTestExpr(expr, expected)
+    addTableApiTestExpr(exprString, expected)
+    if (expected == nullable) {
+      testTableNullable(expr, exprString)
+    }
+  }
+
+  private def addTableApiTestExpr(tableApiString: String, expected: String): Unit = {
+    addTableApiTestExpr(ExpressionParser.parseExpression(tableApiString), expected)
+  }
+
+  private def addTableApiTestExpr(tableApiExpr: Expression, expected: String): Unit = {
+    // create RelNode from Table API expression
+    val relNode = relBuilder
+        .queryOperation(tEnv.scan(tableName).select(tableApiExpr).getQueryOperation).build()
+    addTestExpr(relNode, expected, tableApiExpr.asSummaryString())
+  }
+
+  def testSqlNullable(nullUdf: String): Unit = {
+    addSqlTestExpr(
+      s"CASE WHEN ($nullUdf) is null THEN '$nullable' ELSE '$notNullable' END", nullable)
+  }
+
   def testSqlApi(
       sqlExpr: String,
       expected: String): Unit = {
     addSqlTestExpr(sqlExpr, expected)
+    if (expected == nullable) {
+      testSqlNullable(sqlExpr)
+    }
+  }
+
+  def testTableNullable(nullExpr: Expression, nullExprString: String): Unit = {
+    val retExpr = ExpressionBuilder.ifThenElse(nullExpr.isNull, nullable, notNullable)
+    addTableApiTestExpr(retExpr, nullable)
+    val retStrExpr = ifThenElse(
+      ExpressionParser.parseExpression(nullExprString).isNull, nullable, notNullable)
+    addTableApiTestExpr(retStrExpr, nullable)
   }
 
   def testData: Row
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala
index 116c40b..7a3b224 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ScalarOperatorsTestBase.scala
@@ -21,6 +21,7 @@ package org.apache.flink.table.planner.expressions.utils
 import org.apache.flink.api.common.typeinfo.Types
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.dataformat.Decimal
+import org.apache.flink.table.functions.ScalarFunction
 import org.apache.flink.table.planner.utils.DateTimeTestUtil._
 import org.apache.flink.table.runtime.typeutils.DecimalTypeInfo
 import org.apache.flink.types.Row
@@ -72,4 +73,8 @@ abstract class ScalarOperatorsTestBase extends ExpressionTestBase {
       /* 17 */ DecimalTypeInfo.of(19, 1)
     )
   }
+
+  override def functions: Map[String, ScalarFunction] = Map(
+    "shouldNotExecuteFunc" -> ShouldNotExecuteFunc
+  )
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala
index 485878a..9faed1e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/userDefinedScalarFunctions.scala
@@ -342,6 +342,28 @@ object Func20 extends ScalarFunction {
 }
 
 @SerialVersionUID(1L)
+object Func21 extends ScalarFunction {
+  def eval(p: People): String = {
+    p.name
+  }
+
+  def eval(p: Student): String = {
+    "student#" + p.name
+  }
+}
+
+@SerialVersionUID(1L)
+object Func22 extends ScalarFunction {
+  def eval(a: Array[People]): String = {
+    a.head.name
+  }
+
+  def eval(a: Array[Student]): String = {
+    "student#" + a.head.name
+  }
+}
+
+@SerialVersionUID(1L)
 object Func23 extends ScalarFunction {
   def eval(a: Integer, b: JLong, c: String): Row = {
     Row.of("star", a, b, c)
@@ -410,3 +432,9 @@ class SplitUDF(deterministic: Boolean) extends ScalarFunction {
   override def isDeterministic: Boolean = deterministic
 
 }
+
+class People(val name: String)
+
+class Student(name: String) extends People(name)
+
+class GraduatedStudent(name: String) extends Student(name)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala
index 23ec7ba..86dbe45 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ArrayTypeValidationTest.scala
@@ -19,6 +19,7 @@
 package org.apache.flink.table.planner.expressions.validation
 
 import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.expressions.utils.ArrayTypeTestBase
 
 import org.junit.Test
@@ -26,16 +27,31 @@ import org.junit.Test
 class ArrayTypeValidationTest extends ArrayTypeTestBase {
 
   @Test(expected = classOf[ValidationException])
+  def testImplicitTypeCastTableApi(): Unit = {
+    testTableApi(array(1.0, 2.0f), "FAIL", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
   def testImplicitTypeCastArraySql(): Unit = {
     testSqlApi("ARRAY['string', 12]", "FAIL")
   }
 
   @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")
   }
@@ -46,6 +62,28 @@ class ArrayTypeValidationTest extends ArrayTypeTestBase {
   }
 
   @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)",
@@ -53,6 +91,11 @@ class ArrayTypeValidationTest extends ArrayTypeTestBase {
   }
 
   @Test(expected = classOf[ValidationException])
+  def testCardinalityOnNonArray(): Unit = {
+    testTableApi('f0.cardinality(), "FAIL", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
   def testCardinalityOnNonArraySql(): Unit = {
     testSqlApi("CARDINALITY(f0)", "FAIL")
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/CompositeAccessValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/CompositeAccessValidationTest.scala
index 3657fce..687700e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/CompositeAccessValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/CompositeAccessValidationTest.scala
@@ -19,6 +19,7 @@
 package org.apache.flink.table.planner.expressions.validation
 
 import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.expressions.utils.CompositeTypeTestBase
 
 import org.junit.Test
@@ -34,6 +35,26 @@ class CompositeAccessValidationTest extends CompositeTypeTestBase {
   def testWrongSqlField(): Unit = {
     testSqlApi("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")
+  }
 }
 
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/MapTypeValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/MapTypeValidationTest.scala
index 05c01f8..52a62c7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/MapTypeValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/MapTypeValidationTest.scala
@@ -19,6 +19,7 @@
 package org.apache.flink.table.planner.expressions.validation
 
 import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.expressions.utils.MapTypeTestBase
 
 import org.junit.Test
@@ -27,16 +28,23 @@ class MapTypeValidationTest extends MapTypeTestBase {
 
   @Test(expected = classOf[ValidationException])
   def testWrongKeyType(): Unit = {
-    testSqlApi("f2[12]", "FAIL")
+    testAllApis('f2.at(12), "f2.at(12)", "f2[12]", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testIncorrectMapTypeComparison(): Unit = {
+    testAllApis('f1 === 'f3, "f1 === f3", "f1 = f3", "FAIL")
   }
 
   @Test(expected = classOf[ValidationException])
   def testUnsupportedComparisonType(): Unit = {
+    testAllApis('f6 !== 'f2, "f6 !== f2", "f6 != f2", "FAIL")
     testSqlApi("f6 <> f2", "FAIL")
   }
 
   @Test(expected = classOf[ValidationException])
   def testEmptyMap(): Unit = {
+    testAllApis("FAIL", "map()", "MAP[]", "FAIL")
     testSqlApi("MAP[]", "FAIL")
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/RowTypeValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/RowTypeValidationTest.scala
index f1297be..f72de54 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/RowTypeValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/RowTypeValidationTest.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.planner.expressions.validation
 
+import org.apache.flink.table.api.scala._
 import org.apache.flink.table.api.{SqlParserException, ValidationException}
 import org.apache.flink.table.planner.expressions.utils.RowTypeTestBase
 
@@ -32,11 +33,11 @@ class RowTypeValidationTest extends RowTypeTestBase {
 
   @Test(expected = classOf[ValidationException])
   def testNullRowType(): Unit = {
-    testSqlApi("Row(NULL)", "FAIL")
+    testAllApis("FAIL", "row(null)", "Row(NULL)", "FAIL")
   }
 
   @Test(expected = classOf[ValidationException])
   def testSqlRowIllegalAccess(): Unit = {
-    testSqlApi("f5.f2", "FAIL")
+    testAllApis('f5.get("f2"), "f5.get('f2')", "f5.f2", "FAIL")
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala
index 4f791ad..da4763f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarFunctionsValidationTest.scala
@@ -18,11 +18,13 @@
 
 package org.apache.flink.table.planner.expressions.validation
 
+import org.apache.flink.table.api.scala._
 import org.apache.flink.table.api.{SqlParserException, ValidationException}
+import org.apache.flink.table.expressions.TimePointUnit
 import org.apache.flink.table.planner.expressions.utils.ScalarTypesTestBase
 
 import org.apache.calcite.avatica.util.TimeUnit
-import org.junit.Test
+import org.junit.{Ignore, Test}
 
 class ScalarFunctionsValidationTest extends ScalarTypesTestBase {
 
@@ -30,45 +32,75 @@ class ScalarFunctionsValidationTest extends ScalarTypesTestBase {
   // Math functions
   // ----------------------------------------------------------------------------------------------
 
+  @Ignore
   @Test
-  def testInvalidBin1(): Unit = {
+  def testInvalidLog1(): Unit = {
     thrown.expect(classOf[ValidationException])
-    testSqlApi("BIN(f12)", "101010") // float type
+    // invalid arithmetic argument
+    testSqlApi(
+      "LOG(1, 100)",
+      "FAIL"
+    )
   }
 
+  @Ignore
   @Test
-  def testInvalidBin2(): Unit = {
+  def testInvalidLog2(): Unit ={
     thrown.expect(classOf[ValidationException])
+    // invalid arithmetic argument
+    testSqlApi(
+      "LOG(-1)",
+      "FAIL"
+    )
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidBin1(): Unit = {
+    testSqlApi("BIN(f12)", "101010") // float type
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInvalidBin2(): Unit = {
     testSqlApi("BIN(f15)", "101010") // BigDecimal type
   }
 
-  @Test
+  @Test(expected = classOf[ValidationException])
   def testInvalidBin3(): Unit = {
-    thrown.expect(classOf[ValidationException])
     testSqlApi("BIN(f16)", "101010") // Date type
   }
 
+  // ----------------------------------------------------------------------------------------------
+  // String functions
+  // ----------------------------------------------------------------------------------------------
+
+  @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")
+  }
 
   // ----------------------------------------------------------------------------------------------
   // Temporal functions
   // ----------------------------------------------------------------------------------------------
 
-  @Test
-  def testTimestampAddWithWrongTimestampInterval(): Unit = {
-    thrown.expect(classOf[SqlParserException])
-    testSqlApi("TIMESTAMPADD(XXX, 1, timestamp '2016-02-24')", "2016-06-16")
+  @Test(expected = classOf[SqlParserException])
+  def testTimestampAddWithWrongTimestampInterval(): Unit ={
+    testSqlApi("TIMESTAMPADD(XXX, 1, timestamp '2016-02-24'))", "2016-06-16")
   }
 
-  @Test
-  def testTimestampAddWithWrongTimestampFormat(): Unit = {
-    thrown.expect(classOf[SqlParserException])
-    thrown.expectMessage("Illegal TIMESTAMP literal '2016/02/24'")
-    testSqlApi("TIMESTAMPADD(YEAR, 1, timestamp '2016/02/24')", "2016-06-16")
+  @Test(expected = classOf[SqlParserException])
+  def testTimestampAddWithWrongTimestampFormat(): Unit ={
+    testSqlApi("TIMESTAMPADD(YEAR, 1, timestamp '2016-02-24'))", "2016-06-16")
   }
 
-  @Test
-  def testTimestampAddWithWrongQuantity(): Unit = {
-    thrown.expect(classOf[ValidationException])
+  @Test(expected = classOf[ValidationException])
+  def testTimestampAddWithWrongQuantity(): Unit ={
     testSqlApi("TIMESTAMPADD(YEAR, 1.0, timestamp '2016-02-24 12:42:25')", "2016-06-16")
   }
 
@@ -76,6 +108,50 @@ class ScalarFunctionsValidationTest extends ScalarTypesTestBase {
   // Sub-query functions
   // ----------------------------------------------------------------------------------------------
 
+  @Test(expected = classOf[ValidationException])
+  def testInValidationExceptionMoreThanOneTypes(): Unit = {
+    testTableApi(
+      'f2.in('f3, 'f8),
+      "f2.in(f3, f8)",
+      "true"
+    )
+    testTableApi(
+      'f2.in('f3, 'f4, 4),
+      "f2.in(f3, f4, 4)",
+      "false"  // OK if all numeric
+    )
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def scalaInValidationExceptionDifferentOperandsTest(): Unit = {
+    testTableApi(
+      'f1.in("Hi", "Hello world", "Comment#1"),
+      "true",
+      "true"
+    )
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def javaInValidationExceptionDifferentOperandsTest(): Unit = {
+    testTableApi(
+      true,
+      "f1.in('Hi','Hello world','Comment#1')",
+      "true"
+    )
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testTimestampDiffWithWrongTime(): Unit = {
+    testTableApi(
+      timestampDiff(TimePointUnit.DAY, "2016-02-24", "2016-02-27"), "FAIL", "FAIL")
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testTimestampDiffWithWrongTimeAndUnit(): Unit = {
+    testTableApi(
+      timestampDiff(TimePointUnit.MINUTE, "2016-02-24", "2016-02-27"), "FAIL", "FAIL")
+  }
+
   @Test
   def testDOWWithTimeWhichIsUnsupported(): Unit = {
     thrown.expect(classOf[ValidationException])
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarOperatorsValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarOperatorsValidationTest.scala
new file mode 100644
index 0000000..9bc2ea8
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/validation/ScalarOperatorsValidationTest.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.planner.expressions.validation
+
+import org.apache.flink.table.api.ValidationException
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.planner.expressions.utils.ScalarOperatorsTestBase
+
+import org.junit.Test
+
+class ScalarOperatorsValidationTest extends ScalarOperatorsTestBase {
+
+  @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")
+  }
+
+  // ----------------------------------------------------------------------------------------------
+  // Sub-query functions
+  // ----------------------------------------------------------------------------------------------
+
+  @Test(expected = classOf[ValidationException])
+  def testInMoreThanOneTypes(): Unit = {
+    testTableApi(
+      'f2.in('f3, 'f4, 4),
+      "FAIL",
+      "FAIL"
+    )
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testInDifferentOperands(): Unit = {
+    testTableApi(
+      'f1.in("Hi", "Hello world", "Comment#1"),
+      "FAIL",
+      "FAIL"
+    )
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testBetweenWithDifferentOperandTypeScala(): Unit = {
+    testTableApi(
+      2.between(1, "a"),
+      "FAIL",
+      "FAIL"
+    )
+  }
+
+  @Test(expected = classOf[ValidationException])
+  def testBetweenWithDifferentOperandTypeJava(): Unit = {
+    testTableApi(
+      "FAIL",
+      "2.between(1, 'a')",
+      "FAIL"
+    )
+  }
+}