davidradl commented on code in PR #27222:
URL: https://github.com/apache/flink/pull/27222#discussion_r2513584962
##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateMaterializedTableConverter.java:
##########
@@ -18,234 +18,109 @@
package org.apache.flink.table.planner.operations.converters;
-import org.apache.flink.sql.parser.SqlConstraintValidator;
import org.apache.flink.sql.parser.ddl.SqlCreateMaterializedTable;
-import org.apache.flink.sql.parser.ddl.SqlRefreshMode;
-import org.apache.flink.sql.parser.ddl.constraint.SqlTableConstraint;
-import org.apache.flink.sql.parser.error.SqlValidateException;
+import org.apache.flink.sql.parser.ddl.SqlTableColumn.SqlRegularColumn;
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.Column;
-import org.apache.flink.table.catalog.IntervalFreshness;
-import org.apache.flink.table.catalog.ObjectIdentifier;
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.operations.Operation;
import
org.apache.flink.table.operations.materializedtable.CreateMaterializedTableOperation;
-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.flink.table.planner.operations.converters.table.MergeTableAsUtil;
import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
-import java.util.Arrays;
+import java.util.HashSet;
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;
-import static
org.apache.flink.table.catalog.IntervalFreshness.validateFreshnessForCron;
-
-/** A converter for {@link SqlCreateMaterializedTable}. */
+/**
+ * A converter for {@link SqlCreateMaterializedTable} to {@link
CreateMaterializedTableOperation}.
+ */
public class SqlCreateMaterializedTableConverter
- implements SqlNodeConverter<SqlCreateMaterializedTable> {
+ extends
AbstractCreateMaterializedTableConverter<SqlCreateMaterializedTable> {
@Override
public Operation convertSqlNode(
SqlCreateMaterializedTable sqlCreateMaterializedTable,
ConvertContext context) {
- UnresolvedIdentifier unresolvedIdentifier =
-
UnresolvedIdentifier.of(sqlCreateMaterializedTable.fullTableName());
- ObjectIdentifier identifier =
-
context.getCatalogManager().qualifyIdentifier(unresolvedIdentifier);
-
- // get comment
- String tableComment =
-
OperationConverterUtils.getComment(sqlCreateMaterializedTable.getComment());
-
- // get options
- final Map<String, String> tableOptions =
-
OperationConverterUtils.getProperties(sqlCreateMaterializedTable.getPropertyList());
-
- // get freshness
- IntervalFreshness intervalFreshness =
- Optional.ofNullable(sqlCreateMaterializedTable.getFreshness())
-
.map(MaterializedTableUtils::getMaterializedTableFreshness)
- .orElse(null);
-
- // Get the logical refresh mode from SQL
- SqlRefreshMode sqlRefreshMode =
-
Optional.ofNullable(sqlCreateMaterializedTable.getRefreshMode())
- .map(mode -> mode.getValueAs(SqlRefreshMode.class))
- .orElse(null);
-
- final LogicalRefreshMode logicalRefreshMode =
-
MaterializedTableUtils.deriveLogicalRefreshMode(sqlRefreshMode);
-
- // get the physical refresh mode from SQL
- final RefreshMode refreshMode =
- sqlRefreshMode == null
- ? null
- :
MaterializedTableUtils.fromSqltoRefreshMode(sqlRefreshMode);
-
- if (CatalogMaterializedTable.RefreshMode.FULL == refreshMode &&
intervalFreshness != null) {
- validateFreshnessForCron(intervalFreshness);
- }
-
- // get query schema and definition query
- SqlNode selectQuery = sqlCreateMaterializedTable.getAsQuery();
- SqlNode validatedQuery =
context.getSqlValidator().validate(selectQuery);
-
- String definitionQuery = context.toQuotedSqlString(validatedQuery);
-
- PlannerQueryOperation queryOperation =
- new PlannerQueryOperation(
- context.toRelRoot(validatedQuery).project(), () ->
definitionQuery);
-
- // get schema
- ResolvedSchema resolvedSchema = queryOperation.getResolvedSchema();
- Schema.Builder builder =
Schema.newBuilder().fromResolvedSchema(resolvedSchema);
-
- // get and verify partition key
- List<String> partitionKeys =
- OperationConverterUtils.getColumnNames(
- sqlCreateMaterializedTable.getPartitionKeyList());
- verifyPartitioningColumnsExist(
- resolvedSchema,
- partitionKeys,
- tableOptions.keySet().stream()
- .filter(k -> k.startsWith(PARTITION_FIELDS))
- .collect(Collectors.toSet()));
-
- // verify and build primary key
- sqlCreateMaterializedTable
- .getTableConstraint()
- .ifPresent(
- sqlTableConstraint ->
- verifyAndBuildPrimaryKey(
- builder, resolvedSchema,
sqlTableConstraint));
-
- Optional<TableDistribution> tableDistribution =
-
Optional.ofNullable(sqlCreateMaterializedTable.getDistribution())
-
.map(OperationConverterUtils::getDistributionFromSqlDistribution);
-
- CatalogMaterializedTable materializedTable =
- CatalogMaterializedTable.newBuilder()
- .schema(builder.build())
- .comment(tableComment)
- .distribution(tableDistribution.orElse(null))
- .partitionKeys(partitionKeys)
- .options(tableOptions)
- .definitionQuery(definitionQuery)
- .freshness(intervalFreshness)
- .logicalRefreshMode(logicalRefreshMode)
- .refreshMode(refreshMode)
-
.refreshStatus(CatalogMaterializedTable.RefreshStatus.INITIALIZING)
- .build();
-
return new CreateMaterializedTableOperation(
- identifier,
-
context.getCatalogManager().resolveCatalogMaterializedTable(materializedTable));
+ getIdentifier(sqlCreateMaterializedTable, context),
+
getResolvedCatalogMaterializedTable(sqlCreateMaterializedTable, context));
}
- private static void verifyPartitioningColumnsExist(
- ResolvedSchema resolvedSchema,
- List<String> partitionKeys,
- Set<String> partitionFieldOptions) {
- // verify partition key whether exists
- for (String partitionKey : partitionKeys) {
- if (!resolvedSchema.getColumn(partitionKey).isPresent()) {
- throw new ValidationException(
- String.format(
- "Partition column '%s' not defined in the
query schema. Available columns: [%s].",
- partitionKey,
- resolvedSchema.getColumnNames().stream()
- .collect(Collectors.joining("', '",
"'", "'"))));
+ @Override
+ protected MergeContext getMergeContext(
+ SqlCreateMaterializedTable sqlCreateMaterializedTable,
ConvertContext context) {
+ return new MergeContext() {
+ private final MergeTableAsUtil mergeTableAsUtil = new
MergeTableAsUtil(context);
+ private final String definitionQuery =
+
SqlCreateMaterializedTableConverter.this.getDerivedDefinitionQuery(
+ sqlCreateMaterializedTable, context);
+ private final ResolvedSchema querySchema =
+
SqlCreateMaterializedTableConverter.this.getQueryResolvedSchema(
+ sqlCreateMaterializedTable, context);
+
+ @Override
+ public Schema getMergedSchema() {
+ final Set<String> querySchemaColumnNames =
+ new HashSet<>(querySchema.getColumnNames());
+ final SqlNodeList sqlNodeList =
sqlCreateMaterializedTable.getColumnList();
+ for (SqlNode column : sqlNodeList) {
+ if (!(column instanceof SqlRegularColumn)) {
+ continue;
+ }
+
+ SqlRegularColumn physicalColumn = (SqlRegularColumn)
column;
+ if
(!querySchemaColumnNames.contains(physicalColumn.getName().getSimple())) {
+ throw new ValidationException(
+ "Physical columns defined in DDL must be used
in query. Defined in DDL and not used in a query column '"
Review Comment:
I am thinking this Validation message could be simplified.
Invalid as physical column {} is defined in the DDL, but is not used in a
query column.
--
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]