twalthr commented on a change in pull request #14996: URL: https://github.com/apache/flink/pull/14996#discussion_r582783271
########## File path: flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java ########## @@ -0,0 +1,328 @@ +/* + * 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.catalog; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Schema.UnresolvedComputedColumn; +import org.apache.flink.table.api.Schema.UnresolvedMetadataColumn; +import org.apache.flink.table.api.Schema.UnresolvedPhysicalColumn; +import org.apache.flink.table.api.Schema.UnresolvedPrimaryKey; +import org.apache.flink.table.api.Schema.UnresolvedWatermarkSpec; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Column.ComputedColumn; +import org.apache.flink.table.catalog.Column.MetadataColumn; +import org.apache.flink.table.catalog.Column.PhysicalColumn; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.LocalReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.resolver.ExpressionResolver.ExpressionResolverBuilder; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.TimestampKind; +import org.apache.flink.table.types.logical.TimestampType; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.flink.table.expressions.ApiExpressionUtils.localRef; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.hasRoot; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isProctimeAttribute; +import static org.apache.flink.table.types.utils.DataTypeUtils.replaceLogicalType; + +/** Default implementation of {@link SchemaResolver}. */ +@Internal +class DefaultSchemaResolver implements SchemaResolver { + + private final boolean isStreamingMode; + private final boolean supportsMetadata; + private final DataTypeFactory dataTypeFactory; + private final ExpressionResolverBuilder resolverBuilder; + + DefaultSchemaResolver( + boolean isStreamingMode, + boolean supportsMetadata, + DataTypeFactory dataTypeFactory, + ExpressionResolverBuilder resolverBuilder) { + this.isStreamingMode = isStreamingMode; + this.supportsMetadata = supportsMetadata; + this.dataTypeFactory = dataTypeFactory; + this.resolverBuilder = resolverBuilder; + } + + public SchemaResolver withMetadata(boolean supportsMetadata) { + return new DefaultSchemaResolver( + isStreamingMode, supportsMetadata, dataTypeFactory, resolverBuilder); + } + + @Override + public ResolvedSchema resolve(Schema schema) { + final List<Column> columns = resolveColumns(schema.getColumns()); + + final List<WatermarkSpec> watermarkSpecs = + resolveWatermarkSpecs(schema.getWatermarkSpecs(), columns); + + final List<Column> columnsWithRowtime = adjustRowtimeAttributes(watermarkSpecs, columns); + + final UniqueConstraint primaryKey = + resolvePrimaryKey(schema.getPrimaryKey().orElse(null), columnsWithRowtime); + + return new ResolvedSchema(columnsWithRowtime, watermarkSpecs, primaryKey); + } + + @Override + public boolean isStreamingMode() { + return isStreamingMode; + } + + @Override + public boolean supportsMetadata() { + return supportsMetadata; + } + + // -------------------------------------------------------------------------------------------- + + private List<Column> resolveColumns(List<Schema.UnresolvedColumn> unresolvedColumns) { + final List<Column> resolvedColumns = new ArrayList<>(); + for (Schema.UnresolvedColumn unresolvedColumn : unresolvedColumns) { + final Column column; + if (unresolvedColumn instanceof UnresolvedPhysicalColumn) { + column = resolvePhysicalColumn((UnresolvedPhysicalColumn) unresolvedColumn); + } else if (unresolvedColumn instanceof UnresolvedMetadataColumn) { + column = resolveMetadataColumn((UnresolvedMetadataColumn) unresolvedColumn); + } else if (unresolvedColumn instanceof UnresolvedComputedColumn) { + column = + resolveComputedColumn( + (UnresolvedComputedColumn) unresolvedColumn, resolvedColumns); + } else { + throw new IllegalArgumentException("Unknown unresolved column type."); + } + resolvedColumns.add(column); + } + + validateDuplicateColumns(resolvedColumns); + + return resolvedColumns; + } + + private PhysicalColumn resolvePhysicalColumn(UnresolvedPhysicalColumn unresolvedColumn) { + return Column.physical( + unresolvedColumn.getColumnName(), + dataTypeFactory.createDataType(unresolvedColumn.getDataType())); + } + + private MetadataColumn resolveMetadataColumn(UnresolvedMetadataColumn unresolvedColumn) { + if (!supportsMetadata) { + throw new ValidationException( + "Metadata columns are not supported in a schema at the current location."); + } + return Column.metadata( + unresolvedColumn.getColumnName(), + dataTypeFactory.createDataType(unresolvedColumn.getDataType()), + unresolvedColumn.getMetadataKey(), + unresolvedColumn.isVirtual()); + } + + private ComputedColumn resolveComputedColumn( + UnresolvedComputedColumn unresolvedColumn, List<Column> inputColumns) { + final ResolvedExpression resolvedExpression; + try { + resolvedExpression = resolveExpression(inputColumns, unresolvedColumn.getExpression()); + } catch (Exception e) { + throw new ValidationException( + String.format( + "Invalid expression for computed column '%s'.", + unresolvedColumn.getColumnName()), + e); + } + return Column.computed(unresolvedColumn.getColumnName(), resolvedExpression); + } + + private void validateDuplicateColumns(List<Column> columns) { + final List<String> names = + columns.stream().map(Column::getName).collect(Collectors.toList()); + final List<String> duplicates = + names.stream() + .filter(name -> Collections.frequency(names, name) > 1) + .distinct() + .collect(Collectors.toList()); + if (duplicates.size() > 0) { + throw new ValidationException( + String.format( + "Schema must not contain duplicate column names. Found duplicates: %s", + duplicates)); + } + } + + private List<WatermarkSpec> resolveWatermarkSpecs( + List<UnresolvedWatermarkSpec> unresolvedWatermarkSpecs, List<Column> inputColumns) { + if (unresolvedWatermarkSpecs.size() == 0) { + return Collections.emptyList(); + } + if (unresolvedWatermarkSpecs.size() > 1) { + throw new ValidationException("Multiple watermark definitions are not supported yet."); + } + final UnresolvedWatermarkSpec watermarkSpec = unresolvedWatermarkSpecs.get(0); + + // validate time attribute + final String timeColumn = watermarkSpec.getColumnName(); + validateTimeColumn(timeColumn, inputColumns); + + // resolve watermark expression + final ResolvedExpression watermarkExpression; + try { + watermarkExpression = + resolveExpression(inputColumns, watermarkSpec.getWatermarkExpression()); + } catch (Exception e) { + throw new ValidationException( + String.format( + "Invalid expression for watermark '%s'.", watermarkSpec.toString()), + e); + } + validateWatermarkExpression(watermarkExpression.getOutputDataType().getLogicalType()); + + return Collections.singletonList( + new WatermarkSpec(watermarkSpec.getColumnName(), watermarkExpression)); + } + + private void validateTimeColumn(String columnName, List<Column> columns) { + final Optional<Column> timeColumn = + columns.stream().filter(c -> c.getName().equals(columnName)).findFirst(); + if (!timeColumn.isPresent()) { Review comment: I did this for code readability, using `orElseThrow` will blow up the formatting ---------------------------------------------------------------- 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]
