JingsongLi commented on a change in pull request #9485: 
[FLINK-13775][table-planner-blink] Refactor 
ExpressionConverter(RexNodeConverter) in blink
URL: https://github.com/apache/flink/pull/9485#discussion_r319337008
 
 

 ##########
 File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/converter/ExpressionConverter.java
 ##########
 @@ -0,0 +1,322 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.converter;
+
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.dataformat.Decimal;
+import org.apache.flink.table.expressions.CallExpression;
+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.LocalReferenceExpression;
+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.ValueLiteralExpression;
+import org.apache.flink.table.planner.calcite.FlinkContext;
+import org.apache.flink.table.planner.calcite.FlinkRelBuilder;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.calcite.RexFieldVariable;
+import org.apache.flink.table.planner.expressions.RexNodeExpression;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.DateString;
+import org.apache.calcite.util.TimeString;
+import org.apache.calcite.util.TimestampString;
+import org.apache.calcite.util.TimestampWithTimeZoneString;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Date;
+import java.util.List;
+import java.util.Optional;
+import java.util.TimeZone;
+
+import static 
org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType;
+
+/**
+ * Visit expression to generator {@link RexNode}.
+ */
+public class ExpressionConverter implements ExpressionVisitor<RexNode> {
+
+       private static final List<CallExpressionConvertRule> 
FUNCTION_CONVERT_CHAIN = Arrays.asList(
+               new ScalarFunctionConvertRule(),
+               new OverConvertRule(),
+               new DefinedConvertRule(),
+               new CustomizedConvertRule()
+       );
+
+       private final RelBuilder relBuilder;
+       private final FlinkTypeFactory typeFactory;
+
+       public ExpressionConverter(RelBuilder relBuilder) {
+               this.relBuilder = relBuilder;
+               this.typeFactory = (FlinkTypeFactory) 
relBuilder.getRexBuilder().getTypeFactory();
+       }
+
+       @Override
+       public RexNode visit(CallExpression call) {
+               for (CallExpressionConvertRule rule : FUNCTION_CONVERT_CHAIN) {
+                       Optional<RexNode> converted = rule.convert(call, 
newFunctionContext());
+                       if (converted.isPresent()) {
+                               return converted.get();
+                       }
+               }
+               throw new RuntimeException("Unknown call expression: " + call);
+       }
+
+       @Override
+       public RexNode visit(ValueLiteralExpression valueLiteral) {
+               LogicalType type = 
fromDataTypeToLogicalType(valueLiteral.getOutputDataType());
+               RexBuilder rexBuilder = relBuilder.getRexBuilder();
+               FlinkTypeFactory typeFactory = (FlinkTypeFactory) 
relBuilder.getTypeFactory();
+               if (valueLiteral.isNull()) {
+                       return relBuilder.getRexBuilder()
+                                       .makeCast(
+                                                       
typeFactory.createFieldTypeFromLogicalType(type),
+                                                       
relBuilder.getRexBuilder().constantNull());
+               }
+
+               switch (type.getTypeRoot()) {
+                       case DECIMAL:
+                               DecimalType dt = (DecimalType) type;
+                               BigDecimal bigDecimal = 
extractValue(valueLiteral, BigDecimal.class);
+                               RelDataType decType = 
relBuilder.getTypeFactory().createSqlType(SqlTypeName.DECIMAL,
+                                               dt.getPrecision(), 
dt.getScale());
+                               return 
relBuilder.getRexBuilder().makeExactLiteral(bigDecimal, decType);
+                       case BIGINT:
+                               // create BIGINT literals for long type
+                               BigDecimal bigint = extractValue(valueLiteral, 
BigDecimal.class);
+                               return 
relBuilder.getRexBuilder().makeBigintLiteral(bigint);
+                       case FLOAT:
+                               //Float/Double type should be liked as java 
type here.
+                               return 
relBuilder.getRexBuilder().makeApproxLiteral(
+                                               extractValue(valueLiteral, 
BigDecimal.class),
+                                               
relBuilder.getTypeFactory().createSqlType(SqlTypeName.FLOAT));
+                       case DOUBLE:
+                               //Float/Double type should be liked as java 
type here.
+                               return rexBuilder.makeApproxLiteral(
+                                               extractValue(valueLiteral, 
BigDecimal.class),
+                                               
relBuilder.getTypeFactory().createSqlType(SqlTypeName.DOUBLE));
+                       case DATE:
+                               return 
relBuilder.getRexBuilder().makeDateLiteral(DateString.fromCalendarFields(
+                                               
valueAsCalendar(extractValue(valueLiteral, java.sql.Date.class))));
+                       case TIME_WITHOUT_TIME_ZONE:
+                               return 
relBuilder.getRexBuilder().makeTimeLiteral(TimeString.fromCalendarFields(
+                                               
valueAsCalendar(extractValue(valueLiteral, java.sql.Time.class))), 0);
+                       case TIMESTAMP_WITHOUT_TIME_ZONE:
+                               return 
relBuilder.getRexBuilder().makeTimestampLiteral(TimestampString.fromCalendarFields(
+                                               
valueAsCalendar(extractValue(valueLiteral, java.sql.Timestamp.class))), 3);
+                       case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                               TimeZone timeZone = 
TimeZone.getTimeZone(((FlinkContext) ((FlinkRelBuilder) this.relBuilder)
+                                               
.getCluster().getPlanner().getContext()).getTableConfig().getLocalTimeZone());
+                               return 
this.relBuilder.getRexBuilder().makeTimestampWithLocalTimeZoneLiteral(
+                                               new TimestampWithTimeZoneString(
+                                                               
TimestampString.fromMillisSinceEpoch(
+                                                                               
extractValue(valueLiteral, java.time.Instant.class).toEpochMilli()),
+                                                               timeZone)
+                                                               
.withTimeZone(DateTimeUtils.UTC_ZONE)
+                                                               
.getLocalTimestampString(), 3);
+                       case INTERVAL_YEAR_MONTH:
+                               return 
this.relBuilder.getRexBuilder().makeIntervalLiteral(
+                                               
BigDecimal.valueOf(extractValue(valueLiteral, Integer.class)),
+                                               new 
SqlIntervalQualifier(TimeUnit.YEAR, TimeUnit.MONTH, SqlParserPos.ZERO));
+                       case INTERVAL_DAY_TIME:
+                               return 
this.relBuilder.getRexBuilder().makeIntervalLiteral(
+                                               
BigDecimal.valueOf(extractValue(valueLiteral, Long.class)),
+                                               new 
SqlIntervalQualifier(TimeUnit.DAY, TimeUnit.SECOND, SqlParserPos.ZERO));
+                       default:
+                               break;
+               }
+               Object object = extractValue(valueLiteral, Object.class);
+               if (object instanceof TimePointUnit) {
+                       TimeUnit value = 
timePointUnitToTimeUnit((TimePointUnit) object);
+                       return relBuilder.getRexBuilder().makeFlag(value);
+               } else if (object instanceof TimeIntervalUnit) {
+                       TimeUnitRange value = 
intervalUnitToUnitRange((TimeIntervalUnit) object);
+                       return relBuilder.getRexBuilder().makeFlag(value);
+               } else {
+                       return relBuilder.literal(extractValue(valueLiteral, 
Object.class));
+               }
+       }
+
+       @Override
+       public RexNode visit(FieldReferenceExpression fieldReference) {
+               // We can not use inputCount+inputIndex+FieldIndex to construct 
field of calcite.
 
 Review comment:
   I created https://issues.apache.org/jira/browse/FLINK-13902

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to