raminqaf commented on code in PR #27222: URL: https://github.com/apache/flink/pull/27222#discussion_r2514944764
########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractCreateMaterializedTableConverter.java: ########## @@ -0,0 +1,216 @@ +/* + * 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.operations.converters; + +import org.apache.flink.sql.parser.ddl.SqlCreateMaterializedTable; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.CatalogMaterializedTable; +import org.apache.flink.table.catalog.CatalogMaterializedTable.LogicalRefreshMode; +import org.apache.flink.table.catalog.CatalogMaterializedTable.RefreshMode; +import org.apache.flink.table.catalog.IntervalFreshness; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.TableDistribution; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.planner.operations.PlannerQueryOperation; +import org.apache.flink.table.planner.utils.MaterializedTableUtils; +import org.apache.flink.table.planner.utils.OperationConverterUtils; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; + +import org.apache.calcite.sql.SqlNode; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.flink.table.api.config.MaterializedTableConfigOptions.DATE_FORMATTER; +import static org.apache.flink.table.api.config.MaterializedTableConfigOptions.PARTITION_FIELDS; + +/** + * Abstract class for converting {@link SqlCreateMaterializedTable} and it's children to create + * materialized table operations. + */ +public abstract class AbstractCreateMaterializedTableConverter<T extends SqlCreateMaterializedTable> + implements SqlNodeConverter<T> { + /** Context of create table converters while mering source and derived items. */ + protected interface MergeContext { + Schema getMergedSchema(); + + Map<String, String> getMergedTableOptions(); + + List<String> getMergedPartitionKeys(); + + Optional<TableDistribution> getMergedTableDistribution(); + + String getMergedDefinitionQuery(); + + ResolvedSchema getMergedQuerySchema(); + } + + protected abstract MergeContext getMergeContext( + T sqlCreateMaterializedTable, ConvertContext context); + + protected final Optional<TableDistribution> getDerivedTableDistribution( + T sqlCreateMaterializedTable) { + return Optional.ofNullable(sqlCreateMaterializedTable.getDistribution()) + .map(OperationConverterUtils::getDistributionFromSqlDistribution); + } + + protected final List<String> getDerivedPartitionKeys(T sqlCreateMaterializedTable) { + return OperationConverterUtils.getColumnNames( + sqlCreateMaterializedTable.getPartitionKeyList()); + } + + protected final Map<String, String> getDerivedTableOptions(T sqlCreateMaterializedTable) { + return OperationConverterUtils.getProperties(sqlCreateMaterializedTable.getPropertyList()); + } + + protected final IntervalFreshness getDerivedFreshness(T sqlCreateMaterializedTable) { + return Optional.ofNullable(sqlCreateMaterializedTable.getFreshness()) + .map(MaterializedTableUtils::getMaterializedTableFreshness) + .orElse(null); + } + + protected final ResolvedSchema getQueryResolvedSchema( + T sqlCreateMaterializedTable, ConvertContext context) { + SqlNode selectQuery = sqlCreateMaterializedTable.getAsQuery(); + SqlNode validateQuery = context.getSqlValidator().validate(selectQuery); + + PlannerQueryOperation queryOperation = + new PlannerQueryOperation( + context.toRelRoot(validateQuery).project(), + () -> context.toQuotedSqlString(validateQuery)); + return queryOperation.getResolvedSchema(); + } + + protected final LogicalRefreshMode getDerivedLogicalRefreshMode(T sqlCreateMaterializedTable) { + return MaterializedTableUtils.deriveLogicalRefreshMode( + sqlCreateMaterializedTable.getRefreshMode()); + } + + protected final RefreshMode getDerivedRefreshMode(LogicalRefreshMode logicalRefreshMode) { + return MaterializedTableUtils.fromLogicalRefreshModeToRefreshMode(logicalRefreshMode); + } + + protected final String getDerivedDefinitionQuery( + T sqlCreateMaterializedTable, ConvertContext context) { + SqlNode selectQuery = sqlCreateMaterializedTable.getAsQuery(); + SqlNode validatedQuery = context.getSqlValidator().validate(selectQuery); + return context.toQuotedSqlString(validatedQuery); + } + + protected final String getComment(T sqlCreateMaterializedTable) { + return OperationConverterUtils.getComment(sqlCreateMaterializedTable.getComment()); + } + + protected final ResolvedCatalogMaterializedTable getResolvedCatalogMaterializedTable( + T sqlCreateMaterializedTable, ConvertContext context) { + final MergeContext mergeContext = getMergeContext(sqlCreateMaterializedTable, context); + final List<String> partitionKeys = mergeContext.getMergedPartitionKeys(); + final Schema schema = mergeContext.getMergedSchema(); + final ResolvedSchema querySchema = mergeContext.getMergedQuerySchema(); + final Map<String, String> tableOptions = mergeContext.getMergedTableOptions(); + verifyPartitioningColumnsExist(querySchema, partitionKeys, tableOptions); + + final TableDistribution distribution = + mergeContext.getMergedTableDistribution().orElse(null); + final String comment = getComment(sqlCreateMaterializedTable); + final String definitionQuery = mergeContext.getMergedDefinitionQuery(); + final IntervalFreshness intervalFreshness = getDerivedFreshness(sqlCreateMaterializedTable); + final LogicalRefreshMode logicalRefreshMode = + getDerivedLogicalRefreshMode(sqlCreateMaterializedTable); + final RefreshMode refreshMode = getDerivedRefreshMode(logicalRefreshMode); + return context.getCatalogManager() + .resolveCatalogMaterializedTable( + CatalogMaterializedTable.newBuilder() + .schema(schema) + .comment(comment) + .distribution(distribution) + .partitionKeys(partitionKeys) + .options(tableOptions) + .definitionQuery(definitionQuery) Review Comment: Please use `expandedQuery` since `definitionQuery` is deprecated. Please provide a value for `originalQuery` as well -- 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
