diff --git a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd index ff10d47109a946..02ba1a1aaaaf01 100644 --- a/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd +++ b/flink-table/flink-sql-parser/src/main/codegen/data/Parser.tdd @@ -40,6 +40,7 @@ "org.apache.flink.sql.parser.ddl.SqlAlterDatabase" "org.apache.flink.sql.parser.ddl.SqlAlterFunction" "org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTable" + "org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableAsQuery" "org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableFreshness" "org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableOptions" "org.apache.flink.sql.parser.ddl.SqlAlterMaterializedTableRefreshMode" 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 b02be2ad72ced8..f371e6b0dd6847 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 @@ -2009,6 +2009,7 @@ SqlAlterMaterializedTable SqlAlterMaterializedTable() : SqlNodeList propertyKeyList = SqlNodeList.EMPTY; SqlNodeList partSpec = SqlNodeList.EMPTY; SqlNode freshness = null; + SqlNode asQuery = null; } { { startPos = getPos();} @@ -2089,6 +2090,15 @@ SqlAlterMaterializedTable SqlAlterMaterializedTable() : tableIdentifier, propertyKeyList); } + | + + asQuery = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) + { + return new SqlAlterMaterializedTableAsQuery( + startPos.plus(getPos()), + tableIdentifier, + asQuery); + } ) } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableAsQuery.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableAsQuery.java new file mode 100644 index 00000000000000..7bc10e2da1a549 --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/SqlAlterMaterializedTableAsQuery.java @@ -0,0 +1,58 @@ +/* + * 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.sql.parser.ddl; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.util.ImmutableNullableList; + +import java.util.List; + +/** + * SqlNode to describe the ALTER TABLE [catalogName.] [dataBasesName.]tableName AS <query> + * statement. + */ +public class SqlAlterMaterializedTableAsQuery extends SqlAlterMaterializedTable { + + private final SqlNode asQuery; + + public SqlAlterMaterializedTableAsQuery( + SqlParserPos pos, SqlIdentifier tableName, SqlNode asQuery) { + super(pos, tableName); + this.asQuery = asQuery; + } + + public SqlNode getAsQuery() { + return asQuery; + } + + @Override + public List getOperandList() { + return ImmutableNullableList.of(getTableName(), asQuery); + } + + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + super.unparse(writer, leftPrec, rightPrec); + writer.keyword("AS"); + asQuery.unparse(writer, leftPrec, rightPrec); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/MaterializedTableStatementParserTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/MaterializedTableStatementParserTest.java index e738060d882daf..dc4b429206e250 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/MaterializedTableStatementParserTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/MaterializedTableStatementParserTest.java @@ -218,6 +218,7 @@ void testAlterMaterializedTableSuspend() { .fails( "Encountered \"\" at line 1, column 28.\n" + "Was expecting one of:\n" + + " \"AS\" ...\n" + " \"RESET\" ...\n" + " \"SET\" ...\n" + " \"SUSPEND\" ...\n" @@ -384,6 +385,27 @@ void testAlterMaterializedTableReset() { + " "); } + @Test + void testAlterMaterializedTableAsQuery() { + final String sql = "ALTER MATERIALIZED TABLE tbl1 AS SELECT * FROM t"; + final String expected = "ALTER MATERIALIZED TABLE `TBL1` AS SELECT *\nFROM `T`"; + sql(sql).ok(expected); + + final String sql2 = "ALTER MATERIALIZED TABLE tbl1 AS SELECT * FROM t A^S^"; + sql(sql2) + .fails( + "Encountered \"\" at line 1, column 51.\n" + + "Was expecting one of:\n" + + " ...\n" + + " ...\n" + + " ...\n" + + " ...\n" + + " ...\n" + + " ...\n" + + " ...\n" + + " "); + } + @Test void testDropMaterializedTable() { final String sql = "DROP MATERIALIZED TABLE tbl1";