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/29 06:43:41 UTC

[flink] 07/08: [FLINK-13774][table-planner-blink] Remove unresolved expression in RexNodeConverter

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

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

commit d2085a18169da7bf84fdff98c5f743a8e75b34a7
Author: JingsongLi <lz...@aliyun.com>
AuthorDate: Thu Aug 22 12:58:01 2019 +0200

    [FLINK-13774][table-planner-blink] Remove unresolved expression in RexNodeConverter
---
 .../planner/expressions/RexNodeConverter.java      | 58 ++--------------------
 1 file changed, 5 insertions(+), 53 deletions(-)

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 68b7b7f..731d550 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
@@ -31,8 +31,6 @@ import org.apache.flink.table.expressions.TableReferenceExpression;
 import org.apache.flink.table.expressions.TimeIntervalUnit;
 import org.apache.flink.table.expressions.TimePointUnit;
 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;
@@ -121,8 +119,8 @@ import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoT
 /**
  * Visit expression to generator {@link RexNode}.
  *
- * <p>TODO actually we should use {@link ResolvedExpressionVisitor} here as it is the output of the API.
- * we will update it after introduce Expression resolve in AggCodeGen.
+ * <p>TODO remove blink expressions(like {@link ResolvedAggInputReference}) and use
+ * {@link ResolvedExpressionVisitor}.
  */
 public class RexNodeConverter implements ExpressionVisitor<RexNode> {
 
@@ -356,7 +354,7 @@ public class RexNodeConverter implements ExpressionVisitor<RexNode> {
 			FunctionDefinition def = call.getFunctionDefinition();
 			if (conversionsOfBuiltInFunc.containsKey(def)) {
 				RexNodeConversion conversion = conversionsOfBuiltInFunc.get(def);
-				return conversion.convert(call);
+				return conversion.convert(call.getChildren());
 			} else {
 				throw new UnsupportedOperationException(def.toString());
 			}
@@ -877,16 +875,12 @@ public class RexNodeConverter implements ExpressionVisitor<RexNode> {
 
 	@Override
 	public RexNode visit(Expression other) {
-		if (other instanceof UnresolvedReferenceExpression) {
-			return visitUnresolvedReferenceExpression((UnresolvedReferenceExpression) other);
-		} else if (other instanceof ResolvedAggInputReference) {
+		if (other instanceof ResolvedAggInputReference) {
 			return visitResolvedAggInputReference((ResolvedAggInputReference) other);
 		} else if (other instanceof ResolvedAggLocalReference) {
 			return visitResolvedAggLocalReference((ResolvedAggLocalReference) other);
 		} else if (other instanceof ResolvedDistinctKeyReference) {
 			return visitResolvedDistinctKeyReference((ResolvedDistinctKeyReference) other);
-		} else if (other instanceof UnresolvedCallExpression) {
-			return visit((UnresolvedCallExpression) other);
 		} else if (other instanceof RexNodeExpression) {
 			return ((RexNodeExpression) other).getRexNode();
 		} else {
@@ -894,10 +888,6 @@ public class RexNodeConverter implements ExpressionVisitor<RexNode> {
 		}
 	}
 
-	private RexNode visitUnresolvedReferenceExpression(UnresolvedReferenceExpression field) {
-		return relBuilder.field(field.getName());
-	}
-
 	private RexNode visitResolvedAggInputReference(ResolvedAggInputReference reference) {
 		// using index to resolve field directly, name used in toString only
 		return new RexInputRef(
@@ -922,34 +912,6 @@ public class RexNodeConverter implements ExpressionVisitor<RexNode> {
 				type);
 	}
 
-	private RexNode visit(UnresolvedCallExpression call) {
-		FunctionDefinition func = call.getFunctionDefinition();
-		switch (func.getKind()) {
-			case SCALAR:
-				if (func instanceof ScalarFunctionDefinition) {
-					ScalarFunction scalaFunc = ((ScalarFunctionDefinition) func).getScalarFunction();
-					List<RexNode> child = convertCallChildren(call.getChildren());
-					SqlFunction sqlFunction = UserDefinedFunctionUtils.createScalarSqlFunction(
-							scalaFunc.functionIdentifier(),
-							scalaFunc.toString(),
-							scalaFunc,
-							typeFactory);
-					return relBuilder.call(sqlFunction, child);
-				} else {
-					FunctionDefinition def = call.getFunctionDefinition();
-					if (conversionsOfBuiltInFunc.containsKey(def)) {
-						RexNodeConversion conversion = conversionsOfBuiltInFunc.get(def);
-						return conversion.convert(call);
-					} else {
-						throw new UnsupportedOperationException(def.toString());
-					}
-				}
-
-			default:
-				throw new UnsupportedOperationException();
-		}
-	}
-
 	private RexNode createCollation(RexNode node, RelFieldCollation.Direction direction,
 			RelFieldCollation.NullDirection nullDirection, Set<SqlKind> kinds) {
 		switch (node.getKind()) {
@@ -1025,20 +987,10 @@ public class RexNodeConverter implements ExpressionVisitor<RexNode> {
 	}
 
 	/**
-	 * RexNodeConversion to define how to convert a {@link CallExpression} or a {@link UnresolvedCallExpression} which
+	 * RexNodeConversion to define how to convert a {@link CallExpression} which
 	 * has built-in FunctionDefinition to RexNode.
 	 */
 	private interface RexNodeConversion {
-
 		RexNode convert(List<Expression> children);
-
-		default RexNode convert(CallExpression expression) {
-			return convert(expression.getChildren());
-		}
-
-		default RexNode convert(UnresolvedCallExpression unresolvedCallExpression) {
-			return convert(unresolvedCallExpression.getChildren());
-		}
 	}
-
 }