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 338a0337c9889..88d3450d16f51 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 @@ -50,10 +50,10 @@ "org.apache.flink.sql.parser.ddl.SqlAlterDatabase" "org.apache.flink.sql.parser.ddl.SqlAlterFunction" "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTable" + "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableAsQuery" "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableDistribution.SqlAlterMaterializedTableAddDistribution" "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableDistribution.SqlAlterMaterializedTableModifyDistribution" "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableDropDistribution" - "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableAsQuery" "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableFreshness" "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableOptions" "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableRefreshMode" @@ -62,6 +62,10 @@ "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableResume" "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSuspend" "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableAddSchema" + "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableDropColumn" + "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableDropConstraint" + "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableDropPrimaryKey" + "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableDropWatermark" "org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableModifySchema" "org.apache.flink.sql.parser.ddl.materializedtable.SqlCreateOrAlterMaterializedTable" "org.apache.flink.sql.parser.ddl.materializedtable.SqlDropMaterializedTable" @@ -75,11 +79,11 @@ "org.apache.flink.sql.parser.ddl.table.SqlAlterTableAdd" "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDistribution.SqlAlterTableAddDistribution" "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDistribution.SqlAlterTableModifyDistribution" - "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDropColumn" - "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDropConstraint" "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDropDistribution" - "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDropPrimaryKey" - "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDropWatermark" + "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDrop.SqlAlterTableDropConstraint" + "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDrop.SqlAlterTableDropColumn" + "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDrop.SqlAlterTableDropPrimaryKey" + "org.apache.flink.sql.parser.ddl.table.SqlAlterTableDrop.SqlAlterTableDropWatermark" "org.apache.flink.sql.parser.ddl.table.SqlAlterTableModify" "org.apache.flink.sql.parser.ddl.table.SqlAlterTableOptions" "org.apache.flink.sql.parser.ddl.table.SqlAlterTableRename" 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 193dd959d39d5..362c9a8e106a4 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 @@ -1018,7 +1018,7 @@ SqlAlterTable SqlAlterTable() : startPos.plus(getPos()), tableIdentifier, new SqlNodeList( - Collections.singletonList(columnName), + List.of(columnName), getPos()), ifExists); } @@ -2041,6 +2041,7 @@ SqlAlterMaterializedTable SqlAlterMaterializedTable() : SqlNodeList partSpec = SqlNodeList.EMPTY; SqlNode freshness = null; SqlNode asQuery = null; + SqlIdentifier constraintName; AlterTableSchemaContext ctx = new AlterTableSchemaContext(); } { @@ -2184,11 +2185,51 @@ SqlAlterMaterializedTable SqlAlterMaterializedTable() : ctx.watermark); } | - { + + ( + { SqlIdentifier columnName = null; } + columnName = CompoundIdentifier() { + return new SqlAlterMaterializedTableDropColumn( + startPos.plus(getPos()), + tableIdentifier, + new SqlNodeList( + List.of(columnName), + getPos())); + } + | + { Pair columnWithTypePair = null; } + columnWithTypePair = ParenthesizedCompoundIdentifierList() { + return new SqlAlterMaterializedTableDropColumn( + startPos.plus(getPos()), + tableIdentifier, + columnWithTypePair.getKey()); + } + | + { return new SqlAlterMaterializedTableDropDistribution( - startPos.plus(getPos()), - tableIdentifier); + startPos.plus(getPos()), + tableIdentifier); + } + | + { + return new SqlAlterMaterializedTableDropPrimaryKey( + startPos.plus(getPos()), + tableIdentifier); } + | + constraintName = SimpleIdentifier() { + return new SqlAlterMaterializedTableDropConstraint( + startPos.plus(getPos()), + tableIdentifier, + constraintName); + } + | + { + return new SqlAlterMaterializedTableDropWatermark( + startPos.plus(getPos()), + tableIdentifier); + } + ) ) } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlAlterMaterializedTableDropDistribution.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlAlterMaterializedTableDropDistribution.java index 223771bd6c152..994b97abf9e15 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlAlterMaterializedTableDropDistribution.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlAlterMaterializedTableDropDistribution.java @@ -23,7 +23,6 @@ import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.parser.SqlParserPos; -import java.util.Collections; import java.util.List; /** @@ -38,7 +37,7 @@ public SqlAlterMaterializedTableDropDistribution(SqlParserPos pos, SqlIdentifier @Override public List getOperandList() { - return Collections.emptyList(); + return List.of(name); } @Override diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlAlterMaterializedTableSchema.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlAlterMaterializedTableSchema.java index bfc12536db7e7..51285718806f3 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlAlterMaterializedTableSchema.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlAlterMaterializedTableSchema.java @@ -42,7 +42,7 @@ /** * Abstract class to describe statements which are used to alter schema for materialized tables. See - * examples in javadoc for {@link SqlAlterMaterializedTableAddSchema}. + * examples in javadoc for child classes. */ public abstract class SqlAlterMaterializedTableSchema extends SqlAlterMaterializedTable implements ExtendedSqlNode { @@ -181,4 +181,130 @@ protected String getAlterOperation() { return "MODIFY"; } } + + /** + * Abstract class to describe statements which are used to drop schema components while altering + * schema of materialized tables. See examples in javadoc for child classes. + */ + public abstract static class SqlAlterMaterializedTableDropSchema + extends SqlAlterMaterializedTableSchema { + public SqlAlterMaterializedTableDropSchema( + SqlParserPos pos, SqlIdentifier materializedTableName) { + super(pos, materializedTableName, SqlNodeList.EMPTY, List.of(), null); + } + + protected abstract void unparseDropOperation(SqlWriter writer, int leftPrec, int rightPrec); + + @Override + protected String getAlterOperation() { + return "DROP"; + } + + @Override + public final void unparseAlterOperation(SqlWriter writer, int leftPrec, int rightPrec) { + super.unparseAlterOperation(writer, leftPrec, rightPrec); + unparseDropOperation(writer, leftPrec, rightPrec); + } + } + + /** ALTER MATERIALIZED TABLE [catalog_name.][db_name.]materialized_table_name DROP WATERMARK. */ + public static class SqlAlterMaterializedTableDropWatermark + extends SqlAlterMaterializedTableDropSchema { + public SqlAlterMaterializedTableDropWatermark( + SqlParserPos pos, SqlIdentifier materializedTableName) { + super(pos, materializedTableName); + } + + @Override + public List getOperandList() { + return List.of(name); + } + + @Override + public void unparseDropOperation(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("WATERMARK"); + } + } + + /** + * ALTER MATERIALIZED TABLE [catalog_name.][db_name.]materialized_table_name DROP PRIMARY KEY. + */ + public static class SqlAlterMaterializedTableDropPrimaryKey + extends SqlAlterMaterializedTableDropSchema { + public SqlAlterMaterializedTableDropPrimaryKey( + SqlParserPos pos, SqlIdentifier materializedTableName) { + super(pos, materializedTableName); + } + + @Override + public void unparseDropOperation(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("PRIMARY KEY"); + } + } + + /** + * ALTER MATERIALIZED TABLE [catalog_name.][db_name.]materialized_table_name DROP CONSTRAINT + * constraint_name. + */ + public static class SqlAlterMaterializedTableDropConstraint + extends SqlAlterMaterializedTableDropSchema { + private final SqlIdentifier constraintName; + + public SqlAlterMaterializedTableDropConstraint( + SqlParserPos pos, SqlIdentifier tableName, SqlIdentifier constraintName) { + super(pos, tableName); + this.constraintName = constraintName; + } + + public SqlIdentifier getConstraintName() { + return constraintName; + } + + @Override + public void unparseDropOperation(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("CONSTRAINT"); + constraintName.unparse(writer, leftPrec, rightPrec); + } + } + + /** + * SqlNode to describe ALTER MATERIALIZED TABLE materialized_table_name DROP column clause. + * + *

Example: DDL like the below for drop column. + * + *

{@code
+     * -- drop single column
+     * ALTER MATERIALIZED TABLE prod.db.sample DROP col1;
+     *
+     * -- drop multiple columns
+     * ALTER MATERIALIZED TABLE prod.db.sample DROP (col1, col2, col3);
+     * }
+ */ + public static class SqlAlterMaterializedTableDropColumn + extends SqlAlterMaterializedTableDropSchema { + + private final SqlNodeList columnList; + + public SqlAlterMaterializedTableDropColumn( + SqlParserPos pos, SqlIdentifier tableName, SqlNodeList columnList) { + super(pos, tableName); + this.columnList = columnList; + } + + public SqlNodeList getColumnList() { + return columnList; + } + + @Override + public List getOperandList() { + return List.of(name, columnList); + } + + @Override + public void unparseDropOperation(SqlWriter writer, int leftPrec, int rightPrec) { + // unparse materialized table column + SqlUnparseUtils.unparseTableSchema( + columnList, List.of(), null, writer, leftPrec, rightPrec); + } + } } diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlDropMaterializedTable.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlDropMaterializedTable.java index cfbc168bb1ed7..91b4d7bddbee5 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlDropMaterializedTable.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/materializedtable/SqlDropMaterializedTable.java @@ -22,11 +22,14 @@ import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlSpecialOperator; import org.apache.calcite.sql.SqlWriter; import org.apache.calcite.sql.parser.SqlParserPos; +import java.util.List; + /** DROP MATERIALIZED TABLE DDL sql call. */ public class SqlDropMaterializedTable extends SqlDropObject { @@ -38,6 +41,11 @@ public SqlDropMaterializedTable( super(OPERATOR, pos, tableIdentifier, ifExists); } + @Override + public List getOperandList() { + return List.of(name); + } + @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { writer.keyword("DROP MATERIALIZED TABLE"); diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDrop.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDrop.java new file mode 100644 index 0000000000000..c2681f4cd64f3 --- /dev/null +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDrop.java @@ -0,0 +1,187 @@ +/* + * 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.table; + +import org.apache.flink.sql.parser.SqlUnparseUtils; +import org.apache.flink.sql.parser.ddl.SqlWatermark; +import org.apache.flink.sql.parser.ddl.constraint.SqlTableConstraint; + +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.parser.SqlParserPos; + +import javax.annotation.Nullable; + +import java.util.List; + +/** + * SqlNode to describe ALTER TABLE [IF EXISTS ]table_name DROP column/constraint/watermark clause. + * + *

Example: DDL like the below for dropping column/constraint/watermark. + * + *

{@code
+ * -- drop a column (only drop of non persisted is allowed)
+ * ALTER MATERIALIZED TABLE materializedTable DROP col1;
+ *
+ * -- drop several columns
+ * ALTER MATERIALIZED TABLE materializedTable DROP (col1, col2, col3);
+ *
+ * -- drop a primary key
+ * ALTER MATERIALIZED TABLE materializedTable DROP PRIMARY KEY;
+ *
+ * -- drop a constraint by name
+ * ALTER MATERIALIZED TABLE materializedTable DROP CONSTRAINT constraint_name;
+ *
+ * -- drop a watermark
+ * ALTER MATERIALIZED TABLE materializedTable DROP WATERMARK;
+ * }
+ */ +public abstract class SqlAlterTableDrop extends SqlAlterTableSchema { + + public SqlAlterTableDrop( + SqlParserPos pos, + SqlIdentifier tableName, + SqlNodeList modifiedColumns, + List constraints, + @Nullable SqlWatermark watermark, + boolean ifTableExists) { + super(pos, tableName, modifiedColumns, constraints, watermark, ifTableExists); + } + + public SqlAlterTableDrop(SqlParserPos pos, SqlIdentifier tableName, boolean ifTableExists) { + super(pos, tableName, SqlNodeList.EMPTY, List.of(), null, ifTableExists); + } + + protected abstract void unparseDropOperation(SqlWriter writer, int leftPrec, int rightPrec); + + @Override + public List getOperandList() { + return List.of(tableIdentifier); + } + + @Override + protected String getAlterOperation() { + return "DROP"; + } + + @Override + public void unparseAlterOperation(SqlWriter writer, int leftPrec, int rightPrec) { + super.unparseAlterOperation(writer, leftPrec, rightPrec); + unparseDropOperation(writer, leftPrec, rightPrec); + } + + /** ALTER TABLE [IF EXISTS ][catalog_name.][db_name.]table_name DROP PRIMARY KEY. */ + public static class SqlAlterTableDropPrimaryKey extends SqlAlterTableDrop { + + public SqlAlterTableDropPrimaryKey( + SqlParserPos pos, SqlIdentifier tableName, boolean ifTableExists) { + super(pos, tableName, ifTableExists); + } + + @Override + protected void unparseDropOperation(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("PRIMARY KEY"); + } + } + + /** + * ALTER TABLE [IF EXISTS ][catalog_name.][db_name.]table_name DROP CONSTRAINT constraint_name. + */ + public static class SqlAlterTableDropConstraint extends SqlAlterTableDrop { + private final SqlIdentifier constraintName; + + public SqlAlterTableDropConstraint( + SqlParserPos pos, + SqlIdentifier tableName, + SqlIdentifier constraintName, + boolean ifTableExists) { + super(pos, tableName, ifTableExists); + this.constraintName = constraintName; + } + + public SqlIdentifier getConstraintName() { + return constraintName; + } + + @Override + protected void unparseDropOperation(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("CONSTRAINT"); + constraintName.unparse(writer, leftPrec, rightPrec); + } + } + + /** ALTER TABLE [IF EXISTS ][catalog_name.][db_name.]table_name DROP WATERMARK. */ + public static class SqlAlterTableDropWatermark extends SqlAlterTableDrop { + + public SqlAlterTableDropWatermark( + SqlParserPos pos, SqlIdentifier tableName, boolean ifTableExists) { + super(pos, tableName, ifTableExists); + } + + @Override + protected void unparseDropOperation(SqlWriter writer, int leftPrec, int rightPrec) { + writer.keyword("WATERMARK"); + } + } + + /** + * SqlNode to describe ALTER TABLE [IF EXISTS ]table_name DROP column clause. + * + *

Example: DDL like the below for drop column. + * + *

{@code
+     * -- drop single column
+     * ALTER TABLE prod.db.sample DROP col1;
+     *
+     * -- drop multiple columns
+     * ALTER TABLE prod.db.sample DROP (col1, col2, col3);
+     * }
+ */ + public static class SqlAlterTableDropColumn extends SqlAlterTableDrop { + + private final SqlNodeList columnList; + + public SqlAlterTableDropColumn( + SqlParserPos pos, + SqlIdentifier tableName, + SqlNodeList columnList, + boolean ifTableExists) { + super(pos, tableName, ifTableExists); + this.columnList = columnList; + } + + @Override + public List getOperandList() { + return List.of(tableIdentifier, columnList); + } + + public SqlNodeList getColumnList() { + return columnList; + } + + @Override + protected void unparseDropOperation(SqlWriter writer, int leftPrec, int rightPrec) { + // unparse table column + SqlUnparseUtils.unparseTableSchema( + columnList, List.of(), null, writer, leftPrec, rightPrec); + } + } +} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropColumn.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropColumn.java deleted file mode 100644 index fadc5c6efed75..0000000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropColumn.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.table; - -import org.apache.flink.sql.parser.SqlUnparseUtils; - -import org.apache.calcite.sql.SqlIdentifier; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlNodeList; -import org.apache.calcite.sql.SqlWriter; -import org.apache.calcite.sql.parser.SqlParserPos; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - -/** - * SqlNode to describe ALTER TABLE [IF EXISTS] table_name DROP column clause. - * - *

Example: DDL like the below for drop column. - * - *

{@code
- * -- drop single column
- * ALTER TABLE prod.db.sample DROP col1;
- *
- * -- drop multiple columns
- * ALTER TABLE prod.db.sample DROP (col1, col2, col3);
- * }
- */ -public class SqlAlterTableDropColumn extends SqlAlterTable { - - private final SqlNodeList columnList; - - public SqlAlterTableDropColumn( - SqlParserPos pos, - SqlIdentifier tableName, - SqlNodeList columnList, - boolean ifTableExists) { - super(pos, tableName, ifTableExists); - this.columnList = columnList; - } - - @Override - public List getOperandList() { - return Arrays.asList(tableIdentifier, columnList); - } - - public SqlNodeList getColumnList() { - return columnList; - } - - @Override - public void unparseAlterOperation(SqlWriter writer, int leftPrec, int rightPrec) { - super.unparseAlterOperation(writer, leftPrec, rightPrec); - writer.keyword("DROP"); - // unparse table column - SqlUnparseUtils.unparseTableSchema( - columnList, Collections.emptyList(), null, writer, leftPrec, rightPrec); - } -} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropConstraint.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropConstraint.java deleted file mode 100644 index e3483643992b1..0000000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropConstraint.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.table; - -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; - -/** ALTER TABLE [IF EXISTS] [catalog_name.][db_name.]table_name DROP CONSTRAINT constraint_name. */ -public class SqlAlterTableDropConstraint extends SqlAlterTable { - - private final SqlIdentifier constraintName; - - /** - * Creates an alter table drop constraint node. - * - * @param pos Parser position - * @param tableName Table name - * @param constraintName Constraint name - * @param ifTableExists Whether IF EXISTS is specified - */ - public SqlAlterTableDropConstraint( - SqlParserPos pos, - SqlIdentifier tableName, - SqlIdentifier constraintName, - boolean ifTableExists) { - super(pos, tableName, ifTableExists); - this.constraintName = constraintName; - } - - public SqlIdentifier getConstraintName() { - return constraintName; - } - - @Override - public List getOperandList() { - return ImmutableNullableList.of(getTableName(), this.constraintName); - } - - @Override - public void unparseAlterOperation(SqlWriter writer, int leftPrec, int rightPrec) { - super.unparseAlterOperation(writer, leftPrec, rightPrec); - writer.keyword("DROP CONSTRAINT"); - this.constraintName.unparse(writer, leftPrec, rightPrec); - } -} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropPrimaryKey.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropPrimaryKey.java deleted file mode 100644 index f8a12f47112b4..0000000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropPrimaryKey.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.table; - -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 java.util.Collections; -import java.util.List; - -/** ALTER TABLE [IF EXISTS] [catalog_name.][db_name.]table_name DROP PRIMARY KEY. */ -public class SqlAlterTableDropPrimaryKey extends SqlAlterTable { - - public SqlAlterTableDropPrimaryKey( - SqlParserPos pos, SqlIdentifier tableName, boolean ifTableExists) { - super(pos, tableName, ifTableExists); - } - - @Override - public List getOperandList() { - return Collections.singletonList(tableIdentifier); - } - - @Override - public void unparseAlterOperation(SqlWriter writer, int leftPrec, int rightPrec) { - super.unparseAlterOperation(writer, leftPrec, rightPrec); - writer.keyword("DROP PRIMARY KEY"); - } -} diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropWatermark.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropWatermark.java deleted file mode 100644 index 28b1364abc478..0000000000000 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/ddl/table/SqlAlterTableDropWatermark.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * 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.table; - -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 java.util.Collections; -import java.util.List; - -/** - * SqlNode to describe ALTER TABLE [IF EXISTS] table_name DROP watermark clause. - * - *

Example: DDL like the below for drop watermark. - * - *

{@code
- * -- drop watermark
- *  ALTER TABLE prod.db.sample DROP WATERMARK;
- * }
- */ -public class SqlAlterTableDropWatermark extends SqlAlterTable { - - public SqlAlterTableDropWatermark( - SqlParserPos pos, SqlIdentifier tableName, boolean ifTableExists) { - super(pos, tableName, ifTableExists); - } - - @Override - public List getOperandList() { - return Collections.emptyList(); - } - - @Override - public void unparseAlterOperation(SqlWriter writer, int leftPrec, int rightPrec) { - super.unparseAlterOperation(writer, leftPrec, rightPrec); - writer.keyword("DROP WATERMARK"); - } -} 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 91d5f3c0d33c8..1ea1855e223d8 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 @@ -619,6 +619,27 @@ void testAddDistribution() { .ok("ALTER MATERIALIZED TABLE `MT1` ADD DISTRIBUTION BY (`A`, `H`)"); } + @Test + void testAlterMaterializedTableDrop() { + sql("alter materialized table mt1 drop distribution") + .ok("ALTER MATERIALIZED TABLE `MT1` DROP DISTRIBUTION"); + + sql("alter materialized table mt1 drop primary key") + .ok("ALTER MATERIALIZED TABLE `MT1` DROP PRIMARY KEY"); + + sql("alter materialized table mt1 drop constraint pk_mt") + .ok("ALTER MATERIALIZED TABLE `MT1` DROP CONSTRAINT `PK_MT`"); + + sql("alter materialized table mt1 drop watermark") + .ok("ALTER MATERIALIZED TABLE `MT1` DROP WATERMARK"); + + sql("alter materialized table mt1 drop c1") + .ok("ALTER MATERIALIZED TABLE `MT1` DROP (\n `C1`\n)"); + + sql("alter materialized table mt1 drop (c1, c2, c3)") + .ok("ALTER MATERIALIZED TABLE `MT1` DROP (\n `C1`,\n `C2`,\n `C3`\n)"); + } + @Test void testDropMaterializedTable() { final String sql = "DROP MATERIALIZED TABLE tbl1"; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlNodeToOperationConversion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlNodeToOperationConversion.java index 0f79073262530..4fcc3d7340283 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlNodeToOperationConversion.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/SqlNodeToOperationConversion.java @@ -20,12 +20,9 @@ import org.apache.flink.sql.parser.ddl.SqlAddJar; import org.apache.flink.sql.parser.ddl.SqlAlterDatabase; -import org.apache.flink.sql.parser.ddl.SqlAlterFunction; import org.apache.flink.sql.parser.ddl.SqlCompilePlan; import org.apache.flink.sql.parser.ddl.SqlCreateDatabase; -import org.apache.flink.sql.parser.ddl.SqlCreateFunction; import org.apache.flink.sql.parser.ddl.SqlDropDatabase; -import org.apache.flink.sql.parser.ddl.SqlDropFunction; import org.apache.flink.sql.parser.ddl.SqlRemoveJar; import org.apache.flink.sql.parser.ddl.SqlReplaceTableAs; import org.apache.flink.sql.parser.ddl.SqlReset; @@ -35,9 +32,6 @@ import org.apache.flink.sql.parser.ddl.SqlUseModules; import org.apache.flink.sql.parser.ddl.catalog.SqlDropCatalog; import org.apache.flink.sql.parser.ddl.catalog.SqlUseCatalog; -import org.apache.flink.sql.parser.ddl.resource.SqlResource; -import org.apache.flink.sql.parser.ddl.resource.SqlResourceType; -import org.apache.flink.sql.parser.ddl.table.SqlAnalyzeTable; import org.apache.flink.sql.parser.ddl.table.SqlCreateTableAs; import org.apache.flink.sql.parser.ddl.table.SqlDropTable; import org.apache.flink.sql.parser.ddl.view.SqlDropView; @@ -61,38 +55,20 @@ import org.apache.flink.sql.parser.dql.SqlShowJobs; import org.apache.flink.sql.parser.dql.SqlShowModules; import org.apache.flink.sql.parser.dql.SqlUnloadModule; -import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Catalog; -import org.apache.flink.table.catalog.CatalogBaseTable; -import org.apache.flink.table.catalog.CatalogBaseTable.TableKind; import org.apache.flink.table.catalog.CatalogDatabase; import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.CatalogFunction; -import org.apache.flink.table.catalog.CatalogFunctionImpl; import org.apache.flink.table.catalog.CatalogManager; -import org.apache.flink.table.catalog.CatalogPartitionSpec; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ContextResolvedTable; -import org.apache.flink.table.catalog.FunctionLanguage; import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.abilities.SupportsDeletePushDown; import org.apache.flink.table.connector.source.abilities.SupportsRowLevelModificationScan; -import org.apache.flink.table.expressions.CallExpression; -import org.apache.flink.table.expressions.Expression; -import org.apache.flink.table.expressions.FieldReferenceExpression; import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.expressions.ValueLiteralExpression; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; -import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.operations.BeginStatementSetOperation; import org.apache.flink.table.operations.CompileAndExecutePlanOperation; import org.apache.flink.table.operations.DeleteFromFilterOperation; @@ -124,32 +100,20 @@ import org.apache.flink.table.operations.command.ShowJarsOperation; import org.apache.flink.table.operations.command.ShowJobsOperation; import org.apache.flink.table.operations.command.StopJobOperation; -import org.apache.flink.table.operations.ddl.AlterCatalogFunctionOperation; import org.apache.flink.table.operations.ddl.AlterDatabaseOperation; -import org.apache.flink.table.operations.ddl.AnalyzeTableOperation; import org.apache.flink.table.operations.ddl.CompilePlanOperation; -import org.apache.flink.table.operations.ddl.CreateCatalogFunctionOperation; import org.apache.flink.table.operations.ddl.CreateDatabaseOperation; -import org.apache.flink.table.operations.ddl.CreateTempSystemFunctionOperation; -import org.apache.flink.table.operations.ddl.DropCatalogFunctionOperation; import org.apache.flink.table.operations.ddl.DropCatalogOperation; import org.apache.flink.table.operations.ddl.DropDatabaseOperation; import org.apache.flink.table.operations.ddl.DropTableOperation; -import org.apache.flink.table.operations.ddl.DropTempSystemFunctionOperation; import org.apache.flink.table.operations.ddl.DropViewOperation; import org.apache.flink.table.operations.utils.LikeType; import org.apache.flink.table.operations.utils.ShowLikeOperator; -import org.apache.flink.table.operations.utils.ValidationUtils; import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.hint.FlinkHints; import org.apache.flink.table.planner.operations.converters.SqlNodeConverters; import org.apache.flink.table.planner.utils.OperationConverterUtils; import org.apache.flink.table.planner.utils.RowLevelModificationContextUtils; -import org.apache.flink.table.resource.ResourceType; -import org.apache.flink.table.resource.ResourceUri; -import org.apache.flink.table.types.DataType; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.StringUtils; import org.apache.calcite.rel.RelRoot; import org.apache.calcite.rel.hint.HintStrategyTable; @@ -166,16 +130,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Time; -import java.sql.Timestamp; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -266,12 +221,6 @@ private static Optional convertValidatedSqlNode( return Optional.of(converter.convertShowColumns((SqlShowColumns) validated)); } else if (validated instanceof SqlDropView) { return Optional.of(converter.convertDropView((SqlDropView) validated)); - } else if (validated instanceof SqlCreateFunction) { - return Optional.of(converter.convertCreateFunction((SqlCreateFunction) validated)); - } else if (validated instanceof SqlDropFunction) { - return Optional.of(converter.convertDropFunction((SqlDropFunction) validated)); - } else if (validated instanceof SqlAlterFunction) { - return Optional.of(converter.convertAlterFunction((SqlAlterFunction) validated)); } else if (validated instanceof SqlShowCreateTable) { return Optional.of(converter.convertShowCreateTable((SqlShowCreateTable) validated)); } else if (validated instanceof SqlShowCreateMaterializedTable) { @@ -315,8 +264,6 @@ private static Optional convertValidatedSqlNode( } else if (validated instanceof SqlCompileAndExecutePlan) { return Optional.of( converter.convertCompileAndExecutePlan((SqlCompileAndExecutePlan) validated)); - } else if (validated instanceof SqlAnalyzeTable) { - return Optional.of(converter.convertAnalyzeTable((SqlAnalyzeTable) validated)); } else if (validated instanceof SqlStopJob) { return Optional.of(converter.convertStopJob((SqlStopJob) validated)); } else if (validated instanceof SqlDelete) { @@ -355,130 +302,6 @@ private Operation convertDropTable(SqlDropTable sqlDropTable) { identifier, sqlDropTable.getIfExists(), sqlDropTable.isTemporary()); } - /** Convert CREATE FUNCTION statement. */ - private Operation convertCreateFunction(SqlCreateFunction sqlCreateFunction) { - UnresolvedIdentifier unresolvedIdentifier = - UnresolvedIdentifier.of(sqlCreateFunction.getFullName()); - List resourceUris = getFunctionResources(sqlCreateFunction.getResourceInfos()); - final Map options = sqlCreateFunction.getProperties(); - if (sqlCreateFunction.isSystemFunction()) { - return new CreateTempSystemFunctionOperation( - unresolvedIdentifier.getObjectName(), - sqlCreateFunction.getFunctionClassName().getValueAs(String.class), - sqlCreateFunction.isIfNotExists(), - parseLanguage(sqlCreateFunction.getFunctionLanguage()), - resourceUris, - options); - } else { - FunctionLanguage language = parseLanguage(sqlCreateFunction.getFunctionLanguage()); - CatalogFunction catalogFunction = - new CatalogFunctionImpl( - sqlCreateFunction.getFunctionClassName().getValueAs(String.class), - language, - resourceUris, - options); - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); - - return new CreateCatalogFunctionOperation( - identifier, - catalogFunction, - sqlCreateFunction.isIfNotExists(), - sqlCreateFunction.isTemporary()); - } - } - - private List getFunctionResources(List sqlResources) { - return sqlResources.stream() - .map(SqlResource.class::cast) - .map( - sqlResource -> { - // get resource type - SqlResourceType sqlResourceType = - sqlResource.getResourceType().getValueAs(SqlResourceType.class); - ResourceType resourceType; - switch (sqlResourceType) { - case FILE: - resourceType = ResourceType.FILE; - break; - case JAR: - resourceType = ResourceType.JAR; - break; - case ARCHIVE: - resourceType = ResourceType.ARCHIVE; - break; - default: - throw new ValidationException( - String.format( - "Unsupported resource type: .", - sqlResourceType)); - } - // get resource path - String path = sqlResource.getResourcePath().getValueAs(String.class); - return new ResourceUri(resourceType, path); - }) - .collect(Collectors.toList()); - } - - /** Convert ALTER FUNCTION statement. */ - private Operation convertAlterFunction(SqlAlterFunction sqlAlterFunction) { - if (sqlAlterFunction.isSystemFunction()) { - throw new ValidationException("Alter temporary system function is not supported"); - } - - FunctionLanguage language = parseLanguage(sqlAlterFunction.getFunctionLanguage()); - CatalogFunction catalogFunction = - new CatalogFunctionImpl( - sqlAlterFunction.getFunctionClassName().getValueAs(String.class), language); - - UnresolvedIdentifier unresolvedIdentifier = - UnresolvedIdentifier.of(sqlAlterFunction.getFullName()); - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); - return new AlterCatalogFunctionOperation( - identifier, - catalogFunction, - sqlAlterFunction.isIfExists(), - sqlAlterFunction.isTemporary()); - } - - /** Convert DROP FUNCTION statement. */ - private Operation convertDropFunction(SqlDropFunction sqlDropFunction) { - UnresolvedIdentifier unresolvedIdentifier = - UnresolvedIdentifier.of(sqlDropFunction.getFullName()); - if (sqlDropFunction.isSystemFunction()) { - return new DropTempSystemFunctionOperation( - unresolvedIdentifier.getObjectName(), sqlDropFunction.getIfExists()); - } else { - ObjectIdentifier identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); - - return new DropCatalogFunctionOperation( - identifier, sqlDropFunction.getIfExists(), sqlDropFunction.isTemporary()); - } - } - - /** - * Converts language string to the FunctionLanguage. - * - * @param languageString the language string from SQL parser - * @return supported FunctionLanguage otherwise raise UnsupportedOperationException. - * @throws UnsupportedOperationException if the languageString is not parsable or language is - * not supported - */ - private FunctionLanguage parseLanguage(String languageString) { - if (StringUtils.isNullOrWhitespaceOnly(languageString)) { - return FunctionLanguage.JAVA; - } - - FunctionLanguage language; - try { - language = FunctionLanguage.valueOf(languageString); - } catch (IllegalArgumentException e) { - throw new UnsupportedOperationException( - String.format("Unrecognized function language string %s", languageString), e); - } - - return language; - } - /** Convert statement set into statement. */ private StatementSetOperation convertSqlStatementSet(SqlStatementSet statementSet) { return new StatementSetOperation( @@ -815,198 +638,6 @@ private Operation convertCompileAndExecutePlan(SqlCompileAndExecutePlan compileA compileAndExecutePlan.getOperandList().get(0))); } - private Operation convertAnalyzeTable(SqlAnalyzeTable analyzeTable) { - UnresolvedIdentifier unresolvedIdentifier = - UnresolvedIdentifier.of(analyzeTable.fullTableName()); - ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); - Optional optionalCatalogTable = - catalogManager.getTable(tableIdentifier); - if (!optionalCatalogTable.isPresent() || optionalCatalogTable.get().isTemporary()) { - throw new ValidationException( - String.format( - "Table %s doesn't exist or is a temporary table.", tableIdentifier)); - } - CatalogBaseTable baseTable = optionalCatalogTable.get().getResolvedTable(); - ValidationUtils.validateTableKind(baseTable, TableKind.TABLE, "analyze table"); - - CatalogTable table = (CatalogTable) baseTable; - ResolvedSchema schema = - baseTable.getUnresolvedSchema().resolve(catalogManager.getSchemaResolver()); - - LinkedHashMap partitions = analyzeTable.getPartitions(); - List targetPartitionSpecs = null; - if (table.isPartitioned()) { - if (!partitions.keySet().equals(new HashSet<>(table.getPartitionKeys()))) { - throw new ValidationException( - String.format( - "Invalid ANALYZE TABLE statement. For partition table, all partition keys should be specified explicitly. " - + "The given partition keys: [%s] are not match the target partition keys: [%s].", - String.join(",", partitions.keySet()), - String.join(",", table.getPartitionKeys()))); - } - - try { - targetPartitionSpecs = getPartitionSpecs(tableIdentifier, schema, partitions); - } catch (Exception e) { - throw new ValidationException(e.getMessage(), e); - } - } else if (!partitions.isEmpty()) { - throw new ValidationException( - String.format( - "Invalid ANALYZE TABLE statement. Table: %s is not a partition table, while partition values are given.", - tableIdentifier)); - } - - List columns = analyzeTable.getColumnNames(); - List targetColumns; - if (analyzeTable.isAllColumns()) { - Preconditions.checkArgument(columns.isEmpty()); - // computed column and metadata column will be ignored - targetColumns = - schema.getColumns().stream() - .filter(Column::isPhysical) - .collect(Collectors.toList()); - } else if (!columns.isEmpty()) { - targetColumns = - columns.stream() - .map( - c -> { - Optional colOpt = schema.getColumn(c); - if (!colOpt.isPresent()) { - throw new ValidationException( - String.format( - "Column: %s does not exist in the table: %s.", - c, tableIdentifier)); - } - Column col = colOpt.get(); - if (col instanceof Column.ComputedColumn) { - throw new ValidationException( - String.format( - "Column: %s is a computed column, ANALYZE TABLE does not support computed column.", - c)); - } else if (col instanceof Column.MetadataColumn) { - throw new ValidationException( - String.format( - "Column: %s is a metadata column, ANALYZE TABLE does not support metadata column.", - c)); - } else if (col instanceof Column.PhysicalColumn) { - return col; - } else { - throw new ValidationException( - "Unknown column class: " - + col.getClass().getSimpleName()); - } - }) - .collect(Collectors.toList()); - } else { - targetColumns = Collections.emptyList(); - } - - return new AnalyzeTableOperation(tableIdentifier, targetPartitionSpecs, targetColumns); - } - - private List getPartitionSpecs( - ObjectIdentifier tableIdentifier, - ResolvedSchema schema, - LinkedHashMap partitions) - throws TableNotPartitionedException, TableNotExistException { - List filters = new ArrayList<>(); - for (Map.Entry entry : partitions.entrySet()) { - if (entry.getValue() != null) { - CallExpression call = - CallExpression.temporary( - FunctionIdentifier.of("="), - BuiltInFunctionDefinitions.EQUALS, - Arrays.asList( - getPartitionKeyExpr(schema, entry.getKey()), - getPartitionValueExpr( - schema, entry.getKey(), entry.getValue())), - DataTypes.BOOLEAN()); - filters.add(call); - } - } - if (filters.isEmpty()) { - return catalogManager - .getCatalog(tableIdentifier.getCatalogName()) - .get() - .listPartitions(tableIdentifier.toObjectPath()); - } else { - return catalogManager - .getCatalog(tableIdentifier.getCatalogName()) - .get() - .listPartitionsByFilter(tableIdentifier.toObjectPath(), filters); - } - } - - private FieldReferenceExpression getPartitionKeyExpr( - ResolvedSchema schema, String partitionKey) { - int fieldIndex = schema.getColumnNames().indexOf(partitionKey); - if (fieldIndex < 0) { - throw new ValidationException( - String.format( - "Partition: %s does not exist in the schema: %s", - partitionKey, schema.getColumnNames())); - } - return new FieldReferenceExpression( - partitionKey, schema.getColumnDataTypes().get(fieldIndex), 0, fieldIndex); - } - - private ValueLiteralExpression getPartitionValueExpr( - ResolvedSchema schema, String partitionKey, String partitionValue) { - int fieldIndex = schema.getColumnNames().indexOf(partitionKey); - if (fieldIndex < 0) { - throw new ValidationException( - String.format( - "Partition: %s does not exist in the schema: %s", - partitionKey, schema.getColumnNames())); - } - DataType dataType = schema.getColumnDataTypes().get(fieldIndex); - if (partitionValue == null) { - return new ValueLiteralExpression(null, dataType.nullable()); - } - Object value; - switch (dataType.getLogicalType().getTypeRoot()) { - case CHAR: - case VARCHAR: - value = partitionValue; - break; - case TINYINT: - value = Byte.valueOf(partitionValue); - break; - case SMALLINT: - value = Short.valueOf(partitionValue); - break; - case INTEGER: - value = Integer.valueOf(partitionValue); - break; - case BIGINT: - value = Long.valueOf(partitionValue); - break; - case FLOAT: - value = Float.valueOf(partitionValue); - break; - case DOUBLE: - value = Double.valueOf(partitionValue); - break; - case DECIMAL: - value = new BigDecimal(partitionValue); - break; - case DATE: - value = Date.valueOf(partitionValue); - break; - case TIME_WITHOUT_TIME_ZONE: - value = Time.valueOf(partitionValue); - break; - case TIMESTAMP_WITHOUT_TIME_ZONE: - value = Timestamp.valueOf(partitionValue); - break; - default: - throw new UnsupportedOperationException( - "Unsupported partition value type: " + dataType.getLogicalType()); - } - return new ValueLiteralExpression(value, dataType.notNull()); - } - private Operation convertShowJobs(SqlShowJobs sqlStopJob) { return new ShowJobsOperation(); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/MergeTableAsUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/MergeTableAsUtil.java similarity index 99% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/MergeTableAsUtil.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/MergeTableAsUtil.java index 98536ff68e009..3264e3c392a23 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/MergeTableAsUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/MergeTableAsUtil.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters.table; +package org.apache.flink.table.planner.operations.converters; import org.apache.flink.sql.parser.ddl.SqlTableColumn; import org.apache.flink.sql.parser.ddl.SqlTableColumn.SqlComputedColumn; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/MergeTableLikeUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/MergeTableLikeUtil.java similarity index 98% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/MergeTableLikeUtil.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/MergeTableLikeUtil.java index 7fdcf65dd452c..a5d722282fd7c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/MergeTableLikeUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/MergeTableLikeUtil.java @@ -16,8 +16,9 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters.table; +package org.apache.flink.table.planner.operations.converters; +import org.apache.flink.annotation.Internal; import org.apache.flink.sql.parser.ddl.SqlTableColumn; import org.apache.flink.sql.parser.ddl.SqlTableColumn.SqlComputedColumn; import org.apache.flink.sql.parser.ddl.SqlTableColumn.SqlMetadataColumn; @@ -38,7 +39,6 @@ import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; -import org.apache.flink.table.planner.operations.converters.SqlNodeConverter; import org.apache.flink.table.types.logical.LogicalType; import org.apache.calcite.rel.type.RelDataType; @@ -56,7 +56,8 @@ import java.util.function.Function; /** A utility class with logic for handling the {@code CREATE TABLE ... LIKE} clause. */ -class MergeTableLikeUtil { +@Internal +public class MergeTableLikeUtil { /** Default merging strategy if given option was not provided explicitly by the user. */ private static final HashMap defaultMergingStrategies = new HashMap<>(); @@ -75,7 +76,7 @@ class MergeTableLikeUtil { private final Function escapeExpression; private final DataTypeFactory dataTypeFactory; - MergeTableLikeUtil( + public MergeTableLikeUtil( SqlValidator validator, Function escapeExpression, DataTypeFactory dataTypeFactory) { @@ -84,7 +85,7 @@ class MergeTableLikeUtil { this.dataTypeFactory = dataTypeFactory; } - MergeTableLikeUtil(SqlNodeConverter.ConvertContext context) { + public MergeTableLikeUtil(SqlNodeConverter.ConvertContext context) { this( context.getSqlValidator(), context::toQuotedSqlString, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SchemaBuilderUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaBuilderUtil.java similarity index 99% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SchemaBuilderUtil.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaBuilderUtil.java index 553f9a82f1eff..cfb5fb8abdbc9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SchemaBuilderUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaBuilderUtil.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters.table; +package org.apache.flink.table.planner.operations.converters; import org.apache.flink.sql.parser.ddl.SqlTableColumn.SqlComputedColumn; import org.apache.flink.sql.parser.ddl.SqlTableColumn.SqlMetadataColumn; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaConverter.java index c1c2becdbe877..a902dc376444f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaConverter.java @@ -227,7 +227,8 @@ private void applyColumnPosition(List alterColumns) { for (SqlNode alterColumn : alterColumns) { SqlTableColumnPosition columnPosition = (SqlTableColumnPosition) alterColumn; SqlTableColumn column = columnPosition.getColumn(); - String columnName = getColumnName(column.getName()); + String columnName = + OperationConverterUtils.extractSimpleColumnName(column.getName(), exMsgPrefix); if (!alterColNames.add(columnName)) { throw new ValidationException( String.format( @@ -241,11 +242,8 @@ protected String getReferencedColumn(SqlTableColumnPosition columnPosition) { SqlIdentifier referencedIdent = columnPosition.getAfterReferencedColumn(); Preconditions.checkNotNull( referencedIdent, String.format("%sCould not refer to a null column", exMsgPrefix)); - if (!referencedIdent.isSimple()) { - throw new UnsupportedOperationException( - String.format("%sAlter nested row type is not supported yet.", exMsgPrefix)); - } - String referencedName = referencedIdent.getSimple(); + String referencedName = + OperationConverterUtils.extractSimpleColumnName(referencedIdent, exMsgPrefix); if (!sortedColumnNames.contains(referencedName)) { throw new ValidationException( String.format( @@ -299,16 +297,6 @@ protected abstract void updatePositionAndCollectColumnChange( protected abstract void checkAndCollectWatermarkChange(); - protected String getColumnName(SqlIdentifier identifier) { - if (!identifier.isSimple()) { - throw new UnsupportedOperationException( - String.format( - "%sAlter nested row type %s is not supported yet.", - exMsgPrefix, identifier)); - } - return identifier.getSimple(); - } - protected T unwrap(Optional value) { return value.orElseThrow(() -> new TableException("The value should never be empty.")); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaReferencesManager.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaReferencesManager.java index 77b54156a467f..88e63447a56dd 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaReferencesManager.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SchemaReferencesManager.java @@ -22,7 +22,9 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedCatalogBaseTable; +import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.TableChange; import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.planner.expressions.ColumnReferenceFinder; import org.apache.flink.table.utils.EncodingUtils; @@ -31,6 +33,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.function.BiConsumer; import java.util.function.Function; @@ -84,16 +87,19 @@ private SchemaReferencesManager( this.distributionKeys = distributionKeys; } - public static SchemaReferencesManager create(ResolvedCatalogTable catalogTable) { + private static SchemaReferencesManager create( + ResolvedCatalogBaseTable catalogBaseTable, + Set partitionKeys, + Set distributionKeys) { Map> columnToReferences = new HashMap<>(); Map> columnToDependencies = new HashMap<>(); - catalogTable.getResolvedSchema().getColumns().stream() + catalogBaseTable.getResolvedSchema().getColumns().stream() .filter(column -> column instanceof Column.ComputedColumn) .forEach( column -> { Set referencedColumns = ColumnReferenceFinder.findReferencedColumn( - column.getName(), catalogTable.getResolvedSchema()); + column.getName(), catalogBaseTable.getResolvedSchema()); for (String referencedColumn : referencedColumns) { columnToReferences .computeIfAbsent(referencedColumn, key -> new HashSet<>()) @@ -105,22 +111,44 @@ public static SchemaReferencesManager create(ResolvedCatalogTable catalogTable) }); return new SchemaReferencesManager( - new HashSet<>(catalogTable.getResolvedSchema().getColumnNames()), + new HashSet<>(catalogBaseTable.getResolvedSchema().getColumnNames()), columnToReferences, columnToDependencies, - catalogTable + catalogBaseTable .getResolvedSchema() .getPrimaryKey() .map(constraint -> new HashSet<>(constraint.getColumns())) .orElse(new HashSet<>()), ColumnReferenceFinder.findWatermarkReferencedColumn( - catalogTable.getResolvedSchema()), - new HashSet<>(catalogTable.getPartitionKeys()), - new HashSet<>( - catalogTable - .getDistribution() - .map(TableDistribution::getBucketKeys) - .orElse(List.of()))); + catalogBaseTable.getResolvedSchema()), + partitionKeys, + distributionKeys); + } + + public static SchemaReferencesManager create(ResolvedCatalogBaseTable catalogTable) { + final Set partitionKeys; + final Set distributionKeys; + if (catalogTable instanceof ResolvedCatalogTable) { + partitionKeys = new HashSet<>(((ResolvedCatalogTable) catalogTable).getPartitionKeys()); + distributionKeys = + toDistributionKeys(((ResolvedCatalogTable) catalogTable).getDistribution()); + } else if (catalogTable instanceof ResolvedCatalogMaterializedTable) { + partitionKeys = + new HashSet<>( + ((ResolvedCatalogMaterializedTable) catalogTable).getPartitionKeys()); + distributionKeys = + toDistributionKeys( + ((ResolvedCatalogMaterializedTable) catalogTable).getDistribution()); + } else { + throw new UnsupportedOperationException( + "Schema references manager is unsupported for " + catalogTable.getClass()); + } + return create(catalogTable, partitionKeys, distributionKeys); + } + + private static Set toDistributionKeys(Optional tableDistribution) { + return new HashSet<>( + tableDistribution.map(TableDistribution::getBucketKeys).orElse(List.of())); } public void dropColumn(String columnName, Supplier errorMsg) { @@ -222,4 +250,28 @@ public static void buildUpdatedPrimaryKey( builder.primaryKeyNamed(constrainName, newPrimaryKeyNames); }); } + + public static Schema buildSchemaForAlterSchemaDrop( + ResolvedCatalogBaseTable oldTable, + List tableChanges, + Set columnsToDrop) { + Schema.Builder schemaBuilder = Schema.newBuilder(); + SchemaReferencesManager.buildUpdatedColumn( + schemaBuilder, + oldTable, + (builder, column) -> { + if (!columnsToDrop.contains(column.getName())) { + builder.fromColumns(List.of(column)); + } + }); + if (tableChanges.stream().noneMatch(c -> c instanceof TableChange.DropConstraint)) { + SchemaReferencesManager.buildUpdatedPrimaryKey( + schemaBuilder, oldTable, Function.identity()); + } + + if (tableChanges.stream().noneMatch(c -> c instanceof TableChange.DropWatermark)) { + SchemaReferencesManager.buildUpdatedWatermark(schemaBuilder, oldTable); + } + return schemaBuilder.build(); + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterFunctionConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterFunctionConverter.java new file mode 100644 index 0000000000000..43690d7d69600 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterFunctionConverter.java @@ -0,0 +1,56 @@ +/* + * 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.SqlAlterFunction; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogFunctionImpl; +import org.apache.flink.table.catalog.FunctionLanguage; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.ddl.AlterCatalogFunctionOperation; +import org.apache.flink.table.planner.utils.OperationConverterUtils; + +/** Convert ALTER FUNCTION statement. */ +public class SqlAlterFunctionConverter implements SqlNodeConverter { + @Override + public Operation convertSqlNode(SqlAlterFunction sqlAlterFunction, ConvertContext context) { + if (sqlAlterFunction.isSystemFunction()) { + throw new ValidationException("Alter temporary system function is not supported"); + } + + FunctionLanguage language = + OperationConverterUtils.parseLanguage(sqlAlterFunction.getFunctionLanguage()); + CatalogFunction catalogFunction = + new CatalogFunctionImpl( + sqlAlterFunction.getFunctionClassName().getValueAs(String.class), language); + + UnresolvedIdentifier unresolvedIdentifier = + UnresolvedIdentifier.of(sqlAlterFunction.getFullName()); + ObjectIdentifier identifier = + context.getCatalogManager().qualifyIdentifier(unresolvedIdentifier); + return new AlterCatalogFunctionOperation( + identifier, + catalogFunction, + sqlAlterFunction.isIfExists(), + sqlAlterFunction.isTemporary()); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateFunctionConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateFunctionConverter.java new file mode 100644 index 0000000000000..293e89d6106ec --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateFunctionConverter.java @@ -0,0 +1,72 @@ +/* + * 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.SqlCreateFunction; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogFunctionImpl; +import org.apache.flink.table.catalog.FunctionLanguage; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.ddl.CreateCatalogFunctionOperation; +import org.apache.flink.table.operations.ddl.CreateTempSystemFunctionOperation; +import org.apache.flink.table.planner.utils.OperationConverterUtils; +import org.apache.flink.table.resource.ResourceUri; + +import java.util.List; +import java.util.Map; + +/** Convert CREATE FUNCTION statement. */ +public class SqlCreateFunctionConverter implements SqlNodeConverter { + @Override + public Operation convertSqlNode(SqlCreateFunction sqlCreateFunction, ConvertContext context) { + UnresolvedIdentifier unresolvedIdentifier = + UnresolvedIdentifier.of(sqlCreateFunction.getFullName()); + List resourceUris = + OperationConverterUtils.getFunctionResources(sqlCreateFunction.getResourceInfos()); + final Map options = sqlCreateFunction.getProperties(); + final FunctionLanguage language = + OperationConverterUtils.parseLanguage(sqlCreateFunction.getFunctionLanguage()); + if (sqlCreateFunction.isSystemFunction()) { + return new CreateTempSystemFunctionOperation( + unresolvedIdentifier.getObjectName(), + sqlCreateFunction.getFunctionClassName().getValueAs(String.class), + sqlCreateFunction.isIfNotExists(), + language, + resourceUris, + options); + } else { + CatalogFunction catalogFunction = + new CatalogFunctionImpl( + sqlCreateFunction.getFunctionClassName().getValueAs(String.class), + language, + resourceUris, + options); + ObjectIdentifier identifier = + context.getCatalogManager().qualifyIdentifier(unresolvedIdentifier); + + return new CreateCatalogFunctionOperation( + identifier, + catalogFunction, + sqlCreateFunction.isIfNotExists(), + sqlCreateFunction.isTemporary()); + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateModelConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateModelConverter.java index 895143fad741e..c2278a99b696d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateModelConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateModelConverter.java @@ -28,7 +28,6 @@ import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateModelOperation; -import org.apache.flink.table.planner.operations.converters.table.SchemaBuilderUtil; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlNodeList; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlDropFunctionConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlDropFunctionConverter.java new file mode 100644 index 0000000000000..f11c476b5a5eb --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlDropFunctionConverter.java @@ -0,0 +1,45 @@ +/* + * 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.SqlDropFunction; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.ddl.DropCatalogFunctionOperation; +import org.apache.flink.table.operations.ddl.DropTempSystemFunctionOperation; + +/** Convert DROP FUNCTION statement. */ +public class SqlDropFunctionConverter implements SqlNodeConverter { + @Override + public Operation convertSqlNode(SqlDropFunction sqlDropFunction, ConvertContext context) { + UnresolvedIdentifier unresolvedIdentifier = + UnresolvedIdentifier.of(sqlDropFunction.getFullName()); + if (sqlDropFunction.isSystemFunction()) { + return new DropTempSystemFunctionOperation( + unresolvedIdentifier.getObjectName(), sqlDropFunction.getIfExists()); + } else { + ObjectIdentifier identifier = + context.getCatalogManager().qualifyIdentifier(unresolvedIdentifier); + + return new DropCatalogFunctionOperation( + identifier, sqlDropFunction.getIfExists(), sqlDropFunction.isTemporary()); + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConverters.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConverters.java index 44521e56f54b4..256dd03ccc1f7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConverters.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlNodeConverters.java @@ -21,24 +21,38 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.planner.operations.converters.SqlAlterMaterializedTableSchemaConverter.SqlAlterMaterializedTableAddSchemaConverter; -import org.apache.flink.table.planner.operations.converters.SqlAlterMaterializedTableSchemaConverter.SqlAlterMaterializedTableModifySchemaConverter; import org.apache.flink.table.planner.operations.converters.SqlNodeConverter.ConvertContext; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableAddDistributionConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableAsQueryConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableDropDistributionConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableDropSchemaConverter.SqlAlterMaterializedTableDropConstraintConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableDropSchemaConverter.SqlAlterMaterializedTableDropPrimaryKeyConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableDropSchemaConverter.SqlAlterMaterializedTableDropWatermarkConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableDropSchemaConverter.SqlAlterMaterializedTableSchemaDropColumnConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableModifyDistributionConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableRefreshConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableResumeConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableSchemaConverter.SqlAlterMaterializedTableAddSchemaConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableSchemaConverter.SqlAlterMaterializedTableModifySchemaConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlAlterMaterializedTableSuspendConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlCreateOrAlterMaterializedTableConverter; +import org.apache.flink.table.planner.operations.converters.materializedtable.SqlDropMaterializedTableConverter; import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableAddDistributionConverter; import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableAddPartitionConverter; -import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableDropColumnConverter; -import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableDropConstraintConverter; import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableDropDistributionConverter; import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableDropPartitionConverter; -import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableDropPrimaryKeyConverter; -import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableDropWatermarkConverter; import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableModifyDistributionConverter; import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableOptionsConverter; import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableRenameColumnConverter; import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableRenameConverter; import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableResetConverter; -import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableSchemaAddConverter; -import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableSchemaModifyConverter; +import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableSchemaConverter.SqlAlterTableSchemaAddConverter; +import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableSchemaConverter.SqlAlterTableSchemaDropColumnConverter; +import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableSchemaConverter.SqlAlterTableSchemaDropConstraintConverter; +import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableSchemaConverter.SqlAlterTableSchemaDropPrimaryKeyConverter; +import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableSchemaConverter.SqlAlterTableSchemaDropWatermarkConverter; +import org.apache.flink.table.planner.operations.converters.table.SqlAlterTableSchemaConverter.SqlAlterTableSchemaModifyConverter; +import org.apache.flink.table.planner.operations.converters.table.SqlAnalyzeTableConverter; import org.apache.flink.table.planner.operations.converters.table.SqlCreateTableAsConverter; import org.apache.flink.table.planner.operations.converters.table.SqlCreateTableConverter; import org.apache.flink.table.planner.operations.converters.table.SqlCreateTableLikeConverter; @@ -62,6 +76,9 @@ public class SqlNodeConverters { static { // register all the converters here register(new SqlDescribeFunctionConverter()); + register(new SqlAlterFunctionConverter()); + register(new SqlCreateFunctionConverter()); + register(new SqlDropFunctionConverter()); register(new SqlDescribeJobConverter()); register(new SqlProcedureCallConverter()); register(new SqlQueryConverter()); @@ -78,21 +95,22 @@ public class SqlNodeConverters { } private static void registerTableConverters() { + register(new SqlAnalyzeTableConverter()); register(new SqlAlterTableAddDistributionConverter()); register(new SqlAlterTableAddPartitionConverter()); - register(new SqlAlterTableDropPrimaryKeyConverter()); - register(new SqlAlterTableDropColumnConverter()); - register(new SqlAlterTableDropConstraintConverter()); + register(new SqlAlterTableSchemaDropColumnConverter()); register(new SqlAlterTableDropDistributionConverter()); register(new SqlAlterTableDropPartitionConverter()); - register(new SqlAlterTableDropWatermarkConverter()); register(new SqlAlterTableModifyDistributionConverter()); register(new SqlAlterTableOptionsConverter()); register(new SqlAlterTableRenameColumnConverter()); register(new SqlAlterTableRenameConverter()); register(new SqlAlterTableResetConverter()); - register(new SqlAlterTableSchemaModifyConverter()); register(new SqlAlterTableSchemaAddConverter()); + register(new SqlAlterTableSchemaDropConstraintConverter()); + register(new SqlAlterTableSchemaDropPrimaryKeyConverter()); + register(new SqlAlterTableSchemaDropWatermarkConverter()); + register(new SqlAlterTableSchemaModifyConverter()); register(new SqlCreateTableAsConverter()); register(new SqlCreateTableConverter()); register(new SqlCreateTableLikeConverter()); @@ -124,6 +142,10 @@ private static void registerMaterializedTableConverters() { register(new SqlAlterMaterializedTableAsQueryConverter()); register(new SqlAlterMaterializedTableDropDistributionConverter()); register(new SqlAlterMaterializedTableModifyDistributionConverter()); + register(new SqlAlterMaterializedTableDropPrimaryKeyConverter()); + register(new SqlAlterMaterializedTableDropConstraintConverter()); + register(new SqlAlterMaterializedTableSchemaDropColumnConverter()); + register(new SqlAlterMaterializedTableDropWatermarkConverter()); register(new SqlAlterMaterializedTableModifySchemaConverter()); register(new SqlAlterMaterializedTableRefreshConverter()); register(new SqlAlterMaterializedTableResumeConverter()); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractAlterMaterializedTableConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/AbstractAlterMaterializedTableConverter.java similarity index 96% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractAlterMaterializedTableConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/AbstractAlterMaterializedTableConverter.java index 4892a8b0b530b..fca8c516be262 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractAlterMaterializedTableConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/AbstractAlterMaterializedTableConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTable; import org.apache.flink.table.api.ValidationException; @@ -29,6 +29,7 @@ import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.utils.ValidationUtils; +import org.apache.flink.table.planner.operations.converters.SqlNodeConverter; import java.util.Optional; import java.util.function.Consumer; 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/materializedtable/AbstractCreateMaterializedTableConverter.java similarity index 98% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/AbstractCreateMaterializedTableConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/AbstractCreateMaterializedTableConverter.java index b7cdc48d66cf0..e04d98fb7984e 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/materializedtable/AbstractCreateMaterializedTableConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.materializedtable.SqlCreateMaterializedTable; import org.apache.flink.table.api.Schema; @@ -32,6 +32,7 @@ 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.operations.converters.SqlNodeConverter; import org.apache.flink.table.planner.utils.MaterializedTableUtils; import org.apache.flink.table.planner.utils.OperationConverterUtils; import org.apache.flink.table.types.logical.LogicalType; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableAddDistributionConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableAddDistributionConverter.java similarity index 97% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableAddDistributionConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableAddDistributionConverter.java index 5fb1b946314d6..9f95e4602ad7e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableAddDistributionConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableAddDistributionConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableDistribution; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableDistribution.SqlAlterMaterializedTableAddDistribution; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableAsQueryConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableAsQueryConverter.java similarity index 98% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableAsQueryConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableAsQueryConverter.java index 864dda6cc8564..ab8856dbe8af6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableAsQueryConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableAsQueryConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableAsQuery; import org.apache.flink.table.api.Schema; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableDropDistributionConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableDropDistributionConverter.java similarity index 96% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableDropDistributionConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableDropDistributionConverter.java index 2ab8f5efaaf25..1596907ab7296 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableDropDistributionConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableDropDistributionConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableDropDistribution; import org.apache.flink.table.api.ValidationException; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableDropSchemaConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableDropSchemaConverter.java new file mode 100644 index 0000000000000..63f5874262eee --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableDropSchemaConverter.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.materializedtable; + +import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableDropColumn; +import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableDropConstraint; +import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableDropPrimaryKey; +import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableDropSchema; +import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableDropWatermark; +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.Column; +import org.apache.flink.table.catalog.ResolvedCatalogMaterializedTable; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.materializedtable.AlterMaterializedTableChangeOperation; +import org.apache.flink.table.planner.operations.converters.SchemaReferencesManager; +import org.apache.flink.table.planner.utils.MaterializedTableUtils; +import org.apache.flink.table.planner.utils.OperationConverterUtils; + +import org.apache.calcite.sql.SqlIdentifier; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Abstract class for converters to convert {@code ALTER MATERIALIZED TABLE DROP } + * to generate an updated Schema. + */ +public abstract class SqlAlterMaterializedTableDropSchemaConverter< + T extends SqlAlterMaterializedTableDropSchema> + extends AbstractAlterMaterializedTableConverter { + @Override + protected Operation convertToOperation( + T alterTableSchema, ResolvedCatalogMaterializedTable oldTable, ConvertContext context) { + Set columnsToDrop = getColumnsToDrop(alterTableSchema); + List tableChanges = + validateAndGatherDropChanges(alterTableSchema, oldTable, columnsToDrop, context); + Schema schema = + SchemaReferencesManager.buildSchemaForAlterSchemaDrop( + oldTable, tableChanges, columnsToDrop); + + CatalogMaterializedTable mtWithUpdatedSchemaAndQuery = + buildUpdatedMaterializedTable(oldTable, builder -> builder.schema(schema)); + + return new AlterMaterializedTableChangeOperation( + resolveIdentifier(alterTableSchema, context), + tableChanges, + mtWithUpdatedSchemaAndQuery); + } + + protected abstract List validateAndGatherDropChanges( + T alterTableSchema, + ResolvedCatalogMaterializedTable oldTable, + Set columnsToDrop, + ConvertContext context); + + protected abstract Set getColumnsToDrop(T alterTableSchema); + + /** + * Convert {@code ALTER TABLE MATERIALIZED TABLE DROP PRIMARY KEY} to generate an updated + * Schema. + */ + public static class SqlAlterMaterializedTableDropPrimaryKeyConverter + extends SqlAlterMaterializedTableDropSchemaConverter< + SqlAlterMaterializedTableDropPrimaryKey> { + @Override + protected List validateAndGatherDropChanges( + SqlAlterMaterializedTableDropPrimaryKey alterTableSchema, + ResolvedCatalogMaterializedTable oldTable, + Set columnsToDrop, + ConvertContext context) { + return OperationConverterUtils.validateAndGatherDropPrimaryKey( + oldTable, EX_MSG_PREFIX, "materialized table"); + } + + @Override + protected Set getColumnsToDrop( + SqlAlterMaterializedTableDropPrimaryKey alterTableSchema) { + return Set.of(); + } + } + + /** + * Convert {@code ALTER TABLE MATERIALIZED TABLE DROP CONSTRAINT constraint_name} to generate an + * updated Schema. + */ + public static class SqlAlterMaterializedTableDropConstraintConverter + extends SqlAlterMaterializedTableDropSchemaConverter< + SqlAlterMaterializedTableDropConstraint> { + + @Override + protected List validateAndGatherDropChanges( + SqlAlterMaterializedTableDropConstraint dropConstraint, + ResolvedCatalogMaterializedTable oldTable, + Set columnsToDrop, + ConvertContext context) { + return OperationConverterUtils.validateAndGatherDropConstraintChanges( + oldTable, + dropConstraint.getConstraintName(), + EX_MSG_PREFIX, + "materialized table"); + } + + @Override + protected Set getColumnsToDrop( + SqlAlterMaterializedTableDropConstraint alterTableSchema) { + return Set.of(); + } + } + + /** + * Convert {@code ALTER TABLE MATERIALIZED TABLE DROP WATERMARK} to generate an updated Schema. + */ + public static class SqlAlterMaterializedTableDropWatermarkConverter + extends SqlAlterMaterializedTableDropSchemaConverter< + SqlAlterMaterializedTableDropWatermark> { + + @Override + protected List validateAndGatherDropChanges( + SqlAlterMaterializedTableDropWatermark alterTableSchema, + ResolvedCatalogMaterializedTable oldTable, + Set columnsToDrop, + ConvertContext context) { + return OperationConverterUtils.validateAndGatherDropWatermarkChanges( + oldTable, EX_MSG_PREFIX, "materialized table"); + } + + @Override + protected Set getColumnsToDrop( + SqlAlterMaterializedTableDropWatermark alterTableSchema) { + return Set.of(); + } + } + + /** + * Convert {@code ALTER TABLE MATERIALIZED TABLE DROP column_name} in case of one column and + * {@code ALTER TABLE MATERIALIZED TABLE DROP (column_name [, column_name2])} in case of several + * columns to generate an updated Schema. + */ + public static class SqlAlterMaterializedTableSchemaDropColumnConverter + extends SqlAlterMaterializedTableDropSchemaConverter< + SqlAlterMaterializedTableDropColumn> { + + @Override + protected List validateAndGatherDropChanges( + SqlAlterMaterializedTableDropColumn alterTableSchema, + ResolvedCatalogMaterializedTable oldTable, + Set columnsToDrop, + ConvertContext context) { + List tableChanges = + OperationConverterUtils.validateAndGatherDropColumn( + oldTable, columnsToDrop, EX_MSG_PREFIX); + validateColumnsUsedInQuery(oldTable, alterTableSchema, context); + for (Column column : oldTable.getResolvedSchema().getColumns()) { + if (column.isPersisted() && columnsToDrop.contains(column.getName())) { + throw new ValidationException( + String.format( + "%sThe column `%s` is a persisted column. Dropping of persisted columns is not supported.", + EX_MSG_PREFIX, column.getName())); + } + } + return tableChanges; + } + + @Override + protected Set getColumnsToDrop( + SqlAlterMaterializedTableDropColumn alterTableSchema) { + return OperationConverterUtils.getColumnNames( + alterTableSchema.getColumnList(), EX_MSG_PREFIX); + } + + private static void validateColumnsUsedInQuery( + ResolvedCatalogMaterializedTable oldTable, + SqlAlterMaterializedTableDropColumn alterTableSchema, + ConvertContext context) { + final ResolvedSchema querySchema = + MaterializedTableUtils.getQueryOperationResolvedSchema(oldTable, context); + + Set querySchemaColumnNames = new HashSet<>(querySchema.getColumnNames()); + querySchemaColumnNames.retainAll( + alterTableSchema.getColumnList().stream() + .map(c -> ((SqlIdentifier) c).getSimple()) + .collect(Collectors.toList())); + if (!querySchemaColumnNames.isEmpty()) { + throw new ValidationException( + String.format( + "%sColumn(s) (%s) are used in query.", + EX_MSG_PREFIX, + querySchemaColumnNames.stream() + .map(c -> "'" + c + "'") + .collect(Collectors.joining(", ")))); + } + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableModifyDistributionConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableModifyDistributionConverter.java similarity index 97% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableModifyDistributionConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableModifyDistributionConverter.java index a5c864688b623..1cf0ff095f5f3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableModifyDistributionConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableModifyDistributionConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableDistribution.SqlAlterMaterializedTableModifyDistribution; import org.apache.flink.table.api.ValidationException; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableRefreshConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableRefreshConverter.java similarity index 92% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableRefreshConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableRefreshConverter.java index f23aa7d2a2327..6fc95c0071ad9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableRefreshConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableRefreshConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.SqlParseUtils; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableRefresh; @@ -24,6 +24,7 @@ import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.materializedtable.AlterMaterializedTableRefreshOperation; +import org.apache.flink.table.planner.operations.converters.SqlNodeConverter; import java.util.Map; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableResumeConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableResumeConverter.java similarity index 92% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableResumeConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableResumeConverter.java index 40f0f026a44bd..628482d281b3d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableResumeConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableResumeConverter.java @@ -16,13 +16,14 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableResume; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.materializedtable.AlterMaterializedTableResumeOperation; +import org.apache.flink.table.planner.operations.converters.SqlNodeConverter; import java.util.Map; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableSchemaConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableSchemaConverter.java similarity index 96% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableSchemaConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableSchemaConverter.java index 909793e7e777e..7999c0c2cca64 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableSchemaConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableSchemaConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSchema.SqlAlterMaterializedTableAddSchema; @@ -37,6 +37,9 @@ import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.materializedtable.AlterMaterializedTableChangeOperation; +import org.apache.flink.table.planner.operations.converters.SchemaAddConverter; +import org.apache.flink.table.planner.operations.converters.SchemaConverter; +import org.apache.flink.table.planner.operations.converters.SchemaModifyConverter; import org.apache.flink.table.planner.utils.MaterializedTableUtils; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.utils.LogicalTypeCasts; @@ -72,7 +75,7 @@ protected Operation convertToOperation( return new AlterMaterializedTableChangeOperation( resolveIdentifier(alterTableSchema, context), - converter.changesCollector, + converter.getChangesCollector(), mtWithUpdatedSchemaAndQuery); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableSuspendConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableSuspendConverter.java similarity index 91% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableSuspendConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableSuspendConverter.java index 73d10e29e9bfe..d0cba0d2b9817 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlAlterMaterializedTableSuspendConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlAlterMaterializedTableSuspendConverter.java @@ -16,13 +16,14 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.materializedtable.SqlAlterMaterializedTableSuspend; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.materializedtable.AlterMaterializedTableSuspendOperation; +import org.apache.flink.table.planner.operations.converters.SqlNodeConverter; /** A converter for {@link SqlAlterMaterializedTableSuspend}. */ public class SqlAlterMaterializedTableSuspendConverter diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateOrAlterMaterializedTableConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlCreateOrAlterMaterializedTableConverter.java similarity index 98% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateOrAlterMaterializedTableConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlCreateOrAlterMaterializedTableConverter.java index 8af209da282c0..013198c796010 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlCreateOrAlterMaterializedTableConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlCreateOrAlterMaterializedTableConverter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.SqlTableColumn.SqlRegularColumn; import org.apache.flink.sql.parser.ddl.materializedtable.SqlCreateOrAlterMaterializedTable; @@ -38,7 +38,7 @@ import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.materializedtable.AlterMaterializedTableAsQueryOperation; import org.apache.flink.table.operations.materializedtable.CreateMaterializedTableOperation; -import org.apache.flink.table.planner.operations.converters.table.MergeTableAsUtil; +import org.apache.flink.table.planner.operations.converters.MergeTableAsUtil; import org.apache.flink.table.planner.utils.MaterializedTableUtils; import org.apache.calcite.sql.SqlNode; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlDropMaterializedTableConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlDropMaterializedTableConverter.java similarity index 92% rename from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlDropMaterializedTableConverter.java rename to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlDropMaterializedTableConverter.java index 4a76cab24b4a3..9b4de074cfb70 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/SqlDropMaterializedTableConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/materializedtable/SqlDropMaterializedTableConverter.java @@ -16,13 +16,14 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters; +package org.apache.flink.table.planner.operations.converters.materializedtable; import org.apache.flink.sql.parser.ddl.materializedtable.SqlDropMaterializedTable; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.materializedtable.DropMaterializedTableOperation; +import org.apache.flink.table.planner.operations.converters.SqlNodeConverter; /** A converter for {@link SqlDropMaterializedTable}. */ public class SqlDropMaterializedTableConverter diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/AbstractAlterTableConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/AbstractAlterTableConverter.java index c1a80f9c8f642..6723ec8f708b3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/AbstractAlterTableConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/AbstractAlterTableConverter.java @@ -21,7 +21,7 @@ import org.apache.flink.sql.parser.ddl.table.SqlAlterTable; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogBaseTable.TableKind; import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ContextResolvedTable; @@ -36,8 +36,6 @@ import org.apache.flink.table.operations.utils.ValidationUtils; import org.apache.flink.table.planner.operations.converters.SqlNodeConverter; -import org.apache.calcite.sql.SqlIdentifier; - import java.util.List; import java.util.Optional; @@ -65,9 +63,7 @@ public final Operation convertSqlNode(T sqlAlterTable, ConvertContext context) { "Table %s doesn't exist or is a temporary table.", tableIdentifier)); } ValidationUtils.validateTableKind( - optionalCatalogTable.get().getTable(), - CatalogBaseTable.TableKind.TABLE, - "alter table"); + optionalCatalogTable.get().getTable(), TableKind.TABLE, "alter table"); return convertToOperation( sqlAlterTable, optionalCatalogTable.get().getResolvedTable(), context); @@ -99,16 +95,6 @@ protected final Operation buildAlterTableChangeOperation( alterTable.ifTableExists()); } - protected static String getColumnName(SqlIdentifier identifier) { - if (!identifier.isSimple()) { - throw new UnsupportedOperationException( - String.format( - "%sAlter nested row type %s is not supported yet.", - EX_MSG_PREFIX, identifier)); - } - return identifier.getSimple(); - } - protected final ObjectIdentifier resolveIdentifier(SqlAlterTable node, ConvertContext context) { UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of(node.fullTableName()); return context.getCatalogManager().qualifyIdentifier(unresolvedIdentifier); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropColumnConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropColumnConverter.java deleted file mode 100644 index 3a669cd0f57bd..0000000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropColumnConverter.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.table; - -import org.apache.flink.sql.parser.ddl.table.SqlAlterTableDropColumn; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.ResolvedCatalogBaseTable; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.catalog.TableChange; -import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.planner.operations.converters.SchemaReferencesManager; - -import org.apache.calcite.sql.SqlIdentifier; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collectors; - -/** Convert ALTER TABLE DROP (col1 [, col2, ...]) to generate an updated Schema. */ -public class SqlAlterTableDropColumnConverter - extends AbstractAlterTableConverter { - @Override - protected Operation convertToOperation( - SqlAlterTableDropColumn dropColumn, - ResolvedCatalogTable oldTable, - ConvertContext context) { - Set columnsToDrop = new HashSet<>(); - dropColumn - .getColumnList() - .forEach( - identifier -> { - String name = getColumnName((SqlIdentifier) identifier); - if (!columnsToDrop.add(name)) { - throw new ValidationException( - String.format( - "%sDuplicate column `%s`.", EX_MSG_PREFIX, name)); - } - }); - - SchemaReferencesManager referencesManager = SchemaReferencesManager.create(oldTable); - // Sort by dependencies count from smallest to largest. For example, when dropping column a, - // b(b as a+1), the order should be: [b, a] after sort. - List sortedColumnsToDrop = - columnsToDrop.stream() - .sorted( - Comparator.comparingInt( - col -> - referencesManager.getColumnDependencyCount( - (String) col)) - .reversed()) - .collect(Collectors.toList()); - List tableChanges = new ArrayList<>(sortedColumnsToDrop.size()); - for (String columnToDrop : sortedColumnsToDrop) { - referencesManager.dropColumn(columnToDrop, () -> EX_MSG_PREFIX); - tableChanges.add(TableChange.dropColumn(columnToDrop)); - } - - final Schema schema = getUpdatedSchema(oldTable, columnsToDrop); - - return buildAlterTableChangeOperation( - dropColumn, tableChanges, schema, oldTable, context.getCatalogManager()); - } - - private Schema getUpdatedSchema( - ResolvedCatalogBaseTable oldTable, Set columnsToDrop) { - Schema.Builder schemaBuilder = Schema.newBuilder(); - SchemaReferencesManager.buildUpdatedColumn( - schemaBuilder, - oldTable, - (builder, column) -> { - if (!columnsToDrop.contains(column.getName())) { - builder.fromColumns(Collections.singletonList(column)); - } - }); - SchemaReferencesManager.buildUpdatedPrimaryKey( - schemaBuilder, oldTable, Function.identity()); - SchemaReferencesManager.buildUpdatedWatermark(schemaBuilder, oldTable); - return schemaBuilder.build(); - } -} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropConstraintConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropConstraintConverter.java deleted file mode 100644 index e5b4124e99764..0000000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropConstraintConverter.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.table; - -import org.apache.flink.sql.parser.ddl.table.SqlAlterTableDropConstraint; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.catalog.TableChange; -import org.apache.flink.table.catalog.UniqueConstraint; -import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.planner.operations.converters.SchemaReferencesManager; - -import org.apache.calcite.sql.SqlIdentifier; - -import java.util.List; -import java.util.Optional; - -/** Convert ALTER TABLE DROP CONSTRAINT constraint_name to generate an updated {@link Schema}. */ -public class SqlAlterTableDropConstraintConverter - extends AbstractAlterTableConverter { - @Override - protected Operation convertToOperation( - SqlAlterTableDropConstraint dropConstraint, - ResolvedCatalogTable oldTable, - ConvertContext context) { - Optional pkConstraint = oldTable.getResolvedSchema().getPrimaryKey(); - if (pkConstraint.isEmpty()) { - throw new ValidationException( - String.format( - "%sThe base table does not define any primary key.", EX_MSG_PREFIX)); - } - SqlIdentifier constraintIdentifier = dropConstraint.getConstraintName(); - String constraintName = pkConstraint.get().getName(); - if (constraintIdentifier != null - && !constraintIdentifier.getSimple().equals(constraintName)) { - throw new ValidationException( - String.format( - "%sThe base table does not define a primary key constraint named '%s'. " - + "Available constraint name: ['%s'].", - EX_MSG_PREFIX, constraintIdentifier.getSimple(), constraintName)); - } - - Schema.Builder schemaBuilder = Schema.newBuilder(); - SchemaReferencesManager.buildUpdatedColumn( - schemaBuilder, oldTable, (builder, column) -> builder.fromColumns(List.of(column))); - SchemaReferencesManager.buildUpdatedWatermark(schemaBuilder, oldTable); - - return buildAlterTableChangeOperation( - dropConstraint, - List.of(TableChange.dropConstraint(constraintName)), - schemaBuilder.build(), - oldTable, - context.getCatalogManager()); - } -} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropPrimaryKeyConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropPrimaryKeyConverter.java deleted file mode 100644 index 3797dc7a592dd..0000000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropPrimaryKeyConverter.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * 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.table; - -import org.apache.flink.sql.parser.ddl.table.SqlAlterTableDropPrimaryKey; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.catalog.TableChange; -import org.apache.flink.table.catalog.UniqueConstraint; -import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.planner.operations.converters.SchemaReferencesManager; - -import java.util.List; -import java.util.Optional; - -/** Convert ALTER TABLE DROP PRIMARY KEY to generate an updated Schema. */ -public class SqlAlterTableDropPrimaryKeyConverter - extends AbstractAlterTableConverter { - - public Operation convertToOperation( - SqlAlterTableDropPrimaryKey sqlAlterTable, - ResolvedCatalogTable oldTable, - ConvertContext context) { - Optional pkConstraint = oldTable.getResolvedSchema().getPrimaryKey(); - - if (pkConstraint.isEmpty()) { - throw new ValidationException( - String.format( - "%sThe base table does not define any primary key.", EX_MSG_PREFIX)); - } - - Schema.Builder schemaBuilder = Schema.newBuilder(); - SchemaReferencesManager.buildUpdatedColumn( - schemaBuilder, oldTable, (builder, column) -> builder.fromColumns(List.of(column))); - SchemaReferencesManager.buildUpdatedWatermark(schemaBuilder, oldTable); - - return buildAlterTableChangeOperation( - sqlAlterTable, - List.of(TableChange.dropConstraint(pkConstraint.get().getName())), - schemaBuilder.build(), - oldTable, - context.getCatalogManager()); - } -} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropWatermarkConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropWatermarkConverter.java deleted file mode 100644 index 69d9c5e9a99dc..0000000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableDropWatermarkConverter.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * 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.table; - -import org.apache.flink.sql.parser.ddl.table.SqlAlterTableDropWatermark; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.catalog.TableChange; -import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.planner.operations.converters.SchemaReferencesManager; - -import java.util.List; -import java.util.function.Function; - -/** Convert ALTER TABLE DROP WATERMARK to generate an updated {@link Schema}. */ -public class SqlAlterTableDropWatermarkConverter - extends AbstractAlterTableConverter { - @Override - protected Operation convertToOperation( - SqlAlterTableDropWatermark dropWatermark, - ResolvedCatalogTable oldTable, - ConvertContext context) { - if (oldTable.getResolvedSchema().getWatermarkSpecs().isEmpty()) { - throw new ValidationException( - String.format( - "%sThe base table does not define any watermark strategy.", - EX_MSG_PREFIX)); - } - - Schema.Builder schemaBuilder = Schema.newBuilder(); - SchemaReferencesManager.buildUpdatedColumn( - schemaBuilder, oldTable, (builder, column) -> builder.fromColumns(List.of(column))); - SchemaReferencesManager.buildUpdatedPrimaryKey( - schemaBuilder, oldTable, Function.identity()); - - return buildAlterTableChangeOperation( - dropWatermark, - List.of(TableChange.dropWatermark()), - schemaBuilder.build(), - oldTable, - context.getCatalogManager()); - } -} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableRenameColumnConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableRenameColumnConverter.java index dc5570faff182..ac222161d4621 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableRenameColumnConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableRenameColumnConverter.java @@ -26,6 +26,7 @@ import org.apache.flink.table.catalog.TableChange; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.planner.operations.converters.SchemaReferencesManager; +import org.apache.flink.table.planner.utils.OperationConverterUtils; import java.util.List; @@ -37,8 +38,12 @@ protected Operation convertToOperation( SqlAlterTableRenameColumn renameColumn, ResolvedCatalogTable oldTable, ConvertContext context) { - String oldColumnName = getColumnName(renameColumn.getOldColumnIdentifier()); - String newColumnName = getColumnName(renameColumn.getNewColumnIdentifier()); + String oldColumnName = + OperationConverterUtils.extractSimpleColumnName( + renameColumn.getOldColumnIdentifier(), EX_MSG_PREFIX); + String newColumnName = + OperationConverterUtils.extractSimpleColumnName( + renameColumn.getNewColumnIdentifier(), EX_MSG_PREFIX); SchemaReferencesManager.create(oldTable) .checkReferences(oldColumnName, () -> EX_MSG_PREFIX); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableSchemaAddConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableSchemaAddConverter.java deleted file mode 100644 index 4b6a094ea3f54..0000000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableSchemaAddConverter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.table; - -import org.apache.flink.sql.parser.ddl.table.SqlAlterTableAdd; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.planner.operations.converters.SchemaAddConverter; -import org.apache.flink.table.planner.operations.converters.SchemaConverter; - -/** - * Convert ALTER TABLE ADD (<schema_component> [, <schema_component>, ...]) to generate - * an updated Schema. - */ -public class SqlAlterTableSchemaAddConverter - extends SqlAlterTableSchemaConverter { - @Override - protected SchemaConverter createSchemaConverter( - ResolvedCatalogTable oldTable, ConvertContext context) { - return new SchemaAddConverter(oldTable, context); - } -} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableSchemaConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableSchemaConverter.java index 33109cc401395..219acb6fae03c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableSchemaConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableSchemaConverter.java @@ -18,14 +18,29 @@ package org.apache.flink.table.planner.operations.converters.table; +import org.apache.flink.sql.parser.ddl.table.SqlAlterTableAdd; +import org.apache.flink.sql.parser.ddl.table.SqlAlterTableDrop.SqlAlterTableDropColumn; +import org.apache.flink.sql.parser.ddl.table.SqlAlterTableDrop.SqlAlterTableDropConstraint; +import org.apache.flink.sql.parser.ddl.table.SqlAlterTableDrop.SqlAlterTableDropPrimaryKey; +import org.apache.flink.sql.parser.ddl.table.SqlAlterTableDrop.SqlAlterTableDropWatermark; +import org.apache.flink.sql.parser.ddl.table.SqlAlterTableModify; import org.apache.flink.sql.parser.ddl.table.SqlAlterTableSchema; +import org.apache.flink.table.api.Schema; import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.TableChange; import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.operations.converters.SchemaAddConverter; import org.apache.flink.table.planner.operations.converters.SchemaConverter; +import org.apache.flink.table.planner.operations.converters.SchemaModifyConverter; +import org.apache.flink.table.planner.operations.converters.SchemaReferencesManager; +import org.apache.flink.table.planner.utils.OperationConverterUtils; + +import java.util.List; +import java.util.Set; /** - * Abstract class for converters to convert ALTER TABLE ADD | MODIFY (<schema_component> [, - * <schema_component>, ...]) to generate an updated Schema. + * Abstract class for converters to convert {@code ALTER TABLE ADD | MODIFY + * (<schema_component> [, <schema_component>, ...])} to generate an updated Schema. */ public abstract class SqlAlterTableSchemaConverter extends AbstractAlterTableConverter { @@ -47,4 +62,136 @@ protected Operation convertToOperation( protected abstract SchemaConverter createSchemaConverter( ResolvedCatalogTable oldTable, ConvertContext context); + + /** + * Convert {@code ALTER TABLE ADD ( [, ])} to generate an + * updated Schema. + */ + public static class SqlAlterTableSchemaAddConverter + extends SqlAlterTableSchemaConverter { + @Override + protected SchemaConverter createSchemaConverter( + ResolvedCatalogTable oldTable, ConvertContext context) { + return new SchemaAddConverter(oldTable, context); + } + } + + /** + * Convert {@code ALTER TABLE MODIFY ( [, ])} to generate an + * updated Schema. + */ + public static class SqlAlterTableSchemaModifyConverter + extends SqlAlterTableSchemaConverter { + @Override + protected SchemaModifyConverter createSchemaConverter( + ResolvedCatalogTable oldTable, ConvertContext context) { + return new SchemaModifyConverter(oldTable, context); + } + } + + /** Convert {@code ALTER TABLE DROP} to generate an updated Schema. */ + public abstract static class SqlAlterTableSchemaDropConverter + extends AbstractAlterTableConverter { + @Override + protected Operation convertToOperation( + T alterTableSchema, ResolvedCatalogTable oldTable, ConvertContext context) { + Set columnsToDrop = getColumnsToDrop(alterTableSchema); + List tableChanges = + validateAndGatherDropChanges(alterTableSchema, oldTable, columnsToDrop); + Schema schema = + SchemaReferencesManager.buildSchemaForAlterSchemaDrop( + oldTable, tableChanges, columnsToDrop); + + return buildAlterTableChangeOperation( + alterTableSchema, tableChanges, schema, oldTable, context.getCatalogManager()); + } + + protected abstract List validateAndGatherDropChanges( + T alterTableSchema, ResolvedCatalogTable oldTable, Set columnsToDrop); + + protected abstract Set getColumnsToDrop(T alterTableSchema); + } + + /** Convert {@code ALTER TABLE DROP PRIMARY KEY} to generate an updated Schema. */ + public static class SqlAlterTableSchemaDropPrimaryKeyConverter + extends SqlAlterTableSchemaDropConverter { + + @Override + protected List validateAndGatherDropChanges( + SqlAlterTableDropPrimaryKey alterTableSchema, + ResolvedCatalogTable oldTable, + Set columnsToDrop) { + return OperationConverterUtils.validateAndGatherDropPrimaryKey( + oldTable, EX_MSG_PREFIX, "table"); + } + + @Override + protected Set getColumnsToDrop(SqlAlterTableDropPrimaryKey alterTableSchema) { + return Set.of(); + } + } + + /** Convert {@code ALTER TABLE DROP WATERMARK} to generate an updated Schema. */ + public static class SqlAlterTableSchemaDropWatermarkConverter + extends SqlAlterTableSchemaDropConverter { + + @Override + protected List validateAndGatherDropChanges( + SqlAlterTableDropWatermark alterTableSchema, + ResolvedCatalogTable oldTable, + Set columnsToDrop) { + return OperationConverterUtils.validateAndGatherDropWatermarkChanges( + oldTable, EX_MSG_PREFIX, "table"); + } + + @Override + protected Set getColumnsToDrop(SqlAlterTableDropWatermark alterTableSchema) { + return Set.of(); + } + } + + /** + * Convert {@code ALTER TABLE DROP column_name} in case of one column and {@code ALTER TABLE + * DROP (column_name1 [, column_name2])} in case of multiple columns to generate an updated + * Schema. + */ + public static class SqlAlterTableSchemaDropColumnConverter + extends SqlAlterTableSchemaDropConverter { + + @Override + protected List validateAndGatherDropChanges( + SqlAlterTableDropColumn alterTableSchema, + ResolvedCatalogTable oldTable, + Set columnsToDrop) { + return OperationConverterUtils.validateAndGatherDropColumn( + oldTable, columnsToDrop, EX_MSG_PREFIX); + } + + @Override + protected Set getColumnsToDrop(SqlAlterTableDropColumn dropColumn) { + return OperationConverterUtils.getColumnNames( + dropColumn.getColumnList(), EX_MSG_PREFIX); + } + } + + /** + * Convert {@code ALTER TABLE DROP CONSTRAINT constraint_name} to generate an updated Schema. + */ + public static class SqlAlterTableSchemaDropConstraintConverter + extends SqlAlterTableSchemaDropConverter { + + @Override + protected List validateAndGatherDropChanges( + SqlAlterTableDropConstraint dropConstraint, + ResolvedCatalogTable oldTable, + Set columnsToDrop) { + return OperationConverterUtils.validateAndGatherDropConstraintChanges( + oldTable, dropConstraint.getConstraintName(), EX_MSG_PREFIX, "table"); + } + + @Override + protected Set getColumnsToDrop(SqlAlterTableDropConstraint alterTableSchema) { + return Set.of(); + } + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableSchemaModifyConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableSchemaModifyConverter.java deleted file mode 100644 index f21879ac061cc..0000000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAlterTableSchemaModifyConverter.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.table; - -import org.apache.flink.sql.parser.ddl.table.SqlAlterTableModify; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.planner.operations.converters.SchemaModifyConverter; - -/** - * Convert ALTER TABLE MODIFY (<schema_component> [, <schema_component>, ...]) to - * generate an updated Schema. - */ -public class SqlAlterTableSchemaModifyConverter - extends SqlAlterTableSchemaConverter { - @Override - protected SchemaModifyConverter createSchemaConverter( - ResolvedCatalogTable oldTable, ConvertContext context) { - return new SchemaModifyConverter(oldTable, context); - } -} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAnalyzeTableConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAnalyzeTableConverter.java new file mode 100644 index 0000000000000..fea06b175ea45 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlAnalyzeTableConverter.java @@ -0,0 +1,260 @@ +/* + * 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.table; + +import org.apache.flink.sql.parser.ddl.table.SqlAnalyzeTable; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ContextResolvedTable; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionIdentifier; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.ddl.AnalyzeTableOperation; +import org.apache.flink.table.operations.utils.ValidationUtils; +import org.apache.flink.table.planner.operations.converters.SqlNodeConverter; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Time; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** Convert ANALYZE TABLE statement. */ +public class SqlAnalyzeTableConverter implements SqlNodeConverter { + @Override + public Operation convertSqlNode(SqlAnalyzeTable analyzeTable, ConvertContext context) { + UnresolvedIdentifier unresolvedIdentifier = + UnresolvedIdentifier.of(analyzeTable.fullTableName()); + CatalogManager catalogManager = context.getCatalogManager(); + ObjectIdentifier tableIdentifier = catalogManager.qualifyIdentifier(unresolvedIdentifier); + Optional optionalCatalogTable = + catalogManager.getTable(tableIdentifier); + if (optionalCatalogTable.isEmpty() || optionalCatalogTable.get().isTemporary()) { + throw new ValidationException( + String.format( + "Table %s doesn't exist or is a temporary table.", tableIdentifier)); + } + CatalogBaseTable baseTable = optionalCatalogTable.get().getResolvedTable(); + ValidationUtils.validateTableKind( + baseTable, CatalogBaseTable.TableKind.TABLE, "analyze table"); + + CatalogTable table = (CatalogTable) baseTable; + ResolvedSchema schema = + baseTable.getUnresolvedSchema().resolve(catalogManager.getSchemaResolver()); + + LinkedHashMap partitions = analyzeTable.getPartitions(); + List targetPartitionSpecs = null; + if (table.isPartitioned()) { + if (!partitions.keySet().equals(new HashSet<>(table.getPartitionKeys()))) { + throw new ValidationException( + String.format( + "Invalid ANALYZE TABLE statement. For partition table, all partition keys should be specified explicitly. " + + "The given partition keys: [%s] are not match the target partition keys: [%s].", + String.join(",", partitions.keySet()), + String.join(",", table.getPartitionKeys()))); + } + + try { + targetPartitionSpecs = + getPartitionSpecs(tableIdentifier, schema, partitions, catalogManager); + } catch (Exception e) { + throw new ValidationException(e.getMessage(), e); + } + } else if (!partitions.isEmpty()) { + throw new ValidationException( + String.format( + "Invalid ANALYZE TABLE statement. Table: %s is not a partition table, while partition values are given.", + tableIdentifier)); + } + + List columns = analyzeTable.getColumnNames(); + List targetColumns; + if (analyzeTable.isAllColumns()) { + Preconditions.checkArgument(columns.isEmpty()); + // computed column and metadata column will be ignored + targetColumns = + schema.getColumns().stream() + .filter(Column::isPhysical) + .collect(Collectors.toList()); + } else if (!columns.isEmpty()) { + targetColumns = + columns.stream() + .map( + c -> { + Optional colOpt = schema.getColumn(c); + if (colOpt.isEmpty()) { + throw new ValidationException( + String.format( + "Column: %s does not exist in the table: %s.", + c, tableIdentifier)); + } + Column col = colOpt.get(); + if (col instanceof Column.ComputedColumn) { + throw new ValidationException( + String.format( + "Column: %s is a computed column, ANALYZE TABLE does not support computed column.", + c)); + } else if (col instanceof Column.MetadataColumn) { + throw new ValidationException( + String.format( + "Column: %s is a metadata column, ANALYZE TABLE does not support metadata column.", + c)); + } else if (col instanceof Column.PhysicalColumn) { + return col; + } else { + throw new ValidationException( + "Unknown column class: " + + col.getClass().getSimpleName()); + } + }) + .collect(Collectors.toList()); + } else { + targetColumns = Collections.emptyList(); + } + + return new AnalyzeTableOperation(tableIdentifier, targetPartitionSpecs, targetColumns); + } + + private List getPartitionSpecs( + ObjectIdentifier tableIdentifier, + ResolvedSchema schema, + LinkedHashMap partitions, + CatalogManager catalogManager) + throws TableNotPartitionedException, TableNotExistException { + List filters = new ArrayList<>(); + for (Map.Entry entry : partitions.entrySet()) { + if (entry.getValue() != null) { + CallExpression call = + CallExpression.temporary( + FunctionIdentifier.of("="), + BuiltInFunctionDefinitions.EQUALS, + Arrays.asList( + getPartitionKeyExpr(schema, entry.getKey()), + getPartitionValueExpr( + schema, entry.getKey(), entry.getValue())), + DataTypes.BOOLEAN()); + filters.add(call); + } + } + if (filters.isEmpty()) { + return catalogManager + .getCatalog(tableIdentifier.getCatalogName()) + .get() + .listPartitions(tableIdentifier.toObjectPath()); + } else { + return catalogManager + .getCatalog(tableIdentifier.getCatalogName()) + .get() + .listPartitionsByFilter(tableIdentifier.toObjectPath(), filters); + } + } + + private FieldReferenceExpression getPartitionKeyExpr( + ResolvedSchema schema, String partitionKey) { + int fieldIndex = schema.getColumnNames().indexOf(partitionKey); + if (fieldIndex < 0) { + throw new ValidationException( + String.format( + "Partition: %s does not exist in the schema: %s", + partitionKey, schema.getColumnNames())); + } + return new FieldReferenceExpression( + partitionKey, schema.getColumnDataTypes().get(fieldIndex), 0, fieldIndex); + } + + private ValueLiteralExpression getPartitionValueExpr( + ResolvedSchema schema, String partitionKey, String partitionValue) { + int fieldIndex = schema.getColumnNames().indexOf(partitionKey); + if (fieldIndex < 0) { + throw new ValidationException( + String.format( + "Partition: %s does not exist in the schema: %s", + partitionKey, schema.getColumnNames())); + } + DataType dataType = schema.getColumnDataTypes().get(fieldIndex); + if (partitionValue == null) { + return new ValueLiteralExpression(null, dataType.nullable()); + } + Object value; + switch (dataType.getLogicalType().getTypeRoot()) { + case CHAR: + case VARCHAR: + value = partitionValue; + break; + case TINYINT: + value = Byte.valueOf(partitionValue); + break; + case SMALLINT: + value = Short.valueOf(partitionValue); + break; + case INTEGER: + value = Integer.valueOf(partitionValue); + break; + case BIGINT: + value = Long.valueOf(partitionValue); + break; + case FLOAT: + value = Float.valueOf(partitionValue); + break; + case DOUBLE: + value = Double.valueOf(partitionValue); + break; + case DECIMAL: + value = new BigDecimal(partitionValue); + break; + case DATE: + value = Date.valueOf(partitionValue); + break; + case TIME_WITHOUT_TIME_ZONE: + value = Time.valueOf(partitionValue); + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + value = Timestamp.valueOf(partitionValue); + break; + default: + throw new UnsupportedOperationException( + "Unsupported partition value type: " + dataType.getLogicalType()); + } + return new ValueLiteralExpression(value, dataType.notNull()); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableAsConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableAsConverter.java index c19d7237da91b..8fa2702a329f4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableAsConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableAsConverter.java @@ -32,6 +32,7 @@ import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.operations.PlannerQueryOperation; import org.apache.flink.table.planner.operations.SqlNodeToOperationConversion; +import org.apache.flink.table.planner.operations.converters.MergeTableAsUtil; import org.apache.calcite.sql.SqlNode; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableConverter.java index 3d9d25c780d30..6eae8819bff67 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableConverter.java @@ -27,6 +27,7 @@ import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateTableOperation; +import org.apache.flink.table.planner.operations.converters.MergeTableLikeUtil; import java.util.Collections; import java.util.List; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableLikeConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableLikeConverter.java index c9fe60d32e382..2088b4fdebdb6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableLikeConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlCreateTableLikeConverter.java @@ -33,6 +33,7 @@ import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.ddl.CreateTableOperation; +import org.apache.flink.table.planner.operations.converters.MergeTableLikeUtil; import java.util.Collections; import java.util.List; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlReplaceTableAsConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlReplaceTableAsConverter.java index 319f8d705dd25..438575d8f9407 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlReplaceTableAsConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/operations/converters/table/SqlReplaceTableAsConverter.java @@ -32,6 +32,7 @@ import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; import org.apache.flink.table.planner.operations.PlannerQueryOperation; import org.apache.flink.table.planner.operations.SqlNodeToOperationConversion; +import org.apache.flink.table.planner.operations.converters.MergeTableAsUtil; import java.util.List; import java.util.Map; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/MaterializedTableUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/MaterializedTableUtils.java index 838991bbf1366..42dea1379a22d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/MaterializedTableUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/MaterializedTableUtils.java @@ -150,6 +150,18 @@ public static List validateAndExtractNewColumns( return newAddedColumns; } + public static ResolvedSchema getQueryOperationResolvedSchema( + ResolvedCatalogMaterializedTable oldTable, ConvertContext context) { + final SqlNode originalQuery = + context.getFlinkPlanner().parser().parse(oldTable.getOriginalQuery()); + final SqlNode validateQuery = context.getSqlValidator().validate(originalQuery); + final PlannerQueryOperation queryOperation = + new PlannerQueryOperation( + context.toRelRoot(validateQuery).project(), + () -> context.toQuotedSqlString(validateQuery)); + return queryOperation.getResolvedSchema(); + } + public static void validatePersistedColumnsUsedByQuery( ResolvedCatalogMaterializedTable oldTable, SqlAlterMaterializedTableSchema alterTableSchema, @@ -159,15 +171,8 @@ public static void validatePersistedColumnsUsedByQuery( return; } - final SqlNode originalQuery = - context.getFlinkPlanner().parser().parse(oldTable.getOriginalQuery()); - final SqlNode validateQuery = context.getSqlValidator().validate(originalQuery); - final PlannerQueryOperation queryOperation = - new PlannerQueryOperation( - context.toRelRoot(validateQuery).project(), - () -> context.toQuotedSqlString(validateQuery)); - - validatePersistedColumnsUsedByQuery(sqlNodeList, queryOperation.getResolvedSchema()); + final ResolvedSchema querySchema = getQueryOperationResolvedSchema(oldTable, context); + validatePersistedColumnsUsedByQuery(sqlNodeList, querySchema); } public static void validatePersistedColumnsUsedByQuery( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/OperationConverterUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/OperationConverterUtils.java index ff25ff6b0e068..82a3a15724a6b 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/OperationConverterUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/OperationConverterUtils.java @@ -20,8 +20,19 @@ import org.apache.flink.sql.parser.SqlParseUtils; import org.apache.flink.sql.parser.ddl.SqlDistribution; +import org.apache.flink.sql.parser.ddl.resource.SqlResource; +import org.apache.flink.sql.parser.ddl.resource.SqlResourceType; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.FunctionLanguage; +import org.apache.flink.table.catalog.ResolvedCatalogBaseTable; +import org.apache.flink.table.catalog.TableChange; import org.apache.flink.table.catalog.TableDistribution; +import org.apache.flink.table.catalog.UniqueConstraint; import org.apache.flink.table.planner.calcite.FlinkPlannerImpl; +import org.apache.flink.table.planner.operations.converters.SchemaReferencesManager; +import org.apache.flink.table.resource.ResourceType; +import org.apache.flink.table.resource.ResourceUri; +import org.apache.flink.util.StringUtils; import org.apache.calcite.sql.SqlDialect; import org.apache.calcite.sql.SqlIdentifier; @@ -32,7 +43,13 @@ import org.apache.calcite.sql.parser.SqlParser; import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashSet; import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; /** Utils methods for converting sql to operations. */ public class OperationConverterUtils { @@ -69,4 +86,153 @@ public static String getQuotedSqlString(SqlNode sqlNode, FlinkPlannerImpl flinkP .withIdentifierQuoteString(parserConfig.quoting().string)); return sqlNode.toSqlString(dialect).getSql(); } + + public static Set getColumnNames(SqlNodeList sqlNodeList, String errMsgPrefix) { + Set distinctNames = new HashSet<>(); + for (SqlNode sqlNode : sqlNodeList) { + String name = extractSimpleColumnName((SqlIdentifier) sqlNode, errMsgPrefix); + if (!distinctNames.add(name)) { + throw new ValidationException( + String.format("%sDuplicate column `%s`.", errMsgPrefix, name)); + } + } + return distinctNames; + } + + public static String extractSimpleColumnName(SqlIdentifier identifier, String exMsgPrefix) { + if (!identifier.isSimple()) { + throw new UnsupportedOperationException( + String.format( + "%sAltering the nested row type `%s` is not supported yet.", + exMsgPrefix, String.join("`.`", identifier.names))); + } + return identifier.getSimple(); + } + + public static List validateAndGatherDropWatermarkChanges( + ResolvedCatalogBaseTable oldTable, String exMsgPrefix, String tableKindStr) { + if (oldTable.getResolvedSchema().getWatermarkSpecs().isEmpty()) { + throw new ValidationException( + String.format( + "%sThe current %s does not define any watermark strategy.", + exMsgPrefix, tableKindStr)); + } + + return List.of(TableChange.dropWatermark()); + } + + public static List validateAndGatherDropConstraintChanges( + ResolvedCatalogBaseTable oldTable, + SqlIdentifier constraint, + String exMsgPrefix, + String tableKindStr) { + Optional pkConstraint = oldTable.getResolvedSchema().getPrimaryKey(); + if (pkConstraint.isEmpty()) { + throw new ValidationException( + String.format( + "%sThe current %s does not define any primary key.", + exMsgPrefix, tableKindStr)); + } + String constraintName = pkConstraint.get().getName(); + if (constraint != null && !constraint.getSimple().equals(constraintName)) { + throw new ValidationException( + String.format( + "%sThe current %s does not define a primary key constraint named '%s'. " + + "Available constraint name: ['%s'].", + exMsgPrefix, tableKindStr, constraint.getSimple(), constraintName)); + } + + return List.of(TableChange.dropConstraint(constraintName)); + } + + public static List validateAndGatherDropPrimaryKey( + ResolvedCatalogBaseTable oldTable, String exMsgPrefix, String tableKindStr) { + Optional pkConstraint = oldTable.getResolvedSchema().getPrimaryKey(); + + if (pkConstraint.isEmpty()) { + throw new ValidationException( + String.format( + "%sThe current %s does not define any primary key.", + exMsgPrefix, tableKindStr)); + } + + return List.of(TableChange.dropConstraint(pkConstraint.get().getName())); + } + + public static List validateAndGatherDropColumn( + ResolvedCatalogBaseTable oldTable, Set columnsToDrop, String exMsgPrefix) { + SchemaReferencesManager referencesManager = SchemaReferencesManager.create(oldTable); + // Sort by dependencies count from smallest to largest. For example, when dropping + // column a, + // b(b as a+1), the order should be: [b, a] after sort. + Comparator comparator = + Comparator.comparingInt( + col -> referencesManager.getColumnDependencyCount((String) col)) + .reversed(); + List sortedColumnsToDrop = + columnsToDrop.stream().sorted(comparator).collect(Collectors.toList()); + List tableChanges = new ArrayList<>(sortedColumnsToDrop.size()); + for (String columnToDrop : sortedColumnsToDrop) { + referencesManager.dropColumn(columnToDrop, () -> exMsgPrefix); + tableChanges.add(TableChange.dropColumn(columnToDrop)); + } + + return tableChanges; + } + + public static List getFunctionResources(List sqlResources) { + return sqlResources.stream() + .map(SqlResource.class::cast) + .map( + sqlResource -> { + // get resource type + SqlResourceType sqlResourceType = + sqlResource.getResourceType().getValueAs(SqlResourceType.class); + ResourceType resourceType; + switch (sqlResourceType) { + case FILE: + resourceType = ResourceType.FILE; + break; + case JAR: + resourceType = ResourceType.JAR; + break; + case ARCHIVE: + resourceType = ResourceType.ARCHIVE; + break; + default: + throw new ValidationException( + String.format( + "Unsupported resource type: .", + sqlResourceType)); + } + // get resource path + String path = sqlResource.getResourcePath().getValueAs(String.class); + return new ResourceUri(resourceType, path); + }) + .collect(Collectors.toList()); + } + + /** + * Converts language string to the FunctionLanguage. + * + * @param languageString the language string from SQL parser + * @return supported FunctionLanguage otherwise raise UnsupportedOperationException. + * @throws UnsupportedOperationException if the languageString is not parsable or language is + * not supported + */ + public static FunctionLanguage parseLanguage(String languageString) { + if (StringUtils.isNullOrWhitespaceOnly(languageString)) { + return FunctionLanguage.JAVA; + } + + FunctionLanguage language; + try { + language = FunctionLanguage.valueOf(languageString); + } catch (IllegalArgumentException e) { + throw new UnsupportedOperationException( + String.format("Unrecognized function language string %s", languageString), e); + } + + return language; + } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java index abaeac9072bd2..12496179523db 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlDdlToOperationConverterTest.java @@ -1228,7 +1228,8 @@ void testAlterTableRenameColumn() throws Exception { // rename nested column assertThatThrownBy(() -> parse("alter table tb1 rename e.f1 to e.f11")) .isInstanceOf(UnsupportedOperationException.class) - .hasMessageContaining("Alter nested row type e.f1 is not supported yet."); + .hasMessageContaining( + "Altering the nested row type `e`.`f1` is not supported yet."); // rename column with duplicate name assertThatThrownBy(() -> parse("alter table tb1 rename c to a")) @@ -1298,7 +1299,8 @@ void testFailedToAlterTableDropColumn() throws Exception { // drop a nested column assertThatThrownBy(() -> parse("alter table tb1 drop e.f2")) .isInstanceOf(UnsupportedOperationException.class) - .hasMessageContaining("Alter nested row type e.f2 is not supported yet."); + .hasMessageContaining( + "Altering the nested row type `e`.`f2` is not supported yet."); // drop a column which generates a computed column assertThatThrownBy(() -> parse("alter table tb1 drop a")) @@ -1359,15 +1361,20 @@ void testFailedToAlterTableDropConstraint() throws Exception { prepareTable("tb1", 0); assertThatThrownBy(() -> parse("alter table tb1 drop primary key")) .isInstanceOf(ValidationException.class) - .hasMessageContaining("The base table does not define any primary key."); + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table does not define any primary key."); assertThatThrownBy(() -> parse("alter table tb1 drop constraint ct")) .isInstanceOf(ValidationException.class) - .hasMessageContaining("The base table does not define any primary key."); + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table does not define any primary key."); prepareTable("tb2", 1); assertThatThrownBy(() -> parse("alter table tb2 drop constraint ct2")) .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "The base table does not define a primary key constraint named 'ct2'. Available constraint name: ['ct1']."); + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table does not define a primary key constraint named 'ct2'. Available constraint name: ['ct1']."); checkAlterNonExistTable("alter table %s nonexistent drop primary key"); checkAlterNonExistTable("alter table %s nonexistent drop constraint ct"); } @@ -1583,7 +1590,9 @@ void testFailedToAlterTableDropWatermark() throws Exception { prepareTable("tb1", false); assertThatThrownBy(() -> parse("alter table tb1 drop watermark")) .isInstanceOf(ValidationException.class) - .hasMessageContaining("The base table does not define any watermark strategy."); + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table does not define any watermark strategy."); checkAlterNonExistTable("alter table %s nonexistent drop watermark"); } @@ -1645,16 +1654,19 @@ void testFailedToAlterTableAddColumn() throws Exception { // add an inner field to a nested row assertThatThrownBy(() -> parse("alter table tb1 add (e.f3 string)")) .isInstanceOf(UnsupportedOperationException.class) - .hasMessageContaining("Alter nested row type e.f3 is not supported yet."); + .hasMessageContaining( + "Altering the nested row type `e`.`f3` is not supported yet."); // refer to a nested inner field assertThatThrownBy(() -> parse("alter table tb1 add (x string after e.f2)")) .isInstanceOf(UnsupportedOperationException.class) - .hasMessageContaining("Alter nested row type is not supported yet."); + .hasMessageContaining( + "Altering the nested row type `e`.`f2` is not supported yet."); assertThatThrownBy(() -> parse("alter table tb1 add (e.f3 string after e.f1)")) .isInstanceOf(UnsupportedOperationException.class) - .hasMessageContaining("Alter nested row type e.f3 is not supported yet."); + .hasMessageContaining( + "Altering the nested row type `e`.`f3` is not supported yet."); checkAlterNonExistTable("alter table %s nonexistent add a bigint not null"); } @@ -1779,8 +1791,9 @@ void testFailedToAlterTableAddPk() throws Exception { assertThatThrownBy(() -> parse("alter table tb1 add primary key(c) not enforced")) .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "The current table has already defined the primary key constraint [`a`]. " + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table has already defined the primary key constraint [`a`]. " + "You might want to drop it before adding a new one."); assertThatThrownBy( @@ -1788,27 +1801,30 @@ void testFailedToAlterTableAddPk() throws Exception { parse( "alter table tb1 add x string not null primary key not enforced")) .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "The current table has already defined the primary key constraint [`a`]. " - + "You might want to drop it before adding a new one"); + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table has already defined the primary key constraint [`a`]. " + + "You might want to drop it before adding a new one."); // the original table has composite pk prepareTable("tb2", 2); assertThatThrownBy(() -> parse("alter table tb2 add primary key(c) not enforced")) .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "The current table has already defined the primary key constraint [`a`, `b`]. " - + "You might want to drop it before adding a new one"); + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table has already defined the primary key constraint [`a`, `b`]. " + + "You might want to drop it before adding a new one."); assertThatThrownBy( () -> parse( "alter table tb2 add x string not null primary key not enforced")) .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "The current table has already defined the primary key constraint [`a`, `b`]. " - + "You might want to drop it before adding a new one"); + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table has already defined the primary key constraint [`a`, `b`]. " + + "You might want to drop it before adding a new one."); // the original table does not define pk prepareTable("tb3", 0); @@ -1950,8 +1966,9 @@ void testFailedToAlterTableAddWatermark() throws Exception { assertThatThrownBy(() -> parse("alter table tb2 add watermark for ts as ts")) .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "The current table has already defined the watermark strategy " + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table has already defined the watermark strategy " + "`ts` AS ts - interval '5' seconds. " + "You might want to drop it before adding a new one."); checkAlterNonExistTable("alter table %s nonexistent add watermark for ts as ts"); @@ -2101,16 +2118,19 @@ void testFailedToAlterTableModifyColumn() throws Exception { // modify an inner field to a nested row assertThatThrownBy(() -> parse("alter table tb2 modify (e.f0 string)")) .isInstanceOf(UnsupportedOperationException.class) - .hasMessageContaining("Alter nested row type e.f0 is not supported yet."); + .hasMessageContaining( + "Altering the nested row type `e`.`f0` is not supported yet."); // refer to a nested inner field assertThatThrownBy(() -> parse("alter table tb2 modify (g string after e.f2)")) .isInstanceOf(UnsupportedOperationException.class) - .hasMessageContaining("Alter nested row type is not supported yet."); + .hasMessageContaining( + "Altering the nested row type `e`.`f2` is not supported yet."); assertThatThrownBy(() -> parse("alter table tb2 modify (e.f0 string after e.f1)")) .isInstanceOf(UnsupportedOperationException.class) - .hasMessageContaining("Alter nested row type e.f0 is not supported yet."); + .hasMessageContaining( + "Altering the nested row type `e`.`f0` is not supported yet."); checkAlterNonExistTable("alter table %s nonexistent modify a int first"); } @@ -2275,8 +2295,9 @@ void testFailedToAlterTableModifyPk() throws Exception { parse( "alter table tb1 modify constraint ct primary key (b) not enforced")) .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "The current table does not define any primary key constraint. You might want to add a new one."); + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table does not define any primary key constraint. You might want to add a new one."); prepareTable("tb2", 1); @@ -2435,8 +2456,9 @@ void testFailedToAlterTableModifyWatermark() throws Exception { parse( "alter table tb1 modify watermark for a as to_timestamp(a) - interval '1' minute")) .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "The current table does not define any watermark. You might want to add a new one."); + .hasMessage( + "Failed to execute ALTER TABLE statement.\n" + + "The current table does not define any watermark. You might want to add a new one."); prepareTable("tb2", true); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java index 839a815af4adb..73c2fe2a16ea9 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java @@ -133,16 +133,17 @@ void before() throws TableAlreadyExistException, DatabaseNotExistException { + "AS SELECT t1.* FROM t1"; createMaterializedTableInCatalog(sqlWithWatermark, "base_mtbl_with_watermark"); - // MATERIALIZED TABLE with METADATA column + // MATERIALIZED TABLE with METADATA column and distribution final String sqlWithMetadataColumn = "CREATE MATERIALIZED TABLE base_mtbl_with_metadata (\n" + " t AS current_timestamp," + " m STRING METADATA VIRTUAL," + + " m_p STRING METADATA," + " CONSTRAINT ct1 PRIMARY KEY(a) NOT ENFORCED," + " WATERMARK FOR t as current_timestamp - INTERVAL '5' SECOND" + ")\n" + "COMMENT 'materialized table comment'\n" - + "PARTITIONED BY (a, d)\n" + + "DISTRIBUTED BY HASH (b) INTO 7 BUCKETS\n" + "WITH (\n" + " 'connector' = 'filesystem', \n" + " 'format' = 'json'\n" @@ -371,8 +372,8 @@ void testFullRefreshMode() { } @ParameterizedTest - @MethodSource("testDataForCreateMaterializedTableFailedCase") - void createMaterializedTableFailedCase(TestSpec spec) { + @MethodSource("testDataForCreateAlterMaterializedTableFailedCase") + void createAlterMaterializedTableFailedCase(TestSpec spec) { assertThatThrownBy(() -> parse(spec.sql)) .as(spec.sql) .isInstanceOf(spec.expectedException) @@ -699,7 +700,7 @@ void testCreateOrAlterMaterializedTableForExistingTable() throws TableNotExistEx "f", DataTypes.VARCHAR(Integer.MAX_VALUE)))); } - private static Collection testDataForCreateMaterializedTableFailedCase() { + private static Collection testDataForCreateAlterMaterializedTableFailedCase() { final Collection list = new ArrayList<>(); list.addAll(createWithInvalidSchema()); list.addAll(createWithInvalidFreshness()); @@ -707,6 +708,7 @@ private static Collection testDataForCreateMaterializedTableFailedCase list.addAll(alterAddWithInvalidSchema()); list.addAll(alterModifyWithInvalidSchema()); list.addAll(alterQuery()); + list.addAll(alterDrop()); return list; } @@ -992,10 +994,63 @@ private static List createWithInvalidFreshness() { "Materialized table freshness only support SECOND, MINUTE, HOUR, DAY as the time unit.")); } + private static Collection alterDrop() { + return List.of( + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl DROP WATERMARK", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "The current materialized table does not define any watermark strategy."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl_without_constraint DROP PRIMARY KEY", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "The current materialized table does not define any primary key."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl DROP CONSTRAINT invalid_constraint_name", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "The current materialized table does not define a primary key constraint named 'invalid_constraint_name'. Available constraint name: ['ct1']."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl DROP invalid_column_name", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "The column `invalid_column_name` does not exist in the base table."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl DROP invalid_column_name", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "The column `invalid_column_name` does not exist in the base table."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl DROP (a, b, a)", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "Duplicate column `a`."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl DROP a", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "The column `a` is used as the partition keys."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl_with_metadata DROP a", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "The column `a` is used as the primary key."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl_with_metadata DROP b", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "The column `b` is used as a distribution key."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl_with_metadata DROP t", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "The column `t` is referenced by watermark expression."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl_with_metadata DROP d", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "Column(s) ('d') are used in query."), + TestSpec.of( + "ALTER MATERIALIZED TABLE base_mtbl_with_metadata DROP m_p", + "Failed to execute ALTER MATERIALIZED TABLE statement.\n" + + "The column `m_p` is a persisted column. Dropping of persisted columns is not supported.")); + } + private static Collection alterSuccessCase() { List list = new ArrayList<>(); list.addAll(alterAddSchemaSuccessCase()); list.addAll(alterModifySchemaSuccessCase()); + list.addAll(alterDropSchemaSuccessCase()); return list; } @@ -1063,6 +1118,7 @@ private static Collection alterModifySchemaSuccessCase() { "(\n" + " `t` AS [CURRENT_TIMESTAMP],\n" + " `m` STRING METADATA VIRTUAL,\n" + + " `m_p` STRING METADATA,\n" + " `a` BIGINT NOT NULL,\n" + " `b` STRING,\n" + " `c` INT,\n" @@ -1073,6 +1129,54 @@ private static Collection alterModifySchemaSuccessCase() { return list; } + private static Collection alterDropSchemaSuccessCase() { + final Collection list = new ArrayList<>(); + list.add( + TestSpec.withExpectedSchema( + "ALTER MATERIALIZED TABLE base_mtbl DROP PRIMARY KEY", + "(\n" + + " `a` BIGINT NOT NULL,\n" + + " `b` STRING,\n" + + " `c` INT,\n" + + " `d` STRING\n" + + ")")); + list.add( + TestSpec.withExpectedSchema( + "ALTER MATERIALIZED TABLE base_mtbl DROP CONSTRAINT ct1", + "(\n" + + " `a` BIGINT NOT NULL,\n" + + " `b` STRING,\n" + + " `c` INT,\n" + + " `d` STRING\n" + + ")")); + list.add( + TestSpec.withExpectedSchema( + "ALTER MATERIALIZED TABLE base_mtbl_with_watermark DROP WATERMARK", + "(\n" + + " `t` AS [CURRENT_TIMESTAMP],\n" + + " `a` BIGINT NOT NULL,\n" + + " `b` STRING,\n" + + " `c` INT,\n" + + " `d` STRING,\n" + + " CONSTRAINT `ct1` PRIMARY KEY (`a`) NOT ENFORCED\n" + + ")")); + list.add( + TestSpec.withExpectedSchema( + // drop metadata virtual column + "ALTER MATERIALIZED TABLE base_mtbl_with_metadata DROP m", + "(\n" + + " `t` AS [CURRENT_TIMESTAMP],\n" + + " `m_p` STRING METADATA,\n" + + " `a` BIGINT NOT NULL,\n" + + " `b` STRING,\n" + + " `c` INT,\n" + + " `d` STRING,\n" + + " WATERMARK FOR `t` AS [CURRENT_TIMESTAMP - INTERVAL '5' SECOND],\n" + + " CONSTRAINT `ct1` PRIMARY KEY (`a`) NOT ENFORCED\n" + + ")")); + return list; + } + private static Collection testDataWithDifferentSchemasSuccessCase() { final Collection list = new ArrayList<>(); list.addAll(createOrAlter(CREATE_OPERATION)); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/converters/table/MergeTableLikeUtilTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/converters/MergeTableLikeUtilTest.java similarity index 87% rename from flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/converters/table/MergeTableLikeUtilTest.java rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/converters/MergeTableLikeUtilTest.java index 3aa4670636cc1..bdd39709ebd13 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/converters/table/MergeTableLikeUtilTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/converters/MergeTableLikeUtilTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.table.planner.operations.converters.table; +package org.apache.flink.table.planner.operations.converters; import org.apache.flink.sql.parser.ddl.SqlTableColumn.SqlComputedColumn; import org.apache.flink.sql.parser.ddl.SqlTableColumn.SqlMetadataColumn; @@ -54,7 +54,6 @@ import org.junit.jupiter.api.Test; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -85,7 +84,7 @@ void mergePhysicalColumns() { .build(); List derivedColumns = - Arrays.asList( + List.of( regularColumn("three", DataTypes.INT()), regularColumn("four", DataTypes.STRING())); @@ -94,7 +93,7 @@ void mergePhysicalColumns() { getDefaultMergingStrategies(), sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null); Schema expectedSchema = @@ -113,7 +112,7 @@ void mergeWithIncludeFailsOnDuplicateColumn() { Schema sourceSchema = Schema.newBuilder().column("one", DataTypes.INT()).build(); List derivedColumns = - Arrays.asList( + List.of( regularColumn("one", DataTypes.INT()), regularColumn("four", DataTypes.STRING())); @@ -123,7 +122,7 @@ void mergeWithIncludeFailsOnDuplicateColumn() { getDefaultMergingStrategies(), sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null)) .isInstanceOf(ValidationException.class) .hasMessage("A column named 'one' already exists in the base table."); @@ -134,7 +133,7 @@ void mergeWithIncludeFailsOnDuplicateRegularColumn() { Schema sourceSchema = Schema.newBuilder().column("one", DataTypes.INT()).build(); List derivedColumns = - Arrays.asList( + List.of( regularColumn("two", DataTypes.INT()), regularColumn("two", DataTypes.INT()), regularColumn("four", DataTypes.STRING())); @@ -145,7 +144,7 @@ void mergeWithIncludeFailsOnDuplicateRegularColumn() { getDefaultMergingStrategies(), sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null)) .isInstanceOf(ValidationException.class) .hasMessage("A regular Column named 'two' already exists in the table."); @@ -156,7 +155,7 @@ void mergeWithIncludeFailsOnDuplicateRegularColumnAndComputeColumn() { Schema sourceSchema = Schema.newBuilder().column("one", DataTypes.INT()).build(); List derivedColumns = - Arrays.asList( + List.of( regularColumn("two", DataTypes.INT()), computedColumn("three", plus("two", "3")), regularColumn("three", DataTypes.INT()), @@ -168,7 +167,7 @@ void mergeWithIncludeFailsOnDuplicateRegularColumnAndComputeColumn() { getDefaultMergingStrategies(), sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null)) .isInstanceOf(ValidationException.class) .hasMessage( @@ -181,7 +180,7 @@ void mergeWithIncludeFailsOnDuplicateRegularColumnAndMetadataColumn() { Schema sourceSchema = Schema.newBuilder().column("one", DataTypes.INT()).build(); List derivedColumns = - Arrays.asList( + List.of( metadataColumn("two", DataTypes.INT(), true), computedColumn("three", plus("two", "3")), regularColumn("two", DataTypes.INT()), @@ -193,7 +192,7 @@ void mergeWithIncludeFailsOnDuplicateRegularColumnAndMetadataColumn() { getDefaultMergingStrategies(), sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null)) .isInstanceOf(ValidationException.class) .hasMessage( @@ -210,7 +209,7 @@ void mergeGeneratedColumns() { .build(); List derivedColumns = - Arrays.asList( + List.of( regularColumn("three", DataTypes.INT()), computedColumn("four", plus("one", "3"))); @@ -219,7 +218,7 @@ void mergeGeneratedColumns() { getDefaultMergingStrategies(), sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null); Schema expectedSchema = @@ -243,7 +242,7 @@ void mergeMetadataColumns() { .build(); List derivedColumns = - Arrays.asList( + List.of( regularColumn("three", DataTypes.INT()), metadataColumn("four", DataTypes.INT(), true)); @@ -252,7 +251,7 @@ void mergeMetadataColumns() { getDefaultMergingStrategies(), sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null); Schema expectedSchema = @@ -275,8 +274,7 @@ void mergeIncludingGeneratedColumnsFailsOnDuplicate() { .columnByExpression("two", "one + 1") .build(); - List derivedColumns = - Collections.singletonList(computedColumn("two", plus("one", "3"))); + List derivedColumns = List.of(computedColumn("two", plus("one", "3"))); assertThatThrownBy( () -> @@ -284,7 +282,7 @@ void mergeIncludingGeneratedColumnsFailsOnDuplicate() { getDefaultMergingStrategies(), sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null)) .isInstanceOf(ValidationException.class) .hasMessage( @@ -301,8 +299,7 @@ void mergeIncludingMetadataColumnsFailsOnDuplicate() { .columnByMetadata("two", DataTypes.INT()) .build(); - List derivedColumns = - Collections.singletonList(metadataColumn("two", DataTypes.INT(), false)); + List derivedColumns = List.of(metadataColumn("two", DataTypes.INT(), false)); assertThatThrownBy( () -> @@ -310,7 +307,7 @@ void mergeIncludingMetadataColumnsFailsOnDuplicate() { getDefaultMergingStrategies(), sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null)) .isInstanceOf(ValidationException.class) .hasMessage( @@ -327,19 +324,13 @@ void mergeExcludingGeneratedColumnsDuplicate() { .columnByExpression("two", "one + 1") .build(); - List derivedColumns = - Collections.singletonList(computedColumn("two", plus("one", "3"))); + List derivedColumns = List.of(computedColumn("two", plus("one", "3"))); Map mergingStrategies = getDefaultMergingStrategies(); mergingStrategies.put(FeatureOption.GENERATED, MergingStrategy.EXCLUDING); Schema mergedSchema = - util.mergeTables( - mergingStrategies, - sourceSchema, - derivedColumns, - Collections.emptyList(), - null); + util.mergeTables(mergingStrategies, sourceSchema, derivedColumns, List.of(), null); Schema expectedSchema = Schema.newBuilder() @@ -358,19 +349,13 @@ void mergeExcludingMetadataColumnsDuplicate() { .columnByMetadata("two", DataTypes.INT()) .build(); - List derivedColumns = - Collections.singletonList(metadataColumn("two", DataTypes.BOOLEAN(), false)); + List derivedColumns = List.of(metadataColumn("two", DataTypes.BOOLEAN(), false)); Map mergingStrategies = getDefaultMergingStrategies(); mergingStrategies.put(FeatureOption.METADATA, MergingStrategy.EXCLUDING); Schema mergedSchema = - util.mergeTables( - mergingStrategies, - sourceSchema, - derivedColumns, - Collections.emptyList(), - null); + util.mergeTables(mergingStrategies, sourceSchema, derivedColumns, List.of(), null); Schema expectedSchema = Schema.newBuilder() @@ -389,19 +374,13 @@ void mergeOverwritingGeneratedColumnsDuplicate() { .columnByExpression("two", "one + 1") .build(); - List derivedColumns = - Collections.singletonList(computedColumn("two", plus("one", "3"))); + List derivedColumns = List.of(computedColumn("two", plus("one", "3"))); Map mergingStrategies = getDefaultMergingStrategies(); mergingStrategies.put(FeatureOption.GENERATED, MergingStrategy.OVERWRITING); Schema mergedSchema = - util.mergeTables( - mergingStrategies, - sourceSchema, - derivedColumns, - Collections.emptyList(), - null); + util.mergeTables(mergingStrategies, sourceSchema, derivedColumns, List.of(), null); Schema expectedSchema = Schema.newBuilder() @@ -420,19 +399,13 @@ void mergeOverwritingMetadataColumnsDuplicate() { .columnByMetadata("two", DataTypes.INT()) .build(); - List derivedColumns = - Collections.singletonList(metadataColumn("two", DataTypes.BOOLEAN(), true)); + List derivedColumns = List.of(metadataColumn("two", DataTypes.BOOLEAN(), true)); Map mergingStrategies = getDefaultMergingStrategies(); mergingStrategies.put(FeatureOption.METADATA, MergingStrategy.OVERWRITING); Schema mergedSchema = - util.mergeTables( - mergingStrategies, - sourceSchema, - derivedColumns, - Collections.emptyList(), - null); + util.mergeTables(mergingStrategies, sourceSchema, derivedColumns, List.of(), null); Schema expectedSchema = Schema.newBuilder() @@ -451,8 +424,7 @@ void mergeOverwritingPhysicalColumnWithGeneratedColumn() { .column("two", DataTypes.INT()) .build(); - List derivedColumns = - Collections.singletonList(computedColumn("two", plus("one", "3"))); + List derivedColumns = List.of(computedColumn("two", plus("one", "3"))); Map mergingStrategies = getDefaultMergingStrategies(); mergingStrategies.put(FeatureOption.GENERATED, MergingStrategy.OVERWRITING); @@ -463,7 +435,7 @@ void mergeOverwritingPhysicalColumnWithGeneratedColumn() { mergingStrategies, sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null)) .isInstanceOf(ValidationException.class) .hasMessage( @@ -480,8 +452,7 @@ void mergeOverwritingComputedColumnWithMetadataColumn() { .columnByExpression("two", "one + 3") .build(); - List derivedColumns = - Collections.singletonList(metadataColumn("two", DataTypes.BOOLEAN(), false)); + List derivedColumns = List.of(metadataColumn("two", DataTypes.BOOLEAN(), false)); Map mergingStrategies = getDefaultMergingStrategies(); mergingStrategies.put(FeatureOption.METADATA, MergingStrategy.OVERWRITING); @@ -492,7 +463,7 @@ void mergeOverwritingComputedColumnWithMetadataColumn() { mergingStrategies, sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null)) .isInstanceOf(ValidationException.class) .hasMessage( @@ -511,7 +482,7 @@ void mergeWatermarks() { .build(); List derivedColumns = - Arrays.asList( + List.of( regularColumn("three", DataTypes.INT()), computedColumn("four", plus("one", "3"))); @@ -520,7 +491,7 @@ void mergeWatermarks() { getDefaultMergingStrategies(), sourceSchema, derivedColumns, - Collections.emptyList(), + List.of(), null); Schema expectedSchema = @@ -546,7 +517,7 @@ void mergeIncludingWatermarksFailsOnDuplicate() { .build(); List derivedWatermarkSpecs = - Collections.singletonList( + List.of( new SqlWatermark( SqlParserPos.ZERO, identifier("timestamp"), @@ -557,7 +528,7 @@ void mergeIncludingWatermarksFailsOnDuplicate() { util.mergeTables( getDefaultMergingStrategies(), sourceSchema, - Collections.emptyList(), + List.of(), derivedWatermarkSpecs, null)) .isInstanceOf(ValidationException.class) @@ -577,7 +548,7 @@ void mergeExcludingWatermarksDuplicate() { .build(); List derivedWatermarkSpecs = - Collections.singletonList( + List.of( new SqlWatermark( SqlParserPos.ZERO, identifier("timestamp"), @@ -588,11 +559,7 @@ void mergeExcludingWatermarksDuplicate() { Schema mergedSchema = util.mergeTables( - mergingStrategies, - sourceSchema, - Collections.emptyList(), - derivedWatermarkSpecs, - null); + mergingStrategies, sourceSchema, List.of(), derivedWatermarkSpecs, null); Schema expectedSchema = Schema.newBuilder() @@ -614,7 +581,7 @@ void mergeOverwritingWatermarksDuplicate() { .build(); List derivedWatermarkSpecs = - Collections.singletonList( + List.of( new SqlWatermark( SqlParserPos.ZERO, identifier("timestamp"), @@ -625,11 +592,7 @@ void mergeOverwritingWatermarksDuplicate() { Schema mergedSchema = util.mergeTables( - mergingStrategies, - sourceSchema, - Collections.emptyList(), - derivedWatermarkSpecs, - null); + mergingStrategies, sourceSchema, List.of(), derivedWatermarkSpecs, null); Schema expectedSchema = Schema.newBuilder() @@ -653,11 +616,7 @@ void mergeConstraintsFromBaseTable() { Schema mergedSchema = util.mergeTables( - getDefaultMergingStrategies(), - sourceSchema, - Collections.emptyList(), - Collections.emptyList(), - null); + getDefaultMergingStrategies(), sourceSchema, List.of(), List.of(), null); Schema expectedSchema = Schema.newBuilder() @@ -683,8 +642,8 @@ void mergeConstraintsFromDerivedTable() { util.mergeTables( getDefaultMergingStrategies(), sourceSchema, - Collections.emptyList(), - Collections.emptyList(), + List.of(), + List.of(), primaryKey("one", "two")); Schema expectedSchema = @@ -713,8 +672,8 @@ void mergeIncludingConstraintsFailsOnDuplicate() { util.mergeTables( getDefaultMergingStrategies(), sourceSchema, - Collections.emptyList(), - Collections.emptyList(), + List.of(), + List.of(), primaryKey("one", "two"))) .isInstanceOf(ValidationException.class) .hasMessage( @@ -739,8 +698,8 @@ void mergeExcludingConstraintsOnDuplicate() { util.mergeTables( mergingStrategies, sourceSchema, - Collections.emptyList(), - Collections.emptyList(), + List.of(), + List.of(), primaryKey("one", "two")); Schema expectedSchema = @@ -757,7 +716,7 @@ void mergeExcludingConstraintsOnDuplicate() { @Test void mergeDistributionFromBaseTable() { Optional sourceDistribution = - Optional.of(TableDistribution.ofHash(Collections.singletonList("a"), 3)); + Optional.of(TableDistribution.ofHash(List.of("a"), 3)); Optional mergePartitions = util.mergeDistribution( getDefaultMergingStrategies().get(FeatureOption.DISTRIBUTION), @@ -770,7 +729,7 @@ void mergeDistributionFromBaseTable() { @Test void mergeDistributionFromDerivedTable() { Optional derivedDistribution = - Optional.of(TableDistribution.ofHash(Collections.singletonList("a"), 3)); + Optional.of(TableDistribution.ofHash(List.of("a"), 3)); Optional mergePartitions = util.mergeDistribution( getDefaultMergingStrategies().get(FeatureOption.DISTRIBUTION), @@ -783,9 +742,9 @@ void mergeDistributionFromDerivedTable() { @Test void mergeIncludingDistributionFailsOnDuplicate() { Optional sourceDistribution = - Optional.of(TableDistribution.ofHash(Collections.singletonList("a"), 3)); + Optional.of(TableDistribution.ofHash(List.of("a"), 3)); Optional derivedDistribution = - Optional.of(TableDistribution.ofHash(Collections.singletonList("b"), 3)); + Optional.of(TableDistribution.ofHash(List.of("b"), 3)); assertThatThrownBy( () -> @@ -802,9 +761,9 @@ void mergeIncludingDistributionFailsOnDuplicate() { @Test void mergeExcludingDistributionOnDuplicate() { Optional sourceDistribution = - Optional.of(TableDistribution.ofHash(Collections.singletonList("a"), 3)); + Optional.of(TableDistribution.ofHash(List.of("a"), 3)); Optional derivedDistribution = - Optional.of(TableDistribution.ofHash(Collections.singletonList("b"), 3)); + Optional.of(TableDistribution.ofHash(List.of("b"), 3)); Optional mergedPartitions = util.mergeDistribution( @@ -815,23 +774,23 @@ void mergeExcludingDistributionOnDuplicate() { @Test void mergePartitionsFromBaseTable() { - List sourcePartitions = Arrays.asList("col1", "col2"); + List sourcePartitions = List.of("col1", "col2"); List mergePartitions = util.mergePartitions( getDefaultMergingStrategies().get(FeatureOption.PARTITIONS), sourcePartitions, - Collections.emptyList()); + List.of()); assertThat(mergePartitions).isEqualTo(sourcePartitions); } @Test void mergePartitionsFromDerivedTable() { - List derivedPartitions = Arrays.asList("col1", "col2"); + List derivedPartitions = List.of("col1", "col2"); List mergePartitions = util.mergePartitions( getDefaultMergingStrategies().get(FeatureOption.PARTITIONS), - Collections.emptyList(), + List.of(), derivedPartitions); assertThat(mergePartitions).isEqualTo(derivedPartitions); @@ -839,8 +798,8 @@ void mergePartitionsFromDerivedTable() { @Test void mergeIncludingPartitionsFailsOnDuplicate() { - List sourcePartitions = Arrays.asList("col3", "col4"); - List derivedPartitions = Arrays.asList("col1", "col2"); + List sourcePartitions = List.of("col3", "col4"); + List derivedPartitions = List.of("col1", "col2"); assertThatThrownBy( () -> @@ -856,8 +815,8 @@ void mergeIncludingPartitionsFailsOnDuplicate() { @Test void mergeExcludingPartitionsOnDuplicate() { - List sourcePartitions = Arrays.asList("col3", "col4"); - List derivedPartitions = Arrays.asList("col1", "col2"); + List sourcePartitions = List.of("col3", "col4"); + List derivedPartitions = List.of("col1", "col2"); List mergedPartitions = util.mergePartitions( @@ -952,7 +911,7 @@ void mergeOverwritingOptionsDuplicate() { @Test void defaultMergeStrategies() { Map mergingStrategies = - util.computeMergingStrategies(Collections.emptyList()); + util.computeMergingStrategies(List.of()); assertThat(mergingStrategies.get(FeatureOption.OPTIONS)) .isEqualTo(MergingStrategy.OVERWRITING); @@ -969,8 +928,7 @@ void defaultMergeStrategies() { @Test void includingAllMergeStrategyExpansion() { List inputOptions = - Collections.singletonList( - new SqlTableLikeOption(MergingStrategy.INCLUDING, FeatureOption.ALL)); + List.of(new SqlTableLikeOption(MergingStrategy.INCLUDING, FeatureOption.ALL)); Map mergingStrategies = util.computeMergingStrategies(inputOptions); @@ -990,8 +948,7 @@ void includingAllMergeStrategyExpansion() { @Test void excludingAllMergeStrategyExpansion() { List inputOptions = - Collections.singletonList( - new SqlTableLikeOption(MergingStrategy.EXCLUDING, FeatureOption.ALL)); + List.of(new SqlTableLikeOption(MergingStrategy.EXCLUDING, FeatureOption.ALL)); Map mergingStrategies = util.computeMergingStrategies(inputOptions); @@ -1011,7 +968,7 @@ void excludingAllMergeStrategyExpansion() { @Test void includingAllOverwriteOptionsMergeStrategyExpansion() { List inputOptions = - Arrays.asList( + List.of( new SqlTableLikeOption(MergingStrategy.EXCLUDING, FeatureOption.ALL), new SqlTableLikeOption( MergingStrategy.INCLUDING, FeatureOption.CONSTRAINTS)); @@ -1034,7 +991,7 @@ void includingAllOverwriteOptionsMergeStrategyExpansion() { } private Map getDefaultMergingStrategies() { - return util.computeMergingStrategies(Collections.emptyList()); + return util.computeMergingStrategies(List.of()); } private SqlNode regularColumn(String name, DataType type) {