dawidwys commented on a change in pull request #11290: [FLINK-16379][table] Introduce fromValues in TableEnvironment URL: https://github.com/apache/flink/pull/11290#discussion_r410162710
########## File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/ValuesOperationFactory.java ########## @@ -0,0 +1,356 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.operations.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionDefaultVisitor; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.expressions.resolver.ExpressionResolver; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.operations.QueryOperation; +import org.apache.flink.table.operations.ValuesQueryOperation; +import org.apache.flink.table.types.CollectionDataType; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.FieldsDataType; +import org.apache.flink.table.types.KeyValueDataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.utils.LogicalTypeCasts; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.table.types.logical.utils.LogicalTypeGeneralization; +import org.apache.flink.table.types.utils.TypeConversions; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.apache.flink.table.expressions.ApiExpressionUtils.valueLiteral; + +/** + * Utility class for creating valid {@link ValuesQueryOperation} operation. + */ +@Internal +class ValuesOperationFactory { + /** + * Creates a valid {@link ValuesQueryOperation} operation. + * + * <p>It derives a row type based on {@link LogicalTypeGeneralization}. It flattens any + * row constructors. It does not flatten ROWs which are a result of e.g. a function call. + * + * <p>The resulting schema can be provided manually. If it is not, the schema will be automatically derived from + * the types of the expressions. + */ + QueryOperation create( + @Nullable TableSchema expectedSchema, + List<ResolvedExpression> resolvedExpressions, + ExpressionResolver.PostResolverFactory postResolverFactory) { + List<List<ResolvedExpression>> resolvedRows = unwrapFromRowConstructor(resolvedExpressions); + + if (expectedSchema != null) { + verifyAllSameSize(resolvedRows, expectedSchema.getFieldCount()); + } + + TableSchema schema = Optional.ofNullable(expectedSchema) + .orElseGet(() -> extractSchema(resolvedRows)); + + List<List<ResolvedExpression>> castedExpressions = resolvedRows.stream() + .map(row -> convertTopLevelExpressionToExpectedRowType(postResolverFactory, schema.getFieldDataTypes(), row)) + .collect(Collectors.toList()); + + return new ValuesQueryOperation(castedExpressions, schema); + } + + private TableSchema extractSchema(List<List<ResolvedExpression>> resolvedRows) { + DataType[] dataTypes = findRowType(resolvedRows); + String[] fieldNames = IntStream.range(0, dataTypes.length) + .mapToObj(i -> "f" + i) + .toArray(String[]::new); + return TableSchema.builder() + .fields(fieldNames, dataTypes) + .build(); + } + + private List<ResolvedExpression> convertTopLevelExpressionToExpectedRowType( + ExpressionResolver.PostResolverFactory postResolverFactory, + DataType[] dataTypes, + List<ResolvedExpression> row) { + return IntStream.range(0, row.size()) + .mapToObj(i -> { + boolean typesMatch = row.get(i) + .getOutputDataType() + .getLogicalType() + .equals(dataTypes[i].getLogicalType()); + if (typesMatch) { + return row.get(i); + } + + ResolvedExpression castedExpr = row.get(i); + DataType targetType = dataTypes[i]; + + return convertToExpectedType(castedExpr, targetType, postResolverFactory) + .orElseThrow(() -> new ValidationException(String.format( + "Could not cast the value of the %d column: [ %s ] of a row: %s to the requested type: %s", + i, + castedExpr.asSummaryString(), + row.stream() + .map(ResolvedExpression::asSummaryString) + .collect(Collectors.joining(", ", "[ ", " ]")), + targetType.getLogicalType().asSummaryString()))); + }) + .collect(Collectors.toList()); + } + + private Optional<ResolvedExpression> convertToExpectedType( + ResolvedExpression castedExpr, + DataType targetType, + ExpressionResolver.PostResolverFactory postResolverFactory) { + + // if the expression is a literal try converting the literal in place instead of casting + if (castedExpr instanceof ValueLiteralExpression) { + // Assign a type to a null literal + if (LogicalTypeChecks.hasRoot(castedExpr.getOutputDataType().getLogicalType(), LogicalTypeRoot.NULL)) { + return Optional.of(valueLiteral(null, targetType)); + } + + Optional<Object> value = ((ValueLiteralExpression) castedExpr).getValueAs(Object.class); + if (value.isPresent() && targetType.getLogicalType().supportsInputConversion(value.get().getClass())) { + ValueLiteralExpression convertedLiteral = valueLiteral( + value.get(), + targetType.notNull().bridgedTo(value.get().getClass())); + if (targetType.getLogicalType().isNullable()) { + return Optional.of(postResolverFactory.cast(convertedLiteral, targetType)); + } else { + return Optional.of(convertedLiteral); + } + } + } + + if (castedExpr instanceof CallExpression) { + FunctionDefinition functionDefinition = ((CallExpression) castedExpr).getFunctionDefinition(); + if (functionDefinition == BuiltInFunctionDefinitions.ROW && targetType instanceof FieldsDataType) { + return convertRowToExpectedType(castedExpr, (FieldsDataType) targetType, postResolverFactory); + } else if (functionDefinition == BuiltInFunctionDefinitions.ARRAY && + LogicalTypeChecks.hasRoot(targetType.getLogicalType(), LogicalTypeRoot.ARRAY)) { + return convertArrayToExpectedType(castedExpr, (CollectionDataType) targetType, postResolverFactory); + } else if (functionDefinition == BuiltInFunctionDefinitions.MAP && + LogicalTypeChecks.hasRoot(targetType.getLogicalType(), LogicalTypeRoot.MAP)) { + return convertMapToExpectedType(castedExpr, (KeyValueDataType) targetType, postResolverFactory); + } + } + + // We might not be able to cast to the expected type if the expected type was provided by the user + // we ignore nullability constraints here, as we let users override what we expect there, e.g. they + // might know that a certain function will not produce nullable values for a given input + if (LogicalTypeCasts.supportsExplicitCast( + castedExpr.getOutputDataType().getLogicalType().copy(true), + targetType.getLogicalType().copy(true))) { + return Optional.of(postResolverFactory.cast(castedExpr, targetType)); + } else { + return Optional.empty(); + } + } + + private Optional<ResolvedExpression> convertRowToExpectedType( + ResolvedExpression castedExpr, + FieldsDataType targetType, + ExpressionResolver.PostResolverFactory postResolverFactory) { + DataType[] targetTypes = ((RowType) targetType.getLogicalType()).getFieldNames() + .stream() + .map(name -> targetType.getFieldDataTypes().get(name)) Review comment: How about a follow-up issue for this? I am not sure how to do it properly now. I am not sure where should we change the type of the map. The map is a parameter to `FieldsDataType`, so we would have to change it whenever the `FieldsDataType is constructed. It has also the downside that we add an implicit contract to the parameters that the map is order in the same way as the logical type. Moreover we apply the same pattern in other locations: e.g. `org.apache.flink.table.types.utils.LegacyTypeInfoDataTypeConverter#convertToRowTypeInfo` `org.apache.flink.table.types.utils.DataTypeUtils.DataTypeTransformer#visit(org.apache.flink.table.types.FieldsDataType)`... If we change the contract, we should also fix those locations. ---------------------------------------------------------------- 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: us...@infra.apache.org With regards, Apache Git Services