|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.flink.table.planner.operations.converters; |
| 20 | + |
| 21 | +import org.apache.flink.sql.parser.ddl.SqlCreateMaterializedTable; |
| 22 | +import org.apache.flink.table.api.Schema; |
| 23 | +import org.apache.flink.table.api.ValidationException; |
| 24 | +import org.apache.flink.table.catalog.CatalogMaterializedTable; |
| 25 | +import org.apache.flink.table.catalog.CatalogMaterializedTable.LogicalRefreshMode; |
| 26 | +import org.apache.flink.table.catalog.CatalogMaterializedTable.RefreshMode; |
| 27 | +import org.apache.flink.table.catalog.IntervalFreshness; |
| 28 | +import org.apache.flink.table.catalog.ObjectIdentifier; |
| 29 | +import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; |
| 30 | +import org.apache.flink.table.catalog.ResolvedSchema; |
| 31 | +import org.apache.flink.table.catalog.TableDistribution; |
| 32 | +import org.apache.flink.table.catalog.UnresolvedIdentifier; |
| 33 | +import org.apache.flink.table.planner.operations.PlannerQueryOperation; |
| 34 | +import org.apache.flink.table.planner.utils.MaterializedTableUtils; |
| 35 | +import org.apache.flink.table.planner.utils.OperationConverterUtils; |
| 36 | +import org.apache.flink.table.types.logical.LogicalType; |
| 37 | +import org.apache.flink.table.types.logical.LogicalTypeFamily; |
| 38 | + |
| 39 | +import org.apache.calcite.sql.SqlNode; |
| 40 | + |
| 41 | +import java.util.List; |
| 42 | +import java.util.Map; |
| 43 | +import java.util.Optional; |
| 44 | +import java.util.Set; |
| 45 | +import java.util.stream.Collectors; |
| 46 | + |
| 47 | +import static org.apache.flink.table.api.config.MaterializedTableConfigOptions.DATE_FORMATTER; |
| 48 | +import static org.apache.flink.table.api.config.MaterializedTableConfigOptions.PARTITION_FIELDS; |
| 49 | + |
| 50 | +/** |
| 51 | + * Abstract class for converting {@link SqlCreateMaterializedTable} and it's children to create |
| 52 | + * materialized table operations. |
| 53 | + */ |
| 54 | +public abstract class AbstractCreateMaterializedTableConverter<T extends SqlCreateMaterializedTable> |
| 55 | + implements SqlNodeConverter<T> { |
| 56 | + /** Context of create table converters while merging source and derived items. */ |
| 57 | + protected interface MergeContext { |
| 58 | + Schema getMergedSchema(); |
| 59 | + |
| 60 | + Map<String, String> getMergedTableOptions(); |
| 61 | + |
| 62 | + List<String> getMergedPartitionKeys(); |
| 63 | + |
| 64 | + Optional<TableDistribution> getMergedTableDistribution(); |
| 65 | + |
| 66 | + String getMergedOriginalQuery(); |
| 67 | + |
| 68 | + String getMergedExpandedQuery(); |
| 69 | + |
| 70 | + ResolvedSchema getMergedQuerySchema(); |
| 71 | + } |
| 72 | + |
| 73 | + protected abstract MergeContext getMergeContext( |
| 74 | + T sqlCreateMaterializedTable, ConvertContext context); |
| 75 | + |
| 76 | + protected final Optional<TableDistribution> getDerivedTableDistribution( |
| 77 | + T sqlCreateMaterializedTable) { |
| 78 | + return Optional.ofNullable(sqlCreateMaterializedTable.getDistribution()) |
| 79 | + .map(OperationConverterUtils::getDistributionFromSqlDistribution); |
| 80 | + } |
| 81 | + |
| 82 | + protected final List<String> getDerivedPartitionKeys(T sqlCreateMaterializedTable) { |
| 83 | + return OperationConverterUtils.getColumnNames( |
| 84 | + sqlCreateMaterializedTable.getPartitionKeyList()); |
| 85 | + } |
| 86 | + |
| 87 | + protected final Map<String, String> getDerivedTableOptions(T sqlCreateMaterializedTable) { |
| 88 | + return OperationConverterUtils.getProperties(sqlCreateMaterializedTable.getPropertyList()); |
| 89 | + } |
| 90 | + |
| 91 | + protected final IntervalFreshness getDerivedFreshness(T sqlCreateMaterializedTable) { |
| 92 | + return Optional.ofNullable(sqlCreateMaterializedTable.getFreshness()) |
| 93 | + .map(MaterializedTableUtils::getMaterializedTableFreshness) |
| 94 | + .orElse(null); |
| 95 | + } |
| 96 | + |
| 97 | + protected final ResolvedSchema getQueryResolvedSchema( |
| 98 | + T sqlCreateMaterializedTable, ConvertContext context) { |
| 99 | + SqlNode selectQuery = sqlCreateMaterializedTable.getAsQuery(); |
| 100 | + SqlNode validateQuery = context.getSqlValidator().validate(selectQuery); |
| 101 | + |
| 102 | + PlannerQueryOperation queryOperation = |
| 103 | + new PlannerQueryOperation( |
| 104 | + context.toRelRoot(validateQuery).project(), |
| 105 | + () -> context.toQuotedSqlString(validateQuery)); |
| 106 | + return queryOperation.getResolvedSchema(); |
| 107 | + } |
| 108 | + |
| 109 | + protected final LogicalRefreshMode getDerivedLogicalRefreshMode(T sqlCreateMaterializedTable) { |
| 110 | + return MaterializedTableUtils.deriveLogicalRefreshMode( |
| 111 | + sqlCreateMaterializedTable.getRefreshMode()); |
| 112 | + } |
| 113 | + |
| 114 | + protected final RefreshMode getDerivedRefreshMode(LogicalRefreshMode logicalRefreshMode) { |
| 115 | + return MaterializedTableUtils.fromLogicalRefreshModeToRefreshMode(logicalRefreshMode); |
| 116 | + } |
| 117 | + |
| 118 | + protected final String getDerivedOriginalQuery( |
| 119 | + T sqlCreateMaterializedTable, ConvertContext context) { |
| 120 | + SqlNode selectQuery = sqlCreateMaterializedTable.getAsQuery(); |
| 121 | + return context.toQuotedSqlString(selectQuery); |
| 122 | + } |
| 123 | + |
| 124 | + protected final String getDerivedExpandedQuery( |
| 125 | + T sqlCreateMaterializedTable, ConvertContext context) { |
| 126 | + SqlNode selectQuery = sqlCreateMaterializedTable.getAsQuery(); |
| 127 | + SqlNode validatedQuery = context.getSqlValidator().validate(selectQuery); |
| 128 | + return context.expandSqlIdentifiers(context.toQuotedSqlString(validatedQuery)); |
| 129 | + } |
| 130 | + |
| 131 | + protected final String getComment(T sqlCreateMaterializedTable) { |
| 132 | + return OperationConverterUtils.getComment(sqlCreateMaterializedTable.getComment()); |
| 133 | + } |
| 134 | + |
| 135 | + protected final ResolvedCatalogMaterializedTable getResolvedCatalogMaterializedTable( |
| 136 | + T sqlCreateMaterializedTable, ConvertContext context) { |
| 137 | + final MergeContext mergeContext = getMergeContext(sqlCreateMaterializedTable, context); |
| 138 | + final List<String> partitionKeys = mergeContext.getMergedPartitionKeys(); |
| 139 | + final Schema schema = mergeContext.getMergedSchema(); |
| 140 | + final ResolvedSchema querySchema = mergeContext.getMergedQuerySchema(); |
| 141 | + final Map<String, String> tableOptions = mergeContext.getMergedTableOptions(); |
| 142 | + verifyPartitioningColumnsExist(querySchema, partitionKeys, tableOptions); |
| 143 | + |
| 144 | + final TableDistribution distribution = |
| 145 | + mergeContext.getMergedTableDistribution().orElse(null); |
| 146 | + final String comment = getComment(sqlCreateMaterializedTable); |
| 147 | + |
| 148 | + final String originalQuery = mergeContext.getMergedOriginalQuery(); |
| 149 | + final String expandedQuery = mergeContext.getMergedExpandedQuery(); |
| 150 | + |
| 151 | + final IntervalFreshness intervalFreshness = getDerivedFreshness(sqlCreateMaterializedTable); |
| 152 | + |
| 153 | + final LogicalRefreshMode logicalRefreshMode = |
| 154 | + getDerivedLogicalRefreshMode(sqlCreateMaterializedTable); |
| 155 | + |
| 156 | + final RefreshMode refreshMode = getDerivedRefreshMode(logicalRefreshMode); |
| 157 | + |
| 158 | + return context.getCatalogManager() |
| 159 | + .resolveCatalogMaterializedTable( |
| 160 | + CatalogMaterializedTable.newBuilder() |
| 161 | + .schema(schema) |
| 162 | + .comment(comment) |
| 163 | + .distribution(distribution) |
| 164 | + .partitionKeys(partitionKeys) |
| 165 | + .options(tableOptions) |
| 166 | + .originalQuery(originalQuery) |
| 167 | + .expandedQuery(expandedQuery) |
| 168 | + .freshness(intervalFreshness) |
| 169 | + .logicalRefreshMode(logicalRefreshMode) |
| 170 | + .refreshMode(refreshMode) |
| 171 | + .refreshStatus(CatalogMaterializedTable.RefreshStatus.INITIALIZING) |
| 172 | + .build()); |
| 173 | + } |
| 174 | + |
| 175 | + protected final ObjectIdentifier getIdentifier( |
| 176 | + SqlCreateMaterializedTable node, ConvertContext context) { |
| 177 | + UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(node.fullTableName()); |
| 178 | + return context.getCatalogManager().qualifyIdentifier(unresolvedIdentifier); |
| 179 | + } |
| 180 | + |
| 181 | + private void verifyPartitioningColumnsExist( |
| 182 | + ResolvedSchema schema, List<String> partitionKeys, Map<String, String> tableOptions) { |
| 183 | + final Set<String> partitionFieldOptions = |
| 184 | + tableOptions.keySet().stream() |
| 185 | + .filter(k -> k.startsWith(PARTITION_FIELDS)) |
| 186 | + .collect(Collectors.toSet()); |
| 187 | + |
| 188 | + for (String partitionKey : partitionKeys) { |
| 189 | + if (schema.getColumn(partitionKey).isEmpty()) { |
| 190 | + throw new ValidationException( |
| 191 | + String.format( |
| 192 | + "Partition column '%s' not defined in the query schema. Available columns: [%s].", |
| 193 | + partitionKey, |
| 194 | + schema.getColumnNames().stream() |
| 195 | + .collect(Collectors.joining("', '", "'", "'")))); |
| 196 | + } |
| 197 | + } |
| 198 | + |
| 199 | + // verify partition key used by materialized table partition option |
| 200 | + // partition.fields.#.date-formatter whether exist |
| 201 | + for (String partitionOption : partitionFieldOptions) { |
| 202 | + String partitionKey = |
| 203 | + partitionOption.substring( |
| 204 | + PARTITION_FIELDS.length() + 1, |
| 205 | + partitionOption.length() - (DATE_FORMATTER.length() + 1)); |
| 206 | + // partition key used in option partition.fields.#.date-formatter must be existed |
| 207 | + if (!partitionKeys.contains(partitionKey)) { |
| 208 | + throw new ValidationException( |
| 209 | + String.format( |
| 210 | + "Column '%s' referenced by materialized table option '%s' isn't a partition column. Available partition columns: [%s].", |
| 211 | + partitionKey, |
| 212 | + partitionOption, |
| 213 | + partitionKeys.stream() |
| 214 | + .collect(Collectors.joining("', '", "'", "'")))); |
| 215 | + } |
| 216 | + |
| 217 | + // partition key used in option partition.fields.#.date-formatter must be string type |
| 218 | + LogicalType partitionKeyType = |
| 219 | + schema.getColumn(partitionKey).get().getDataType().getLogicalType(); |
| 220 | + if (!partitionKeyType |
| 221 | + .getTypeRoot() |
| 222 | + .getFamilies() |
| 223 | + .contains(LogicalTypeFamily.CHARACTER_STRING)) { |
| 224 | + throw new ValidationException( |
| 225 | + String.format( |
| 226 | + "Materialized table option '%s' only supports referring to char, varchar and string type partition column. Column %s type is %s.", |
| 227 | + partitionOption, partitionKey, partitionKeyType.asSummaryString())); |
| 228 | + } |
| 229 | + } |
| 230 | + } |
| 231 | +} |
0 commit comments