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 1e9232296eaef0e1f3a8ba32b416d26143a501c5
Author: JingsongLi <lzljs3620...@aliyun.com>
AuthorDate: Mon Aug 19 16:21:51 2019 +0200

    [FLINK-13774][table-planner-blink] Blink extended expressions should 
implement ResolvedExpression
---
 .../expressions/ResolvedAggInputReference.java       | 20 +++++++++++++++++---
 .../expressions/ResolvedAggLocalReference.java       | 20 +++++++++++++++++---
 .../expressions/ResolvedDistinctKeyReference.java    | 20 +++++++++++++++++---
 3 files changed, 51 insertions(+), 9 deletions(-)

diff --git 
a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedAggInputReference.java
 
b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedAggInputReference.java
index 5430da6..2ad8177 100644
--- 
a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedAggInputReference.java
+++ 
b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedAggInputReference.java
@@ -21,18 +21,22 @@ package org.apache.flink.table.planner.expressions;
 import org.apache.flink.table.expressions.Expression;
 import org.apache.flink.table.expressions.ExpressionVisitor;
 import org.apache.flink.table.expressions.FieldReferenceExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.util.Preconditions;
 
 import java.util.Collections;
 import java.util.List;
 
+import static 
org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType;
+
 /**
  * Normally we should use {@link FieldReferenceExpression} to represent an 
input field.
  * {@link FieldReferenceExpression} uses name to locate the field, in 
aggregate case, we want to use
  * field index.
  */
-public class ResolvedAggInputReference implements Expression {
+public class ResolvedAggInputReference implements ResolvedExpression {
 
        private final String name;
        private final int index;
@@ -57,8 +61,13 @@ public class ResolvedAggInputReference implements Expression 
{
        }
 
        @Override
-       public String asSummaryString() {
-               return name;
+       public DataType getOutputDataType() {
+               return fromLogicalTypeToDataType(resultType);
+       }
+
+       @Override
+       public List<ResolvedExpression> getResolvedChildren() {
+               return Collections.emptyList();
        }
 
        @Override
@@ -67,6 +76,11 @@ public class ResolvedAggInputReference implements Expression 
{
        }
 
        @Override
+       public String asSummaryString() {
+               return name;
+       }
+
+       @Override
        public <R> R accept(ExpressionVisitor<R> visitor) {
                return visitor.visit(this);
        }
diff --git 
a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedAggLocalReference.java
 
b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedAggLocalReference.java
index 205c578..055ed10 100644
--- 
a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedAggLocalReference.java
+++ 
b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedAggLocalReference.java
@@ -20,11 +20,15 @@ package org.apache.flink.table.planner.expressions;
 
 import org.apache.flink.table.expressions.Expression;
 import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalType;
 
 import java.util.Collections;
 import java.util.List;
 
+import static 
org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType;
+
 /**
  * Special reference which represent a local filed, such as aggregate buffers 
or constants.
  * We are stored as class members, so the field can be referenced directly.
@@ -32,7 +36,7 @@ import java.util.List;
  *
  * <p>See {@link 
org.apache.flink.table.planner.codegen.ExprCodeGenerator#visitLocalRef}.
  */
-public class ResolvedAggLocalReference implements Expression {
+public class ResolvedAggLocalReference implements ResolvedExpression {
 
        private final String fieldTerm;
        private final String nullTerm;
@@ -57,8 +61,13 @@ public class ResolvedAggLocalReference implements Expression 
{
        }
 
        @Override
-       public String asSummaryString() {
-               return fieldTerm;
+       public DataType getOutputDataType() {
+               return fromLogicalTypeToDataType(resultType);
+       }
+
+       @Override
+       public List<ResolvedExpression> getResolvedChildren() {
+               return Collections.emptyList();
        }
 
        @Override
@@ -67,6 +76,11 @@ public class ResolvedAggLocalReference implements Expression 
{
        }
 
        @Override
+       public String asSummaryString() {
+               return fieldTerm;
+       }
+
+       @Override
        public <R> R accept(ExpressionVisitor<R> visitor) {
                return visitor.visit(this);
        }
diff --git 
a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedDistinctKeyReference.java
 
b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedDistinctKeyReference.java
index 8b99601..d55905f 100644
--- 
a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedDistinctKeyReference.java
+++ 
b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/ResolvedDistinctKeyReference.java
@@ -20,16 +20,20 @@ package org.apache.flink.table.planner.expressions;
 
 import org.apache.flink.table.expressions.Expression;
 import org.apache.flink.table.expressions.ExpressionVisitor;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.LogicalType;
 import org.apache.flink.util.Preconditions;
 
 import java.util.Collections;
 import java.util.List;
 
+import static 
org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType;
+
 /**
  * Resolved distinct key reference.
  */
-public class ResolvedDistinctKeyReference implements Expression {
+public class ResolvedDistinctKeyReference implements ResolvedExpression {
 
        private final String name;
        private final LogicalType resultType;
@@ -48,8 +52,13 @@ public class ResolvedDistinctKeyReference implements 
Expression {
        }
 
        @Override
-       public String asSummaryString() {
-               return name;
+       public DataType getOutputDataType() {
+               return fromLogicalTypeToDataType(resultType);
+       }
+
+       @Override
+       public List<ResolvedExpression> getResolvedChildren() {
+               return Collections.emptyList();
        }
 
        @Override
@@ -58,6 +67,11 @@ public class ResolvedDistinctKeyReference implements 
Expression {
        }
 
        @Override
+       public String asSummaryString() {
+               return name;
+       }
+
+       @Override
        public <R> R accept(ExpressionVisitor<R> visitor) {
                return visitor.visit(this);
        }

Reply via email to