This is an automated email from the ASF dual-hosted git repository. snuyanzin pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit 29cc69d3308632ac5e1842b7182a89698d667d4e Author: Sergey Nuyanzin <[email protected]> AuthorDate: Tue Nov 11 13:50:34 2025 +0100 [hotfix][table] Simplify parsing of `REFRESH MODE` for `MATERIALIZED TABLE` --- .../flink-sql-parser/src/main/codegen/includes/parserImpls.ftl | 6 +++--- .../apache/flink/sql/parser/ddl/SqlCreateMaterializedTable.java | 9 ++++----- .../converters/AbstractCreateMaterializedTableConverter.java | 9 ++------- 3 files changed, 9 insertions(+), 15 deletions(-) diff --git a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl index f7818dc81d8..3bcaa7314d4 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl +++ b/flink-table/flink-sql-parser/src/main/codegen/includes/parserImpls.ftl @@ -1866,7 +1866,7 @@ SqlCreate SqlCreateMaterializedTable(Span s, boolean replace, boolean isTemporar SqlNodeList partitionColumns = SqlNodeList.EMPTY; SqlNodeList propertyList = SqlNodeList.EMPTY; SqlNode freshness = null; - SqlLiteral refreshMode = null; + SqlRefreshMode refreshMode = null; SqlNode asQuery = null; SqlParserPos pos = startPos; boolean isColumnsIdentifiersOnly = false; @@ -1934,12 +1934,12 @@ SqlCreate SqlCreateMaterializedTable(Span s, boolean replace, boolean isTemporar ( <FULL> { - refreshMode = SqlRefreshMode.FULL.symbol(getPos()); + refreshMode = SqlRefreshMode.FULL; } | <CONTINUOUS> { - refreshMode = SqlRefreshMode.CONTINUOUS.symbol(getPos()); + refreshMode = SqlRefreshMode.CONTINUOUS; } ) ] diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateMaterializedTable.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateMaterializedTable.java index 687226a6220..b1ebd322914 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateMaterializedTable.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlCreateMaterializedTable.java @@ -29,7 +29,6 @@ import org.apache.calcite.sql.SqlCreate; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlIntervalLiteral; import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlLiteral; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; import org.apache.calcite.sql.SqlOperator; @@ -69,7 +68,7 @@ public class SqlCreateMaterializedTable extends SqlCreate implements ExtendedSql private final @Nullable SqlIntervalLiteral freshness; - private final @Nullable SqlLiteral refreshMode; + private final @Nullable SqlRefreshMode refreshMode; private final SqlNode asQuery; @@ -84,7 +83,7 @@ public class SqlCreateMaterializedTable extends SqlCreate implements ExtendedSql SqlNodeList partitionKeyList, SqlNodeList propertyList, @Nullable SqlIntervalLiteral freshness, - @Nullable SqlLiteral refreshMode, + @Nullable SqlRefreshMode refreshMode, SqlNode asQuery) { super(OPERATOR, pos, false, false); this.tableName = requireNonNull(tableName, "tableName should not be null"); @@ -162,7 +161,7 @@ public class SqlCreateMaterializedTable extends SqlCreate implements ExtendedSql } @Nullable - public SqlLiteral getRefreshMode() { + public SqlRefreshMode getRefreshMode() { return refreshMode; } @@ -241,7 +240,7 @@ public class SqlCreateMaterializedTable extends SqlCreate implements ExtendedSql writer.newlineAndIndent(); writer.keyword("REFRESH_MODE"); writer.keyword("="); - refreshMode.unparse(writer, leftPrec, rightPrec); + writer.keyword(refreshMode.name()); } writer.newlineAndIndent(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractCreateMaterializedTableConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractCreateMaterializedTableConverter.java index fb61e48b7ef..36e3e150915 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractCreateMaterializedTableConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractCreateMaterializedTableConverter.java @@ -19,7 +19,6 @@ package org.apache.flink.table.planner.operations.converters; import org.apache.flink.sql.parser.ddl.SqlCreateMaterializedTable; -import org.apache.flink.sql.parser.ddl.SqlRefreshMode; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.CatalogMaterializedTable; @@ -106,12 +105,8 @@ public abstract class AbstractCreateMaterializedTableConverter<T extends SqlCrea } protected final LogicalRefreshMode getDerivedLogicalRefreshMode(T sqlCreateMaterializedTable) { - SqlRefreshMode sqlRefreshMode = - Optional.ofNullable(sqlCreateMaterializedTable.getRefreshMode()) - .map(mode -> mode.getValueAs(SqlRefreshMode.class)) - .orElse(null); - - return MaterializedTableUtils.deriveLogicalRefreshMode(sqlRefreshMode); + return MaterializedTableUtils.deriveLogicalRefreshMode( + sqlCreateMaterializedTable.getRefreshMode()); } protected final RefreshMode getDerivedRefreshMode(LogicalRefreshMode logicalRefreshMode) {
