diff --git a/flink-table/flink-sql-parser/pom.xml b/flink-table/flink-sql-parser/pom.xml index 366d6d8ea1f2f..433146608cad9 100644 --- a/flink-table/flink-sql-parser/pom.xml +++ b/flink-table/flink-sql-parser/pom.xml @@ -36,7 +36,7 @@ under the License. -Djunit.platform.reflection.search.useLegacySemantics=true diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserAlterTableTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserAlterTableTest.java new file mode 100644 index 0000000000000..006fa783d0d39 --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserAlterTableTest.java @@ -0,0 +1,493 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** ALTER TABLE parser tests. */ +@Execution(CONCURRENT) +class FlinkSqlParserAlterTableTest extends FlinkSqlParserTestBase { + + @Test + void testAlterTable() { + sql("alter table t1 rename to t2").ok("ALTER TABLE `T1` RENAME TO `T2`"); + sql("alter table if exists t1 rename to t2") + .ok("ALTER TABLE IF EXISTS `T1` RENAME TO `T2`"); + sql("alter table c1.d1.t1 rename to t2").ok("ALTER TABLE `C1`.`D1`.`T1` RENAME TO `T2`"); + sql("alter table if exists c1.d1.t1 rename to t2") + .ok("ALTER TABLE IF EXISTS `C1`.`D1`.`T1` RENAME TO `T2`"); + + sql("alter table t1 set ('key1'='value1')") + .ok("ALTER TABLE `T1` SET (\n" + " 'key1' = 'value1'\n" + ")"); + sql("alter table if exists t1 set ('key1'='value1')") + .ok("ALTER TABLE IF EXISTS `T1` SET (\n" + " 'key1' = 'value1'\n" + ")"); + + sql("alter table t1 add constraint ct1 primary key(a, b)") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " CONSTRAINT `CT1` PRIMARY KEY (`A`, `B`)\n" + + ")") + .node( + new ValidationMatcher() + .fails( + "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. " + + "ENFORCED/NOT ENFORCED controls if the constraint checks are performed on the incoming/outgoing data. " + + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); + sql("alter table t1 add constraint ct1 primary key(a, b) not enforced") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " CONSTRAINT `CT1` PRIMARY KEY (`A`, `B`) NOT ENFORCED\n" + + ")"); + sql("alter table if exists t1 add constraint ct1 primary key(a, b) not enforced") + .ok( + "ALTER TABLE IF EXISTS `T1` ADD (\n" + + " CONSTRAINT `CT1` PRIMARY KEY (`A`, `B`) NOT ENFORCED\n" + + ")"); + sql("alter table t1 " + "add unique(a, b)") + .ok("ALTER TABLE `T1` ADD (\n" + " UNIQUE (`A`, `B`)\n" + ")") + .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); + sql("alter table if exists t1 " + "add unique(a, b)") + .ok("ALTER TABLE IF EXISTS `T1` ADD (\n" + " UNIQUE (`A`, `B`)\n" + ")"); + + sql("alter table t1 drop constraint ct1").ok("ALTER TABLE `T1` DROP CONSTRAINT `CT1`"); + sql("alter table if exists t1 drop constraint ct1") + .ok("ALTER TABLE IF EXISTS `T1` DROP CONSTRAINT `CT1`"); + + sql("alter table t1 rename a to b").ok("ALTER TABLE `T1` RENAME `A` TO `B`"); + sql("alter table if exists t1 rename a to b") + .ok("ALTER TABLE IF EXISTS `T1` RENAME `A` TO `B`"); + sql("alter table if exists t1 rename a.x to a.y") + .ok("ALTER TABLE IF EXISTS `T1` RENAME `A`.`X` TO `A`.`Y`"); + } + + @Test + void testAlterTableAddNestedColumn() { + // add a row column + sql("alter table t1 add new_column array comment 'new_column docs'") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " `NEW_COLUMN` ARRAY< ROW(`F0` INTEGER, `F1` BIGINT) > COMMENT 'new_column docs'\n" + + ")"); + + sql("alter table t1 add (new_row row(f0 int, f1 bigint) comment 'new_column docs', f2 as new_row.f0 + 1)") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " `NEW_ROW` ROW(`F0` INTEGER, `F1` BIGINT) COMMENT 'new_column docs',\n" + + " `F2` AS (`NEW_ROW`.`F0` + 1)\n" + + ")"); + + // add a field to the row + sql("alter table t1 add (new_row.f2 array)") + .ok("ALTER TABLE `T1` ADD (\n" + " `NEW_ROW`.`F2` ARRAY< INTEGER >\n" + ")"); + + // add a field to the row with after + sql("alter table t1 add (new_row.f2 array after new_row.f0)") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " `NEW_ROW`.`F2` ARRAY< INTEGER > AFTER `NEW_ROW`.`F0`\n" + + ")"); + } + + @Test + void testAlterTableAddSingleColumn() { + sql("alter table if exists t1 add new_column int not null") + .ok( + "ALTER TABLE IF EXISTS `T1` ADD (\n" + + " `NEW_COLUMN` INTEGER NOT NULL\n" + + ")"); + sql("alter table t1 add new_column string comment 'new_column docs'") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " `NEW_COLUMN` STRING COMMENT 'new_column docs'\n" + + ")"); + sql("alter table t1 add new_column string comment 'new_column docs' first") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " `NEW_COLUMN` STRING COMMENT 'new_column docs' FIRST\n" + + ")"); + sql("alter table t1 add new_column string comment 'new_column docs' after id") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " `NEW_COLUMN` STRING COMMENT 'new_column docs' AFTER `ID`\n" + + ")"); + // add compute column + sql("alter table t1 add col_int as col_a - col_b after col_b") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " `COL_INT` AS (`COL_A` - `COL_B`) AFTER `COL_B`\n" + + ")"); + // add metadata column + sql("alter table t1 add col_int int metadata from 'mk1' virtual comment 'comment_metadata' after col_b") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " `COL_INT` INTEGER METADATA FROM 'mk1' VIRTUAL COMMENT 'comment_metadata' AFTER `COL_B`\n" + + ")"); + } + + @Test + void testAlterTableAddWatermark() { + sql("alter table if exists t1 add watermark for ts as ts") + .ok("ALTER TABLE IF EXISTS `T1` ADD (\n" + " WATERMARK FOR `TS` AS `TS`\n" + ")"); + sql("alter table t1 add watermark for ts as ts - interval '1' second") + .ok( + "ALTER TABLE `T1` ADD (\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" + + ")"); + sql("alter table default_database.t1 add watermark for ts as ts - interval '1' second") + .ok( + "ALTER TABLE `DEFAULT_DATABASE`.`T1` ADD (\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" + + ")"); + sql("alter table default_catalog.default_database.t1 add watermark for ts as ts - interval '1' second") + .ok( + "ALTER TABLE `DEFAULT_CATALOG`.`DEFAULT_DATABASE`.`T1` ADD (\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" + + ")"); + + sql("alter table default_catalog.default_database.t1 add (\n" + + "watermark for ts as ts - interval '1' second,\n" + + "^watermark^ for f1 as now()\n" + + ")") + .fails("Multiple WATERMARK declarations are not supported yet."); + } + + @Test + void testAlterTableAddDistribution() { + sql("alter table t1 add DISTRIBUTION BY HASH(a) INTO 6 BUCKETS") + .ok("ALTER TABLE `T1` ADD DISTRIBUTION BY HASH(`A`) INTO 6 BUCKETS"); + + sql("alter table t1 add DISTRIBUTION BY HASH(a, h) INTO 6 BUCKETS") + .ok("ALTER TABLE `T1` ADD DISTRIBUTION BY HASH(`A`, `H`) INTO 6 BUCKETS"); + + sql("alter table tbl1 add DISTRIBUTION BY RANGE(a, h) INTO 6 BUCKETS") + .ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY RANGE(`A`, `H`) INTO 6 BUCKETS"); + + sql("alter table tbl1 add DISTRIBUTION BY ^RANDOM^(a, h) INTO 6 BUCKETS") + .fails("(?s).*Encountered \"RANDOM\" at line 1, column 38.*"); + + sql("alter table tbl1 add DISTRIBUTION BY (a, h) INTO 6 BUCKETS") + .ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY (`A`, `H`) INTO 6 BUCKETS"); + + sql("alter table tbl1 add DISTRIBUTION BY RANGE(a, h)") + .ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY RANGE(`A`, `H`)"); + + sql("alter table tbl1 add DISTRIBUTION BY (a, h)") + .ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY (`A`, `H`)"); + } + + @Test + void testAlterTableModifyDistribution() { + sql("alter table t1 modify DISTRIBUTION BY HASH(a) INTO 6 BUCKETS") + .ok("ALTER TABLE `T1` MODIFY DISTRIBUTION BY HASH(`A`) INTO 6 BUCKETS"); + + sql("alter table tbl1 modify DISTRIBUTION BY HASH(a, h) INTO 6 BUCKETS") + .ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY HASH(`A`, `H`) INTO 6 BUCKETS"); + + sql("alter table tbl1 modify DISTRIBUTION BY RANGE(a, h) INTO 6 BUCKETS") + .ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY RANGE(`A`, `H`) INTO 6 BUCKETS"); + + sql("alter table tbl1 modify DISTRIBUTION BY ^RANDOM^(a, h) INTO 6 BUCKETS") + .fails("(?s).*Encountered \"RANDOM\" at line 1, column 41.*"); + + sql("alter table tbl1 modify DISTRIBUTION BY (a, h) INTO 6 BUCKETS") + .ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY (`A`, `H`) INTO 6 BUCKETS"); + + sql("alter table tbl1 modify DISTRIBUTION BY RANGE(a, h)") + .ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY RANGE(`A`, `H`)"); + + sql("alter table tbl1 modify DISTRIBUTION BY (a, h)") + .ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY (`A`, `H`)"); + } + + @Test + void testAlterTableDropDistribution() { + sql("alter table t1 drop DISTRIBUTION").ok("ALTER TABLE `T1` DROP DISTRIBUTION"); + } + + @Test + void testAlterTableAddMultipleColumn() { + final String sql1 = + "alter table t1 add (\n" + + "col_int int,\n" + + "log_ts string comment 'log timestamp string' first,\n" + + "ts AS to_timestamp(log_ts) after log_ts,\n" + + "col_meta int metadata from 'mk1' virtual comment 'comment_str' after col_b,\n" + + "primary key (id) not enforced,\n" + + "unique(a, b),\n" + + "watermark for ts as ts - interval '3' second\n" + + ")"; + final String expected1 = + "ALTER TABLE `T1` ADD (\n" + + " `COL_INT` INTEGER,\n" + + " `LOG_TS` STRING COMMENT 'log timestamp string' FIRST,\n" + + " `TS` AS `TO_TIMESTAMP`(`LOG_TS`) AFTER `LOG_TS`,\n" + + " `COL_META` INTEGER METADATA FROM 'mk1' VIRTUAL COMMENT 'comment_str' AFTER `COL_B`,\n" + + " PRIMARY KEY (`ID`) NOT ENFORCED,\n" + + " UNIQUE (`A`, `B`),\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '3' SECOND)\n" + + ")"; + sql(sql1).ok(expected1); + + final String sql2 = + "alter table t1 add (\n" + + "col_int int primary key not enforced,\n" + + "log_ts string comment 'log timestamp string' first,\n" + + "ts AS to_timestamp(log_ts) after log_ts,\n" + + "col_meta int metadata from 'mk1' virtual comment 'comment_str' after col_b,\n" + + "primary key (id) not enforced,\n" + + "unique (a, b),\n" + + "watermark for ts as ts - interval '3' second\n" + + ")"; + sql(sql2).node(new ValidationMatcher().fails("Duplicate primary key definition")); + } + + @Test + public void testAlterTableModifySingleColumn() { + sql("alter table if exists t1 modify new_column string comment 'new_column docs'") + .ok( + "ALTER TABLE IF EXISTS `T1` MODIFY (\n" + + " `NEW_COLUMN` STRING COMMENT 'new_column docs'\n" + + ")"); + sql("alter table t1 modify new_column string comment 'new_column docs'") + .ok( + "ALTER TABLE `T1` MODIFY (\n" + + " `NEW_COLUMN` STRING COMMENT 'new_column docs'\n" + + ")"); + sql("alter table t1 modify new_column string comment 'new_column docs' first") + .ok( + "ALTER TABLE `T1` MODIFY (\n" + + " `NEW_COLUMN` STRING COMMENT 'new_column docs' FIRST\n" + + ")"); + sql("alter table t1 modify new_column string comment 'new_column docs' after id") + .ok( + "ALTER TABLE `T1` MODIFY (\n" + + " `NEW_COLUMN` STRING COMMENT 'new_column docs' AFTER `ID`\n" + + ")"); + // modify column type + sql("alter table t1 modify new_column array not null") + .ok( + "ALTER TABLE `T1` MODIFY (\n" + + " `NEW_COLUMN` ARRAY< STRING NOT NULL > NOT NULL\n" + + ")"); + + // modify compute column + sql("alter table t1 modify col_int as col_a - col_b after col_b") + .ok( + "ALTER TABLE `T1` MODIFY (\n" + + " `COL_INT` AS (`COL_A` - `COL_B`) AFTER `COL_B`\n" + + ")"); + // modify metadata column + sql("alter table t1 modify col_int int metadata from 'mk1' virtual comment 'comment_metadata' after col_b") + .ok( + "ALTER TABLE `T1` MODIFY (\n" + + " `COL_INT` INTEGER METADATA FROM 'mk1' VIRTUAL COMMENT 'comment_metadata' AFTER `COL_B`\n" + + ")"); + + // modify nested column + sql("alter table t1 modify row_column.f0 int not null comment 'change nullability'") + .ok( + "ALTER TABLE `T1` MODIFY (\n" + + " `ROW_COLUMN`.`F0` INTEGER NOT NULL COMMENT 'change nullability'\n" + + ")"); + + // modify nested column, shift position + sql("alter table t1 modify row_column.f0 int after row_column.f2") + .ok( + "ALTER TABLE `T1` MODIFY (\n" + + " `ROW_COLUMN`.`F0` INTEGER AFTER `ROW_COLUMN`.`F2`\n" + + ")"); + } + + @Test + void testAlterTableModifyWatermark() { + sql("alter table if exists t1 modify watermark for ts as ts") + .ok( + "ALTER TABLE IF EXISTS `T1` MODIFY (\n" + + " WATERMARK FOR `TS` AS `TS`\n" + + ")"); + sql("alter table t1 modify watermark for ts as ts - interval '1' second") + .ok( + "ALTER TABLE `T1` MODIFY (\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" + + ")"); + sql("alter table default_database.t1 modify watermark for ts as ts - interval '1' second") + .ok( + "ALTER TABLE `DEFAULT_DATABASE`.`T1` MODIFY (\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" + + ")"); + sql("alter table default_catalog.default_database.t1 modify watermark for ts as ts - interval '1' second") + .ok( + "ALTER TABLE `DEFAULT_CATALOG`.`DEFAULT_DATABASE`.`T1` MODIFY (\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" + + ")"); + + sql("alter table default_catalog.default_database.t1 modify (\n" + + "watermark for ts as ts - interval '1' second,\n" + + "^watermark^ for f1 as now()\n" + + ")") + .fails("Multiple WATERMARK declarations are not supported yet."); + } + + @Test + void testAlterTableModifyConstraint() { + sql("alter table t1 modify constraint ct1 primary key(a, b) not enforced") + .ok( + "ALTER TABLE `T1` MODIFY (\n" + + " CONSTRAINT `CT1` PRIMARY KEY (`A`, `B`) NOT ENFORCED\n" + + ")"); + sql("alter table t1 modify unique(a, b)") + .ok("ALTER TABLE `T1` MODIFY (\n" + " UNIQUE (`A`, `B`)\n" + ")"); + } + + @Test + public void testAlterTableModifyMultipleColumn() { + final String sql1 = + "alter table t1 modify (\n" + + "col_int int,\n" + + "log_ts string comment 'log timestamp string' first,\n" + + "ts AS to_timestamp(log_ts) after log_ts,\n" + + "col_meta int metadata from 'mk1' virtual comment 'comment_str' after col_b,\n" + + "primary key (id) not enforced,\n" + + "unique(a, b),\n" + + "watermark for ts as ts - interval '3' second\n" + + ")"; + final String expected1 = + "ALTER TABLE `T1` MODIFY (\n" + + " `COL_INT` INTEGER,\n" + + " `LOG_TS` STRING COMMENT 'log timestamp string' FIRST,\n" + + " `TS` AS `TO_TIMESTAMP`(`LOG_TS`) AFTER `LOG_TS`,\n" + + " `COL_META` INTEGER METADATA FROM 'mk1' VIRTUAL COMMENT 'comment_str' AFTER `COL_B`,\n" + + " PRIMARY KEY (`ID`) NOT ENFORCED,\n" + + " UNIQUE (`A`, `B`),\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '3' SECOND)\n" + + ")"; + sql(sql1).ok(expected1); + } + + @Test + public void testAlterTableDropSingleColumn() { + sql("alter table if exists t1 drop id") + .ok("ALTER TABLE IF EXISTS `T1` DROP (\n" + " `ID`\n" + ")"); + sql("alter table t1 drop id").ok("ALTER TABLE `T1` DROP (\n" + " `ID`\n" + ")"); + + sql("alter table t1 drop (id)").ok("ALTER TABLE `T1` DROP (\n" + " `ID`\n" + ")"); + + sql("alter table t1 drop tuple.id") + .ok("ALTER TABLE `T1` DROP (\n" + " `TUPLE`.`ID`\n" + ")"); + } + + @Test + public void testAlterTableDropMultipleColumn() { + sql("alter table if exists t1 drop (id, ts, tuple.f0, tuple.f1)") + .ok( + "ALTER TABLE IF EXISTS `T1` DROP (\n" + + " `ID`,\n" + + " `TS`,\n" + + " `TUPLE`.`F0`,\n" + + " `TUPLE`.`F1`\n" + + ")"); + sql("alter table t1 drop (id, ts, tuple.f0, tuple.f1)") + .ok( + "ALTER TABLE `T1` DROP (\n" + + " `ID`,\n" + + " `TS`,\n" + + " `TUPLE`.`F0`,\n" + + " `TUPLE`.`F1`\n" + + ")"); + } + + @Test + public void testAlterTableDropPrimaryKey() { + sql("alter table if exists t1 drop primary key") + .ok("ALTER TABLE IF EXISTS `T1` DROP PRIMARY KEY"); + sql("alter table t1 drop primary key").ok("ALTER TABLE `T1` DROP PRIMARY KEY"); + } + + @Test + public void testAlterTableDropConstraint() { + sql("alter table if exists t1 drop constraint ct") + .ok("ALTER TABLE IF EXISTS `T1` DROP CONSTRAINT `CT`"); + sql("alter table t1 drop constraint ct").ok("ALTER TABLE `T1` DROP CONSTRAINT `CT`"); + + sql("alter table t1 drop constrain^t^") + .fails("(?s).*Encountered \"\" at line 1, column 30.\n.*"); + } + + @Test + public void testAlterTableDropWatermark() { + sql("alter table if exists t1 drop watermark") + .ok("ALTER TABLE IF EXISTS `T1` DROP WATERMARK"); + sql("alter table t1 drop watermark").ok("ALTER TABLE `T1` DROP WATERMARK"); + } + + @Test + void testAlterTableReset() { + sql("alter table if exists t1 reset ('key1')") + .ok("ALTER TABLE IF EXISTS `T1` RESET (\n 'key1'\n)"); + + sql("alter table t1 reset ('key1')").ok("ALTER TABLE `T1` RESET (\n 'key1'\n)"); + + sql("alter table t1 reset ('key1', 'key2')") + .ok("ALTER TABLE `T1` RESET (\n 'key1',\n 'key2'\n)"); + + sql("alter table t1 reset()").ok("ALTER TABLE `T1` RESET (\n)"); + } + + @Test + public void testAddPartition() { + sql("alter table c1.d1.tbl add partition (p1=1,p2='a')") + .ok("ALTER TABLE `C1`.`D1`.`TBL`\n" + "ADD\n" + "PARTITION (`P1` = 1, `P2` = 'a')"); + + sql("alter table tbl add partition (p1=1,p2='a') with ('k1'='v1')") + .ok( + "ALTER TABLE `TBL`\n" + + "ADD\n" + + "PARTITION (`P1` = 1, `P2` = 'a') WITH ('k1' = 'v1')"); + + sql("alter table tbl add if not exists partition (p=1) partition (p=2) with ('k1' = 'v1')") + .ok( + "ALTER TABLE `TBL`\n" + + "ADD IF NOT EXISTS\n" + + "PARTITION (`P` = 1)\n" + + "PARTITION (`P` = 2) WITH ('k1' = 'v1')"); + } + + @Test + public void testDropPartition() { + sql("alter table c1.d1.tbl drop if exists partition (p=1)") + .ok("ALTER TABLE `C1`.`D1`.`TBL`\n" + "DROP IF EXISTS\n" + "PARTITION (`P` = 1)"); + sql("alter table tbl drop partition (p1='a',p2=1), partition(p1='b',p2=2)") + .ok( + "ALTER TABLE `TBL`\n" + + "DROP\n" + + "PARTITION (`P1` = 'a', `P2` = 1),\n" + + "PARTITION (`P1` = 'b', `P2` = 2)"); + sql("alter table tbl drop partition (p1='a',p2=1), " + + "partition(p1='b',p2=2), partition(p1='c',p2=3)") + .ok( + "ALTER TABLE `TBL`\n" + + "DROP\n" + + "PARTITION (`P1` = 'a', `P2` = 1),\n" + + "PARTITION (`P1` = 'b', `P2` = 2),\n" + + "PARTITION (`P1` = 'c', `P2` = 3)"); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCalciteTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCalciteTest.java new file mode 100644 index 0000000000000..81345b6f90c74 --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCalciteTest.java @@ -0,0 +1,136 @@ +/* + * 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; + +import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl; + +import org.apache.calcite.sql.parser.SqlParserFixture; +import org.apache.calcite.sql.parser.SqlParserTest; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** + * Verifies that Flink's extended SQL parser correctly handles all standard SQL syntax defined in + * Calcite's {@link SqlParserTest}. Methods overridden with empty bodies represent Calcite syntax + * that Flink intentionally does not support. + */ +@Execution(CONCURRENT) +class FlinkSqlParserCalciteTest extends SqlParserTest { + + @Override + public SqlParserFixture fixture() { + return super.fixture().withConfig(c -> c.withParserFactory(FlinkSqlParserImpl.FACTORY)); + } + + // -- Calcite syntax not supported by Flink's parser -- + + @Test + void testArrayFunction() {} + + @Test + void testArrayQueryConstructor() {} + + @Test + void testPercentileCont() {} + + @Test + void testPercentileContBigQuery() {} + + @Test + void testPercentileDisc() {} + + @Test + void testPercentileDiscBigQuery() {} + + @Test + void testMapQueryConstructor() {} + + @Test + void testMultisetQueryConstructor() {} + + @Disabled + @Test + void testDescribeSchema() {} + + @Disabled + @Test + void testDescribeStatement() {} + + @Disabled + @Test + void testGroupConcat() {} + + @Disabled + @Test + void testExplainAsDot() {} + + @Disabled + @Test + void testStringAgg() {} + + // -- Calcite syntax overridden by Flink (tested in split test classes) -- + + @Test + void testArrayAgg() {} + + @Test + void testCastAsRowType() {} + + @Test + void testDescribeTable() {} + + @Test + void testExplain() {} + + @Test + void testExplainJsonFormat() {} + + @Test + void testExplainWithImpl() {} + + @Test + void testExplainWithoutImpl() {} + + @Test + void testExplainWithType() {} + + @Test + void testExplainAsXml() {} + + @Test + void testExplainAsJson() {} + + @Test + void testExplainInsert() {} + + @Test + void testExplainUpsert() {} + + @Test + void testSqlOptions() {} + + @Test + void testFromValuesWithoutParens() {} + + @Test + void testUnnest() {} +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCatalogTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCatalogTest.java new file mode 100644 index 0000000000000..68ff24e468770 --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCatalogTest.java @@ -0,0 +1,208 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +import java.util.Locale; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** Catalog and database parser tests. */ +@Execution(CONCURRENT) +class FlinkSqlParserCatalogTest extends FlinkSqlParserTestBase { + + @Test + void testShowCatalogs() { + sql("show catalogs").ok("SHOW CATALOGS"); + + sql("show catalogs like '%'").ok("SHOW CATALOGS LIKE '%'"); + sql("show catalogs not like '%'").ok("SHOW CATALOGS NOT LIKE '%'"); + + sql("show catalogs ilike '%'").ok("SHOW CATALOGS ILIKE '%'"); + sql("show catalogs not ilike '%'").ok("SHOW CATALOGS NOT ILIKE '%'"); + + sql("show catalogs ^likes^").fails("(?s).*Encountered \"likes\" at line 1, column 15.\n.*"); + sql("show catalogs not ^likes^") + .fails("(?s).*Encountered \"likes\" at line 1, column 19" + ".\n" + ".*"); + sql("show catalogs ^ilikes^") + .fails("(?s).*Encountered \"ilikes\" at line 1, column 15.\n.*"); + sql("show catalogs not ^ilikes^") + .fails("(?s).*Encountered \"ilikes\" at line 1, column 19" + ".\n" + ".*"); + } + + @Test + void testShowCurrentCatalog() { + sql("show current catalog").ok("SHOW CURRENT CATALOG"); + } + + @Test + void testDescribeCatalog() { + sql("describe catalog a").ok("DESCRIBE CATALOG `A`"); + sql("describe catalog extended a").ok("DESCRIBE CATALOG EXTENDED `A`"); + + sql("desc catalog a").ok("DESCRIBE CATALOG `A`"); + sql("desc catalog extended a").ok("DESCRIBE CATALOG EXTENDED `A`"); + } + + @Test + void testAlterCatalog() { + sql("alter catalog a set ('k1'='v1', 'k2'='v2')") + .ok("ALTER CATALOG `A` SET (\n" + " 'k1' = 'v1',\n" + " 'k2' = 'v2'\n" + ")"); + sql("alter catalog a reset ('k1')").ok("ALTER CATALOG `A` RESET (\n" + " 'k1'\n" + ")"); + sql("alter catalog a comment 'comment1'").ok("ALTER CATALOG `A` COMMENT 'comment1'"); + } + + // END + + @Test + void testUseCatalog() { + sql("use catalog a").ok("USE CATALOG `A`"); + } + + @ParameterizedTest + @CsvSource({"true,true", "true,false", "false,true", "false,false"}) + void testCreateCatalog(boolean ifNotExists, boolean comment) { + final String ifNotExistsClause = ifNotExists ? "if not exists " : ""; + final String commentClause = comment ? "\ncomment 'HELLO'" : ""; + + sql("create catalog " + + ifNotExistsClause + + "c1" + + commentClause + + "\nWITH (\n" + + " 'key1'='value1',\n" + + " 'key2'='value2'\n" + + " )\n") + .ok( + "CREATE CATALOG " + + ifNotExistsClause.toUpperCase(Locale.ROOT) + + "`C1`" + + commentClause.toUpperCase(Locale.ROOT) + + "\nWITH (\n" + + " 'key1' = 'value1',\n" + + " 'key2' = 'value2'\n" + + ")"); + } + + @Test + void testShowCreateCatalog() { + sql("show create catalog c1").ok("SHOW CREATE CATALOG `C1`"); + } + + @Test + void testDropCatalog() { + sql("drop catalog c1").ok("DROP CATALOG `C1`"); + } + + @Test + void testShowDataBases() { + sql("show databases").ok("SHOW DATABASES"); + + sql("show databases like '%'").ok("SHOW DATABASES LIKE '%'"); + sql("show databases not like '%'").ok("SHOW DATABASES NOT LIKE '%'"); + + sql("show databases from c1").ok("SHOW DATABASES FROM `C1`"); + sql("show databases in c1").ok("SHOW DATABASES IN `C1`"); + + sql("show databases from c1 like '%'").ok("SHOW DATABASES FROM `C1` LIKE '%'"); + sql("show databases from c1 ilike '%'").ok("SHOW DATABASES FROM `C1` ILIKE '%'"); + sql("show databases in c1 like '%'").ok("SHOW DATABASES IN `C1` LIKE '%'"); + sql("show databases in c1 ilike '%'").ok("SHOW DATABASES IN `C1` ILIKE '%'"); + + sql("show databases from c1 not like '%'").ok("SHOW DATABASES FROM `C1` NOT LIKE '%'"); + sql("show databases from c1 not ilike '%'").ok("SHOW DATABASES FROM `C1` NOT ILIKE '%'"); + sql("show databases in c1 not like '%'").ok("SHOW DATABASES IN `C1` NOT LIKE '%'"); + sql("show databases in c1 not ilike '%'").ok("SHOW DATABASES IN `C1` NOT ILIKE '%'"); + + sql("show databases ^likes^") + .fails("(?s).*Encountered \"likes\" at line 1, column 16.\n.*"); + sql("show databases not ^likes^") + .fails("(?s).*Encountered \"likes\" at line 1, column 20" + ".\n" + ".*"); + sql("show databases ^ilikes^") + .fails("(?s).*Encountered \"ilikes\" at line 1, column 16.\n.*"); + sql("show databases not ^ilikes^") + .fails("(?s).*Encountered \"ilikes\" at line 1, column 20" + ".\n" + ".*"); + } + + @Test + void testShowCurrentDatabase() { + sql("show current database").ok("SHOW CURRENT DATABASE"); + } + + @Test + void testUseDataBase() { + sql("use default_db").ok("USE `DEFAULT_DB`"); + sql("use defaultCatalog.default_db").ok("USE `DEFAULTCATALOG`.`DEFAULT_DB`"); + } + + @Test + void testCreateDatabase() { + sql("create database db1").ok("CREATE DATABASE `DB1`"); + sql("create database if not exists db1").ok("CREATE DATABASE IF NOT EXISTS `DB1`"); + sql("create database catalog1.db1").ok("CREATE DATABASE `CATALOG1`.`DB1`"); + final String sql = "create database db1 comment 'test create database'"; + final String expected = "CREATE DATABASE `DB1`\n" + "COMMENT 'test create database'"; + sql(sql).ok(expected); + final String sql1 = + "create database db1 comment 'test create database'" + + "with ( 'key1' = 'value1', 'key2.a' = 'value2.a')"; + final String expected1 = + "CREATE DATABASE `DB1`\n" + + "COMMENT 'test create database'" + + "\nWITH (\n" + + " 'key1' = 'value1',\n" + + " 'key2.a' = 'value2.a'\n" + + ")"; + sql(sql1).ok(expected1); + } + + @Test + void testDropDatabase() { + sql("drop database db1").ok("DROP DATABASE `DB1` RESTRICT"); + sql("drop database catalog1.db1").ok("DROP DATABASE `CATALOG1`.`DB1` RESTRICT"); + sql("drop database db1 RESTRICT").ok("DROP DATABASE `DB1` RESTRICT"); + sql("drop database db1 CASCADE").ok("DROP DATABASE `DB1` CASCADE"); + } + + @Test + void testAlterDatabase() { + final String sql = "alter database db1 set ('key1' = 'value1','key2.a' = 'value2.a')"; + final String expected = + "ALTER DATABASE `DB1` SET (\n" + + " 'key1' = 'value1',\n" + + " 'key2.a' = 'value2.a'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testDescribeDatabase() { + sql("describe database db1").ok("DESCRIBE DATABASE `DB1`"); + sql("describe database catalog1.db1").ok("DESCRIBE DATABASE `CATALOG1`.`DB1`"); + sql("describe database extended db1").ok("DESCRIBE DATABASE EXTENDED `DB1`"); + + sql("desc database db1").ok("DESCRIBE DATABASE `DB1`"); + sql("desc database catalog1.db1").ok("DESCRIBE DATABASE `CATALOG1`.`DB1`"); + sql("desc database extended db1").ok("DESCRIBE DATABASE EXTENDED `DB1`"); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserConnectionTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserConnectionTest.java new file mode 100644 index 0000000000000..04c3e7dc0566f --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserConnectionTest.java @@ -0,0 +1,275 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** CONNECTION parser tests. */ +@Execution(CONCURRENT) +class FlinkSqlParserConnectionTest extends FlinkSqlParserTestBase { + + // ===================================================================================== + + @Test + void testCreateConnection() { + sql("create connection conn1\n" + + " COMMENT 'connection_comment'\n" + + " WITH (\n" + + " 'type'='basic',\n" + + " 'url'='http://example.com',\n" + + " 'username'='user1',\n" + + " 'password'='pass1'\n" + + " )\n") + .ok( + "CREATE CONNECTION `CONN1`\n" + + "COMMENT 'connection_comment'\n" + + "WITH (\n" + + " 'type' = 'basic',\n" + + " 'url' = 'http://example.com',\n" + + " 'username' = 'user1',\n" + + " 'password' = 'pass1'\n" + + ")"); + } + + @Test + void testCreateConnectionIfNotExists() { + sql("create connection if not exists conn1\n" + + " WITH (\n" + + " 'type'='bearer',\n" + + " 'token'='my_token'\n" + + " )\n") + .ok( + "CREATE CONNECTION IF NOT EXISTS `CONN1`\n" + + "WITH (\n" + + " 'type' = 'bearer',\n" + + " 'token' = 'my_token'\n" + + ")"); + } + + @Test + void testCreateTemporaryConnection() { + sql("create temporary connection conn1\n" + + " WITH (\n" + + " 'type'='oauth',\n" + + " 'client_id'='client1'\n" + + " )\n") + .ok( + "CREATE TEMPORARY CONNECTION `CONN1`\n" + + "WITH (\n" + + " 'type' = 'oauth',\n" + + " 'client_id' = 'client1'\n" + + ")"); + } + + @Test + void testCreateSystemConnection() { + sql("create ^system^ connection conn1\n" + + " WITH (\n" + + " 'type'='basic',\n" + + " 'url'='http://example.com'\n" + + " )\n") + .fails( + "(?s)CREATE SYSTEM CONNECTION is not supported, " + + "system connections can only be registered as temporary " + + "connections, you can use CREATE TEMPORARY SYSTEM CONNECTION " + + "instead\\..*"); + } + + @Test + void testCreateTemporarySystemConnection() { + sql("create temporary system connection conn1\n" + + " WITH (\n" + + " 'type'='custom_type',\n" + + " 'api_key'='key123'\n" + + " )\n") + .ok( + "CREATE TEMPORARY SYSTEM CONNECTION `CONN1`\n" + + "WITH (\n" + + " 'type' = 'custom_type',\n" + + " 'api_key' = 'key123'\n" + + ")"); + } + + @Test + void testCreateConnectionWithQualifiedName() { + sql("create connection catalog1.db1.conn1\n" + + " WITH ('type'='basic', 'url'='http://example.com')\n") + .ok( + "CREATE CONNECTION `CATALOG1`.`DB1`.`CONN1`\n" + + "WITH (\n" + + " 'type' = 'basic',\n" + + " 'url' = 'http://example.com'\n" + + ")"); + } + + @Test + void testDropConnection() { + sql("drop connection conn1").ok("DROP CONNECTION `CONN1`"); + sql("drop connection db1.conn1").ok("DROP CONNECTION `DB1`.`CONN1`"); + sql("drop connection catalog1.db1.conn1").ok("DROP CONNECTION `CATALOG1`.`DB1`.`CONN1`"); + } + + @Test + void testDropConnectionIfExists() { + sql("drop connection if exists catalog1.db1.conn1") + .ok("DROP CONNECTION IF EXISTS `CATALOG1`.`DB1`.`CONN1`"); + } + + @Test + void testDropTemporaryConnection() { + sql("drop temporary connection conn1").ok("DROP TEMPORARY CONNECTION `CONN1`"); + sql("drop temporary connection if exists conn1") + .ok("DROP TEMPORARY CONNECTION IF EXISTS `CONN1`"); + } + + @Test + void testDropTemporarySystemConnection() { + sql("drop temporary system connection conn1") + .ok("DROP TEMPORARY SYSTEM CONNECTION `CONN1`"); + sql("drop temporary system connection if exists conn1") + .ok("DROP TEMPORARY SYSTEM CONNECTION IF EXISTS `CONN1`"); + } + + @Test + void testDropSystemConnection() { + sql("drop ^system^ connection conn1") + .fails( + "(?s)DROP SYSTEM CONNECTION is not supported, " + + "system connections can only be dropped as temporary " + + "connections, you can use DROP TEMPORARY SYSTEM CONNECTION " + + "instead\\..*"); + } + + @Test + void testAlterConnectionSet() { + final String sql = + "alter connection conn1 set ('password' = 'new_password','url' = 'http://new.com')"; + final String expected = + "ALTER CONNECTION `CONN1` SET (\n" + + " 'password' = 'new_password',\n" + + " 'url' = 'http://new.com'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testAlterConnectionSetWithQualifiedName() { + final String sql = "alter connection catalog1.db1.conn1 set ('token' = 'new_token')"; + final String expected = + "ALTER CONNECTION `CATALOG1`.`DB1`.`CONN1` SET (\n" + + " 'token' = 'new_token'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testAlterConnectionRename() { + final String sql = "alter connection conn1 rename to conn2"; + final String expected = "ALTER CONNECTION `CONN1` RENAME TO `CONN2`"; + sql(sql).ok(expected); + } + + @Test + void testAlterConnectionRenameWithQualifiedName() { + final String sql = "alter connection catalog1.db1.conn1 rename to conn2"; + final String expected = "ALTER CONNECTION `CATALOG1`.`DB1`.`CONN1` RENAME TO `CONN2`"; + sql(sql).ok(expected); + } + + @Test + void testAlterConnectionReset() { + final String sql = "alter connection conn1 reset ('password', 'url')"; + final String expected = "ALTER CONNECTION `CONN1` RESET (\n 'password',\n 'url'\n)"; + sql(sql).ok(expected); + } + + @Test + void testAlterConnectionResetWithQualifiedName() { + final String sql = "alter connection catalog1.db1.conn1 reset ('token')"; + final String expected = "ALTER CONNECTION `CATALOG1`.`DB1`.`CONN1` RESET (\n 'token'\n)"; + sql(sql).ok(expected); + } + + @Test + void testAlterConnectionIfExists() { + final String sql = + "alter connection if exists conn1 set ('password' = 'new_password','url' = 'http://new.com')"; + final String expected = + "ALTER CONNECTION IF EXISTS `CONN1` SET (\n" + + " 'password' = 'new_password',\n" + + " 'url' = 'http://new.com'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testAlterConnectionRenameIfExists() { + final String sql = "alter connection if exists conn1 rename to conn2"; + final String expected = "ALTER CONNECTION IF EXISTS `CONN1` RENAME TO `CONN2`"; + sql(sql).ok(expected); + } + + @Test + void testAlterConnectionResetIfExists() { + final String sql = "alter connection if exists conn1 reset ('password', 'url')"; + final String expected = + "ALTER CONNECTION IF EXISTS `CONN1` RESET (\n 'password',\n 'url'\n)"; + sql(sql).ok(expected); + } + + @Test + void testShowConnections() { + sql("show connections").ok("SHOW CONNECTIONS"); + sql("show connections from db1").ok("SHOW CONNECTIONS FROM `DB1`"); + sql("show connections from catalog1.db1").ok("SHOW CONNECTIONS FROM `CATALOG1`.`DB1`"); + sql("show connections in db1").ok("SHOW CONNECTIONS IN `DB1`"); + sql("show connections in catalog1.db1").ok("SHOW CONNECTIONS IN `CATALOG1`.`DB1`"); + } + + @Test + void testShowConnectionsLike() { + sql("show connections like '%conn%'").ok("SHOW CONNECTIONS LIKE '%CONN%'"); + sql("show connections from db1 like 'my_%'").ok("SHOW CONNECTIONS FROM `DB1` LIKE 'MY_%'"); + sql("show connections not like 'temp_%'").ok("SHOW CONNECTIONS NOT LIKE 'TEMP_%'"); + } + + @Test + void testShowCreateConnection() { + sql("show create connection conn1").ok("SHOW CREATE CONNECTION `CONN1`"); + sql("show create connection catalog1.db1.conn1") + .ok("SHOW CREATE CONNECTION `CATALOG1`.`DB1`.`CONN1`"); + } + + @Test + void testDescribeConnection() { + sql("describe connection conn1").ok("DESCRIBE CONNECTION `CONN1`"); + sql("describe connection catalog1.db1.conn1") + .ok("DESCRIBE CONNECTION `CATALOG1`.`DB1`.`CONN1`"); + sql("describe connection extended conn1").ok("DESCRIBE CONNECTION EXTENDED `CONN1`"); + + sql("desc connection conn1").ok("DESCRIBE CONNECTION `CONN1`"); + sql("desc connection catalog1.db1.conn1") + .ok("DESCRIBE CONNECTION `CATALOG1`.`DB1`.`CONN1`"); + sql("desc connection extended catalog1.db1.conn1") + .ok("DESCRIBE CONNECTION EXTENDED `CATALOG1`.`DB1`.`CONN1`"); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCreateTableTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCreateTableTest.java new file mode 100644 index 0000000000000..e412a7082895d --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCreateTableTest.java @@ -0,0 +1,1390 @@ +/* + * 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; + +import org.apache.calcite.sql.parser.SqlParseException; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** CREATE TABLE, DROP TABLE and INSERT parser tests. */ +@Execution(CONCURRENT) +class FlinkSqlParserCreateTableTest extends FlinkSqlParserTestBase { + + @Test + void testCreateTable() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " g as 2 * (a + 1), \n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" + + " b varchar,\n" + + " proc as PROCTIME(), \n" + + " meta STRING METADATA, \n" + + " my_meta STRING METADATA FROM 'meta', \n" + + " my_meta STRING METADATA FROM 'meta' VIRTUAL, \n" + + " meta STRING METADATA VIRTUAL, \n" + + " PRIMARY KEY (a, b)\n" + + ")\n" + + "PARTITIONED BY (a, h)\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`(),\n" + + " `META` STRING METADATA,\n" + + " `MY_META` STRING METADATA FROM 'meta',\n" + + " `MY_META` STRING METADATA FROM 'meta' VIRTUAL,\n" + + " `META` STRING METADATA VIRTUAL,\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ")\n" + + "PARTITIONED BY (`A`, `H`)\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithDistribution() { + final String sql = buildDistributionInput("DISTRIBUTED BY HASH(a, h) INTO 6 BUCKETS"); + final String expected = + buildDistributionOutput("DISTRIBUTED BY HASH(`A`, `H`) INTO 6 BUCKETS\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithRangeDistribution() { + final String sql = buildDistributionInput("DISTRIBUTED BY RANGE(a, h) INTO 6 BUCKETS\n"); + final String expected = + buildDistributionOutput("DISTRIBUTED BY RANGE(`A`, `H`) INTO 6 BUCKETS\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithRandomDistribution() { + final String sql = buildDistributionInput("DISTRIBUTED BY ^RANDOM^(a, h) INTO 6 BUCKETS\n"); + sql(sql).fails("(?s).*Encountered \"RANDOM\" at line 7, column 16.*"); + } + + @Test + void testCreateTableWithDistributionNoAlgorithm() { + final String sql = buildDistributionInput("DISTRIBUTED BY (a, h) INTO 6 BUCKETS\n"); + final String expected = + buildDistributionOutput("DISTRIBUTED BY (`A`, `H`) INTO 6 BUCKETS\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithDistributionAlgorithmWithoutBuckets() { + final String sql = buildDistributionInput("DISTRIBUTED BY RANGE(a, h)\n"); + final String expected = buildDistributionOutput("DISTRIBUTED BY RANGE(`A`, `H`)\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithDistributionNoAlgorithmWithoutBuckets() { + final String sql = buildDistributionInput("DISTRIBUTED BY (a, h)\n"); + final String expected = buildDistributionOutput("DISTRIBUTED BY (`A`, `H`)\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithDistributionIntoBuckets() { + final String sql = buildDistributionInput("DISTRIBUTED INTO 3 BUCKETS\n"); + final String expected = buildDistributionOutput("DISTRIBUTED INTO 3 BUCKETS\n"); + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithDistributionIntoNegativeBuckets() { + final String sql = buildDistributionInput("DISTRIBUTED INTO ^-^3 BUCKETS\n"); + sql(sql).fails("(?s).*Encountered \"-\" at line 7, column 18.*"); + } + + @Test + void testCreateTableWithDistributionIntoDecimalBuckets() { + final String sql = buildDistributionInput("DISTRIBUTED INTO ^3.2^ BUCKETS\n"); + sql(sql).fails("(?s).*Bucket count must be a positive integer.*"); + } + + @Test + void testCreateTableWithBadDistribution() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " g as 2 * (a + 1), \n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" + + " b varchar,\n" + + " proc as PROCTIME(), \n" + + " meta STRING METADATA, \n" + + " my_meta STRING METADATA FROM 'meta', \n" + + " my_meta STRING METADATA FROM 'meta' VIRTUAL, \n" + + " meta STRING METADATA VIRTUAL, \n" + + " PRIMARY KEY (a, b)\n" + + ")\n" + + "DISTRIBUTED \n" + + " ^with^ (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + sql(sql).fails("(?s).*Encountered \"with\" at line 15, column 3.*"); + } + + @Test + void testCreateTableWithDistributionIfNotExists() { + final String sql = + "CREATE TABLE if not exists tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " PRIMARY KEY (a, b)\n" + + ")\n" + + "DISTRIBUTED BY HASH(a, h) INTO 6 BUCKETS" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE IF NOT EXISTS `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ")\n" + + "DISTRIBUTED BY HASH(`A`, `H`) INTO 6 BUCKETS\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableIfNotExists() { + final String sql = + "CREATE TABLE IF NOT EXISTS tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " PRIMARY KEY (a, b)\n" + + ")\n" + + "PARTITIONED BY (a, h)\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE IF NOT EXISTS `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ")\n" + + "PARTITIONED BY (`A`, `H`)\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithComment() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " a bigint comment 'test column comment AAA.',\n" + + " h varchar, \n" + + " g as 2 * (a + 1), \n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" + + " b varchar,\n" + + " proc as PROCTIME(), \n" + + " meta STRING METADATA COMMENT 'c1', \n" + + " my_meta STRING METADATA FROM 'meta' COMMENT 'c2', \n" + + " my_meta STRING METADATA FROM 'meta' VIRTUAL COMMENT 'c3', \n" + + " meta STRING METADATA VIRTUAL COMMENT 'c4', \n" + + " PRIMARY KEY (a, b)\n" + + ")\n" + + "comment 'test table comment ABC.'\n" + + "PARTITIONED BY (a, h)\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT COMMENT 'test column comment AAA.',\n" + + " `H` VARCHAR,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`(),\n" + + " `META` STRING METADATA COMMENT 'c1',\n" + + " `MY_META` STRING METADATA FROM 'meta' COMMENT 'c2',\n" + + " `MY_META` STRING METADATA FROM 'meta' VIRTUAL COMMENT 'c3',\n" + + " `META` STRING METADATA VIRTUAL COMMENT 'c4',\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ")\n" + + "COMMENT 'test table comment ABC.'\n" + + "PARTITIONED BY (`A`, `H`)\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithCommentOnComputedColumn() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " a bigint comment 'test column comment AAA.',\n" + + " h varchar, \n" + + " g as 2 * (a + 1) comment 'test computed column.', \n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" + + " b varchar,\n" + + " proc as PROCTIME(), \n" + + " PRIMARY KEY (a, b)\n" + + ")\n" + + "comment 'test table comment ABC.'\n" + + "PARTITIONED BY (a, h)\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT COMMENT 'test column comment AAA.',\n" + + " `H` VARCHAR,\n" + + " `G` AS (2 * (`A` + 1)) COMMENT 'test computed column.',\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`(),\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ")\n" + + "COMMENT 'test table comment ABC.'\n" + + "PARTITIONED BY (`A`, `H`)\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testTableConstraints() { + final String sql1 = + "CREATE TABLE tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " g as 2 * (a + 1),\n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" + + " b varchar,\n" + + " proc as PROCTIME(),\n" + + " PRIMARY KEY (a, b),\n" + + " UNIQUE (h, g)\n" + + ") with (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected1 = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`(),\n" + + " PRIMARY KEY (`A`, `B`),\n" + + " UNIQUE (`H`, `G`)\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql1) + .ok(expected1) + .node( + new ValidationMatcher() + .fails( + "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. " + + "ENFORCED/NOT ENFORCED controls if the constraint checks are performed on the incoming/outgoing data. " + + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); + + final String sql2 = + "CREATE TABLE tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " g as 2 * (a + 1),\n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" + + " b varchar,\n" + + " proc as PROCTIME(),\n" + + " PRIMARY KEY (a, b) NOT ENFORCED,\n" + + " UNIQUE (h, g)\n" + + ") with (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected2 = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`(),\n" + + " PRIMARY KEY (`A`, `B`) NOT ENFORCED,\n" + + " UNIQUE (`H`, `G`)\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql2) + .ok(expected2) + .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); + + final String sql3 = + "CREATE TABLE tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " g as 2 * (a + 1),\n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" + + " b varchar,\n" + + " proc as PROCTIME(),\n" + + " PRIMARY KEY (a, b) NOT ENFORCED\n" + + ") with (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expectParsed = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`(),\n" + + " PRIMARY KEY (`A`, `B`) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + final String expectValidated = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT NOT NULL,\n" + + " `H` VARCHAR,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR NOT NULL,\n" + + " `PROC` AS `PROCTIME`(),\n" + + " PRIMARY KEY (`A`, `B`) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql3).ok(expectParsed).node(validated(expectValidated)); + } + + @Test + void testColumnConstraints() { + final String sql1 = + "CREATE TABLE tbl1 (\n" + + " a bigint primary key,\n" + + " h varchar unique,\n" + + " g as 2 * (a + 1),\n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" + + " b varchar,\n" + + " proc as PROCTIME()\n" + + ") with (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected1 = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT PRIMARY KEY,\n" + + " `H` VARCHAR UNIQUE,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`()\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql1) + .ok(expected1) + .node( + new ValidationMatcher() + .fails( + "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. " + + "ENFORCED/NOT ENFORCED controls if the constraint checks are performed on the incoming/outgoing data. " + + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); + + final String sql2 = + "CREATE TABLE tbl1 (\n" + + " a bigint primary key not enforced,\n" + + " h varchar unique,\n" + + " g as 2 * (a + 1),\n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" + + " b varchar,\n" + + " proc as PROCTIME()\n" + + ") with (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected2 = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT PRIMARY KEY NOT ENFORCED,\n" + + " `H` VARCHAR UNIQUE,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`()\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql2) + .ok(expected2) + .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); + + final String sql3 = + "CREATE TABLE tbl1 (\n" + + " a bigint primary key not enforced,\n" + + " h varchar,\n" + + " g as 2 * (a + 1),\n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" + + " b varchar,\n" + + " proc as PROCTIME()\n" + + ") with (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expectParsed = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT PRIMARY KEY NOT ENFORCED,\n" + + " `H` VARCHAR,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`()\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + final String expectValidated = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT NOT NULL PRIMARY KEY NOT ENFORCED,\n" + + " `H` VARCHAR,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`()\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql3).ok(expectParsed).node(validated(expectValidated)); + } + + @Test + void testUniqueTableConstraint() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " g as 2 * (a + 1),\n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" + + " b varchar,\n" + + " proc as PROCTIME(),\n" + + " PRIMARY KEY (a, b) NOT ENFORCED,\n" + + " UNIQUE (h, g)\n" + + ") with (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR,\n" + + " `PROC` AS `PROCTIME`(),\n" + + " PRIMARY KEY (`A`, `B`) NOT ENFORCED,\n" + + " UNIQUE (`H`, `G`)\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + sql(sql).node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); + } + + @Test + void testTableConstraintsWithEnforcement() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " a bigint primary key enforced comment 'test column comment AAA.',\n" + + " h varchar constraint ct1 unique not enforced,\n" + + " g as 2 * (a + 1), \n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" + + " b varchar constraint ct2 unique,\n" + + " proc as PROCTIME(),\n" + + " unique (g, ts) not enforced" + + ") with (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT PRIMARY KEY ENFORCED COMMENT 'test column comment AAA.',\n" + + " `H` VARCHAR CONSTRAINT `CT1` UNIQUE NOT ENFORCED,\n" + + " `G` AS (2 * (`A` + 1)),\n" + + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" + + " `B` VARCHAR CONSTRAINT `CT2` UNIQUE,\n" + + " `PROC` AS `PROCTIME`(),\n" + + " UNIQUE (`G`, `TS`) NOT ENFORCED\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testDuplicatePk() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " a bigint comment 'test column comment AAA.',\n" + + " h varchar constraint ct1 primary key,\n" + + " g as 2 * (a + 1), \n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" + + " b varchar,\n" + + " proc as PROCTIME(),\n" + + " constraint ct2 primary key (b, h)" + + ") with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + sql(sql).node(new ValidationMatcher().fails("Duplicate primary key definition")); + } + + @Test + void testCreateTableWithWatermark() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " ts timestamp(3),\n" + + " id varchar, \n" + + " watermark FOR ts AS ts - interval '3' second\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `TS` TIMESTAMP(3),\n" + + " `ID` VARCHAR,\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '3' SECOND)\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithWatermarkOnComputedColumn() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " log_ts varchar,\n" + + " ts as to_timestamp(log_ts), \n" + + " WATERMARK FOR ts AS ts + interval '1' second\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `LOG_TS` VARCHAR,\n" + + " `TS` AS `TO_TIMESTAMP`(`LOG_TS`),\n" + + " WATERMARK FOR `TS` AS (`TS` + INTERVAL '1' SECOND)\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithWatermarkOnNestedField() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " f1 row, q3 boolean>,\n" + + " WATERMARK FOR f1.q2.t1 AS NOW()\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `F1` ROW< `Q1` BIGINT, `Q2` ROW< `T1` TIMESTAMP, `T2` VARCHAR >, `Q3` BOOLEAN >,\n" + + " WATERMARK FOR `F1`.`Q2`.`T1` AS `NOW`()\n" + + ")\n" + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithMultipleWatermark() { + String sql = + "CREATE TABLE tbl1 (\n" + + " f0 bigint,\n" + + " f1 varchar,\n" + + " f2 boolean,\n" + + " WATERMARK FOR f0 AS NOW(),\n" + + " ^WATERMARK^ FOR f1 AS NOW()\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + sql(sql).fails("Multiple WATERMARK declarations are not supported yet."); + } + + @Test + void testCreateTableWithQueryWatermarkExpression() { + String sql = + "CREATE TABLE tbl1 (\n" + + " f0 bigint,\n" + + " f1 varchar,\n" + + " f2 boolean,\n" + + " WATERMARK FOR f0 AS ^(^SELECT f1 FROM tbl1)\n" + + ")\n" + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + sql(sql).fails("Query expression encountered in illegal context"); + } + + @Test + void testCreateTableWithComplexType() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " a ARRAY, \n" + + " b MAP,\n" + + " c ROW,\n" + + " d MULTISET,\n" + + " PRIMARY KEY (a, b) \n" + + ") with (\n" + + " 'x' = 'y', \n" + + " 'asd' = 'data'\n" + + ")\n"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `A` ARRAY< BIGINT >,\n" + + " `B` MAP< INTEGER, VARCHAR >,\n" + + " `C` ROW< `CC0` INTEGER, `CC1` FLOAT, `CC2` VARCHAR >,\n" + + " `D` MULTISET< VARCHAR >,\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ")\n" + + "WITH (\n" + + " 'x' = 'y',\n" + + " 'asd' = 'data'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithNestedComplexType() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " a ARRAY>, \n" + + " b MAP, ARRAY>,\n" + + " c ROW, cc1 float, cc2 varchar>,\n" + + " d MULTISET>,\n" + + " PRIMARY KEY (a, b) \n" + + ") with (\n" + + " 'x' = 'y', \n" + + " 'asd' = 'data'\n" + + ")\n"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `A` ARRAY< ARRAY< BIGINT > >,\n" + + " `B` MAP< MAP< INTEGER, VARCHAR >, ARRAY< VARCHAR > >,\n" + + " `C` ROW< `CC0` ARRAY< INTEGER >, `CC1` FLOAT, `CC2` VARCHAR >,\n" + + " `D` MULTISET< ARRAY< INTEGER > >,\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ")\n" + + "WITH (\n" + + " 'x' = 'y',\n" + + " 'asd' = 'data'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithUserDefinedType() { + final String sql = + "create table t(\n" + + " a catalog1.db1.MyType1,\n" + + " b db2.MyType2\n" + + ") with (\n" + + " 'k1' = 'v1',\n" + + " 'k2' = 'v2'\n" + + ")"; + final String expected = + "CREATE TABLE `T` (\n" + + " `A` `CATALOG1`.`DB1`.`MYTYPE1`,\n" + + " `B` `DB2`.`MYTYPE2`\n" + + ")\n" + + "WITH (\n" + + " 'k1' = 'v1',\n" + + " 'k2' = 'v2'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testInvalidComputedColumn() { + final String sql0 = + "CREATE TABLE t1 (\n" + + " a bigint, \n" + + " b varchar,\n" + + " toTimestamp^(^b, 'yyyy-MM-dd HH:mm:ss'), \n" + + " PRIMARY KEY (a, b) \n" + + ") with (\n" + + " 'x' = 'y', \n" + + " 'asd' = 'data'\n" + + ")\n"; + final String expect0 = + "(?s).*Encountered \"\\(\" at line 4, column 14.\n" + + "Was expecting one of:\n" + + " \"AS\" ...\n" + + " \".\" ...\n" + + " \"STRING\" ...\n" + + ".*"; + sql(sql0).fails(expect0); + // Sub-query computed column expression is forbidden. + final String sql1 = + "CREATE TABLE t1 (\n" + + " a bigint, \n" + + " b varchar,\n" + + " c as ^(^select max(d) from t2), \n" + + " PRIMARY KEY (a, b) \n" + + ") with (\n" + + " 'x' = 'y', \n" + + " 'asd' = 'data'\n" + + ")\n"; + final String expect1 = "(?s).*Query expression encountered in illegal context.*"; + sql(sql1).fails(expect1); + } + + @Test + void testColumnSqlString() { + final String sql = + "CREATE TABLE sls_stream (\n" + + " a bigint, \n" + + " f as a + 1, \n" + + " b varchar,\n" + + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" + + " proc as PROCTIME(),\n" + + " c int,\n" + + " PRIMARY KEY (a, b) \n" + + ") with (\n" + + " 'x' = 'y', \n" + + " 'asd' = 'data'\n" + + ")\n"; + final String expected = + "`A`, (`A` + 1) AS `F`, `B`, " + + "`TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss') AS `TS`, " + + "`PROCTIME`() AS `PROC`, `C`"; + sql(sql).node(new ValidationMatcher().expectColumnSql(expected)); + } + + @Test + void testCreateTableWithMinusInOptionKey() { + final String sql = + "create table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c string\n" + + ") with (\n" + + " 'a-b-c-d124' = 'ab',\n" + + " 'a.b.1.c' = 'aabb',\n" + + " 'a.b-c-connector.e-f.g' = 'ada',\n" + + " 'a.b-c-d.e-1231.g' = 'ada',\n" + + " 'a.b-c-d.*' = 'adad')\n"; + final String expected = + "CREATE TABLE `SOURCE_TABLE` (\n" + + " `A` INTEGER,\n" + + " `B` BIGINT,\n" + + " `C` STRING\n" + + ")\n" + + "WITH (\n" + + " 'a-b-c-d124' = 'ab',\n" + + " 'a.b.1.c' = 'aabb',\n" + + " 'a.b-c-connector.e-f.g' = 'ada',\n" + + " 'a.b-c-d.e-1231.g' = 'ada',\n" + + " 'a.b-c-d.*' = 'adad'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithOptionKeyAsIdentifier() { + final String sql = + "create table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c string\n" + + ") with (\n" + + " ^a^.b.c = 'ab',\n" + + " a.b.c1 = 'aabb')\n"; + sql(sql).fails("(?s).*Encountered \"a\" at line 6, column 3.\n.*"); + } + + @Test + void testCreateTableLikeWithoutOption() { + final String sql = + "create table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c string\n" + + ")\n" + + "LIKE parent_table"; + final String expected = + "CREATE TABLE `SOURCE_TABLE` (\n" + + " `A` INTEGER,\n" + + " `B` BIGINT,\n" + + " `C` STRING\n" + + ")\n" + + "LIKE `PARENT_TABLE`"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableLikeWithConstraints() { + final String sql1 = + "create table source_table(\n" + + " a int primary key,\n" + + " b bigint,\n" + + " c string\n" + + ")\n" + + "LIKE parent_table"; + sql(sql1) + .node( + new ValidationMatcher() + .fails( + "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. ENFORCED/NOT ENFORCED " + + "controls if the constraint checks are performed on the incoming/outgoing data. " + + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); + + final String sql2 = + "create table source_table(\n" + + " a int primary key,\n" + + " b bigint,\n" + + " c string,\n" + + " primary key(a) not enforced\n" + + ")\n" + + "LIKE parent_table"; + + sql(sql2).node(new ValidationMatcher().fails("Duplicate primary key definition")); + + final String sql3 = + "create table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c string,\n" + + " unique (a)\n" + + ")\n" + + "LIKE parent_table"; + + sql(sql3).node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); + } + + @Test + void testCreateTableWithLikeClause() { + final String sql = + "create table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c string\n" + + ")\n" + + "LIKE parent_table (\n" + + " INCLUDING ALL\n" + + " OVERWRITING OPTIONS\n" + + " EXCLUDING PARTITIONS\n" + + " INCLUDING GENERATED\n" + + " INCLUDING METADATA\n" + + ")"; + final String expected = + "CREATE TABLE `SOURCE_TABLE` (\n" + + " `A` INTEGER,\n" + + " `B` BIGINT,\n" + + " `C` STRING\n" + + ")\n" + + "LIKE `PARENT_TABLE` (\n" + + " INCLUDING ALL\n" + + " OVERWRITING OPTIONS\n" + + " EXCLUDING PARTITIONS\n" + + " INCLUDING GENERATED\n" + + " INCLUDING METADATA\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithLikeClauseIncludingDistribution() { + final String sql = + "create table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c string\n" + + ")\n" + + "LIKE parent_table (\n" + + " INCLUDING ALL\n" + + " OVERWRITING OPTIONS\n" + + " INCLUDING DISTRIBUTION\n" + + " EXCLUDING PARTITIONS\n" + + " INCLUDING GENERATED\n" + + " INCLUDING METADATA\n" + + ")"; + final String expected = + "CREATE TABLE `SOURCE_TABLE` (\n" + + " `A` INTEGER,\n" + + " `B` BIGINT,\n" + + " `C` STRING\n" + + ")\n" + + "LIKE `PARENT_TABLE` (\n" + + " INCLUDING ALL\n" + + " OVERWRITING OPTIONS\n" + + " INCLUDING DISTRIBUTION\n" + + " EXCLUDING PARTITIONS\n" + + " INCLUDING GENERATED\n" + + " INCLUDING METADATA\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithLikeClauseExcludingDistribution() { + final String sql = + "create table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c string\n" + + ")\n" + + "LIKE parent_table (\n" + + " INCLUDING ALL\n" + + " OVERWRITING OPTIONS\n" + + " EXCLUDING DISTRIBUTION\n" + + " INCLUDING PARTITIONS\n" + + " INCLUDING GENERATED\n" + + " INCLUDING METADATA\n" + + ")"; + final String expected = + "CREATE TABLE `SOURCE_TABLE` (\n" + + " `A` INTEGER,\n" + + " `B` BIGINT,\n" + + " `C` STRING\n" + + ")\n" + + "LIKE `PARENT_TABLE` (\n" + + " INCLUDING ALL\n" + + " OVERWRITING OPTIONS\n" + + " EXCLUDING DISTRIBUTION\n" + + " INCLUDING PARTITIONS\n" + + " INCLUDING GENERATED\n" + + " INCLUDING METADATA\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithLikeClauseWithoutColumns() { + final String sql = + "" + + "create TEMPORARY table source_table (\n" + + " WATERMARK FOR ts AS ts - INTERVAL '5' SECOND\n" + + ") with (\n" + + " 'scan.startup.mode' = 'specific-offsets',\n" + + " 'scan.startup.specific-offsets' = 'partition:0,offset:1169129'\n" + + ") like t_order_course (\n" + + " OVERWRITING WATERMARKS\n" + + " OVERWRITING OPTIONS\n" + + " EXCLUDING CONSTRAINTS\n" + + ")"; + final String expected = + "CREATE TEMPORARY TABLE `SOURCE_TABLE` (\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '5' SECOND)\n" + + ")\n" + + "WITH (\n" + + " 'scan.startup.mode' = 'specific-offsets',\n" + + " 'scan.startup.specific-offsets' = 'partition:0,offset:1169129'\n" + + ")\n" + + "LIKE `T_ORDER_COURSE` (\n" + + " OVERWRITING WATERMARKS\n" + + " OVERWRITING OPTIONS\n" + + " EXCLUDING CONSTRAINTS\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTemporaryTable() { + final String sql = + "create temporary table source_table(\n" + + " a int,\n" + + " b bigint,\n" + + " c string\n" + + ") with (\n" + + " 'x' = 'y',\n" + + " 'abc' = 'def'\n" + + ")"; + final String expected = + "CREATE TEMPORARY TABLE `SOURCE_TABLE` (\n" + + " `A` INTEGER,\n" + + " `B` BIGINT,\n" + + " `C` STRING\n" + + ")\n" + + "WITH (\n" + + " 'x' = 'y',\n" + + " 'abc' = 'def'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithNoColumns() { + final String sql = + "create table source_table with (\n" + " 'x' = 'y',\n" + " 'abc' = 'def'\n" + ")"; + final String expected = + "CREATE TABLE `SOURCE_TABLE`\nWITH (\n" + + " 'x' = 'y',\n" + + " 'abc' = 'def'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableWithOnlyWaterMark() { + final String sql = + "create table source_table (\n" + + " watermark FOR ts AS ts - interval '3' second\n" + + ") with (\n" + + " 'x' = 'y',\n" + + " 'abc' = 'def'\n" + + ")"; + final String expected = + "CREATE TABLE `SOURCE_TABLE` (\n" + + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '3' SECOND)\n" + + ")\n" + + "WITH (\n" + + " 'x' = 'y',\n" + + " 'abc' = 'def'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testDropTable() { + final String sql = "DROP table catalog1.db1.tbl1"; + final String expected = "DROP TABLE `CATALOG1`.`DB1`.`TBL1`"; + sql(sql).ok(expected); + } + + @Test + void testDropIfExists() { + final String sql = "DROP table IF EXISTS catalog1.db1.tbl1"; + final String expected = "DROP TABLE IF EXISTS `CATALOG1`.`DB1`.`TBL1`"; + sql(sql).ok(expected); + } + + @Test + void testTemporaryDropTable() { + final String sql = "DROP temporary table catalog1.db1.tbl1"; + final String expected = "DROP TEMPORARY TABLE `CATALOG1`.`DB1`.`TBL1`"; + sql(sql).ok(expected); + } + + @Test + void testDropTemporaryIfExists() { + final String sql = "DROP temporary table IF EXISTS catalog1.db1.tbl1"; + final String expected = "DROP TEMPORARY TABLE IF EXISTS `CATALOG1`.`DB1`.`TBL1`"; + sql(sql).ok(expected); + } + + @Test + void testInsertPartitionSpecs() { + final String sql1 = "insert into emps partition (x='ab', y='bc') (x,y) select * from emps"; + final String expected = + "INSERT INTO `EMPS` " + + "PARTITION (`X` = 'ab', `Y` = 'bc')\n" + + "(`X`, `Y`)\n" + + "SELECT *\n" + + "FROM `EMPS`"; + sql(sql1).ok(expected); + final String sql2 = + "insert into emp\n" + + "partition(empno='1', job='job')\n" + + "(empno, ename, job, mgr, hiredate,\n" + + " sal, comm, deptno, slacker)\n" + + "select 'nom', 0, timestamp '1970-01-01 00:00:00',\n" + + " 1, 1, 1, false\n" + + "from (values 'a')"; + sql(sql2) + .ok( + "INSERT INTO `EMP` " + + "PARTITION (`EMPNO` = '1', `JOB` = 'job')\n" + + "(`EMPNO`, `ENAME`, `JOB`, `MGR`, `HIREDATE`, `SAL`," + + " `COMM`, `DEPTNO`, `SLACKER`)\n" + + "SELECT 'nom', 0, TIMESTAMP '1970-01-01 00:00:00', 1, 1, 1, FALSE\n" + + "FROM (VALUES (ROW('a')))"); + final String sql3 = + "insert into empnullables\n" + + "partition(ename='b')\n" + + "(empno, ename)\n" + + "select 1 from (values 'a')"; + sql(sql3) + .ok( + "INSERT INTO `EMPNULLABLES` " + + "PARTITION (`ENAME` = 'b')\n" + + "(`EMPNO`, `ENAME`)\n" + + "SELECT 1\n" + + "FROM (VALUES (ROW('a')))"); + } + + @Test + void testInsertCaseSensitivePartitionSpecs() { + final String expected = + "INSERT INTO `emps` " + + "PARTITION (`x` = 'ab', `y` = 'bc')\n" + + "(`x`, `y`)\n" + + "SELECT *\n" + + "FROM `EMPS`"; + sql("insert into \"emps\" " + + "partition (\"x\"='ab', \"y\"='bc')(\"x\",\"y\") select * from emps") + .ok(expected); + } + + @Test + void testInsertExtendedColumnAsStaticPartition1() { + final String expected = + "INSERT INTO `EMPS` EXTEND (`Z` BOOLEAN) " + + "PARTITION (`Z` = 'ab')\n" + + "(`X`, `Y`)\n" + + "SELECT *\n" + + "FROM `EMPS`"; + sql("insert into emps(z boolean) partition (z='ab') (x,y) select * from emps").ok(expected); + } + + @Test + void testInsertExtendedColumnAsStaticPartition2() { + assertThatThrownBy( + () -> + sql("insert into emps(x, y, z boolean) partition (z='ab') select * from emps") + .node( + new ValidationMatcher() + .fails( + "Extended columns not allowed under the current SQL conformance level"))) + .isInstanceOf(SqlParseException.class); + } + + @Test + void testInsertOverwrite() { + // non-partitioned + final String sql = "INSERT OVERWRITE myDB.myTbl SELECT * FROM src"; + final String expected = "INSERT OVERWRITE `MYDB`.`MYTBL`\n" + "SELECT *\n" + "FROM `SRC`"; + sql(sql).ok(expected); + + // partitioned + final String sql1 = "INSERT OVERWRITE myTbl PARTITION (p1='v1',p2='v2') SELECT * FROM src"; + final String expected1 = + "INSERT OVERWRITE `MYTBL` " + + "PARTITION (`P1` = 'v1', `P2` = 'v2')\n" + + "\n" + + "SELECT *\n" + + "FROM `SRC`"; + sql(sql1).ok(expected1); + } + + @Test + void testInvalidUpsertOverwrite() { + sql("UPSERT ^OVERWRITE^ myDB.myTbl SELECT * FROM src") + .fails("OVERWRITE expression is only used with INSERT statement."); + } + + @Test + void testInsertOnConflict() { + // ON CONFLICT DO ERROR + sql("INSERT INTO t1 SELECT * FROM t2 ON CONFLICT DO ERROR") + .ok("INSERT INTO `T1`\nSELECT *\nFROM `T2`\nON CONFLICT DO ERROR"); + + // ON CONFLICT DO NOTHING + sql("INSERT INTO t1 SELECT * FROM t2 ON CONFLICT DO NOTHING") + .ok("INSERT INTO `T1`\nSELECT *\nFROM `T2`\nON CONFLICT DO NOTHING"); + + // ON CONFLICT DO DEDUPLICATE + sql("INSERT INTO t1 SELECT * FROM t2 ON CONFLICT DO DEDUPLICATE") + .ok("INSERT INTO `T1`\nSELECT *\nFROM `T2`\nON CONFLICT DO DEDUPLICATE"); + + // ON CONFLICT with partition + sql("INSERT INTO t1 PARTITION (p='v') SELECT * FROM t2 ON CONFLICT DO ERROR") + .ok( + "INSERT INTO `T1` PARTITION (`P` = 'v')\n\nSELECT *\nFROM `T2`\nON CONFLICT DO ERROR"); + + // ON CONFLICT with INSERT OVERWRITE (should work) + sql("INSERT OVERWRITE t1 SELECT * FROM t2 ON CONFLICT DO NOTHING") + .ok("INSERT OVERWRITE `T1`\nSELECT *\nFROM `T2`\nON CONFLICT DO NOTHING"); + + // Invalid ON CONFLICT strategy + sql("INSERT INTO t1 SELECT * FROM t2 ON CONFLICT DO ^UPDATE^") + .fails("(?s).*Encountered \"UPDATE\" at line 1, column 48.\n.*"); + } + + private String buildDistributionInput(final String distributionClause) { + return "CREATE TABLE tbl1 (\n" + + " a bigint,\n" + + " h varchar, \n" + + " b varchar,\n" + + " PRIMARY KEY (a, b)\n" + + ")\n" + + distributionClause + + " with (\n" + + " 'connector' = 'kafka', \n" + + " 'kafka.topic' = 'log.test'\n" + + ")\n"; + } + + @Test + void testCreateTableUsingConnection() { + final String sql = + "CREATE TABLE orders (\n" + + " order_id INT,\n" + + " customer_id INT,\n" + + " amount DECIMAL(10, 2)\n" + + ") USING CONNECTION mycat.mydb.mysql_prod\n" + + "WITH (\n" + + " 'connector' = 'jdbc',\n" + + " 'tables' = 'orders'\n" + + ")"; + final String expected = + "CREATE TABLE `ORDERS` (\n" + + " `ORDER_ID` INTEGER,\n" + + " `CUSTOMER_ID` INTEGER,\n" + + " `AMOUNT` DECIMAL(10, 2)\n" + + ")\n" + + "USING CONNECTION `MYCAT`.`MYDB`.`MYSQL_PROD`\n" + + "WITH (\n" + + " 'connector' = 'jdbc',\n" + + " 'tables' = 'orders'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableUsingConnectionWithPartitionAndDistribution() { + final String sql = + "CREATE TABLE tbl1 (\n" + + " a bigint,\n" + + " h varchar,\n" + + " b varchar\n" + + ")\n" + + "DISTRIBUTED BY HASH(a) INTO 3 BUCKETS\n" + + "PARTITIONED BY (a, h)\n" + + "USING CONNECTION cat1.db1.conn1\n" + + "WITH (\n" + + " 'connector' = 'jdbc'\n" + + ")"; + final String expected = + "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " `B` VARCHAR\n" + + ")\n" + + "DISTRIBUTED BY HASH(`A`) INTO 3 BUCKETS\n" + + "PARTITIONED BY (`A`, `H`)\n" + + "USING CONNECTION `CAT1`.`DB1`.`CONN1`\n" + + "WITH (\n" + + " 'connector' = 'jdbc'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testCreateTableLikeUsingConnection() { + final String sql = + "CREATE TABLE t1 (\n" + + " a INT\n" + + ")\n" + + "USING CONNECTION cat1.db1.conn1\n" + + "WITH ('connector' = 'jdbc')\n" + + "LIKE base_table"; + final String expected = + "CREATE TABLE `T1` (\n" + + " `A` INTEGER\n" + + ")\n" + + "USING CONNECTION `CAT1`.`DB1`.`CONN1`\n" + + "WITH (\n" + + " 'connector' = 'jdbc'\n" + + ")\n" + + "LIKE `BASE_TABLE`"; + sql(sql).ok(expected); + } + + private String buildDistributionOutput(final String distributionClause) { + return "CREATE TABLE `TBL1` (\n" + + " `A` BIGINT,\n" + + " `H` VARCHAR,\n" + + " `B` VARCHAR,\n" + + " PRIMARY KEY (`A`, `B`)\n" + + ")\n" + + distributionClause + + "WITH (\n" + + " 'connector' = 'kafka',\n" + + " 'kafka.topic' = 'log.test'\n" + + ")"; + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCtasTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCtasTest.java new file mode 100644 index 0000000000000..496bc3a3aad3a --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserCtasTest.java @@ -0,0 +1,316 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** CREATE TABLE AS SELECT and job parser tests. */ +@Execution(CONCURRENT) +class FlinkSqlParserCtasTest extends FlinkSqlParserTestBase { + + @Test + void testExplainCreateTableAsSelect() { + this.sql("EXPLAIN CREATE TABLE t AS SELECT * FROM b") + .ok("EXPLAIN CREATE TABLE `T`\nAS\nSELECT *\nFROM `B`"); + } + + @Test + void testExplainCreateOrReplaceTableAsSelect() { + this.sql("EXPLAIN CREATE OR REPLACE TABLE t AS SELECT * FROM b") + .ok("EXPLAIN CREATE OR REPLACE TABLE `T`\nAS\nSELECT *\nFROM `B`"); + } + + @Test + void testExplainReplaceTableAsSelect() { + this.sql("EXPLAIN REPLACE TABLE t AS SELECT * FROM b") + .ok("EXPLAIN REPLACE TABLE `T`\nAS\nSELECT *\nFROM `B`"); + } + + @Test + void testCreateTableAsSelectWithoutOptions() { + sql("CREATE TABLE t AS SELECT * FROM b").ok("CREATE TABLE `T`\nAS\nSELECT *\nFROM `B`"); + } + + @Test + void testCreateTableAsSelectWithOptions() { + sql("CREATE TABLE t WITH ('test' = 'zm') AS SELECT * FROM b") + .ok("CREATE TABLE `T`\nWITH (\n 'test' = 'zm'\n)\nAS\nSELECT *\nFROM `B`"); + } + + @Test + void testCreateTableAsSelectWithCreateTableLike() { + sql("CREATE TABLE t (col1 string) WITH ('test' = 'zm') like b ^AS^ SELECT col1 FROM b") + .fails("(?s).*Encountered \"AS\" at line 1, column 58.*"); + } + + @Test + void testCreateTableAsSelectWithTmpTable() { + sql("CREATE TEMPORARY TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node( + new ValidationMatcher() + .fails( + "CREATE TABLE AS SELECT syntax does not support to create temporary table yet.")); + } + + @Test + void testCreateTableAsSelectWithExplicitColumns() { + sql("CREATE TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + } + + @Test + void testCreateTableAsSelectWithWatermark() { + sql("CREATE TABLE t (watermark FOR col1 AS col1 - interval '3' second) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + } + + @Test + void testCreateTableAsSelectWithConstraints() { + sql("CREATE TABLE t (PRIMARY KEY (col1) NOT ENFORCED) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + + sql("CREATE TABLE t (PRIMARY KEY (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node( + new ValidationMatcher() + .fails( + "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. ENFORCED/NOT ENFORCED controls " + + "if the constraint checks are performed on the incoming/outgoing data. " + + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); + + sql("CREATE TABLE t (PRIMARY KEY (col1), PRIMARY KEY (col2) NOT ENFORCED) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().fails("Duplicate primary key definition")); + + sql("CREATE TABLE t (UNIQUE (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); + } + + @Test + void testCreateTableAsSelectWithDistribution() { + sql("CREATE TABLE t DISTRIBUTED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + } + + @Test + void testCreateTableAsSelectWithPartitionKey() { + sql("CREATE TABLE t PARTITIONED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + } + + @Test + void testCreateTableAsSelectWithColumnIdentifiers() { + // test with only column identifiers + sql("CREATE TABLE t (col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + + // test mix of column identifiers and column with types is not allowed + sql("CREATE TABLE t (col1, col2 ^int^) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .fails("(?s).*Encountered \"int\" at line 1, column 28.*"); + } + + @Test + void testUnsupportedCreateTableStatementsWithColumnIdentifiers() { + String expectedErrorMsg = + "Columns identifiers without types in the schema are " + + "supported on CTAS/RTAS statements only."; + + sql("CREATE TABLE t ^(a, h^) WITH " + "('connector' = 'kafka', 'kafka.topic' = 'log.test')") + .fails(expectedErrorMsg); + + sql("CREATE TABLE t ^(a, h^) WITH " + + "('connector' = 'kafka', 'kafka.topic' = 'log.test') " + + "LIKE parent_table") + .fails(expectedErrorMsg); + } + + @Test + void testReplaceTableAsSelectWithColumnIdentifiers() { + // test with only column identifiers + sql("REPLACE TABLE t (col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + + // test mix of column identifiers and column with types is not allowed + sql("REPLACE TABLE t (col1, col2 ^int^) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .fails("(?s).*Encountered \"int\" at line 1, column 29.*"); + } + + @Test + void testReplaceTableAsSelect() { + // test replace table as select without options + sql("REPLACE TABLE t AS SELECT * FROM b").ok("REPLACE TABLE `T`\nAS\nSELECT *\nFROM `B`"); + + // test replace table as select with options + sql("REPLACE TABLE t WITH ('test' = 'zm') AS SELECT * FROM b") + .ok("REPLACE TABLE `T`\nWITH (\n 'test' = 'zm'\n)\nAS\nSELECT *\nFROM `B`"); + + // test replace table as select with tmp table + sql("REPLACE TEMPORARY TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node( + new ValidationMatcher() + .fails( + "REPLACE TABLE AS SELECT syntax does not support temporary table yet.")); + + // test replace table as select with explicit columns + sql("REPLACE TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + + // test replace table as select with watermark + sql("REPLACE TABLE t (watermark FOR ts AS ts - interval '3' second) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + + // test replace table as select with constraints + sql("REPLACE TABLE t (PRIMARY KEY (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node( + new ValidationMatcher() + .fails( + "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. ENFORCED/NOT ENFORCED controls " + + "if the constraint checks are performed on the incoming/outgoing data. " + + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); + + sql("REPLACE TABLE t (PRIMARY KEY (col1), PRIMARY KEY (col2) NOT ENFORCED) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().fails("Duplicate primary key definition")); + + sql("REPLACE TABLE t (UNIQUE (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); + + // test replace table as select with partition key + sql("REPLACE TABLE t PARTITIONED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + + // test replace table as select with distribution + sql("REPLACE TABLE t DISTRIBUTED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + } + + @Test + void testCreateOrReplaceTableAsSelect() { + // test create or replace table as select without options + sql("CREATE OR REPLACE TABLE t AS SELECT * FROM b") + .ok("CREATE OR REPLACE TABLE `T`\nAS\nSELECT *\nFROM `B`"); + + // test create or replace table as select with options + sql("CREATE OR REPLACE TABLE t WITH ('test' = 'zm') AS SELECT * FROM b") + .ok( + "CREATE OR REPLACE TABLE `T`\nWITH (\n 'test' = 'zm'\n)\nAS\nSELECT *\nFROM `B`"); + + // test create or replace table as select with create table like + sql("CREATE OR REPLACE TABLE t (col1 string) WITH ('test' = 'zm') like b ^AS^ SELECT col1 FROM b") + .fails("(?s).*Encountered \"AS\" at line 1, column 69.*"); + + // test create or replace table as select with tmp table + sql("CREATE OR REPLACE TEMPORARY TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node( + new ValidationMatcher() + .fails( + "CREATE OR REPLACE TABLE AS SELECT syntax does not support temporary table yet.")); + + // test create or replace table as select with explicit columns + sql("CREATE OR REPLACE TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + + // test create or replace table as select with watermark + sql("CREATE OR REPLACE TABLE t (watermark FOR ts AS ts - interval '3' second) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + + // test create or replace table as select with constraints + sql("CREATE OR REPLACE TABLE t (PRIMARY KEY (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node( + new ValidationMatcher() + .fails( + "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. ENFORCED/NOT ENFORCED controls " + + "if the constraint checks are performed on the incoming/outgoing data. " + + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); + + sql("CREATE OR REPLACE TABLE t (PRIMARY KEY (col1), PRIMARY KEY (col2) NOT ENFORCED) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().fails("Duplicate primary key definition")); + + sql("CREATE OR REPLACE TABLE t (UNIQUE (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); + + // test create or replace table as select with partition key + sql("CREATE OR REPLACE TABLE t PARTITIONED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + + sql("CREATE OR REPLACE TABLE t DISTRIBUTED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") + .node(new ValidationMatcher().ok()); + } + + @Test + void testShowJobs() { + sql("show jobs").ok("SHOW JOBS"); + } + + @Test + void testStopJob() { + sql("STOP JOB 'myjob'").same(); + sql("STOP JOB 'myjob' WITH SAVEPOINT").same(); + sql("STOP JOB 'myjob' WITH SAVEPOINT WITH DRAIN").same(); + sql("STOP JOB 'myjob' ^WITH DRAIN^") + .fails("WITH DRAIN could only be used after WITH SAVEPOINT."); + sql("STOP JOB 'myjob' ^WITH DRAIN^ WITH SAVEPOINT") + .fails("WITH DRAIN could only be used after WITH SAVEPOINT."); + } + + @Test + void testDescribeJob() { + sql("DESCRIBE JOB 'myjob'").same(); + sql("DESC JOB 'myjob'").ok("DESCRIBE JOB 'myjob'"); + } + + @Test + void testTruncateTable() { + sql("truncate table t1").ok("TRUNCATE TABLE `T1`"); + } + + @Test + void testCreateTableAsWithUsingConnectionFails() { + final String sql = + "^CREATE^ TABLE t1\n" + + "USING CONNECTION cat1.db1.conn1\n" + + "WITH ('connector' = 'jdbc')\n" + + "AS SELECT 1 AS a"; + sql(sql).fails( + "(?s).*USING CONNECTION clause is not supported with " + + "CREATE TABLE AS SELECT or REPLACE TABLE AS SELECT statements\\..*"); + } + + @Test + void testReplaceTableAsWithUsingConnectionFails() { + final String sql = + "REPLACE TABLE t1\n" + + "^USING^ CONNECTION cat1.db1.conn1\n" + + "WITH ('connector' = 'jdbc')\n" + + "AS SELECT 1 AS a"; + sql(sql).fails("(?s).*Encountered \"USING\" at line 2, column 1.\n.*"); + } + + @Test + void testCreateOrReplaceTableAsWithUsingConnectionFails() { + final String sql = + "^CREATE^ OR REPLACE TABLE t1\n" + + "USING CONNECTION cat1.db1.conn1\n" + + "WITH ('connector' = 'jdbc')\n" + + "AS SELECT 1 AS a"; + sql(sql).fails( + "(?s).*USING CONNECTION clause is not supported with " + + "CREATE TABLE AS SELECT or REPLACE TABLE AS SELECT statements\\..*"); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserExecuteTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserExecuteTest.java new file mode 100644 index 0000000000000..7691b3b0167a6 --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserExecuteTest.java @@ -0,0 +1,429 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** EXECUTE, EXPLAIN and statement set parser tests. */ +@Execution(CONCURRENT) +class FlinkSqlParserExecuteTest extends FlinkSqlParserTestBase { + + @Test + void testBeginStatementSet() { + sql("begin statement set").ok("BEGIN STATEMENT SET"); + } + + @Test + void testEnd() { + sql("end").ok("END"); + } + + @Test + void testExecuteStatementSet() { + sql("execute statement set begin insert into t1 select * from t2; insert into t2 select * from t3; end") + .ok( + "EXECUTE STATEMENT SET BEGIN\n" + + "INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`\n" + + ";\n" + + "INSERT INTO `T2`\n" + + "SELECT *\n" + + "FROM `T3`\n" + + ";\n" + + "END"); + } + + @Test + void testExplainStatementSet() { + sql("explain statement set begin insert into t1 select * from t2; insert into t2 select * from t3; end") + .ok( + "EXPLAIN STATEMENT SET BEGIN\n" + + "INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`\n" + + ";\n" + + "INSERT INTO `T2`\n" + + "SELECT *\n" + + "FROM `T3`\n" + + ";\n" + + "END"); + } + + @Test + void testExplainExecuteStatementSet() { + sql("explain execute statement set begin insert into t1 select * from t2; insert into t2 select * from t3; end") + .ok( + "EXPLAIN EXECUTE STATEMENT SET BEGIN\n" + + "INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`\n" + + ";\n" + + "INSERT INTO `T2`\n" + + "SELECT *\n" + + "FROM `T3`\n" + + ";\n" + + "END"); + } + + @Test + void testExecuteStatementSetWithOnConflict() { + sql("execute statement set begin " + + "insert into t1 select * from t2 on conflict do deduplicate; " + + "insert into t3 select * from t4 on conflict do nothing; " + + "end") + .ok( + "EXECUTE STATEMENT SET BEGIN\n" + + "INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`\n" + + "ON CONFLICT DO DEDUPLICATE\n" + + ";\n" + + "INSERT INTO `T3`\n" + + "SELECT *\n" + + "FROM `T4`\n" + + "ON CONFLICT DO NOTHING\n" + + ";\n" + + "END"); + } + + @Test + void testExplainExecuteSelect() { + sql("explain execute select * from emps").ok("EXPLAIN EXECUTE SELECT *\nFROM `EMPS`"); + } + + @Test + void testExplainExecuteInsert() { + sql("explain execute insert into emps1 select * from emps2") + .ok("EXPLAIN EXECUTE INSERT INTO `EMPS1`\nSELECT *\nFROM `EMPS2`"); + } + + @Test + void testExplain() { + String sql = "explain select * from emps"; + String expected = "EXPLAIN SELECT *\nFROM `EMPS`"; + this.sql(sql).ok(expected); + } + + @Test + void testExecuteSelect() { + String sql = "execute select * from emps"; + String expected = "EXECUTE SELECT *\nFROM `EMPS`"; + this.sql(sql).ok(expected); + } + + @Test + void testExplainPlanFor() { + String sql = "explain plan for select * from emps"; + String expected = "EXPLAIN SELECT *\nFROM `EMPS`"; + this.sql(sql).ok(expected); + } + + @Test + void testExplainChangelogMode() { + String sql = "explain changelog_mode select * from emps"; + String expected = "EXPLAIN CHANGELOG_MODE SELECT *\nFROM `EMPS`"; + this.sql(sql).ok(expected); + } + + @Test + void testExplainEstimatedCost() { + String sql = "explain estimated_cost select * from emps"; + String expected = "EXPLAIN ESTIMATED_COST SELECT *\nFROM `EMPS`"; + this.sql(sql).ok(expected); + } + + @Test + void testExplainUnion() { + String sql = "explain estimated_cost select * from emps union all select * from emps"; + String expected = + "EXPLAIN ESTIMATED_COST SELECT *\n" + + "FROM `EMPS`\n" + + "UNION ALL\n" + + "SELECT *\n" + + "FROM `EMPS`"; + this.sql(sql).ok(expected); + } + + @Test + void testExplainJsonFormat() { + // Unsupported feature. Escape the test. + } + + @Test + void testExplainWithImpl() { + // Unsupported feature. Escape the test. + } + + @Test + void testExplainWithoutImpl() { + // Unsupported feature. Escape the test. + } + + @Test + void testExplainWithType() { + // Unsupported feature. Escape the test. + } + + @Test + void testExplainAsXml() { + // Unsupported feature. Escape the test. + } + + @Test + void testSqlOptions() { + // SET/RESET are overridden for Flink SQL + } + + @Test + void testExplainAsJson() { + String sql = "explain json_execution_plan select * from emps"; + String expected = "EXPLAIN JSON_EXECUTION_PLAN SELECT *\n" + "FROM `EMPS`"; + this.sql(sql).ok(expected); + } + + @Test + void testExplainPlanAdvice() { + String sql = "explain plan_advice select * from emps"; + String expected = "EXPLAIN PLAN_ADVICE SELECT *\nFROM `EMPS`"; + this.sql(sql).ok(expected); + } + + @Test + void testExplainAllDetails() { + String sql = + "explain changelog_mode,json_execution_plan,estimated_cost,plan_advice select * from emps"; + String expected = + "EXPLAIN JSON_EXECUTION_PLAN, CHANGELOG_MODE, PLAN_ADVICE, ESTIMATED_COST SELECT *\n" + + "FROM `EMPS`"; + this.sql(sql).ok(expected); + } + + @Test + void testExplainInsert() { + String expected = "EXPLAIN INSERT INTO `EMPS1`\n" + "SELECT *\n" + "FROM `EMPS2`"; + this.sql("explain plan for insert into emps1 select * from emps2").ok(expected); + } + + @Test + void testExecuteInsert() { + String expected = "EXECUTE INSERT INTO `EMPS1`\n" + "SELECT *\n" + "FROM `EMPS2`"; + this.sql("execute insert into emps1 select * from emps2").ok(expected); + } + + @Test + void testExecutePlan() { + sql("execute plan './test.json'").ok("EXECUTE PLAN './test.json'"); + sql("execute plan '/some/absolute/dir/plan.json'") + .ok("EXECUTE PLAN '/some/absolute/dir/plan.json'"); + sql("execute plan 'file:///foo/bar/test.json'") + .ok("EXECUTE PLAN 'file:///foo/bar/test.json'"); + } + + @Test + void testCompilePlan() { + sql("compile plan './test.json' for insert into t1 select * from t2") + .ok( + "COMPILE PLAN './test.json' FOR INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`"); + sql("compile plan './test.json' if not exists for insert into t1 select * from t2") + .ok( + "COMPILE PLAN './test.json' IF NOT EXISTS FOR INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`"); + sql("compile plan 'file:///foo/bar/test.json' if not exists for insert into t1 select * from t2") + .ok( + "COMPILE PLAN 'file:///foo/bar/test.json' IF NOT EXISTS FOR INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`"); + + sql("compile plan './test.json' for statement set " + + "begin insert into t1 select * from t2; insert into t2 select * from t3; end") + .ok( + "COMPILE PLAN './test.json' FOR STATEMENT SET BEGIN\n" + + "INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`\n" + + ";\n" + + "INSERT INTO `T2`\n" + + "SELECT *\n" + + "FROM `T3`\n" + + ";\n" + + "END"); + sql("compile plan './test.json' if not exists for statement set " + + "begin insert into t1 select * from t2; insert into t2 select * from t3; end") + .ok( + "COMPILE PLAN './test.json' IF NOT EXISTS FOR STATEMENT SET BEGIN\n" + + "INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`\n" + + ";\n" + + "INSERT INTO `T2`\n" + + "SELECT *\n" + + "FROM `T3`\n" + + ";\n" + + "END"); + + sql("compile plan 'file:///foo/bar/test.json' if not exists for statement set " + + "begin insert into t1 select * from t2; insert into t2 select * from t3; end") + .ok( + "COMPILE PLAN 'file:///foo/bar/test.json' IF NOT EXISTS FOR STATEMENT SET BEGIN\n" + + "INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`\n" + + ";\n" + + "INSERT INTO `T2`\n" + + "SELECT *\n" + + "FROM `T3`\n" + + ";\n" + + "END"); + } + + @Test + void testCompileAndExecutePlan() { + sql("compile and execute plan './test.json' for insert into t1 select * from t2") + .ok( + "COMPILE AND EXECUTE PLAN './test.json' FOR INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`"); + + sql("compile and execute plan './test.json' for statement set " + + "begin insert into t1 select * from t2; insert into t2 select * from t3; end") + .ok( + "COMPILE AND EXECUTE PLAN './test.json' FOR STATEMENT SET BEGIN\n" + + "INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`\n" + + ";\n" + + "INSERT INTO `T2`\n" + + "SELECT *\n" + + "FROM `T3`\n" + + ";\n" + + "END"); + sql("compile and execute plan 'file:///foo/bar/test.json' for insert into t1 select * from t2") + .ok( + "COMPILE AND EXECUTE PLAN 'file:///foo/bar/test.json' FOR INSERT INTO `T1`\n" + + "SELECT *\n" + + "FROM `T2`"); + } + + @Test + void testExplainUpsert() { + String sql = "explain plan for upsert into emps1 values (1, 2)"; + String expected = "EXPLAIN UPSERT INTO `EMPS1`\n" + "VALUES (ROW(1, 2))"; + this.sql(sql).ok(expected); + } + + @Test + void testExplainPlanForWithExplainDetails() { + String sql = "explain plan for ^json_execution_plan^ upsert into emps1 values (1, 2)"; + this.sql(sql).fails("Non-query expression encountered in illegal context"); + } + + @Test + void testExplainDuplicateExplainDetails() { + String sql = "explain changelog_mode,^changelog_mode^ select * from emps"; + this.sql(sql).fails("Duplicate EXPLAIN DETAIL is not allowed."); + } + + @Test + void testAddJar() { + sql("add Jar './test.sql'").ok("ADD JAR './test.sql'"); + sql("add JAR 'file:///path/to/\nwhatever'").ok("ADD JAR 'file:///path/to/\nwhatever'"); + sql("add JAR 'oss://path/helloworld.go'").ok("ADD JAR 'oss://path/helloworld.go'"); + } + + @Test + void testRemoveJar() { + sql("remove Jar './test.sql'").ok("REMOVE JAR './test.sql'"); + sql("remove JAR 'file:///path/to/\nwhatever'") + .ok("REMOVE JAR 'file:///path/to/\nwhatever'"); + sql("remove JAR 'oss://path/helloworld.go'").ok("REMOVE JAR 'oss://path/helloworld.go'"); + } + + @Test + void testShowJars() { + sql("show jars").ok("SHOW JARS"); + } + + @Test + void testSetReset() { + sql("SET").same(); + sql("SET 'test-key' = 'test-value'").same(); + sql("RESET").same(); + sql("RESET 'test-key'").same(); + } + + @Test + void testTryCast() { + // Simple types + expr("try_cast(a as timestamp)").ok("(TRY_CAST(`A` AS TIMESTAMP))"); + expr("try_cast('abc' as timestamp)").ok("(TRY_CAST('abc' AS TIMESTAMP))"); + + // Complex types + expr("try_cast(a as row(f0 int, f1 varchar))") + .ok("(TRY_CAST(`A` AS ROW(`F0` INTEGER, `F1` VARCHAR)))"); + expr("try_cast(a as row(f0 int array, f1 map, f2 STRING NOT NULL))") + .ok( + "(TRY_CAST(`A` AS ROW(`F0` INTEGER ARRAY, `F1` MAP< STRING, DECIMAL(10, 2) >, `F2` STRING NOT NULL)))"); + } + + @Test + void testAnalyzeTable() { + sql("analyze table emp^s^").fails("(?s).*Encountered \"\" at line 1, column 18.\n.*"); + sql("analyze table emps compute statistics").ok("ANALYZE TABLE `EMPS` COMPUTE STATISTICS"); + sql("analyze table emps partition ^compute^ statistics") + .fails("(?s).*Encountered \"compute\" at line 1, column 30.\n.*"); + sql("analyze table emps partition(^)^ compute statistics") + .fails("(?s).*Encountered \"\\)\" at line 1, column 30.\n.*"); + sql("analyze table emps partition(x='ab') compute statistics") + .ok("ANALYZE TABLE `EMPS` PARTITION (`X` = 'ab') COMPUTE STATISTICS"); + sql("analyze table emps partition(x='ab', y='bc') compute statistics") + .ok("ANALYZE TABLE `EMPS` PARTITION (`X` = 'ab', `Y` = 'bc') COMPUTE STATISTICS"); + sql("analyze table emps compute statistics for column^s^") + .fails("(?s).*Encountered \"\" at line 1, column 49.\n.*"); + sql("analyze table emps compute statistics for columns a") + .ok("ANALYZE TABLE `EMPS` COMPUTE STATISTICS FOR COLUMNS `A`"); + sql("analyze table emps compute statistics for columns a, b") + .ok("ANALYZE TABLE `EMPS` COMPUTE STATISTICS FOR COLUMNS `A`, `B`"); + sql("analyze table emps compute statistics for all columns") + .ok("ANALYZE TABLE `EMPS` COMPUTE STATISTICS FOR ALL COLUMNS"); + sql("analyze table emps partition(x, y) compute statistics for all columns") + .ok("ANALYZE TABLE `EMPS` PARTITION (`X`, `Y`) COMPUTE STATISTICS FOR ALL COLUMNS"); + sql("analyze table emps partition(x='ab', y) compute statistics for all columns") + .ok( + "ANALYZE TABLE `EMPS` PARTITION (`X` = 'ab', `Y`) COMPUTE STATISTICS FOR ALL COLUMNS"); + sql("analyze table emps partition(x, y='cd') compute statistics for all columns") + .ok( + "ANALYZE TABLE `EMPS` PARTITION (`X`, `Y` = 'cd') COMPUTE STATISTICS FOR ALL COLUMNS"); + sql("analyze table emps partition(x=^,^ y) compute statistics for all columns") + .fails("(?s).*Encountered \"\\,\" at line 1, column 32.\n.*"); + } + + @Test + void testExplainCreateTableNoSupported() { + this.sql("EXPLAIN CREATE TABLE t (id int^)^") + .fails( + "Unsupported CREATE OR REPLACE statement for EXPLAIN\\. The statement must define a query using the AS clause \\(i\\.e\\. CTAS/RTAS statements\\)\\."); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java deleted file mode 100644 index 1d1e7d639fa00..0000000000000 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java +++ /dev/null @@ -1,4046 +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; - -import org.apache.flink.sql.parser.error.SqlValidateException; -import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl; - -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.parser.SqlParseException; -import org.apache.calcite.sql.parser.SqlParserFixture; -import org.apache.calcite.sql.parser.SqlParserTest; -import org.apache.calcite.sql.validate.SqlConformanceEnum; -import org.hamcrest.BaseMatcher; -import org.hamcrest.Description; -import org.hamcrest.TypeSafeDiagnosingMatcher; -import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.parallel.Execution; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; - -import java.util.List; -import java.util.Locale; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; - -/** FlinkSqlParserImpl tests. * */ -@Execution(CONCURRENT) -class FlinkSqlParserImplTest extends SqlParserTest { - - public SqlParserFixture fixture() { - return super.fixture().withConfig(c -> c.withParserFactory(FlinkSqlParserImpl.FACTORY)); - } - - @Test - void testArrayFunction() {} - - @Test - void testArrayQueryConstructor() {} - - @Test - void testPercentileCont() {} - - @Test - void testPercentileContBigQuery() {} - - @Test - void testPercentileDisc() {} - - @Test - void testPercentileDiscBigQuery() {} - - @Test - void testMapQueryConstructor() {} - - @Test - void testMultisetQueryConstructor() {} - - @Test - void testShowCatalogs() { - sql("show catalogs").ok("SHOW CATALOGS"); - - sql("show catalogs like '%'").ok("SHOW CATALOGS LIKE '%'"); - sql("show catalogs not like '%'").ok("SHOW CATALOGS NOT LIKE '%'"); - - sql("show catalogs ilike '%'").ok("SHOW CATALOGS ILIKE '%'"); - sql("show catalogs not ilike '%'").ok("SHOW CATALOGS NOT ILIKE '%'"); - - sql("show catalogs ^likes^").fails("(?s).*Encountered \"likes\" at line 1, column 15.\n.*"); - sql("show catalogs not ^likes^") - .fails("(?s).*Encountered \"likes\" at line 1, column 19" + ".\n" + ".*"); - sql("show catalogs ^ilikes^") - .fails("(?s).*Encountered \"ilikes\" at line 1, column 15.\n.*"); - sql("show catalogs not ^ilikes^") - .fails("(?s).*Encountered \"ilikes\" at line 1, column 19" + ".\n" + ".*"); - } - - @Test - void testShowCurrentCatalog() { - sql("show current catalog").ok("SHOW CURRENT CATALOG"); - } - - @Test - void testDescribeCatalog() { - sql("describe catalog a").ok("DESCRIBE CATALOG `A`"); - sql("describe catalog extended a").ok("DESCRIBE CATALOG EXTENDED `A`"); - - sql("desc catalog a").ok("DESCRIBE CATALOG `A`"); - sql("desc catalog extended a").ok("DESCRIBE CATALOG EXTENDED `A`"); - } - - @Test - void testAlterCatalog() { - sql("alter catalog a set ('k1'='v1', 'k2'='v2')") - .ok("ALTER CATALOG `A` SET (\n" + " 'k1' = 'v1',\n" + " 'k2' = 'v2'\n" + ")"); - sql("alter catalog a reset ('k1')").ok("ALTER CATALOG `A` RESET (\n" + " 'k1'\n" + ")"); - sql("alter catalog a comment 'comment1'").ok("ALTER CATALOG `A` COMMENT 'comment1'"); - } - - @Test - void testArrayAgg() { - sql("select\n" - + " array_agg(ename respect nulls order by deptno, ename) as c1,\n" - + " array_agg(ename order by deptno, ename desc) as c2,\n" - + " array_agg(distinct ename) as c3,\n" - + " array_agg(ename) as c4\n" - + "from emp group by gender") - .ok( - "SELECT" - + " ARRAY_AGG(`ENAME` ORDER BY `DEPTNO`, `ENAME`) RESPECT NULLS AS `C1`," - + " ARRAY_AGG(`ENAME` ORDER BY `DEPTNO`, `ENAME` DESC) AS `C2`," - + " ARRAY_AGG(DISTINCT `ENAME`) AS `C3`," - + " ARRAY_AGG(`ENAME`) AS `C4`\n" - + "FROM `EMP`\n" - + "GROUP BY `GENDER`"); - } - - @Test - void testCastAsMapType() { - this.expr("cast(a as map)").ok("CAST(`A` AS MAP< INTEGER, INTEGER >)"); - this.expr("cast(a as map)") - .ok("CAST(`A` AS MAP< INTEGER, VARCHAR ARRAY >)"); - this.expr("cast(a as map>)") - .ok("CAST(`A` AS MAP< VARCHAR MULTISET, MAP< INTEGER, INTEGER > >)"); - } - - // DESCRIBE SCHEMA - @Disabled - @Test - void testDescribeSchema() {} - - // DESCRIBE STATEMENT - @Disabled - @Test - void testDescribeStatement() {} - - // GROUP CONCAT - @Disabled - @Test - void testGroupConcat() {} - - // EXPLAIN AS DOT - @Disabled - @Test - void testExplainAsDot() {} - - // STRING_AGG - @Disabled - @Test - void testStringAgg() {} - - // END - - @Test - void testUseCatalog() { - sql("use catalog a").ok("USE CATALOG `A`"); - } - - @ParameterizedTest - @CsvSource({"true,true", "true,false", "false,true", "false,false"}) - void testCreateCatalog(boolean ifNotExists, boolean comment) { - final String ifNotExistsClause = ifNotExists ? "if not exists " : ""; - final String commentClause = comment ? "\ncomment 'HELLO'" : ""; - - sql("create catalog " - + ifNotExistsClause - + "c1" - + commentClause - + "\nWITH (\n" - + " 'key1'='value1',\n" - + " 'key2'='value2'\n" - + " )\n") - .ok( - "CREATE CATALOG " - + ifNotExistsClause.toUpperCase(Locale.ROOT) - + "`C1`" - + commentClause.toUpperCase(Locale.ROOT) - + "\nWITH (\n" - + " 'key1' = 'value1',\n" - + " 'key2' = 'value2'\n" - + ")"); - } - - @Test - void testShowCreateCatalog() { - sql("show create catalog c1").ok("SHOW CREATE CATALOG `C1`"); - } - - @Test - void testDropCatalog() { - sql("drop catalog c1").ok("DROP CATALOG `C1`"); - } - - @Test - void testShowDataBases() { - sql("show databases").ok("SHOW DATABASES"); - - sql("show databases like '%'").ok("SHOW DATABASES LIKE '%'"); - sql("show databases not like '%'").ok("SHOW DATABASES NOT LIKE '%'"); - - sql("show databases from c1").ok("SHOW DATABASES FROM `C1`"); - sql("show databases in c1").ok("SHOW DATABASES IN `C1`"); - - sql("show databases from c1 like '%'").ok("SHOW DATABASES FROM `C1` LIKE '%'"); - sql("show databases from c1 ilike '%'").ok("SHOW DATABASES FROM `C1` ILIKE '%'"); - sql("show databases in c1 like '%'").ok("SHOW DATABASES IN `C1` LIKE '%'"); - sql("show databases in c1 ilike '%'").ok("SHOW DATABASES IN `C1` ILIKE '%'"); - - sql("show databases from c1 not like '%'").ok("SHOW DATABASES FROM `C1` NOT LIKE '%'"); - sql("show databases from c1 not ilike '%'").ok("SHOW DATABASES FROM `C1` NOT ILIKE '%'"); - sql("show databases in c1 not like '%'").ok("SHOW DATABASES IN `C1` NOT LIKE '%'"); - sql("show databases in c1 not ilike '%'").ok("SHOW DATABASES IN `C1` NOT ILIKE '%'"); - - sql("show databases ^likes^") - .fails("(?s).*Encountered \"likes\" at line 1, column 16.\n.*"); - sql("show databases not ^likes^") - .fails("(?s).*Encountered \"likes\" at line 1, column 20" + ".\n" + ".*"); - sql("show databases ^ilikes^") - .fails("(?s).*Encountered \"ilikes\" at line 1, column 16.\n.*"); - sql("show databases not ^ilikes^") - .fails("(?s).*Encountered \"ilikes\" at line 1, column 20" + ".\n" + ".*"); - } - - @Test - void testShowCurrentDatabase() { - sql("show current database").ok("SHOW CURRENT DATABASE"); - } - - @Test - void testUseDataBase() { - sql("use default_db").ok("USE `DEFAULT_DB`"); - sql("use defaultCatalog.default_db").ok("USE `DEFAULTCATALOG`.`DEFAULT_DB`"); - } - - @Test - void testCreateDatabase() { - sql("create database db1").ok("CREATE DATABASE `DB1`"); - sql("create database if not exists db1").ok("CREATE DATABASE IF NOT EXISTS `DB1`"); - sql("create database catalog1.db1").ok("CREATE DATABASE `CATALOG1`.`DB1`"); - final String sql = "create database db1 comment 'test create database'"; - final String expected = "CREATE DATABASE `DB1`\n" + "COMMENT 'test create database'"; - sql(sql).ok(expected); - final String sql1 = - "create database db1 comment 'test create database'" - + "with ( 'key1' = 'value1', 'key2.a' = 'value2.a')"; - final String expected1 = - "CREATE DATABASE `DB1`\n" - + "COMMENT 'test create database'" - + "\nWITH (\n" - + " 'key1' = 'value1',\n" - + " 'key2.a' = 'value2.a'\n" - + ")"; - sql(sql1).ok(expected1); - } - - @Test - void testDropDatabase() { - sql("drop database db1").ok("DROP DATABASE `DB1` RESTRICT"); - sql("drop database catalog1.db1").ok("DROP DATABASE `CATALOG1`.`DB1` RESTRICT"); - sql("drop database db1 RESTRICT").ok("DROP DATABASE `DB1` RESTRICT"); - sql("drop database db1 CASCADE").ok("DROP DATABASE `DB1` CASCADE"); - } - - @Test - void testAlterDatabase() { - final String sql = "alter database db1 set ('key1' = 'value1','key2.a' = 'value2.a')"; - final String expected = - "ALTER DATABASE `DB1` SET (\n" - + " 'key1' = 'value1',\n" - + " 'key2.a' = 'value2.a'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testDescribeDatabase() { - sql("describe database db1").ok("DESCRIBE DATABASE `DB1`"); - sql("describe database catalog1.db1").ok("DESCRIBE DATABASE `CATALOG1`.`DB1`"); - sql("describe database extended db1").ok("DESCRIBE DATABASE EXTENDED `DB1`"); - - sql("desc database db1").ok("DESCRIBE DATABASE `DB1`"); - sql("desc database catalog1.db1").ok("DESCRIBE DATABASE `CATALOG1`.`DB1`"); - sql("desc database extended db1").ok("DESCRIBE DATABASE EXTENDED `DB1`"); - } - - @Test - void testAlterFunction() { - sql("alter function function1 as 'org.apache.flink.function.function1'") - .ok("ALTER FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1'"); - - sql("alter temporary function function1 as 'org.apache.flink.function.function1'") - .ok( - "ALTER TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1'"); - - sql("alter temporary function function1 as 'org.apache.flink.function.function1' language scala") - .ok( - "ALTER TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE SCALA"); - - sql("alter temporary system function function1 as 'org.apache.flink.function.function1'") - .ok( - "ALTER TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1'"); - - sql("alter temporary system function function1 as 'org.apache.flink.function.function1' language java") - .ok( - "ALTER TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA"); - } - - @Test - void testShowFunctions() { - sql("show functions").ok("SHOW FUNCTIONS"); - sql("show user functions").ok("SHOW USER FUNCTIONS"); - - sql("show functions like '%'").ok("SHOW FUNCTIONS LIKE '%'"); - sql("show functions not like '%'").ok("SHOW FUNCTIONS NOT LIKE '%'"); - sql("show user functions like '%'").ok("SHOW USER FUNCTIONS LIKE '%'"); - sql("show user functions not like '%'").ok("SHOW USER FUNCTIONS NOT LIKE '%'"); - - sql("show functions from db1").ok("SHOW FUNCTIONS FROM `DB1`"); - sql("show user functions from db1").ok("SHOW USER FUNCTIONS FROM `DB1`"); - sql("show functions in db1").ok("SHOW FUNCTIONS IN `DB1`"); - sql("show user functions in db1").ok("SHOW USER FUNCTIONS IN `DB1`"); - - sql("show functions from catalog1.db1").ok("SHOW FUNCTIONS FROM `CATALOG1`.`DB1`"); - sql("show user functions from catalog1.db1") - .ok("SHOW USER FUNCTIONS FROM `CATALOG1`.`DB1`"); - sql("show functions in catalog1.db1").ok("SHOW FUNCTIONS IN `CATALOG1`.`DB1`"); - sql("show user functions in catalog1.db1").ok("SHOW USER FUNCTIONS IN `CATALOG1`.`DB1`"); - - sql("show functions from db1 like '%'").ok("SHOW FUNCTIONS FROM `DB1` LIKE '%'"); - sql("show user functions from db1 like '%'").ok("SHOW USER FUNCTIONS FROM `DB1` LIKE '%'"); - sql("show functions in db1 ilike '%'").ok("SHOW FUNCTIONS IN `DB1` ILIKE '%'"); - sql("show user functions in db1 ilike '%'").ok("SHOW USER FUNCTIONS IN `DB1` ILIKE '%'"); - - sql("show functions from catalog1.db1 ilike '%'") - .ok("SHOW FUNCTIONS FROM `CATALOG1`.`DB1` ILIKE '%'"); - sql("show user functions from catalog1.db1 ilike '%'") - .ok("SHOW USER FUNCTIONS FROM `CATALOG1`.`DB1` ILIKE '%'"); - sql("show functions in catalog1.db1 like '%'") - .ok("SHOW FUNCTIONS IN `CATALOG1`.`DB1` LIKE '%'"); - sql("show user functions in catalog1.db1 like '%'") - .ok("SHOW USER FUNCTIONS IN `CATALOG1`.`DB1` LIKE '%'"); - - sql("show functions from db1 not like '%'").ok("SHOW FUNCTIONS FROM `DB1` NOT LIKE '%'"); - sql("show user functions from db1 not like '%'") - .ok("SHOW USER FUNCTIONS FROM `DB1` NOT LIKE '%'"); - sql("show functions in db1 not ilike '%'").ok("SHOW FUNCTIONS IN `DB1` NOT ILIKE '%'"); - sql("show user functions in db1 not ilike '%'") - .ok("SHOW USER FUNCTIONS IN `DB1` NOT ILIKE '%'"); - - sql("show functions from catalog1.db1 not like '%'") - .ok("SHOW FUNCTIONS FROM `CATALOG1`.`DB1` NOT LIKE '%'"); - sql("show user functions from catalog1.db1 not like '%'") - .ok("SHOW USER FUNCTIONS FROM `CATALOG1`.`DB1` NOT LIKE '%'"); - sql("show functions in catalog1.db1 not ilike '%'") - .ok("SHOW FUNCTIONS IN `CATALOG1`.`DB1` NOT ILIKE '%'"); - sql("show user functions in catalog1.db1 not ilike '%'") - .ok("SHOW USER FUNCTIONS IN `CATALOG1`.`DB1` NOT ILIKE '%'"); - - sql("show functions ^likes^") - .fails("(?s).*Encountered \"likes\" at line 1, column 16.\n.*"); - sql("show functions not ^likes^") - .fails("(?s).*Encountered \"likes\" at line 1, column 20" + ".\n" + ".*"); - sql("show functions ^ilikes^") - .fails("(?s).*Encountered \"ilikes\" at line 1, column 16.\n.*"); - sql("show functions not ^ilikes^") - .fails("(?s).*Encountered \"ilikes\" at line 1, column 20" + ".\n" + ".*"); - } - - @Test - void testShowProcedures() { - sql("show procedures").ok("SHOW PROCEDURES"); - sql("show procedures not like '%'").ok("SHOW PROCEDURES NOT LIKE '%'"); - - sql("show procedures from db1").ok("SHOW PROCEDURES FROM `DB1`"); - sql("show procedures in db1").ok("SHOW PROCEDURES IN `DB1`"); - - sql("show procedures from catalog1.db1").ok("SHOW PROCEDURES FROM `CATALOG1`.`DB1`"); - sql("show procedures in catalog1.db1").ok("SHOW PROCEDURES IN `CATALOG1`.`DB1`"); - - sql("show procedures from db1 like '%'").ok("SHOW PROCEDURES FROM `DB1` LIKE '%'"); - sql("show procedures in db1 ilike '%'").ok("SHOW PROCEDURES IN `DB1` ILIKE '%'"); - - sql("show procedures from catalog1.db1 Ilike '%'") - .ok("SHOW PROCEDURES FROM `CATALOG1`.`DB1` ILIKE '%'"); - sql("show procedures in catalog1.db1 like '%'") - .ok("SHOW PROCEDURES IN `CATALOG1`.`DB1` LIKE '%'"); - - sql("show procedures from db1 not like '%'").ok("SHOW PROCEDURES FROM `DB1` NOT LIKE '%'"); - sql("show procedures in db1 not ilike '%'").ok("SHOW PROCEDURES IN `DB1` NOT ILIKE '%'"); - - sql("show procedures from catalog1.db1 not like '%'") - .ok("SHOW PROCEDURES FROM `CATALOG1`.`DB1` NOT LIKE '%'"); - sql("show procedures in catalog1.db1 not ilike '%'") - .ok("SHOW PROCEDURES IN `CATALOG1`.`DB1` NOT ILIKE '%'"); - - sql("show procedures ^db1^").fails("(?s).*Encountered \"db1\" at line 1, column 17.\n.*"); - sql("show procedures ^catalog1^.db1") - .fails("(?s).*Encountered \"catalog1\" at line 1, column 17.\n.*"); - - sql("show procedures ^search^ db1") - .fails("(?s).*Encountered \"search\" at line 1, column 17.\n.*"); - - sql("show procedures from db1 ^likes^ '%t'") - .fails("(?s).*Encountered \"likes\" at line 1, column 26.\n.*"); - } - - @Test - void testShowTables() { - sql("show tables").ok("SHOW TABLES"); - sql("show tables not like '%'").ok("SHOW TABLES NOT LIKE '%'"); - - sql("show tables from db1").ok("SHOW TABLES FROM `DB1`"); - sql("show tables in db1").ok("SHOW TABLES IN `DB1`"); - - sql("show tables from catalog1.db1").ok("SHOW TABLES FROM `CATALOG1`.`DB1`"); - sql("show tables in catalog1.db1").ok("SHOW TABLES IN `CATALOG1`.`DB1`"); - - sql("show tables from db1 like '%'").ok("SHOW TABLES FROM `DB1` LIKE '%'"); - sql("show tables in db1 like '%'").ok("SHOW TABLES IN `DB1` LIKE '%'"); - - sql("show tables from catalog1.db1 like '%'") - .ok("SHOW TABLES FROM `CATALOG1`.`DB1` LIKE '%'"); - sql("show tables in catalog1.db1 like '%'").ok("SHOW TABLES IN `CATALOG1`.`DB1` LIKE '%'"); - - sql("show tables from db1 not like '%'").ok("SHOW TABLES FROM `DB1` NOT LIKE '%'"); - sql("show tables in db1 not like '%'").ok("SHOW TABLES IN `DB1` NOT LIKE '%'"); - - sql("show tables from catalog1.db1 not like '%'") - .ok("SHOW TABLES FROM `CATALOG1`.`DB1` NOT LIKE '%'"); - sql("show tables in catalog1.db1 not like '%'") - .ok("SHOW TABLES IN `CATALOG1`.`DB1` NOT LIKE '%'"); - - sql("show tables ^db1^").fails("(?s).*Encountered \"db1\" at line 1, column 13.\n.*"); - sql("show tables ^catalog1^.db1") - .fails("(?s).*Encountered \"catalog1\" at line 1, column 13.\n.*"); - - sql("show tables ^search^ db1") - .fails("(?s).*Encountered \"search\" at line 1, column 13.\n.*"); - - sql("show tables from db1 ^likes^ '%t'") - .fails("(?s).*Encountered \"likes\" at line 1, column 22.\n.*"); - } - - @Test - void testShowCreateModel() { - sql("show create model m1").ok("SHOW CREATE MODEL `M1`"); - sql("show create model catalog1.db1.m1").ok("SHOW CREATE MODEL `CATALOG1`.`DB1`.`M1`"); - } - - @Test - void testShowCreateTable() { - sql("show create table tbl").ok("SHOW CREATE TABLE `TBL`"); - sql("show create table catalog1.db1.tbl").ok("SHOW CREATE TABLE `CATALOG1`.`DB1`.`TBL`"); - } - - @Test - void testShowCreateView() { - sql("show create view v1").ok("SHOW CREATE VIEW `V1`"); - sql("show create view db1.v1").ok("SHOW CREATE VIEW `DB1`.`V1`"); - sql("show create view catalog1.db1.v1").ok("SHOW CREATE VIEW `CATALOG1`.`DB1`.`V1`"); - } - - @Test - void testDescribeTable() { - sql("describe tbl").ok("DESCRIBE `TBL`"); - sql("describe catalog1.db1.tbl").ok("DESCRIBE `CATALOG1`.`DB1`.`TBL`"); - sql("describe extended db1").ok("DESCRIBE EXTENDED `DB1`"); - - sql("desc tbl").ok("DESCRIBE `TBL`"); - sql("desc catalog1.db1.tbl").ok("DESCRIBE `CATALOG1`.`DB1`.`TBL`"); - sql("desc extended db1").ok("DESCRIBE EXTENDED `DB1`"); - } - - @Test - void testDescribeModel() { - sql("describe model mdl").ok("DESCRIBE MODEL `MDL`"); - sql("describe model catalog1.db1.mdl").ok("DESCRIBE MODEL `CATALOG1`.`DB1`.`MDL`"); - - sql("desc model mdl").ok("DESCRIBE MODEL `MDL`"); - sql("desc model catalog1.db1.mdl").ok("DESCRIBE MODEL `CATALOG1`.`DB1`.`MDL`"); - } - - @Test - void testDescribeFunction() { - sql("describe function fn").ok("DESCRIBE FUNCTION `FN`"); - sql("describe function catalog1.db1.fn").ok("DESCRIBE FUNCTION `CATALOG1`.`DB1`.`FN`"); - sql("describe function extended fn").ok("DESCRIBE FUNCTION EXTENDED `FN`"); - - sql("desc function fn").ok("DESCRIBE FUNCTION `FN`"); - sql("desc function catalog1.db1.fn").ok("DESCRIBE FUNCTION `CATALOG1`.`DB1`.`FN`"); - sql("desc function extended fn").ok("DESCRIBE FUNCTION EXTENDED `FN`"); - } - - @Test - void testShowColumns() { - sql("show columns from tbl").ok("SHOW COLUMNS FROM `TBL`"); - sql("show columns in tbl").ok("SHOW COLUMNS IN `TBL`"); - - sql("show columns from db1.tbl").ok("SHOW COLUMNS FROM `DB1`.`TBL`"); - sql("show columns in db1.tbl").ok("SHOW COLUMNS IN `DB1`.`TBL`"); - - sql("show columns from catalog1.db1.tbl").ok("SHOW COLUMNS FROM `CATALOG1`.`DB1`.`TBL`"); - sql("show columns in catalog1.db1.tbl").ok("SHOW COLUMNS IN `CATALOG1`.`DB1`.`TBL`"); - - sql("show columns from tbl like '%'").ok("SHOW COLUMNS FROM `TBL` LIKE '%'"); - sql("show columns in tbl like '%'").ok("SHOW COLUMNS IN `TBL` LIKE '%'"); - - sql("show columns from db1.tbl like '%'").ok("SHOW COLUMNS FROM `DB1`.`TBL` LIKE '%'"); - sql("show columns in db1.tbl like '%'").ok("SHOW COLUMNS IN `DB1`.`TBL` LIKE '%'"); - - sql("show columns from catalog1.db1.tbl like '%'") - .ok("SHOW COLUMNS FROM `CATALOG1`.`DB1`.`TBL` LIKE '%'"); - sql("show columns in catalog1.db1.tbl like '%'") - .ok("SHOW COLUMNS IN `CATALOG1`.`DB1`.`TBL` LIKE '%'"); - - sql("show columns from tbl not like '%'").ok("SHOW COLUMNS FROM `TBL` NOT LIKE '%'"); - sql("show columns in tbl not like '%'").ok("SHOW COLUMNS IN `TBL` NOT LIKE '%'"); - - sql("show columns from db1.tbl not like '%'") - .ok("SHOW COLUMNS FROM `DB1`.`TBL` NOT LIKE '%'"); - sql("show columns in db1.tbl not like '%'").ok("SHOW COLUMNS IN `DB1`.`TBL` NOT LIKE '%'"); - - sql("show columns from catalog1.db1.tbl not like '%'") - .ok("SHOW COLUMNS FROM `CATALOG1`.`DB1`.`TBL` NOT LIKE '%'"); - sql("show columns in catalog1.db1.tbl not like '%'") - .ok("SHOW COLUMNS IN `CATALOG1`.`DB1`.`TBL` NOT LIKE '%'"); - } - - @Test - void testAlterTable() { - sql("alter table t1 rename to t2").ok("ALTER TABLE `T1` RENAME TO `T2`"); - sql("alter table if exists t1 rename to t2") - .ok("ALTER TABLE IF EXISTS `T1` RENAME TO `T2`"); - sql("alter table c1.d1.t1 rename to t2").ok("ALTER TABLE `C1`.`D1`.`T1` RENAME TO `T2`"); - sql("alter table if exists c1.d1.t1 rename to t2") - .ok("ALTER TABLE IF EXISTS `C1`.`D1`.`T1` RENAME TO `T2`"); - - sql("alter table t1 set ('key1'='value1')") - .ok("ALTER TABLE `T1` SET (\n" + " 'key1' = 'value1'\n" + ")"); - sql("alter table if exists t1 set ('key1'='value1')") - .ok("ALTER TABLE IF EXISTS `T1` SET (\n" + " 'key1' = 'value1'\n" + ")"); - - sql("alter table t1 add constraint ct1 primary key(a, b)") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " CONSTRAINT `CT1` PRIMARY KEY (`A`, `B`)\n" - + ")") - .node( - new ValidationMatcher() - .fails( - "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. " - + "ENFORCED/NOT ENFORCED controls if the constraint checks are performed on the incoming/outgoing data. " - + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); - sql("alter table t1 add constraint ct1 primary key(a, b) not enforced") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " CONSTRAINT `CT1` PRIMARY KEY (`A`, `B`) NOT ENFORCED\n" - + ")"); - sql("alter table if exists t1 add constraint ct1 primary key(a, b) not enforced") - .ok( - "ALTER TABLE IF EXISTS `T1` ADD (\n" - + " CONSTRAINT `CT1` PRIMARY KEY (`A`, `B`) NOT ENFORCED\n" - + ")"); - sql("alter table t1 " + "add unique(a, b)") - .ok("ALTER TABLE `T1` ADD (\n" + " UNIQUE (`A`, `B`)\n" + ")") - .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); - sql("alter table if exists t1 " + "add unique(a, b)") - .ok("ALTER TABLE IF EXISTS `T1` ADD (\n" + " UNIQUE (`A`, `B`)\n" + ")"); - - sql("alter table t1 drop constraint ct1").ok("ALTER TABLE `T1` DROP CONSTRAINT `CT1`"); - sql("alter table if exists t1 drop constraint ct1") - .ok("ALTER TABLE IF EXISTS `T1` DROP CONSTRAINT `CT1`"); - - sql("alter table t1 rename a to b").ok("ALTER TABLE `T1` RENAME `A` TO `B`"); - sql("alter table if exists t1 rename a to b") - .ok("ALTER TABLE IF EXISTS `T1` RENAME `A` TO `B`"); - sql("alter table if exists t1 rename a.x to a.y") - .ok("ALTER TABLE IF EXISTS `T1` RENAME `A`.`X` TO `A`.`Y`"); - } - - @Test - void testAlterTableAddNestedColumn() { - // add a row column - sql("alter table t1 add new_column array comment 'new_column docs'") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " `NEW_COLUMN` ARRAY< ROW(`F0` INTEGER, `F1` BIGINT) > COMMENT 'new_column docs'\n" - + ")"); - - sql("alter table t1 add (new_row row(f0 int, f1 bigint) comment 'new_column docs', f2 as new_row.f0 + 1)") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " `NEW_ROW` ROW(`F0` INTEGER, `F1` BIGINT) COMMENT 'new_column docs',\n" - + " `F2` AS (`NEW_ROW`.`F0` + 1)\n" - + ")"); - - // add a field to the row - sql("alter table t1 add (new_row.f2 array)") - .ok("ALTER TABLE `T1` ADD (\n" + " `NEW_ROW`.`F2` ARRAY< INTEGER >\n" + ")"); - - // add a field to the row with after - sql("alter table t1 add (new_row.f2 array after new_row.f0)") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " `NEW_ROW`.`F2` ARRAY< INTEGER > AFTER `NEW_ROW`.`F0`\n" - + ")"); - } - - @Test - void testAlterTableAddSingleColumn() { - sql("alter table if exists t1 add new_column int not null") - .ok( - "ALTER TABLE IF EXISTS `T1` ADD (\n" - + " `NEW_COLUMN` INTEGER NOT NULL\n" - + ")"); - sql("alter table t1 add new_column string comment 'new_column docs'") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " `NEW_COLUMN` STRING COMMENT 'new_column docs'\n" - + ")"); - sql("alter table t1 add new_column string comment 'new_column docs' first") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " `NEW_COLUMN` STRING COMMENT 'new_column docs' FIRST\n" - + ")"); - sql("alter table t1 add new_column string comment 'new_column docs' after id") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " `NEW_COLUMN` STRING COMMENT 'new_column docs' AFTER `ID`\n" - + ")"); - // add compute column - sql("alter table t1 add col_int as col_a - col_b after col_b") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " `COL_INT` AS (`COL_A` - `COL_B`) AFTER `COL_B`\n" - + ")"); - // add metadata column - sql("alter table t1 add col_int int metadata from 'mk1' virtual comment 'comment_metadata' after col_b") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " `COL_INT` INTEGER METADATA FROM 'mk1' VIRTUAL COMMENT 'comment_metadata' AFTER `COL_B`\n" - + ")"); - } - - @Test - void testAlterTableAddWatermark() { - sql("alter table if exists t1 add watermark for ts as ts") - .ok("ALTER TABLE IF EXISTS `T1` ADD (\n" + " WATERMARK FOR `TS` AS `TS`\n" + ")"); - sql("alter table t1 add watermark for ts as ts - interval '1' second") - .ok( - "ALTER TABLE `T1` ADD (\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" - + ")"); - sql("alter table default_database.t1 add watermark for ts as ts - interval '1' second") - .ok( - "ALTER TABLE `DEFAULT_DATABASE`.`T1` ADD (\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" - + ")"); - sql("alter table default_catalog.default_database.t1 add watermark for ts as ts - interval '1' second") - .ok( - "ALTER TABLE `DEFAULT_CATALOG`.`DEFAULT_DATABASE`.`T1` ADD (\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" - + ")"); - - sql("alter table default_catalog.default_database.t1 add (\n" - + "watermark for ts as ts - interval '1' second,\n" - + "^watermark^ for f1 as now()\n" - + ")") - .fails("Multiple WATERMARK declarations are not supported yet."); - } - - @Test - void testAlterTableAddDistribution() { - sql("alter table t1 add DISTRIBUTION BY HASH(a) INTO 6 BUCKETS") - .ok("ALTER TABLE `T1` ADD DISTRIBUTION BY HASH(`A`) INTO 6 BUCKETS"); - - sql("alter table t1 add DISTRIBUTION BY HASH(a, h) INTO 6 BUCKETS") - .ok("ALTER TABLE `T1` ADD DISTRIBUTION BY HASH(`A`, `H`) INTO 6 BUCKETS"); - - sql("alter table tbl1 add DISTRIBUTION BY RANGE(a, h) INTO 6 BUCKETS") - .ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY RANGE(`A`, `H`) INTO 6 BUCKETS"); - - sql("alter table tbl1 add DISTRIBUTION BY ^RANDOM^(a, h) INTO 6 BUCKETS") - .fails("(?s).*Encountered \"RANDOM\" at line 1, column 38.*"); - - sql("alter table tbl1 add DISTRIBUTION BY (a, h) INTO 6 BUCKETS") - .ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY (`A`, `H`) INTO 6 BUCKETS"); - - sql("alter table tbl1 add DISTRIBUTION BY RANGE(a, h)") - .ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY RANGE(`A`, `H`)"); - - sql("alter table tbl1 add DISTRIBUTION BY (a, h)") - .ok("ALTER TABLE `TBL1` ADD DISTRIBUTION BY (`A`, `H`)"); - } - - @Test - void testAlterTableModifyDistribution() { - sql("alter table t1 modify DISTRIBUTION BY HASH(a) INTO 6 BUCKETS") - .ok("ALTER TABLE `T1` MODIFY DISTRIBUTION BY HASH(`A`) INTO 6 BUCKETS"); - - sql("alter table tbl1 modify DISTRIBUTION BY HASH(a, h) INTO 6 BUCKETS") - .ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY HASH(`A`, `H`) INTO 6 BUCKETS"); - - sql("alter table tbl1 modify DISTRIBUTION BY RANGE(a, h) INTO 6 BUCKETS") - .ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY RANGE(`A`, `H`) INTO 6 BUCKETS"); - - sql("alter table tbl1 modify DISTRIBUTION BY ^RANDOM^(a, h) INTO 6 BUCKETS") - .fails("(?s).*Encountered \"RANDOM\" at line 1, column 41.*"); - - sql("alter table tbl1 modify DISTRIBUTION BY (a, h) INTO 6 BUCKETS") - .ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY (`A`, `H`) INTO 6 BUCKETS"); - - sql("alter table tbl1 modify DISTRIBUTION BY RANGE(a, h)") - .ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY RANGE(`A`, `H`)"); - - sql("alter table tbl1 modify DISTRIBUTION BY (a, h)") - .ok("ALTER TABLE `TBL1` MODIFY DISTRIBUTION BY (`A`, `H`)"); - } - - @Test - void testAlterTableDropDistribution() { - sql("alter table t1 drop DISTRIBUTION").ok("ALTER TABLE `T1` DROP DISTRIBUTION"); - } - - @Test - void testAlterTableAddMultipleColumn() { - final String sql1 = - "alter table t1 add (\n" - + "col_int int,\n" - + "log_ts string comment 'log timestamp string' first,\n" - + "ts AS to_timestamp(log_ts) after log_ts,\n" - + "col_meta int metadata from 'mk1' virtual comment 'comment_str' after col_b,\n" - + "primary key (id) not enforced,\n" - + "unique(a, b),\n" - + "watermark for ts as ts - interval '3' second\n" - + ")"; - final String expected1 = - "ALTER TABLE `T1` ADD (\n" - + " `COL_INT` INTEGER,\n" - + " `LOG_TS` STRING COMMENT 'log timestamp string' FIRST,\n" - + " `TS` AS `TO_TIMESTAMP`(`LOG_TS`) AFTER `LOG_TS`,\n" - + " `COL_META` INTEGER METADATA FROM 'mk1' VIRTUAL COMMENT 'comment_str' AFTER `COL_B`,\n" - + " PRIMARY KEY (`ID`) NOT ENFORCED,\n" - + " UNIQUE (`A`, `B`),\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '3' SECOND)\n" - + ")"; - sql(sql1).ok(expected1); - - final String sql2 = - "alter table t1 add (\n" - + "col_int int primary key not enforced,\n" - + "log_ts string comment 'log timestamp string' first,\n" - + "ts AS to_timestamp(log_ts) after log_ts,\n" - + "col_meta int metadata from 'mk1' virtual comment 'comment_str' after col_b,\n" - + "primary key (id) not enforced,\n" - + "unique (a, b),\n" - + "watermark for ts as ts - interval '3' second\n" - + ")"; - sql(sql2).node(new ValidationMatcher().fails("Duplicate primary key definition")); - } - - @Test - public void testAlterTableModifySingleColumn() { - sql("alter table if exists t1 modify new_column string comment 'new_column docs'") - .ok( - "ALTER TABLE IF EXISTS `T1` MODIFY (\n" - + " `NEW_COLUMN` STRING COMMENT 'new_column docs'\n" - + ")"); - sql("alter table t1 modify new_column string comment 'new_column docs'") - .ok( - "ALTER TABLE `T1` MODIFY (\n" - + " `NEW_COLUMN` STRING COMMENT 'new_column docs'\n" - + ")"); - sql("alter table t1 modify new_column string comment 'new_column docs' first") - .ok( - "ALTER TABLE `T1` MODIFY (\n" - + " `NEW_COLUMN` STRING COMMENT 'new_column docs' FIRST\n" - + ")"); - sql("alter table t1 modify new_column string comment 'new_column docs' after id") - .ok( - "ALTER TABLE `T1` MODIFY (\n" - + " `NEW_COLUMN` STRING COMMENT 'new_column docs' AFTER `ID`\n" - + ")"); - // modify column type - sql("alter table t1 modify new_column array not null") - .ok( - "ALTER TABLE `T1` MODIFY (\n" - + " `NEW_COLUMN` ARRAY< STRING NOT NULL > NOT NULL\n" - + ")"); - - // modify compute column - sql("alter table t1 modify col_int as col_a - col_b after col_b") - .ok( - "ALTER TABLE `T1` MODIFY (\n" - + " `COL_INT` AS (`COL_A` - `COL_B`) AFTER `COL_B`\n" - + ")"); - // modify metadata column - sql("alter table t1 modify col_int int metadata from 'mk1' virtual comment 'comment_metadata' after col_b") - .ok( - "ALTER TABLE `T1` MODIFY (\n" - + " `COL_INT` INTEGER METADATA FROM 'mk1' VIRTUAL COMMENT 'comment_metadata' AFTER `COL_B`\n" - + ")"); - - // modify nested column - sql("alter table t1 modify row_column.f0 int not null comment 'change nullability'") - .ok( - "ALTER TABLE `T1` MODIFY (\n" - + " `ROW_COLUMN`.`F0` INTEGER NOT NULL COMMENT 'change nullability'\n" - + ")"); - - // modify nested column, shift position - sql("alter table t1 modify row_column.f0 int after row_column.f2") - .ok( - "ALTER TABLE `T1` MODIFY (\n" - + " `ROW_COLUMN`.`F0` INTEGER AFTER `ROW_COLUMN`.`F2`\n" - + ")"); - } - - @Test - void testAlterTableModifyWatermark() { - sql("alter table if exists t1 modify watermark for ts as ts") - .ok( - "ALTER TABLE IF EXISTS `T1` MODIFY (\n" - + " WATERMARK FOR `TS` AS `TS`\n" - + ")"); - sql("alter table t1 modify watermark for ts as ts - interval '1' second") - .ok( - "ALTER TABLE `T1` MODIFY (\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" - + ")"); - sql("alter table default_database.t1 modify watermark for ts as ts - interval '1' second") - .ok( - "ALTER TABLE `DEFAULT_DATABASE`.`T1` MODIFY (\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" - + ")"); - sql("alter table default_catalog.default_database.t1 modify watermark for ts as ts - interval '1' second") - .ok( - "ALTER TABLE `DEFAULT_CATALOG`.`DEFAULT_DATABASE`.`T1` MODIFY (\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '1' SECOND)\n" - + ")"); - - sql("alter table default_catalog.default_database.t1 modify (\n" - + "watermark for ts as ts - interval '1' second,\n" - + "^watermark^ for f1 as now()\n" - + ")") - .fails("Multiple WATERMARK declarations are not supported yet."); - } - - @Test - void testAlterTableModifyConstraint() { - sql("alter table t1 modify constraint ct1 primary key(a, b) not enforced") - .ok( - "ALTER TABLE `T1` MODIFY (\n" - + " CONSTRAINT `CT1` PRIMARY KEY (`A`, `B`) NOT ENFORCED\n" - + ")"); - sql("alter table t1 modify unique(a, b)") - .ok("ALTER TABLE `T1` MODIFY (\n" + " UNIQUE (`A`, `B`)\n" + ")"); - } - - @Test - public void testAlterTableModifyMultipleColumn() { - final String sql1 = - "alter table t1 modify (\n" - + "col_int int,\n" - + "log_ts string comment 'log timestamp string' first,\n" - + "ts AS to_timestamp(log_ts) after log_ts,\n" - + "col_meta int metadata from 'mk1' virtual comment 'comment_str' after col_b,\n" - + "primary key (id) not enforced,\n" - + "unique(a, b),\n" - + "watermark for ts as ts - interval '3' second\n" - + ")"; - final String expected1 = - "ALTER TABLE `T1` MODIFY (\n" - + " `COL_INT` INTEGER,\n" - + " `LOG_TS` STRING COMMENT 'log timestamp string' FIRST,\n" - + " `TS` AS `TO_TIMESTAMP`(`LOG_TS`) AFTER `LOG_TS`,\n" - + " `COL_META` INTEGER METADATA FROM 'mk1' VIRTUAL COMMENT 'comment_str' AFTER `COL_B`,\n" - + " PRIMARY KEY (`ID`) NOT ENFORCED,\n" - + " UNIQUE (`A`, `B`),\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '3' SECOND)\n" - + ")"; - sql(sql1).ok(expected1); - } - - @Test - public void testAlterTableDropSingleColumn() { - sql("alter table if exists t1 drop id") - .ok("ALTER TABLE IF EXISTS `T1` DROP (\n" + " `ID`\n" + ")"); - sql("alter table t1 drop id").ok("ALTER TABLE `T1` DROP (\n" + " `ID`\n" + ")"); - - sql("alter table t1 drop (id)").ok("ALTER TABLE `T1` DROP (\n" + " `ID`\n" + ")"); - - sql("alter table t1 drop tuple.id") - .ok("ALTER TABLE `T1` DROP (\n" + " `TUPLE`.`ID`\n" + ")"); - } - - @Test - public void testAlterTableDropMultipleColumn() { - sql("alter table if exists t1 drop (id, ts, tuple.f0, tuple.f1)") - .ok( - "ALTER TABLE IF EXISTS `T1` DROP (\n" - + " `ID`,\n" - + " `TS`,\n" - + " `TUPLE`.`F0`,\n" - + " `TUPLE`.`F1`\n" - + ")"); - sql("alter table t1 drop (id, ts, tuple.f0, tuple.f1)") - .ok( - "ALTER TABLE `T1` DROP (\n" - + " `ID`,\n" - + " `TS`,\n" - + " `TUPLE`.`F0`,\n" - + " `TUPLE`.`F1`\n" - + ")"); - } - - @Test - public void testAlterTableDropPrimaryKey() { - sql("alter table if exists t1 drop primary key") - .ok("ALTER TABLE IF EXISTS `T1` DROP PRIMARY KEY"); - sql("alter table t1 drop primary key").ok("ALTER TABLE `T1` DROP PRIMARY KEY"); - } - - @Test - public void testAlterTableDropConstraint() { - sql("alter table if exists t1 drop constraint ct") - .ok("ALTER TABLE IF EXISTS `T1` DROP CONSTRAINT `CT`"); - sql("alter table t1 drop constraint ct").ok("ALTER TABLE `T1` DROP CONSTRAINT `CT`"); - - sql("alter table t1 drop constrain^t^") - .fails("(?s).*Encountered \"\" at line 1, column 30.\n.*"); - } - - @Test - public void testAlterTableDropWatermark() { - sql("alter table if exists t1 drop watermark") - .ok("ALTER TABLE IF EXISTS `T1` DROP WATERMARK"); - sql("alter table t1 drop watermark").ok("ALTER TABLE `T1` DROP WATERMARK"); - } - - @Test - void testAlterTableReset() { - sql("alter table if exists t1 reset ('key1')") - .ok("ALTER TABLE IF EXISTS `T1` RESET (\n 'key1'\n)"); - - sql("alter table t1 reset ('key1')").ok("ALTER TABLE `T1` RESET (\n 'key1'\n)"); - - sql("alter table t1 reset ('key1', 'key2')") - .ok("ALTER TABLE `T1` RESET (\n 'key1',\n 'key2'\n)"); - - sql("alter table t1 reset()").ok("ALTER TABLE `T1` RESET (\n)"); - } - - @Test - public void testAddPartition() { - sql("alter table c1.d1.tbl add partition (p1=1,p2='a')") - .ok("ALTER TABLE `C1`.`D1`.`TBL`\n" + "ADD\n" + "PARTITION (`P1` = 1, `P2` = 'a')"); - - sql("alter table tbl add partition (p1=1,p2='a') with ('k1'='v1')") - .ok( - "ALTER TABLE `TBL`\n" - + "ADD\n" - + "PARTITION (`P1` = 1, `P2` = 'a') WITH ('k1' = 'v1')"); - - sql("alter table tbl add if not exists partition (p=1) partition (p=2) with ('k1' = 'v1')") - .ok( - "ALTER TABLE `TBL`\n" - + "ADD IF NOT EXISTS\n" - + "PARTITION (`P` = 1)\n" - + "PARTITION (`P` = 2) WITH ('k1' = 'v1')"); - } - - @Test - public void testDropPartition() { - sql("alter table c1.d1.tbl drop if exists partition (p=1)") - .ok("ALTER TABLE `C1`.`D1`.`TBL`\n" + "DROP IF EXISTS\n" + "PARTITION (`P` = 1)"); - sql("alter table tbl drop partition (p1='a',p2=1), partition(p1='b',p2=2)") - .ok( - "ALTER TABLE `TBL`\n" - + "DROP\n" - + "PARTITION (`P1` = 'a', `P2` = 1),\n" - + "PARTITION (`P1` = 'b', `P2` = 2)"); - sql("alter table tbl drop partition (p1='a',p2=1), " - + "partition(p1='b',p2=2), partition(p1='c',p2=3)") - .ok( - "ALTER TABLE `TBL`\n" - + "DROP\n" - + "PARTITION (`P1` = 'a', `P2` = 1),\n" - + "PARTITION (`P1` = 'b', `P2` = 2),\n" - + "PARTITION (`P1` = 'c', `P2` = 3)"); - } - - @Test - void testCreateTable() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " a bigint,\n" - + " h varchar, \n" - + " g as 2 * (a + 1), \n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" - + " b varchar,\n" - + " proc as PROCTIME(), \n" - + " meta STRING METADATA, \n" - + " my_meta STRING METADATA FROM 'meta', \n" - + " my_meta STRING METADATA FROM 'meta' VIRTUAL, \n" - + " meta STRING METADATA VIRTUAL, \n" - + " PRIMARY KEY (a, b)\n" - + ")\n" - + "PARTITIONED BY (a, h)\n" - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT,\n" - + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`(),\n" - + " `META` STRING METADATA,\n" - + " `MY_META` STRING METADATA FROM 'meta',\n" - + " `MY_META` STRING METADATA FROM 'meta' VIRTUAL,\n" - + " `META` STRING METADATA VIRTUAL,\n" - + " PRIMARY KEY (`A`, `B`)\n" - + ")\n" - + "PARTITIONED BY (`A`, `H`)\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableUsingConnection() { - final String sql = - "CREATE TABLE orders (\n" - + " order_id INT,\n" - + " customer_id INT,\n" - + " amount DECIMAL(10, 2)\n" - + ") USING CONNECTION mycat.mydb.mysql_prod\n" - + "WITH (\n" - + " 'connector' = 'jdbc',\n" - + " 'tables' = 'orders'\n" - + ")"; - final String expected = - "CREATE TABLE `ORDERS` (\n" - + " `ORDER_ID` INTEGER,\n" - + " `CUSTOMER_ID` INTEGER,\n" - + " `AMOUNT` DECIMAL(10, 2)\n" - + ")\n" - + "USING CONNECTION `MYCAT`.`MYDB`.`MYSQL_PROD`\n" - + "WITH (\n" - + " 'connector' = 'jdbc',\n" - + " 'tables' = 'orders'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableUsingConnectionWithPartitionAndDistribution() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " a bigint,\n" - + " h varchar,\n" - + " b varchar\n" - + ")\n" - + "DISTRIBUTED BY HASH(a) INTO 3 BUCKETS\n" - + "PARTITIONED BY (a, h)\n" - + "USING CONNECTION cat1.db1.conn1\n" - + "WITH (\n" - + " 'connector' = 'jdbc'\n" - + ")"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT,\n" - + " `H` VARCHAR,\n" - + " `B` VARCHAR\n" - + ")\n" - + "DISTRIBUTED BY HASH(`A`) INTO 3 BUCKETS\n" - + "PARTITIONED BY (`A`, `H`)\n" - + "USING CONNECTION `CAT1`.`DB1`.`CONN1`\n" - + "WITH (\n" - + " 'connector' = 'jdbc'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableAsWithUsingConnectionFails() { - // CTAS goes through SqlCreateTable with replace=false; the parser explicitly rejects - // USING CONNECTION + AS via ParserResource.usingConnectionWithAsUnsupported(). - final String sql = - "^CREATE^ TABLE t1\n" - + "USING CONNECTION cat1.db1.conn1\n" - + "WITH ('connector' = 'jdbc')\n" - + "AS SELECT 1 AS a"; - sql(sql).fails( - "(?s).*USING CONNECTION clause is not supported with " - + "CREATE TABLE AS SELECT or REPLACE TABLE AS SELECT statements\\..*"); - } - - @Test - void testReplaceTableAsWithUsingConnectionFails() { - // REPLACE TABLE always has an AS clause, so USING CONNECTION is never valid; - // it's not even accepted by the SqlReplaceTable production. - final String sql = - "REPLACE TABLE t1\n" - + "^USING^ CONNECTION cat1.db1.conn1\n" - + "WITH ('connector' = 'jdbc')\n" - + "AS SELECT 1 AS a"; - sql(sql).fails("(?s).*Encountered \"USING\" at line 2, column 1.\n.*"); - } - - @Test - void testCreateOrReplaceTableAsWithUsingConnectionFails() { - // CREATE OR REPLACE TABLE AS goes through SqlCreateTable with replace=true and hits - // the same usingConnectionWithAsUnsupported() error path as CTAS. - final String sql = - "^CREATE^ OR REPLACE TABLE t1\n" - + "USING CONNECTION cat1.db1.conn1\n" - + "WITH ('connector' = 'jdbc')\n" - + "AS SELECT 1 AS a"; - sql(sql).fails( - "(?s).*USING CONNECTION clause is not supported with " - + "CREATE TABLE AS SELECT or REPLACE TABLE AS SELECT statements\\..*"); - } - - @Test - void testCreateTableLikeUsingConnection() { - final String sql = - "CREATE TABLE t1 (\n" - + " a INT\n" - + ")\n" - + "USING CONNECTION cat1.db1.conn1\n" - + "WITH ('connector' = 'jdbc')\n" - + "LIKE base_table"; - final String expected = - "CREATE TABLE `T1` (\n" - + " `A` INTEGER\n" - + ")\n" - + "USING CONNECTION `CAT1`.`DB1`.`CONN1`\n" - + "WITH (\n" - + " 'connector' = 'jdbc'\n" - + ")\n" - + "LIKE `BASE_TABLE`"; - sql(sql).ok(expected); - } - - String buildDistributionInput(final String distributionClause) { - return "CREATE TABLE tbl1 (\n" - + " a bigint,\n" - + " h varchar, \n" - + " b varchar,\n" - + " PRIMARY KEY (a, b)\n" - + ")\n" - + distributionClause - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - } - - String buildDistributionOutput(final String distributionClause) { - return "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT,\n" - + " `H` VARCHAR,\n" - + " `B` VARCHAR,\n" - + " PRIMARY KEY (`A`, `B`)\n" - + ")\n" - + distributionClause - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - } - - @Test - void testCreateTableWithDistribution() { - final String sql = buildDistributionInput("DISTRIBUTED BY HASH(a, h) INTO 6 BUCKETS"); - final String expected = - buildDistributionOutput("DISTRIBUTED BY HASH(`A`, `H`) INTO 6 BUCKETS\n"); - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithRangeDistribution() { - final String sql = buildDistributionInput("DISTRIBUTED BY RANGE(a, h) INTO 6 BUCKETS\n"); - final String expected = - buildDistributionOutput("DISTRIBUTED BY RANGE(`A`, `H`) INTO 6 BUCKETS\n"); - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithRandomDistribution() { - final String sql = buildDistributionInput("DISTRIBUTED BY ^RANDOM^(a, h) INTO 6 BUCKETS\n"); - sql(sql).fails("(?s).*Encountered \"RANDOM\" at line 7, column 16.*"); - } - - @Test - void testCreateTableWithDistributionNoAlgorithm() { - final String sql = buildDistributionInput("DISTRIBUTED BY (a, h) INTO 6 BUCKETS\n"); - final String expected = - buildDistributionOutput("DISTRIBUTED BY (`A`, `H`) INTO 6 BUCKETS\n"); - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithDistributionAlgorithmWithoutBuckets() { - final String sql = buildDistributionInput("DISTRIBUTED BY RANGE(a, h)\n"); - final String expected = buildDistributionOutput("DISTRIBUTED BY RANGE(`A`, `H`)\n"); - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithDistributionNoAlgorithmWithoutBuckets() { - final String sql = buildDistributionInput("DISTRIBUTED BY (a, h)\n"); - final String expected = buildDistributionOutput("DISTRIBUTED BY (`A`, `H`)\n"); - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithDistributionIntoBuckets() { - final String sql = buildDistributionInput("DISTRIBUTED INTO 3 BUCKETS\n"); - final String expected = buildDistributionOutput("DISTRIBUTED INTO 3 BUCKETS\n"); - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithDistributionIntoNegativeBuckets() { - final String sql = buildDistributionInput("DISTRIBUTED INTO ^-^3 BUCKETS\n"); - sql(sql).fails("(?s).*Encountered \"-\" at line 7, column 18.*"); - } - - @Test - void testCreateTableWithDistributionIntoDecimalBuckets() { - final String sql = buildDistributionInput("DISTRIBUTED INTO ^3.2^ BUCKETS\n"); - sql(sql).fails("(?s).*Bucket count must be a positive integer.*"); - } - - @Test - void testCreateTableWithBadDistribution() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " a bigint,\n" - + " h varchar, \n" - + " g as 2 * (a + 1), \n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" - + " b varchar,\n" - + " proc as PROCTIME(), \n" - + " meta STRING METADATA, \n" - + " my_meta STRING METADATA FROM 'meta', \n" - + " my_meta STRING METADATA FROM 'meta' VIRTUAL, \n" - + " meta STRING METADATA VIRTUAL, \n" - + " PRIMARY KEY (a, b)\n" - + ")\n" - + "DISTRIBUTED \n" - + " ^with^ (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - sql(sql).fails("(?s).*Encountered \"with\" at line 15, column 3.*"); - } - - @Test - void testCreateTableWithDistributionIfNotExists() { - final String sql = - "CREATE TABLE if not exists tbl1 (\n" - + " a bigint,\n" - + " h varchar, \n" - + " PRIMARY KEY (a, b)\n" - + ")\n" - + "DISTRIBUTED BY HASH(a, h) INTO 6 BUCKETS" - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected = - "CREATE TABLE IF NOT EXISTS `TBL1` (\n" - + " `A` BIGINT,\n" - + " `H` VARCHAR,\n" - + " PRIMARY KEY (`A`, `B`)\n" - + ")\n" - + "DISTRIBUTED BY HASH(`A`, `H`) INTO 6 BUCKETS\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableIfNotExists() { - final String sql = - "CREATE TABLE IF NOT EXISTS tbl1 (\n" - + " a bigint,\n" - + " h varchar, \n" - + " PRIMARY KEY (a, b)\n" - + ")\n" - + "PARTITIONED BY (a, h)\n" - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected = - "CREATE TABLE IF NOT EXISTS `TBL1` (\n" - + " `A` BIGINT,\n" - + " `H` VARCHAR,\n" - + " PRIMARY KEY (`A`, `B`)\n" - + ")\n" - + "PARTITIONED BY (`A`, `H`)\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithComment() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " a bigint comment 'test column comment AAA.',\n" - + " h varchar, \n" - + " g as 2 * (a + 1), \n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" - + " b varchar,\n" - + " proc as PROCTIME(), \n" - + " meta STRING METADATA COMMENT 'c1', \n" - + " my_meta STRING METADATA FROM 'meta' COMMENT 'c2', \n" - + " my_meta STRING METADATA FROM 'meta' VIRTUAL COMMENT 'c3', \n" - + " meta STRING METADATA VIRTUAL COMMENT 'c4', \n" - + " PRIMARY KEY (a, b)\n" - + ")\n" - + "comment 'test table comment ABC.'\n" - + "PARTITIONED BY (a, h)\n" - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT COMMENT 'test column comment AAA.',\n" - + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`(),\n" - + " `META` STRING METADATA COMMENT 'c1',\n" - + " `MY_META` STRING METADATA FROM 'meta' COMMENT 'c2',\n" - + " `MY_META` STRING METADATA FROM 'meta' VIRTUAL COMMENT 'c3',\n" - + " `META` STRING METADATA VIRTUAL COMMENT 'c4',\n" - + " PRIMARY KEY (`A`, `B`)\n" - + ")\n" - + "COMMENT 'test table comment ABC.'\n" - + "PARTITIONED BY (`A`, `H`)\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithCommentOnComputedColumn() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " a bigint comment 'test column comment AAA.',\n" - + " h varchar, \n" - + " g as 2 * (a + 1) comment 'test computed column.', \n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" - + " b varchar,\n" - + " proc as PROCTIME(), \n" - + " PRIMARY KEY (a, b)\n" - + ")\n" - + "comment 'test table comment ABC.'\n" - + "PARTITIONED BY (a, h)\n" - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT COMMENT 'test column comment AAA.',\n" - + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)) COMMENT 'test computed column.',\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`(),\n" - + " PRIMARY KEY (`A`, `B`)\n" - + ")\n" - + "COMMENT 'test table comment ABC.'\n" - + "PARTITIONED BY (`A`, `H`)\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testTableConstraints() { - final String sql1 = - "CREATE TABLE tbl1 (\n" - + " a bigint,\n" - + " h varchar, \n" - + " g as 2 * (a + 1),\n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" - + " b varchar,\n" - + " proc as PROCTIME(),\n" - + " PRIMARY KEY (a, b),\n" - + " UNIQUE (h, g)\n" - + ") with (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected1 = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT,\n" - + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`(),\n" - + " PRIMARY KEY (`A`, `B`),\n" - + " UNIQUE (`H`, `G`)\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql1) - .ok(expected1) - .node( - new ValidationMatcher() - .fails( - "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. " - + "ENFORCED/NOT ENFORCED controls if the constraint checks are performed on the incoming/outgoing data. " - + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); - - final String sql2 = - "CREATE TABLE tbl1 (\n" - + " a bigint,\n" - + " h varchar, \n" - + " g as 2 * (a + 1),\n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" - + " b varchar,\n" - + " proc as PROCTIME(),\n" - + " PRIMARY KEY (a, b) NOT ENFORCED,\n" - + " UNIQUE (h, g)\n" - + ") with (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected2 = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT,\n" - + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`(),\n" - + " PRIMARY KEY (`A`, `B`) NOT ENFORCED,\n" - + " UNIQUE (`H`, `G`)\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql2) - .ok(expected2) - .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); - - final String sql3 = - "CREATE TABLE tbl1 (\n" - + " a bigint,\n" - + " h varchar, \n" - + " g as 2 * (a + 1),\n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" - + " b varchar,\n" - + " proc as PROCTIME(),\n" - + " PRIMARY KEY (a, b) NOT ENFORCED\n" - + ") with (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expectParsed = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT,\n" - + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`(),\n" - + " PRIMARY KEY (`A`, `B`) NOT ENFORCED\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - final String expectValidated = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT NOT NULL,\n" - + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR NOT NULL,\n" - + " `PROC` AS `PROCTIME`(),\n" - + " PRIMARY KEY (`A`, `B`) NOT ENFORCED\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql3).ok(expectParsed).node(validated(expectValidated)); - } - - @Test - void testColumnConstraints() { - final String sql1 = - "CREATE TABLE tbl1 (\n" - + " a bigint primary key,\n" - + " h varchar unique,\n" - + " g as 2 * (a + 1),\n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" - + " b varchar,\n" - + " proc as PROCTIME()\n" - + ") with (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected1 = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT PRIMARY KEY,\n" - + " `H` VARCHAR UNIQUE,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`()\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql1) - .ok(expected1) - .node( - new ValidationMatcher() - .fails( - "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. " - + "ENFORCED/NOT ENFORCED controls if the constraint checks are performed on the incoming/outgoing data. " - + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); - - final String sql2 = - "CREATE TABLE tbl1 (\n" - + " a bigint primary key not enforced,\n" - + " h varchar unique,\n" - + " g as 2 * (a + 1),\n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" - + " b varchar,\n" - + " proc as PROCTIME()\n" - + ") with (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected2 = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT PRIMARY KEY NOT ENFORCED,\n" - + " `H` VARCHAR UNIQUE,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`()\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql2) - .ok(expected2) - .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); - - final String sql3 = - "CREATE TABLE tbl1 (\n" - + " a bigint primary key not enforced,\n" - + " h varchar,\n" - + " g as 2 * (a + 1),\n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" - + " b varchar,\n" - + " proc as PROCTIME()\n" - + ") with (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expectParsed = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT PRIMARY KEY NOT ENFORCED,\n" - + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`()\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - final String expectValidated = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT NOT NULL PRIMARY KEY NOT ENFORCED,\n" - + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`()\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql3).ok(expectParsed).node(validated(expectValidated)); - } - - @Test - void testUniqueTableConstraint() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " a bigint,\n" - + " h varchar, \n" - + " g as 2 * (a + 1),\n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" - + " b varchar,\n" - + " proc as PROCTIME(),\n" - + " PRIMARY KEY (a, b) NOT ENFORCED,\n" - + " UNIQUE (h, g)\n" - + ") with (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT,\n" - + " `H` VARCHAR,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR,\n" - + " `PROC` AS `PROCTIME`(),\n" - + " PRIMARY KEY (`A`, `B`) NOT ENFORCED,\n" - + " UNIQUE (`H`, `G`)\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql).ok(expected); - sql(sql).node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); - } - - @Test - void testTableConstraintsWithEnforcement() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " a bigint primary key enforced comment 'test column comment AAA.',\n" - + " h varchar constraint ct1 unique not enforced,\n" - + " g as 2 * (a + 1), \n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" - + " b varchar constraint ct2 unique,\n" - + " proc as PROCTIME(),\n" - + " unique (g, ts) not enforced" - + ") with (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `A` BIGINT PRIMARY KEY ENFORCED COMMENT 'test column comment AAA.',\n" - + " `H` VARCHAR CONSTRAINT `CT1` UNIQUE NOT ENFORCED,\n" - + " `G` AS (2 * (`A` + 1)),\n" - + " `TS` AS `TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss'),\n" - + " `B` VARCHAR CONSTRAINT `CT2` UNIQUE,\n" - + " `PROC` AS `PROCTIME`(),\n" - + " UNIQUE (`G`, `TS`) NOT ENFORCED\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testDuplicatePk() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " a bigint comment 'test column comment AAA.',\n" - + " h varchar constraint ct1 primary key,\n" - + " g as 2 * (a + 1), \n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'),\n" - + " b varchar,\n" - + " proc as PROCTIME(),\n" - + " constraint ct2 primary key (b, h)" - + ") with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - sql(sql).node(new ValidationMatcher().fails("Duplicate primary key definition")); - } - - @Test - void testCreateTableWithWatermark() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " ts timestamp(3),\n" - + " id varchar, \n" - + " watermark FOR ts AS ts - interval '3' second\n" - + ")\n" - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `TS` TIMESTAMP(3),\n" - + " `ID` VARCHAR,\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '3' SECOND)\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithWatermarkOnComputedColumn() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " log_ts varchar,\n" - + " ts as to_timestamp(log_ts), \n" - + " WATERMARK FOR ts AS ts + interval '1' second\n" - + ")\n" - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `LOG_TS` VARCHAR,\n" - + " `TS` AS `TO_TIMESTAMP`(`LOG_TS`),\n" - + " WATERMARK FOR `TS` AS (`TS` + INTERVAL '1' SECOND)\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithWatermarkOnNestedField() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " f1 row, q3 boolean>,\n" - + " WATERMARK FOR f1.q2.t1 AS NOW()\n" - + ")\n" - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `F1` ROW< `Q1` BIGINT, `Q2` ROW< `T1` TIMESTAMP, `T2` VARCHAR >, `Q3` BOOLEAN >,\n" - + " WATERMARK FOR `F1`.`Q2`.`T1` AS `NOW`()\n" - + ")\n" - + "WITH (\n" - + " 'connector' = 'kafka',\n" - + " 'kafka.topic' = 'log.test'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithMultipleWatermark() { - String sql = - "CREATE TABLE tbl1 (\n" - + " f0 bigint,\n" - + " f1 varchar,\n" - + " f2 boolean,\n" - + " WATERMARK FOR f0 AS NOW(),\n" - + " ^WATERMARK^ FOR f1 AS NOW()\n" - + ")\n" - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - sql(sql).fails("Multiple WATERMARK declarations are not supported yet."); - } - - @Test - void testCreateTableWithQueryWatermarkExpression() { - String sql = - "CREATE TABLE tbl1 (\n" - + " f0 bigint,\n" - + " f1 varchar,\n" - + " f2 boolean,\n" - + " WATERMARK FOR f0 AS ^(^SELECT f1 FROM tbl1)\n" - + ")\n" - + " with (\n" - + " 'connector' = 'kafka', \n" - + " 'kafka.topic' = 'log.test'\n" - + ")\n"; - sql(sql).fails("Query expression encountered in illegal context"); - } - - @Test - void testCreateTableWithComplexType() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " a ARRAY, \n" - + " b MAP,\n" - + " c ROW,\n" - + " d MULTISET,\n" - + " PRIMARY KEY (a, b) \n" - + ") with (\n" - + " 'x' = 'y', \n" - + " 'asd' = 'data'\n" - + ")\n"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `A` ARRAY< BIGINT >,\n" - + " `B` MAP< INTEGER, VARCHAR >,\n" - + " `C` ROW< `CC0` INTEGER, `CC1` FLOAT, `CC2` VARCHAR >,\n" - + " `D` MULTISET< VARCHAR >,\n" - + " PRIMARY KEY (`A`, `B`)\n" - + ")\n" - + "WITH (\n" - + " 'x' = 'y',\n" - + " 'asd' = 'data'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithNestedComplexType() { - final String sql = - "CREATE TABLE tbl1 (\n" - + " a ARRAY>, \n" - + " b MAP, ARRAY>,\n" - + " c ROW, cc1 float, cc2 varchar>,\n" - + " d MULTISET>,\n" - + " PRIMARY KEY (a, b) \n" - + ") with (\n" - + " 'x' = 'y', \n" - + " 'asd' = 'data'\n" - + ")\n"; - final String expected = - "CREATE TABLE `TBL1` (\n" - + " `A` ARRAY< ARRAY< BIGINT > >,\n" - + " `B` MAP< MAP< INTEGER, VARCHAR >, ARRAY< VARCHAR > >,\n" - + " `C` ROW< `CC0` ARRAY< INTEGER >, `CC1` FLOAT, `CC2` VARCHAR >,\n" - + " `D` MULTISET< ARRAY< INTEGER > >,\n" - + " PRIMARY KEY (`A`, `B`)\n" - + ")\n" - + "WITH (\n" - + " 'x' = 'y',\n" - + " 'asd' = 'data'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithUserDefinedType() { - final String sql = - "create table t(\n" - + " a catalog1.db1.MyType1,\n" - + " b db2.MyType2\n" - + ") with (\n" - + " 'k1' = 'v1',\n" - + " 'k2' = 'v2'\n" - + ")"; - final String expected = - "CREATE TABLE `T` (\n" - + " `A` `CATALOG1`.`DB1`.`MYTYPE1`,\n" - + " `B` `DB2`.`MYTYPE2`\n" - + ")\n" - + "WITH (\n" - + " 'k1' = 'v1',\n" - + " 'k2' = 'v2'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testInvalidComputedColumn() { - final String sql0 = - "CREATE TABLE t1 (\n" - + " a bigint, \n" - + " b varchar,\n" - + " toTimestamp^(^b, 'yyyy-MM-dd HH:mm:ss'), \n" - + " PRIMARY KEY (a, b) \n" - + ") with (\n" - + " 'x' = 'y', \n" - + " 'asd' = 'data'\n" - + ")\n"; - final String expect0 = - "(?s).*Encountered \"\\(\" at line 4, column 14.\n" - + "Was expecting one of:\n" - + " \"AS\" ...\n" - + " \".\" ...\n" - + " \"STRING\" ...\n" - + ".*"; - sql(sql0).fails(expect0); - // Sub-query computed column expression is forbidden. - final String sql1 = - "CREATE TABLE t1 (\n" - + " a bigint, \n" - + " b varchar,\n" - + " c as ^(^select max(d) from t2), \n" - + " PRIMARY KEY (a, b) \n" - + ") with (\n" - + " 'x' = 'y', \n" - + " 'asd' = 'data'\n" - + ")\n"; - final String expect1 = "(?s).*Query expression encountered in illegal context.*"; - sql(sql1).fails(expect1); - } - - @Test - void testColumnSqlString() { - final String sql = - "CREATE TABLE sls_stream (\n" - + " a bigint, \n" - + " f as a + 1, \n" - + " b varchar,\n" - + " ts as toTimestamp(b, 'yyyy-MM-dd HH:mm:ss'), \n" - + " proc as PROCTIME(),\n" - + " c int,\n" - + " PRIMARY KEY (a, b) \n" - + ") with (\n" - + " 'x' = 'y', \n" - + " 'asd' = 'data'\n" - + ")\n"; - final String expected = - "`A`, (`A` + 1) AS `F`, `B`, " - + "`TOTIMESTAMP`(`B`, 'yyyy-MM-dd HH:mm:ss') AS `TS`, " - + "`PROCTIME`() AS `PROC`, `C`"; - sql(sql).node(new ValidationMatcher().expectColumnSql(expected)); - } - - @Test - void testCreateTableWithMinusInOptionKey() { - final String sql = - "create table source_table(\n" - + " a int,\n" - + " b bigint,\n" - + " c string\n" - + ") with (\n" - + " 'a-b-c-d124' = 'ab',\n" - + " 'a.b.1.c' = 'aabb',\n" - + " 'a.b-c-connector.e-f.g' = 'ada',\n" - + " 'a.b-c-d.e-1231.g' = 'ada',\n" - + " 'a.b-c-d.*' = 'adad')\n"; - final String expected = - "CREATE TABLE `SOURCE_TABLE` (\n" - + " `A` INTEGER,\n" - + " `B` BIGINT,\n" - + " `C` STRING\n" - + ")\n" - + "WITH (\n" - + " 'a-b-c-d124' = 'ab',\n" - + " 'a.b.1.c' = 'aabb',\n" - + " 'a.b-c-connector.e-f.g' = 'ada',\n" - + " 'a.b-c-d.e-1231.g' = 'ada',\n" - + " 'a.b-c-d.*' = 'adad'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithOptionKeyAsIdentifier() { - final String sql = - "create table source_table(\n" - + " a int,\n" - + " b bigint,\n" - + " c string\n" - + ") with (\n" - + " ^a^.b.c = 'ab',\n" - + " a.b.c1 = 'aabb')\n"; - sql(sql).fails("(?s).*Encountered \"a\" at line 6, column 3.\n.*"); - } - - @Test - void testCreateTableLikeWithoutOption() { - final String sql = - "create table source_table(\n" - + " a int,\n" - + " b bigint,\n" - + " c string\n" - + ")\n" - + "LIKE parent_table"; - final String expected = - "CREATE TABLE `SOURCE_TABLE` (\n" - + " `A` INTEGER,\n" - + " `B` BIGINT,\n" - + " `C` STRING\n" - + ")\n" - + "LIKE `PARENT_TABLE`"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableLikeWithConstraints() { - final String sql1 = - "create table source_table(\n" - + " a int primary key,\n" - + " b bigint,\n" - + " c string\n" - + ")\n" - + "LIKE parent_table"; - sql(sql1) - .node( - new ValidationMatcher() - .fails( - "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. ENFORCED/NOT ENFORCED " - + "controls if the constraint checks are performed on the incoming/outgoing data. " - + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); - - final String sql2 = - "create table source_table(\n" - + " a int primary key,\n" - + " b bigint,\n" - + " c string,\n" - + " primary key(a) not enforced\n" - + ")\n" - + "LIKE parent_table"; - - sql(sql2).node(new ValidationMatcher().fails("Duplicate primary key definition")); - - final String sql3 = - "create table source_table(\n" - + " a int,\n" - + " b bigint,\n" - + " c string,\n" - + " unique (a)\n" - + ")\n" - + "LIKE parent_table"; - - sql(sql3).node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); - } - - @Test - void testCreateTableWithLikeClause() { - final String sql = - "create table source_table(\n" - + " a int,\n" - + " b bigint,\n" - + " c string\n" - + ")\n" - + "LIKE parent_table (\n" - + " INCLUDING ALL\n" - + " OVERWRITING OPTIONS\n" - + " EXCLUDING PARTITIONS\n" - + " INCLUDING GENERATED\n" - + " INCLUDING METADATA\n" - + ")"; - final String expected = - "CREATE TABLE `SOURCE_TABLE` (\n" - + " `A` INTEGER,\n" - + " `B` BIGINT,\n" - + " `C` STRING\n" - + ")\n" - + "LIKE `PARENT_TABLE` (\n" - + " INCLUDING ALL\n" - + " OVERWRITING OPTIONS\n" - + " EXCLUDING PARTITIONS\n" - + " INCLUDING GENERATED\n" - + " INCLUDING METADATA\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithLikeClauseIncludingDistribution() { - final String sql = - "create table source_table(\n" - + " a int,\n" - + " b bigint,\n" - + " c string\n" - + ")\n" - + "LIKE parent_table (\n" - + " INCLUDING ALL\n" - + " OVERWRITING OPTIONS\n" - + " INCLUDING DISTRIBUTION\n" - + " EXCLUDING PARTITIONS\n" - + " INCLUDING GENERATED\n" - + " INCLUDING METADATA\n" - + ")"; - final String expected = - "CREATE TABLE `SOURCE_TABLE` (\n" - + " `A` INTEGER,\n" - + " `B` BIGINT,\n" - + " `C` STRING\n" - + ")\n" - + "LIKE `PARENT_TABLE` (\n" - + " INCLUDING ALL\n" - + " OVERWRITING OPTIONS\n" - + " INCLUDING DISTRIBUTION\n" - + " EXCLUDING PARTITIONS\n" - + " INCLUDING GENERATED\n" - + " INCLUDING METADATA\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithLikeClauseExcludingDistribution() { - final String sql = - "create table source_table(\n" - + " a int,\n" - + " b bigint,\n" - + " c string\n" - + ")\n" - + "LIKE parent_table (\n" - + " INCLUDING ALL\n" - + " OVERWRITING OPTIONS\n" - + " EXCLUDING DISTRIBUTION\n" - + " INCLUDING PARTITIONS\n" - + " INCLUDING GENERATED\n" - + " INCLUDING METADATA\n" - + ")"; - final String expected = - "CREATE TABLE `SOURCE_TABLE` (\n" - + " `A` INTEGER,\n" - + " `B` BIGINT,\n" - + " `C` STRING\n" - + ")\n" - + "LIKE `PARENT_TABLE` (\n" - + " INCLUDING ALL\n" - + " OVERWRITING OPTIONS\n" - + " EXCLUDING DISTRIBUTION\n" - + " INCLUDING PARTITIONS\n" - + " INCLUDING GENERATED\n" - + " INCLUDING METADATA\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithLikeClauseWithoutColumns() { - final String sql = - "" - + "create TEMPORARY table source_table (\n" - + " WATERMARK FOR ts AS ts - INTERVAL '5' SECOND\n" - + ") with (\n" - + " 'scan.startup.mode' = 'specific-offsets',\n" - + " 'scan.startup.specific-offsets' = 'partition:0,offset:1169129'\n" - + ") like t_order_course (\n" - + " OVERWRITING WATERMARKS\n" - + " OVERWRITING OPTIONS\n" - + " EXCLUDING CONSTRAINTS\n" - + ")"; - final String expected = - "CREATE TEMPORARY TABLE `SOURCE_TABLE` (\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '5' SECOND)\n" - + ")\n" - + "WITH (\n" - + " 'scan.startup.mode' = 'specific-offsets',\n" - + " 'scan.startup.specific-offsets' = 'partition:0,offset:1169129'\n" - + ")\n" - + "LIKE `T_ORDER_COURSE` (\n" - + " OVERWRITING WATERMARKS\n" - + " OVERWRITING OPTIONS\n" - + " EXCLUDING CONSTRAINTS\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTemporaryTable() { - final String sql = - "create temporary table source_table(\n" - + " a int,\n" - + " b bigint,\n" - + " c string\n" - + ") with (\n" - + " 'x' = 'y',\n" - + " 'abc' = 'def'\n" - + ")"; - final String expected = - "CREATE TEMPORARY TABLE `SOURCE_TABLE` (\n" - + " `A` INTEGER,\n" - + " `B` BIGINT,\n" - + " `C` STRING\n" - + ")\n" - + "WITH (\n" - + " 'x' = 'y',\n" - + " 'abc' = 'def'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithNoColumns() { - final String sql = - "create table source_table with (\n" + " 'x' = 'y',\n" + " 'abc' = 'def'\n" + ")"; - final String expected = - "CREATE TABLE `SOURCE_TABLE`\nWITH (\n" - + " 'x' = 'y',\n" - + " 'abc' = 'def'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testCreateTableWithOnlyWaterMark() { - final String sql = - "create table source_table (\n" - + " watermark FOR ts AS ts - interval '3' second\n" - + ") with (\n" - + " 'x' = 'y',\n" - + " 'abc' = 'def'\n" - + ")"; - final String expected = - "CREATE TABLE `SOURCE_TABLE` (\n" - + " WATERMARK FOR `TS` AS (`TS` - INTERVAL '3' SECOND)\n" - + ")\n" - + "WITH (\n" - + " 'x' = 'y',\n" - + " 'abc' = 'def'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testDropTable() { - final String sql = "DROP table catalog1.db1.tbl1"; - final String expected = "DROP TABLE `CATALOG1`.`DB1`.`TBL1`"; - sql(sql).ok(expected); - } - - @Test - void testDropIfExists() { - final String sql = "DROP table IF EXISTS catalog1.db1.tbl1"; - final String expected = "DROP TABLE IF EXISTS `CATALOG1`.`DB1`.`TBL1`"; - sql(sql).ok(expected); - } - - @Test - void testTemporaryDropTable() { - final String sql = "DROP temporary table catalog1.db1.tbl1"; - final String expected = "DROP TEMPORARY TABLE `CATALOG1`.`DB1`.`TBL1`"; - sql(sql).ok(expected); - } - - @Test - void testDropTemporaryIfExists() { - final String sql = "DROP temporary table IF EXISTS catalog1.db1.tbl1"; - final String expected = "DROP TEMPORARY TABLE IF EXISTS `CATALOG1`.`DB1`.`TBL1`"; - sql(sql).ok(expected); - } - - @Test - void testInsertPartitionSpecs() { - final String sql1 = "insert into emps partition (x='ab', y='bc') (x,y) select * from emps"; - final String expected = - "INSERT INTO `EMPS` " - + "PARTITION (`X` = 'ab', `Y` = 'bc')\n" - + "(`X`, `Y`)\n" - + "SELECT *\n" - + "FROM `EMPS`"; - sql(sql1).ok(expected); - final String sql2 = - "insert into emp\n" - + "partition(empno='1', job='job')\n" - + "(empno, ename, job, mgr, hiredate,\n" - + " sal, comm, deptno, slacker)\n" - + "select 'nom', 0, timestamp '1970-01-01 00:00:00',\n" - + " 1, 1, 1, false\n" - + "from (values 'a')"; - sql(sql2) - .ok( - "INSERT INTO `EMP` " - + "PARTITION (`EMPNO` = '1', `JOB` = 'job')\n" - + "(`EMPNO`, `ENAME`, `JOB`, `MGR`, `HIREDATE`, `SAL`," - + " `COMM`, `DEPTNO`, `SLACKER`)\n" - + "SELECT 'nom', 0, TIMESTAMP '1970-01-01 00:00:00', 1, 1, 1, FALSE\n" - + "FROM (VALUES (ROW('a')))"); - final String sql3 = - "insert into empnullables\n" - + "partition(ename='b')\n" - + "(empno, ename)\n" - + "select 1 from (values 'a')"; - sql(sql3) - .ok( - "INSERT INTO `EMPNULLABLES` " - + "PARTITION (`ENAME` = 'b')\n" - + "(`EMPNO`, `ENAME`)\n" - + "SELECT 1\n" - + "FROM (VALUES (ROW('a')))"); - } - - @Test - void testInsertCaseSensitivePartitionSpecs() { - final String expected = - "INSERT INTO `emps` " - + "PARTITION (`x` = 'ab', `y` = 'bc')\n" - + "(`x`, `y`)\n" - + "SELECT *\n" - + "FROM `EMPS`"; - sql("insert into \"emps\" " - + "partition (\"x\"='ab', \"y\"='bc')(\"x\",\"y\") select * from emps") - .ok(expected); - } - - @Test - void testInsertExtendedColumnAsStaticPartition1() { - final String expected = - "INSERT INTO `EMPS` EXTEND (`Z` BOOLEAN) " - + "PARTITION (`Z` = 'ab')\n" - + "(`X`, `Y`)\n" - + "SELECT *\n" - + "FROM `EMPS`"; - sql("insert into emps(z boolean) partition (z='ab') (x,y) select * from emps").ok(expected); - } - - @Test - void testInsertExtendedColumnAsStaticPartition2() { - assertThatThrownBy( - () -> - sql("insert into emps(x, y, z boolean) partition (z='ab') select * from emps") - .node( - new ValidationMatcher() - .fails( - "Extended columns not allowed under the current SQL conformance level"))) - .isInstanceOf(SqlParseException.class); - } - - @Test - void testInsertOverwrite() { - // non-partitioned - final String sql = "INSERT OVERWRITE myDB.myTbl SELECT * FROM src"; - final String expected = "INSERT OVERWRITE `MYDB`.`MYTBL`\n" + "SELECT *\n" + "FROM `SRC`"; - sql(sql).ok(expected); - - // partitioned - final String sql1 = "INSERT OVERWRITE myTbl PARTITION (p1='v1',p2='v2') SELECT * FROM src"; - final String expected1 = - "INSERT OVERWRITE `MYTBL` " - + "PARTITION (`P1` = 'v1', `P2` = 'v2')\n" - + "\n" - + "SELECT *\n" - + "FROM `SRC`"; - sql(sql1).ok(expected1); - } - - @Test - void testInvalidUpsertOverwrite() { - sql("UPSERT ^OVERWRITE^ myDB.myTbl SELECT * FROM src") - .fails("OVERWRITE expression is only used with INSERT statement."); - } - - @Test - void testInsertOnConflict() { - // ON CONFLICT DO ERROR - sql("INSERT INTO t1 SELECT * FROM t2 ON CONFLICT DO ERROR") - .ok("INSERT INTO `T1`\nSELECT *\nFROM `T2`\nON CONFLICT DO ERROR"); - - // ON CONFLICT DO NOTHING - sql("INSERT INTO t1 SELECT * FROM t2 ON CONFLICT DO NOTHING") - .ok("INSERT INTO `T1`\nSELECT *\nFROM `T2`\nON CONFLICT DO NOTHING"); - - // ON CONFLICT DO DEDUPLICATE - sql("INSERT INTO t1 SELECT * FROM t2 ON CONFLICT DO DEDUPLICATE") - .ok("INSERT INTO `T1`\nSELECT *\nFROM `T2`\nON CONFLICT DO DEDUPLICATE"); - - // ON CONFLICT with partition - sql("INSERT INTO t1 PARTITION (p='v') SELECT * FROM t2 ON CONFLICT DO ERROR") - .ok( - "INSERT INTO `T1` PARTITION (`P` = 'v')\n\nSELECT *\nFROM `T2`\nON CONFLICT DO ERROR"); - - // ON CONFLICT with INSERT OVERWRITE (should work) - sql("INSERT OVERWRITE t1 SELECT * FROM t2 ON CONFLICT DO NOTHING") - .ok("INSERT OVERWRITE `T1`\nSELECT *\nFROM `T2`\nON CONFLICT DO NOTHING"); - - // Invalid ON CONFLICT strategy - sql("INSERT INTO t1 SELECT * FROM t2 ON CONFLICT DO ^UPDATE^") - .fails("(?s).*Encountered \"UPDATE\" at line 1, column 48.\n.*"); - } - - @Test - void testCreateView() { - final String sql = "create view v as select col1 from tbl"; - final String expected = "CREATE VIEW `V`\n" + "AS\n" + "SELECT `COL1`\n" + "FROM `TBL`"; - sql(sql).ok(expected); - } - - @Test - void testCreateViewWithInvalidFieldList() { - final String expected = - "(?s).*Encountered \"\\)\" at line 1, column 15.\n" - + "Was expecting one of:\n" - + ".*\n" - + ".*\n" - + ".*\n" - + ".*\n" - + ".*"; - sql("CREATE VIEW V(^)^ AS SELECT * FROM TBL").fails(expected); - } - - @Test - void testCreateViewWithComment() { - final String sql = "create view v COMMENT 'this is a view' as select col1 from tbl"; - final String expected = - "CREATE VIEW `V`\n" - + "COMMENT 'this is a view'\n" - + "AS\n" - + "SELECT `COL1`\n" - + "FROM `TBL`"; - sql(sql).ok(expected); - } - - @Test - void testCreateViewWithFieldNames() { - final String sql = "create view v(col1, col2) as select col3, col4 from tbl"; - final String expected = - "CREATE VIEW `V` (`COL1`, `COL2`)\n" - + "AS\n" - + "SELECT `COL3`, `COL4`\n" - + "FROM `TBL`"; - sql(sql).ok(expected); - } - - @Test - void testCreateViewWithInvalidName() { - final String sql = "create view v(^*^) COMMENT 'this is a view' as select col1 from tbl"; - final String expected = "(?s).*Encountered \"\\*\" at line 1, column 15.*"; - - sql(sql).fails(expected); - } - - @Test - void testCreateTemporaryView() { - final String sql = "create temporary view v as select col1 from tbl"; - final String expected = - "CREATE TEMPORARY VIEW `V`\n" + "AS\n" + "SELECT `COL1`\n" + "FROM `TBL`"; - sql(sql).ok(expected); - } - - @Test - void testCreateTemporaryViewIfNotExists() { - final String sql = "create temporary view if not exists v as select col1 from tbl"; - final String expected = - "CREATE TEMPORARY VIEW IF NOT EXISTS `V`\n" - + "AS\n" - + "SELECT `COL1`\n" - + "FROM `TBL`"; - sql(sql).ok(expected); - } - - @Test - void testCreateViewIfNotExists() { - final String sql = "create view if not exists v as select col1 from tbl"; - final String expected = - "CREATE VIEW IF NOT EXISTS `V`\n" + "AS\n" + "SELECT `COL1`\n" + "FROM `TBL`"; - sql(sql).ok(expected); - } - - @Test - void testDropView() { - final String sql = "DROP VIEW IF EXISTS view_name"; - final String expected = "DROP VIEW IF EXISTS `VIEW_NAME`"; - sql(sql).ok(expected); - } - - @Test - void testDropTemporaryView() { - final String sql = "DROP TEMPORARY VIEW IF EXISTS view_name"; - final String expected = "DROP TEMPORARY VIEW IF EXISTS `VIEW_NAME`"; - sql(sql).ok(expected); - } - - @Test - void testAlterView() { - sql("ALTER VIEW v1 RENAME TO v2").ok("ALTER VIEW `V1` RENAME TO `V2`"); - sql("ALTER VIEW v1 AS SELECT c1, c2 FROM tbl") - .ok("ALTER VIEW `V1`\n" + "AS\n" + "SELECT `C1`, `C2`\n" + "FROM `TBL`"); - } - - @Test - void testShowViews() { - sql("show views").ok("SHOW VIEWS"); - sql("show views not like '%'").ok("SHOW VIEWS NOT LIKE '%'"); - - sql("show views from db1").ok("SHOW VIEWS FROM `DB1`"); - sql("show views in db1").ok("SHOW VIEWS IN `DB1`"); - - sql("show views from catalog1.db1").ok("SHOW VIEWS FROM `CATALOG1`.`DB1`"); - sql("show views in catalog1.db1").ok("SHOW VIEWS IN `CATALOG1`.`DB1`"); - - sql("show views from db1 like '%'").ok("SHOW VIEWS FROM `DB1` LIKE '%'"); - sql("show views in db1 like '%'").ok("SHOW VIEWS IN `DB1` LIKE '%'"); - - sql("show views from catalog1.db1 like '%'") - .ok("SHOW VIEWS FROM `CATALOG1`.`DB1` LIKE '%'"); - sql("show views in catalog1.db1 like '%'").ok("SHOW VIEWS IN `CATALOG1`.`DB1` LIKE '%'"); - - sql("show views from db1 not like '%'").ok("SHOW VIEWS FROM `DB1` NOT LIKE '%'"); - sql("show views in db1 not like '%'").ok("SHOW VIEWS IN `DB1` NOT LIKE '%'"); - - sql("show views from catalog1.db1 not like '%'") - .ok("SHOW VIEWS FROM `CATALOG1`.`DB1` NOT LIKE '%'"); - sql("show views in catalog1.db1 not like '%'") - .ok("SHOW VIEWS IN `CATALOG1`.`DB1` NOT LIKE '%'"); - - sql("show views ^db1^").fails("(?s).*Encountered \"db1\" at line 1, column 12.\n.*"); - sql("show views ^catalog1^.db1") - .fails("(?s).*Encountered \"catalog1\" at line 1, column 12.\n.*"); - - sql("show views ^search^ db1") - .fails("(?s).*Encountered \"search\" at line 1, column 12.\n.*"); - - sql("show views from db1 ^likes^ '%t'") - .fails("(?s).*Encountered \"likes\" at line 1, column 21.\n.*"); - } - - @Test - void testShowPartitions() { - sql("show partitions c1.d1.tbl").ok("SHOW PARTITIONS `C1`.`D1`.`TBL`"); - sql("show partitions tbl partition (p=1)").ok("SHOW PARTITIONS `TBL` PARTITION (`P` = 1)"); - } - - // Override the test because our ROW field type default is nullable, - // which is different with Calcite. - @Test - void testCastAsRowType() { - final String expr = "cast(a as row(f0 int, f1 varchar))"; - final String expected = "CAST(`A` AS ROW(`F0` INTEGER, `F1` VARCHAR))"; - expr(expr).ok(expected); - - final String expr1 = "cast(a as row(f0 int not null, f1 varchar null))"; - final String expected1 = "CAST(`A` AS ROW(`F0` INTEGER NOT NULL, `F1` VARCHAR))"; - expr(expr1).ok(expected1); - - final String expr2 = - "cast(a as row(f0 row(ff0 int not null, ff1 varchar null) null," - + " f1 timestamp not null))"; - final String expected2 = - "CAST(`A` AS ROW(`F0` ROW(`FF0` INTEGER NOT NULL, `FF1` VARCHAR)," - + " `F1` TIMESTAMP NOT NULL))"; - expr(expr2).ok(expected2); - - final String expr3 = "cast(a as row(f0 bigint not null, f1 decimal null) array)"; - final String expected3 = "CAST(`A` AS ROW(`F0` BIGINT NOT NULL, `F1` DECIMAL) ARRAY)"; - expr(expr3).ok(expected3); - - final String expr4 = "cast(a as row(f0 varchar not null, f1 timestamp null) multiset)"; - final String expected4 = "CAST(`A` AS ROW(`F0` VARCHAR NOT NULL, `F1` TIMESTAMP) MULTISET)"; - expr(expr4).ok(expected4); - } - - @Test - void testCreateTableWithNakedTableName() { - String sql = "CREATE TABLE tbl1"; - sql(sql).node(new ValidationMatcher()); - } - - @Test - void testCreateViewWithEmptyFields() { - String sql = "CREATE VIEW v1 AS SELECT 1"; - sql(sql).ok("CREATE VIEW `V1`\n" + "AS\n" + "SELECT 1"); - } - - @Test - void testCreateFunction() { - sql("create function catalog1.db1.function1 as 'org.apache.flink.function.function1'") - .ok( - "CREATE FUNCTION `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.flink.function.function1'"); - - sql("create temporary function catalog1.db1.function1 as 'org.apache.flink.function.function1'") - .ok( - "CREATE TEMPORARY FUNCTION `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.flink.function.function1'"); - - sql("create temporary function db1.function1 as 'org.apache.flink.function.function1'") - .ok( - "CREATE TEMPORARY FUNCTION `DB1`.`FUNCTION1` AS 'org.apache.flink.function.function1'"); - - sql("create temporary function function1 as 'org.apache.flink.function.function1'") - .ok( - "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1'"); - - sql("create temporary function if not exists catalog1.db1.function1 as 'org.apache.flink.function.function1'") - .ok( - "CREATE TEMPORARY FUNCTION IF NOT EXISTS `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.flink.function.function1'"); - - sql("create temporary function function1 as 'org.apache.flink.function.function1' language java") - .ok( - "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA"); - - sql("create temporary system function function1 as 'org.apache.flink.function.function1' language scala") - .ok( - "CREATE TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE SCALA"); - - // Temporary system function always belongs to the system and current session. - sql("create temporary system function catalog1^.^db1.function1 as 'org.apache.flink.function.function1'") - .fails("(?s).*Encountered \".\" at.*"); - - sql("create ^system^ function function1 as 'org.apache.flink.function.function1'") - .fails( - "CREATE SYSTEM FUNCTION is not supported, " - + "system functions can only be registered as temporary " - + "functions, you can use CREATE TEMPORARY SYSTEM FUNCTION instead."); - - // test creating functions with either jar or artifact - for (String usageType : List.of("JAR", "ARTIFACT")) { - sql("create temporary function function1 as 'org.apache.flink.function.function1' language java using " - + usageType - + " 'file:///path/to/test.jar'") - .ok( - "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING " - + usageType - + " 'file:///path/to/test.jar'"); - - sql("create temporary function function1 as 'org.apache.flink.function.function1' language scala using " - + usageType - + " '/path/to/test.jar'") - .ok( - "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE SCALA USING " - + usageType - + " '/path/to/test.jar'"); - - sql("create temporary system function function1 as 'org.apache.flink.function.function1' language scala using " - + usageType - + " '/path/to/test.jar'") - .ok( - "CREATE TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE SCALA USING " - + usageType - + " '/path/to/test.jar'"); - - sql("create function function1 as 'org.apache.flink.function.function1' language java using " - + usageType - + " 'file:///path/to/test.jar', jar 'hdfs:///path/to/test2.jar'") - .ok( - "CREATE FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING " - + usageType - + " 'file:///path/to/test.jar', JAR 'hdfs:///path/to/test2.jar'"); - - sql("create temporary function function1 as 'org.apache.flink.function.function1' language ^sql^ using " - + usageType - + " 'file:///path/to/test.jar'") - .fails( - "CREATE FUNCTION USING JAR/ARTIFACT syntax is not applicable to SQL language."); - - sql("create temporary function function1 as 'org.apache.flink.function.function1' language ^python^ using " - + usageType - + " 'file:///path/to/test.jar'") - .fails( - "CREATE FUNCTION USING JAR/ARTIFACT syntax is not applicable to PYTHON language."); - - sql("create function function1 as 'org.apache.flink.function.function1' language java using " - + usageType - + " 'file:///path/to/test.jar' WITH ('k1' = 'v1', 'k2' = 'v2')") - .ok( - "CREATE FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING " - + usageType - + " 'file:///path/to/test.jar'\nWITH (\n" - + " 'k1' = 'v1',\n" - + " 'k2' = 'v2'\n" - + ")"); - - sql("create temporary function function1 as 'org.apache.flink.function.function1' language java using " - + usageType - + " 'file:///path/to/test.jar' WITH ('k1' = 'v1', 'k2' = 'v2')") - .ok( - "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING " - + usageType - + " 'file:///path/to/test.jar'\nWITH (\n" - + " 'k1' = 'v1',\n" - + " 'k2' = 'v2'\n" - + ")"); - } - - // test mixing jar and artifact keywords - sql("create function function1 as 'org.apache.flink.function.function1' language java using jar 'file:///path/to/test.jar', artifact 'hdfs:///path/to/test2.jar'") - .ok( - "CREATE FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING JAR 'file:///path/to/test.jar', ARTIFACT 'hdfs:///path/to/test2.jar'"); - - sql("create function function1 as 'org.apache.flink.function.function1' language java using artifact 'file:///path/to/test.jar', jar 'hdfs:///path/to/test2.jar'") - .ok( - "CREATE FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING ARTIFACT 'file:///path/to/test.jar', JAR 'hdfs:///path/to/test2.jar'"); - - sql("create temporary function function1 as 'org.apache.flink.function.function1' language java using ^file^ 'file:///path/to/test'") - .fails( - "Encountered \"file\" at line 1, column 98.\n" - + "Was expecting one of:\n" - + " \"ARTIFACT\" ...\n" - + " \"JAR\" ...\n" - + " .*"); - } - - @Test - void testDropTemporaryFunction() { - sql("drop temporary function catalog1.db1.function1") - .ok("DROP TEMPORARY FUNCTION `CATALOG1`.`DB1`.`FUNCTION1`"); - - sql("drop temporary system function catalog1.db1.function1") - .ok("DROP TEMPORARY SYSTEM FUNCTION `CATALOG1`.`DB1`.`FUNCTION1`"); - - sql("drop temporary function if exists catalog1.db1.function1") - .ok("DROP TEMPORARY FUNCTION IF EXISTS `CATALOG1`.`DB1`.`FUNCTION1`"); - - sql("drop temporary system function if exists catalog1.db1.function1") - .ok("DROP TEMPORARY SYSTEM FUNCTION IF EXISTS `CATALOG1`.`DB1`.`FUNCTION1`"); - } - - @Test - void testLoadModule() { - sql("load module core").ok("LOAD MODULE `CORE`"); - - sql("load module dummy with ('k1' = 'v1', 'k2' = 'v2')") - .ok( - "LOAD MODULE `DUMMY`" - + "\nWITH (\n" - + " 'k1' = 'v1',\n" - + " 'k2' = 'v2'\n" - + ")"); - - sql("load module ^'core'^") - .fails("(?s).*Encountered \"\\\\'core\\\\'\" at line 1, column 13.\n.*"); - } - - @Test - void testUnloadModule() { - sql("unload module core").ok("UNLOAD MODULE `CORE`"); - - sql("unload module ^'core'^") - .fails("(?s).*Encountered \"\\\\'core\\\\'\" at line 1, column 15.\n.*"); - } - - @Test - void testUseModules() { - sql("use modules core").ok("USE MODULES `CORE`"); - - sql("use modules x, y, z").ok("USE MODULES `X`, `Y`, `Z`"); - - sql("use modules x^,^").fails("(?s).*Encountered \"\" at line 1, column 14.\n.*"); - - sql("use modules ^'core'^") - .fails("(?s).*Encountered \"\\\\'core\\\\'\" at line 1, column 13.\n.*"); - } - - @Test - void testShowModules() { - sql("show modules").ok("SHOW MODULES"); - - sql("show full modules").ok("SHOW FULL MODULES"); - } - - @Test - void testBeginStatementSet() { - sql("begin statement set").ok("BEGIN STATEMENT SET"); - } - - @Test - void testEnd() { - sql("end").ok("END"); - } - - @Test - void testExecuteStatementSet() { - sql("execute statement set begin insert into t1 select * from t2; insert into t2 select * from t3; end") - .ok( - "EXECUTE STATEMENT SET BEGIN\n" - + "INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`\n" - + ";\n" - + "INSERT INTO `T2`\n" - + "SELECT *\n" - + "FROM `T3`\n" - + ";\n" - + "END"); - } - - @Test - void testExplainStatementSet() { - sql("explain statement set begin insert into t1 select * from t2; insert into t2 select * from t3; end") - .ok( - "EXPLAIN STATEMENT SET BEGIN\n" - + "INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`\n" - + ";\n" - + "INSERT INTO `T2`\n" - + "SELECT *\n" - + "FROM `T3`\n" - + ";\n" - + "END"); - } - - @Test - void testExplainExecuteStatementSet() { - sql("explain execute statement set begin insert into t1 select * from t2; insert into t2 select * from t3; end") - .ok( - "EXPLAIN EXECUTE STATEMENT SET BEGIN\n" - + "INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`\n" - + ";\n" - + "INSERT INTO `T2`\n" - + "SELECT *\n" - + "FROM `T3`\n" - + ";\n" - + "END"); - } - - @Test - void testExecuteStatementSetWithOnConflict() { - sql("execute statement set begin " - + "insert into t1 select * from t2 on conflict do deduplicate; " - + "insert into t3 select * from t4 on conflict do nothing; " - + "end") - .ok( - "EXECUTE STATEMENT SET BEGIN\n" - + "INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`\n" - + "ON CONFLICT DO DEDUPLICATE\n" - + ";\n" - + "INSERT INTO `T3`\n" - + "SELECT *\n" - + "FROM `T4`\n" - + "ON CONFLICT DO NOTHING\n" - + ";\n" - + "END"); - } - - @Test - void testExplainExecuteSelect() { - sql("explain execute select * from emps").ok("EXPLAIN EXECUTE SELECT *\nFROM `EMPS`"); - } - - @Test - void testExplainExecuteInsert() { - sql("explain execute insert into emps1 select * from emps2") - .ok("EXPLAIN EXECUTE INSERT INTO `EMPS1`\nSELECT *\nFROM `EMPS2`"); - } - - @Test - void testExplain() { - String sql = "explain select * from emps"; - String expected = "EXPLAIN SELECT *\nFROM `EMPS`"; - this.sql(sql).ok(expected); - } - - @Test - void testExecuteSelect() { - String sql = "execute select * from emps"; - String expected = "EXECUTE SELECT *\nFROM `EMPS`"; - this.sql(sql).ok(expected); - } - - @Test - void testExplainPlanFor() { - String sql = "explain plan for select * from emps"; - String expected = "EXPLAIN SELECT *\nFROM `EMPS`"; - this.sql(sql).ok(expected); - } - - @Test - void testExplainChangelogMode() { - String sql = "explain changelog_mode select * from emps"; - String expected = "EXPLAIN CHANGELOG_MODE SELECT *\nFROM `EMPS`"; - this.sql(sql).ok(expected); - } - - @Test - void testExplainEstimatedCost() { - String sql = "explain estimated_cost select * from emps"; - String expected = "EXPLAIN ESTIMATED_COST SELECT *\nFROM `EMPS`"; - this.sql(sql).ok(expected); - } - - @Test - void testExplainUnion() { - String sql = "explain estimated_cost select * from emps union all select * from emps"; - String expected = - "EXPLAIN ESTIMATED_COST SELECT *\n" - + "FROM `EMPS`\n" - + "UNION ALL\n" - + "SELECT *\n" - + "FROM `EMPS`"; - this.sql(sql).ok(expected); - } - - @Test - void testExplainJsonFormat() { - // Unsupported feature. Escape the test. - } - - @Test - void testExplainWithImpl() { - // Unsupported feature. Escape the test. - } - - @Test - void testExplainWithoutImpl() { - // Unsupported feature. Escape the test. - } - - @Test - void testExplainWithType() { - // Unsupported feature. Escape the test. - } - - @Test - void testExplainAsXml() { - // Unsupported feature. Escape the test. - } - - @Test - void testSqlOptions() { - // SET/RESET are overridden for Flink SQL - } - - @Test - void testExplainAsJson() { - String sql = "explain json_execution_plan select * from emps"; - String expected = "EXPLAIN JSON_EXECUTION_PLAN SELECT *\n" + "FROM `EMPS`"; - this.sql(sql).ok(expected); - } - - @Test - void testExplainPlanAdvice() { - String sql = "explain plan_advice select * from emps"; - String expected = "EXPLAIN PLAN_ADVICE SELECT *\nFROM `EMPS`"; - this.sql(sql).ok(expected); - } - - @Test - void testExplainAllDetails() { - String sql = - "explain changelog_mode,json_execution_plan,estimated_cost,plan_advice select * from emps"; - String expected = - "EXPLAIN JSON_EXECUTION_PLAN, CHANGELOG_MODE, PLAN_ADVICE, ESTIMATED_COST SELECT *\n" - + "FROM `EMPS`"; - this.sql(sql).ok(expected); - } - - @Test - void testExplainInsert() { - String expected = "EXPLAIN INSERT INTO `EMPS1`\n" + "SELECT *\n" + "FROM `EMPS2`"; - this.sql("explain plan for insert into emps1 select * from emps2").ok(expected); - } - - @Test - void testExecuteInsert() { - String expected = "EXECUTE INSERT INTO `EMPS1`\n" + "SELECT *\n" + "FROM `EMPS2`"; - this.sql("execute insert into emps1 select * from emps2").ok(expected); - } - - @Test - void testExecutePlan() { - sql("execute plan './test.json'").ok("EXECUTE PLAN './test.json'"); - sql("execute plan '/some/absolute/dir/plan.json'") - .ok("EXECUTE PLAN '/some/absolute/dir/plan.json'"); - sql("execute plan 'file:///foo/bar/test.json'") - .ok("EXECUTE PLAN 'file:///foo/bar/test.json'"); - } - - @Test - void testCompilePlan() { - sql("compile plan './test.json' for insert into t1 select * from t2") - .ok( - "COMPILE PLAN './test.json' FOR INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`"); - sql("compile plan './test.json' if not exists for insert into t1 select * from t2") - .ok( - "COMPILE PLAN './test.json' IF NOT EXISTS FOR INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`"); - sql("compile plan 'file:///foo/bar/test.json' if not exists for insert into t1 select * from t2") - .ok( - "COMPILE PLAN 'file:///foo/bar/test.json' IF NOT EXISTS FOR INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`"); - - sql("compile plan './test.json' for statement set " - + "begin insert into t1 select * from t2; insert into t2 select * from t3; end") - .ok( - "COMPILE PLAN './test.json' FOR STATEMENT SET BEGIN\n" - + "INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`\n" - + ";\n" - + "INSERT INTO `T2`\n" - + "SELECT *\n" - + "FROM `T3`\n" - + ";\n" - + "END"); - sql("compile plan './test.json' if not exists for statement set " - + "begin insert into t1 select * from t2; insert into t2 select * from t3; end") - .ok( - "COMPILE PLAN './test.json' IF NOT EXISTS FOR STATEMENT SET BEGIN\n" - + "INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`\n" - + ";\n" - + "INSERT INTO `T2`\n" - + "SELECT *\n" - + "FROM `T3`\n" - + ";\n" - + "END"); - - sql("compile plan 'file:///foo/bar/test.json' if not exists for statement set " - + "begin insert into t1 select * from t2; insert into t2 select * from t3; end") - .ok( - "COMPILE PLAN 'file:///foo/bar/test.json' IF NOT EXISTS FOR STATEMENT SET BEGIN\n" - + "INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`\n" - + ";\n" - + "INSERT INTO `T2`\n" - + "SELECT *\n" - + "FROM `T3`\n" - + ";\n" - + "END"); - } - - @Test - void testCompileAndExecutePlan() { - sql("compile and execute plan './test.json' for insert into t1 select * from t2") - .ok( - "COMPILE AND EXECUTE PLAN './test.json' FOR INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`"); - - sql("compile and execute plan './test.json' for statement set " - + "begin insert into t1 select * from t2; insert into t2 select * from t3; end") - .ok( - "COMPILE AND EXECUTE PLAN './test.json' FOR STATEMENT SET BEGIN\n" - + "INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`\n" - + ";\n" - + "INSERT INTO `T2`\n" - + "SELECT *\n" - + "FROM `T3`\n" - + ";\n" - + "END"); - sql("compile and execute plan 'file:///foo/bar/test.json' for insert into t1 select * from t2") - .ok( - "COMPILE AND EXECUTE PLAN 'file:///foo/bar/test.json' FOR INSERT INTO `T1`\n" - + "SELECT *\n" - + "FROM `T2`"); - } - - @Test - void testExplainUpsert() { - String sql = "explain plan for upsert into emps1 values (1, 2)"; - String expected = "EXPLAIN UPSERT INTO `EMPS1`\n" + "VALUES (ROW(1, 2))"; - this.sql(sql).ok(expected); - } - - @Test - void testExplainPlanForWithExplainDetails() { - String sql = "explain plan for ^json_execution_plan^ upsert into emps1 values (1, 2)"; - this.sql(sql).fails("Non-query expression encountered in illegal context"); - } - - @Test - void testExplainDuplicateExplainDetails() { - String sql = "explain changelog_mode,^changelog_mode^ select * from emps"; - this.sql(sql).fails("Duplicate EXPLAIN DETAIL is not allowed."); - } - - @Test - void testAddJar() { - sql("add Jar './test.sql'").ok("ADD JAR './test.sql'"); - sql("add JAR 'file:///path/to/\nwhatever'").ok("ADD JAR 'file:///path/to/\nwhatever'"); - sql("add JAR 'oss://path/helloworld.go'").ok("ADD JAR 'oss://path/helloworld.go'"); - } - - @Test - void testRemoveJar() { - sql("remove Jar './test.sql'").ok("REMOVE JAR './test.sql'"); - sql("remove JAR 'file:///path/to/\nwhatever'") - .ok("REMOVE JAR 'file:///path/to/\nwhatever'"); - sql("remove JAR 'oss://path/helloworld.go'").ok("REMOVE JAR 'oss://path/helloworld.go'"); - } - - @Test - void testShowJars() { - sql("show jars").ok("SHOW JARS"); - } - - @Test - void testSetReset() { - sql("SET").same(); - sql("SET 'test-key' = 'test-value'").same(); - sql("RESET").same(); - sql("RESET 'test-key'").same(); - } - - @Test - void testTryCast() { - // Simple types - expr("try_cast(a as timestamp)").ok("(TRY_CAST(`A` AS TIMESTAMP))"); - expr("try_cast('abc' as timestamp)").ok("(TRY_CAST('abc' AS TIMESTAMP))"); - - // Complex types - expr("try_cast(a as row(f0 int, f1 varchar))") - .ok("(TRY_CAST(`A` AS ROW(`F0` INTEGER, `F1` VARCHAR)))"); - expr("try_cast(a as row(f0 int array, f1 map, f2 STRING NOT NULL))") - .ok( - "(TRY_CAST(`A` AS ROW(`F0` INTEGER ARRAY, `F1` MAP< STRING, DECIMAL(10, 2) >, `F2` STRING NOT NULL)))"); - } - - @Test - void testAnalyzeTable() { - sql("analyze table emp^s^").fails("(?s).*Encountered \"\" at line 1, column 18.\n.*"); - sql("analyze table emps compute statistics").ok("ANALYZE TABLE `EMPS` COMPUTE STATISTICS"); - sql("analyze table emps partition ^compute^ statistics") - .fails("(?s).*Encountered \"compute\" at line 1, column 30.\n.*"); - sql("analyze table emps partition(^)^ compute statistics") - .fails("(?s).*Encountered \"\\)\" at line 1, column 30.\n.*"); - sql("analyze table emps partition(x='ab') compute statistics") - .ok("ANALYZE TABLE `EMPS` PARTITION (`X` = 'ab') COMPUTE STATISTICS"); - sql("analyze table emps partition(x='ab', y='bc') compute statistics") - .ok("ANALYZE TABLE `EMPS` PARTITION (`X` = 'ab', `Y` = 'bc') COMPUTE STATISTICS"); - sql("analyze table emps compute statistics for column^s^") - .fails("(?s).*Encountered \"\" at line 1, column 49.\n.*"); - sql("analyze table emps compute statistics for columns a") - .ok("ANALYZE TABLE `EMPS` COMPUTE STATISTICS FOR COLUMNS `A`"); - sql("analyze table emps compute statistics for columns a, b") - .ok("ANALYZE TABLE `EMPS` COMPUTE STATISTICS FOR COLUMNS `A`, `B`"); - sql("analyze table emps compute statistics for all columns") - .ok("ANALYZE TABLE `EMPS` COMPUTE STATISTICS FOR ALL COLUMNS"); - sql("analyze table emps partition(x, y) compute statistics for all columns") - .ok("ANALYZE TABLE `EMPS` PARTITION (`X`, `Y`) COMPUTE STATISTICS FOR ALL COLUMNS"); - sql("analyze table emps partition(x='ab', y) compute statistics for all columns") - .ok( - "ANALYZE TABLE `EMPS` PARTITION (`X` = 'ab', `Y`) COMPUTE STATISTICS FOR ALL COLUMNS"); - sql("analyze table emps partition(x, y='cd') compute statistics for all columns") - .ok( - "ANALYZE TABLE `EMPS` PARTITION (`X`, `Y` = 'cd') COMPUTE STATISTICS FOR ALL COLUMNS"); - sql("analyze table emps partition(x=^,^ y) compute statistics for all columns") - .fails("(?s).*Encountered \"\\,\" at line 1, column 32.\n.*"); - } - - @Test - void testExplainCreateTableNoSupported() { - this.sql("EXPLAIN CREATE TABLE t (id int^)^") - .fails( - "Unsupported CREATE OR REPLACE statement for EXPLAIN\\. The statement must define a query using the AS clause \\(i\\.e\\. CTAS/RTAS statements\\)\\."); - } - - @Test - void testExplainCreateTableAsSelect() { - this.sql("EXPLAIN CREATE TABLE t AS SELECT * FROM b") - .ok("EXPLAIN CREATE TABLE `T`\nAS\nSELECT *\nFROM `B`"); - } - - @Test - void testExplainCreateOrReplaceTableAsSelect() { - this.sql("EXPLAIN CREATE OR REPLACE TABLE t AS SELECT * FROM b") - .ok("EXPLAIN CREATE OR REPLACE TABLE `T`\nAS\nSELECT *\nFROM `B`"); - } - - @Test - void testExplainReplaceTableAsSelect() { - this.sql("EXPLAIN REPLACE TABLE t AS SELECT * FROM b") - .ok("EXPLAIN REPLACE TABLE `T`\nAS\nSELECT *\nFROM `B`"); - } - - @Test - void testCreateTableAsSelectWithoutOptions() { - sql("CREATE TABLE t AS SELECT * FROM b").ok("CREATE TABLE `T`\nAS\nSELECT *\nFROM `B`"); - } - - @Test - void testCreateTableAsSelectWithOptions() { - sql("CREATE TABLE t WITH ('test' = 'zm') AS SELECT * FROM b") - .ok("CREATE TABLE `T`\nWITH (\n 'test' = 'zm'\n)\nAS\nSELECT *\nFROM `B`"); - } - - @Test - void testCreateTableAsSelectWithCreateTableLike() { - sql("CREATE TABLE t (col1 string) WITH ('test' = 'zm') like b ^AS^ SELECT col1 FROM b") - .fails("(?s).*Encountered \"AS\" at line 1, column 58.*"); - } - - @Test - void testCreateTableAsSelectWithTmpTable() { - sql("CREATE TEMPORARY TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node( - new ValidationMatcher() - .fails( - "CREATE TABLE AS SELECT syntax does not support to create temporary table yet.")); - } - - @Test - void testCreateTableAsSelectWithExplicitColumns() { - sql("CREATE TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - } - - @Test - void testCreateTableAsSelectWithWatermark() { - sql("CREATE TABLE t (watermark FOR col1 AS col1 - interval '3' second) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - } - - @Test - void testCreateTableAsSelectWithConstraints() { - sql("CREATE TABLE t (PRIMARY KEY (col1) NOT ENFORCED) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - - sql("CREATE TABLE t (PRIMARY KEY (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node( - new ValidationMatcher() - .fails( - "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. ENFORCED/NOT ENFORCED controls " - + "if the constraint checks are performed on the incoming/outgoing data. " - + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); - - sql("CREATE TABLE t (PRIMARY KEY (col1), PRIMARY KEY (col2) NOT ENFORCED) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().fails("Duplicate primary key definition")); - - sql("CREATE TABLE t (UNIQUE (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); - } - - @Test - void testCreateTableAsSelectWithDistribution() { - sql("CREATE TABLE t DISTRIBUTED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - } - - @Test - void testCreateTableAsSelectWithPartitionKey() { - sql("CREATE TABLE t PARTITIONED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - } - - @Test - void testCreateTableAsSelectWithColumnIdentifiers() { - // test with only column identifiers - sql("CREATE TABLE t (col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - - // test mix of column identifiers and column with types is not allowed - sql("CREATE TABLE t (col1, col2 ^int^) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .fails("(?s).*Encountered \"int\" at line 1, column 28.*"); - } - - @Test - void testUnsupportedCreateTableStatementsWithColumnIdentifiers() { - String expectedErrorMsg = - "Columns identifiers without types in the schema are " - + "supported on CTAS/RTAS statements only."; - - sql("CREATE TABLE t ^(a, h^) WITH " + "('connector' = 'kafka', 'kafka.topic' = 'log.test')") - .fails(expectedErrorMsg); - - sql("CREATE TABLE t ^(a, h^) WITH " - + "('connector' = 'kafka', 'kafka.topic' = 'log.test') " - + "LIKE parent_table") - .fails(expectedErrorMsg); - } - - @Test - void testReplaceTableAsSelectWithColumnIdentifiers() { - // test with only column identifiers - sql("REPLACE TABLE t (col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - - // test mix of column identifiers and column with types is not allowed - sql("REPLACE TABLE t (col1, col2 ^int^) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .fails("(?s).*Encountered \"int\" at line 1, column 29.*"); - } - - @Test - void testReplaceTableAsSelect() { - // test replace table as select without options - sql("REPLACE TABLE t AS SELECT * FROM b").ok("REPLACE TABLE `T`\nAS\nSELECT *\nFROM `B`"); - - // test replace table as select with options - sql("REPLACE TABLE t WITH ('test' = 'zm') AS SELECT * FROM b") - .ok("REPLACE TABLE `T`\nWITH (\n 'test' = 'zm'\n)\nAS\nSELECT *\nFROM `B`"); - - // test replace table as select with tmp table - sql("REPLACE TEMPORARY TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node( - new ValidationMatcher() - .fails( - "REPLACE TABLE AS SELECT syntax does not support temporary table yet.")); - - // test replace table as select with explicit columns - sql("REPLACE TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - - // test replace table as select with watermark - sql("REPLACE TABLE t (watermark FOR ts AS ts - interval '3' second) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - - // test replace table as select with constraints - sql("REPLACE TABLE t (PRIMARY KEY (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node( - new ValidationMatcher() - .fails( - "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. ENFORCED/NOT ENFORCED controls " - + "if the constraint checks are performed on the incoming/outgoing data. " - + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); - - sql("REPLACE TABLE t (PRIMARY KEY (col1), PRIMARY KEY (col2) NOT ENFORCED) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().fails("Duplicate primary key definition")); - - sql("REPLACE TABLE t (UNIQUE (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); - - // test replace table as select with partition key - sql("REPLACE TABLE t PARTITIONED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - - // test replace table as select with distribution - sql("REPLACE TABLE t DISTRIBUTED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - } - - @Test - void testCreateOrReplaceTableAsSelect() { - // test create or replace table as select without options - sql("CREATE OR REPLACE TABLE t AS SELECT * FROM b") - .ok("CREATE OR REPLACE TABLE `T`\nAS\nSELECT *\nFROM `B`"); - - // test create or replace table as select with options - sql("CREATE OR REPLACE TABLE t WITH ('test' = 'zm') AS SELECT * FROM b") - .ok( - "CREATE OR REPLACE TABLE `T`\nWITH (\n 'test' = 'zm'\n)\nAS\nSELECT *\nFROM `B`"); - - // test create or replace table as select with create table like - sql("CREATE OR REPLACE TABLE t (col1 string) WITH ('test' = 'zm') like b ^AS^ SELECT col1 FROM b") - .fails("(?s).*Encountered \"AS\" at line 1, column 69.*"); - - // test create or replace table as select with tmp table - sql("CREATE OR REPLACE TEMPORARY TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node( - new ValidationMatcher() - .fails( - "CREATE OR REPLACE TABLE AS SELECT syntax does not support temporary table yet.")); - - // test create or replace table as select with explicit columns - sql("CREATE OR REPLACE TABLE t (col1 string) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - - // test create or replace table as select with watermark - sql("CREATE OR REPLACE TABLE t (watermark FOR ts AS ts - interval '3' second) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - - // test create or replace table as select with constraints - sql("CREATE OR REPLACE TABLE t (PRIMARY KEY (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node( - new ValidationMatcher() - .fails( - "Flink doesn't support ENFORCED mode for PRIMARY KEY constraint. ENFORCED/NOT ENFORCED controls " - + "if the constraint checks are performed on the incoming/outgoing data. " - + "Flink does not own the data therefore the only supported mode is the NOT ENFORCED mode")); - - sql("CREATE OR REPLACE TABLE t (PRIMARY KEY (col1), PRIMARY KEY (col2) NOT ENFORCED) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().fails("Duplicate primary key definition")); - - sql("CREATE OR REPLACE TABLE t (UNIQUE (col1)) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().fails("UNIQUE constraint is not supported yet")); - - // test create or replace table as select with partition key - sql("CREATE OR REPLACE TABLE t PARTITIONED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - - sql("CREATE OR REPLACE TABLE t DISTRIBUTED BY(col1) WITH ('test' = 'zm') AS SELECT col1 FROM b") - .node(new ValidationMatcher().ok()); - } - - @Test - void testShowJobs() { - sql("show jobs").ok("SHOW JOBS"); - } - - @Test - void testStopJob() { - sql("STOP JOB 'myjob'").same(); - sql("STOP JOB 'myjob' WITH SAVEPOINT").same(); - sql("STOP JOB 'myjob' WITH SAVEPOINT WITH DRAIN").same(); - sql("STOP JOB 'myjob' ^WITH DRAIN^") - .fails("WITH DRAIN could only be used after WITH SAVEPOINT."); - sql("STOP JOB 'myjob' ^WITH DRAIN^ WITH SAVEPOINT") - .fails("WITH DRAIN could only be used after WITH SAVEPOINT."); - } - - @Test - void testDescribeJob() { - sql("DESCRIBE JOB 'myjob'").same(); - sql("DESC JOB 'myjob'").ok("DESCRIBE JOB 'myjob'"); - } - - @Test - void testTruncateTable() { - sql("truncate table t1").ok("TRUNCATE TABLE `T1`"); - } - - public static BaseMatcher validated(String validatedSql) { - return new TypeSafeDiagnosingMatcher() { - @Override - protected boolean matchesSafely(SqlNode item, Description mismatchDescription) { - if (item instanceof ExtendedSqlNode) { - try { - ((ExtendedSqlNode) item).validate(); - } catch (SqlValidateException e) { - mismatchDescription.appendText( - "Could not validate the node. Exception: \n"); - mismatchDescription.appendValue(e); - } - - String actual = item.toSqlString(null, true).getSql(); - return actual.equals(validatedSql); - } - mismatchDescription.appendText( - "This matcher can be applied only to ExtendedSqlNode."); - return false; - } - - @Override - public void describeTo(Description description) { - description.appendText( - "The validated node string representation should be equal to: \n"); - description.appendText(validatedSql); - } - }; - } - - @Test - void testShowModels() { - sql("show models").ok("SHOW MODELS"); - sql("show models from db1").ok("SHOW MODELS FROM `DB1`"); - sql("show models from catalog1.db1").ok("SHOW MODELS FROM `CATALOG1`.`DB1`"); - sql("show models in db1").ok("SHOW MODELS IN `DB1`"); - sql("show models in catalog1.db1").ok("SHOW MODELS IN `CATALOG1`.`DB1`"); - } - - @Test - void testDropModel() { - sql("drop model m1").ok("DROP MODEL `M1`"); - sql("drop model db1.m1").ok("DROP MODEL `DB1`.`M1`"); - sql("drop model catalog1.db1.m1").ok("DROP MODEL `CATALOG1`.`DB1`.`M1`"); - } - - @Test - void testDropTemporaryModel() { - sql("drop temporary model m1").ok("DROP TEMPORARY MODEL `M1`"); - sql("drop temporary model if exists m1").ok("DROP TEMPORARY MODEL IF EXISTS `M1`"); - } - - @Test - void testDropModelIfExists() { - sql("drop model if exists catalog1.db1.m1") - .ok("DROP MODEL IF EXISTS `CATALOG1`.`DB1`.`M1`"); - } - - @Test - void testAlterModelSet() { - final String sql = "alter model m1 set ('key1' = 'value1','key2' = 'value2')"; - final String expected = - "ALTER MODEL `M1` SET (\n" - + " 'key1' = 'value1',\n" - + " 'key2' = 'value2'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testAlterModelIfExists() { - final String sql = "alter model if exists m1 set ('key1' = 'value1','key2' = 'value2')"; - final String expected = - "ALTER MODEL IF EXISTS `M1` SET (\n" - + " 'key1' = 'value1',\n" - + " 'key2' = 'value2'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testAlterModelRename() { - final String sql = "alter model m1 rename to m2"; - final String expected = "ALTER MODEL `M1` RENAME TO `M2`"; - sql(sql).ok(expected); - } - - @Test - void testAlterModelRenameIfExists() { - final String sql = "alter model if exists m1 rename to m2"; - final String expected = "ALTER MODEL IF EXISTS `M1` RENAME TO `M2`"; - sql(sql).ok(expected); - } - - @Test - void testAlterModelReset() { - final String sql = "alter model m1 reset ('key1', 'key2')"; - final String expected = "ALTER MODEL `M1` RESET (\n 'key1',\n 'key2'\n)"; - sql(sql).ok(expected); - } - - @Test - void testAlterModelResetIfExists() { - final String sql = "alter model if exists m1 reset ('key1', 'key2')"; - final String expected = "ALTER MODEL IF EXISTS `M1` RESET (\n 'key1',\n 'key2'\n)"; - sql(sql).ok(expected); - } - - @Test - void testCreateModel() { - sql("create model m1\n" - + " INPUT(col1 INT, col2 STRING)\n" - + " OUTPUT(label DOUBLE)\n" - + " COMMENT 'model_comment'\n" - + " WITH (\n" - + " 'key1'='value1',\n" - + " 'key2'='value2'\n" - + " )\n") - .ok( - "CREATE MODEL `M1` INPUT (\n" - + " `COL1` INTEGER,\n" - + " `COL2` STRING\n" - + ") OUTPUT (\n" - + " `LABEL` DOUBLE\n" - + ")\n" - + "COMMENT 'model_comment'" - + "\nWITH (\n" - + " 'key1' = 'value1',\n" - + " 'key2' = 'value2'\n" - + ")"); - } - - @Test - void testCreateModelIfNotExists() { - sql("create model if not exists m1\n" - + " INPUT(col1 INT, col2 STRING)\n" - + " OUTPUT(label DOUBLE)\n" - + " COMMENT 'model_comment'\n" - + " WITH (\n" - + " 'key1'='value1',\n" - + " 'key2'='value2'\n" - + " )\n") - .ok( - "CREATE MODEL IF NOT EXISTS `M1` INPUT (\n" - + " `COL1` INTEGER,\n" - + " `COL2` STRING\n" - + ") OUTPUT (\n" - + " `LABEL` DOUBLE\n" - + ")\n" - + "COMMENT 'model_comment'" - + "\nWITH (\n" - + " 'key1' = 'value1',\n" - + " 'key2' = 'value2'\n" - + ")"); - } - - @Test - void testCreateModelAs() { - sql("create model m1\n" - + " WITH (\n" - + " 'key1'='value1',\n" - + " 'key2'='value2'\n" - + " ) as select f1, f2 from t1\n") - .ok( - "CREATE MODEL `M1`" - + "\nWITH (\n" - + " 'key1' = 'value1',\n" - + " 'key2' = 'value2'\n" - + ")\n" - + "AS\n" - + "SELECT `F1`, `F2`\n" - + "FROM `T1`"); - } - - @Test - void testCreateModelAsIfNotExists() { - sql("create model if not exists m1\n" - + " WITH (\n" - + " 'key1'='value1',\n" - + " 'key2'='value2'\n" - + " ) as select f1, f2 from t1\n") - .ok( - "CREATE MODEL IF NOT EXISTS `M1`" - + "\nWITH (\n" - + " 'key1' = 'value1',\n" - + " 'key2' = 'value2'\n" - + ")\n" - + "AS\n" - + "SELECT `F1`, `F2`\n" - + "FROM `T1`"); - } - - @Test - void testCreateModelAsWithInput() { - sql("create model if not exists m1\n" - + " INPUT(col1 INT, col2 STRING)\n" - + " OUTPUT(label DOUBLE)\n" - + " WITH (\n" - + " 'key1'='value1',\n" - + " 'key2'='value2'\n" - + " ) as select f1, f2 from t1\n") - .ok( - "CREATE MODEL IF NOT EXISTS `M1` INPUT (\n" - + " `COL1` INTEGER,\n" - + " `COL2` STRING\n" - + ") OUTPUT (\n" - + " `LABEL` DOUBLE\n" - + ")" - + "\nWITH (\n" - + " 'key1' = 'value1',\n" - + " 'key2' = 'value2'\n" - + ")\n" - + "AS\n" - + "SELECT `F1`, `F2`\n" - + "FROM `T1`") - .node( - new ValidationMatcher() - .fails( - "CREATE MODEL AS SELECT syntax does not support to specify explicit input columns.")); - } - - @Test - void testCreateModelAsWithOutput() { - sql("create model if not exists m1\n" - + " OUTPUT(label DOUBLE)\n" - + " WITH (\n" - + " 'key1'='value1',\n" - + " 'key2'='value2'\n" - + " ) as select f1, f2 from t1\n") - .ok( - "CREATE MODEL IF NOT EXISTS `M1` OUTPUT (\n" - + " `LABEL` DOUBLE\n" - + ")" - + "\nWITH (\n" - + " 'key1' = 'value1',\n" - + " 'key2' = 'value2'\n" - + ")\n" - + "AS\n" - + "SELECT `F1`, `F2`\n" - + "FROM `T1`") - .node( - new ValidationMatcher() - .fails( - "CREATE MODEL AS SELECT syntax does not support to specify explicit output columns.")); - } - - @Test - void testModelInFunction() { - sql("select * from table(ml_predict(TABLE my_table, MODEL my_model))") - .ok( - "SELECT *\n" - + "FROM TABLE(`ML_PREDICT`((TABLE `MY_TABLE`), MODEL `MY_MODEL`))"); - } - - @Test - void testModelInFunctionWithoutTable() { - sql("select * from func(TABLE my_table, MODEL cat.db.my_model)") - .ok( - "SELECT *\n" - + "FROM TABLE(`FUNC`((TABLE `MY_TABLE`), MODEL `CAT`.`DB`.`MY_MODEL`))"); - } - - @Test - void testModelInFunctionNamedArgs() { - sql("select * from table(ml_predict(INPUT => TABLE my_table, model => MODEL my_model))") - .ok( - "SELECT *\n" - + "FROM TABLE(`ML_PREDICT`(`INPUT` => (TABLE `MY_TABLE`), `MODEL` => (MODEL `MY_MODEL`)))"); - } - - // ===================================================================================== - // Connection DDL/DQL Tests - // ===================================================================================== - - @Test - void testCreateConnection() { - sql("create connection conn1\n" - + " COMMENT 'connection_comment'\n" - + " WITH (\n" - + " 'type'='basic',\n" - + " 'url'='http://example.com',\n" - + " 'username'='user1',\n" - + " 'password'='pass1'\n" - + " )\n") - .ok( - "CREATE CONNECTION `CONN1`\n" - + "COMMENT 'connection_comment'\n" - + "WITH (\n" - + " 'type' = 'basic',\n" - + " 'url' = 'http://example.com',\n" - + " 'username' = 'user1',\n" - + " 'password' = 'pass1'\n" - + ")"); - } - - @Test - void testCreateConnectionIfNotExists() { - sql("create connection if not exists conn1\n" - + " WITH (\n" - + " 'type'='bearer',\n" - + " 'token'='my_token'\n" - + " )\n") - .ok( - "CREATE CONNECTION IF NOT EXISTS `CONN1`\n" - + "WITH (\n" - + " 'type' = 'bearer',\n" - + " 'token' = 'my_token'\n" - + ")"); - } - - @Test - void testCreateTemporaryConnection() { - sql("create temporary connection conn1\n" - + " WITH (\n" - + " 'type'='oauth',\n" - + " 'client_id'='client1'\n" - + " )\n") - .ok( - "CREATE TEMPORARY CONNECTION `CONN1`\n" - + "WITH (\n" - + " 'type' = 'oauth',\n" - + " 'client_id' = 'client1'\n" - + ")"); - } - - @Test - void testCreateSystemConnection() { - sql("create ^system^ connection conn1\n" - + " WITH (\n" - + " 'type'='basic',\n" - + " 'url'='http://example.com'\n" - + " )\n") - .fails( - "(?s)CREATE SYSTEM CONNECTION is not supported, " - + "system connections can only be registered as temporary " - + "connections, you can use CREATE TEMPORARY SYSTEM CONNECTION " - + "instead\\..*"); - } - - @Test - void testCreateTemporarySystemConnection() { - sql("create temporary system connection conn1\n" - + " WITH (\n" - + " 'type'='custom_type',\n" - + " 'api_key'='key123'\n" - + " )\n") - .ok( - "CREATE TEMPORARY SYSTEM CONNECTION `CONN1`\n" - + "WITH (\n" - + " 'type' = 'custom_type',\n" - + " 'api_key' = 'key123'\n" - + ")"); - } - - @Test - void testCreateConnectionWithQualifiedName() { - sql("create connection catalog1.db1.conn1\n" - + " WITH ('type'='basic', 'url'='http://example.com')\n") - .ok( - "CREATE CONNECTION `CATALOG1`.`DB1`.`CONN1`\n" - + "WITH (\n" - + " 'type' = 'basic',\n" - + " 'url' = 'http://example.com'\n" - + ")"); - } - - @Test - void testDropConnection() { - sql("drop connection conn1").ok("DROP CONNECTION `CONN1`"); - sql("drop connection db1.conn1").ok("DROP CONNECTION `DB1`.`CONN1`"); - sql("drop connection catalog1.db1.conn1").ok("DROP CONNECTION `CATALOG1`.`DB1`.`CONN1`"); - } - - @Test - void testDropConnectionIfExists() { - sql("drop connection if exists catalog1.db1.conn1") - .ok("DROP CONNECTION IF EXISTS `CATALOG1`.`DB1`.`CONN1`"); - } - - @Test - void testDropTemporaryConnection() { - sql("drop temporary connection conn1").ok("DROP TEMPORARY CONNECTION `CONN1`"); - sql("drop temporary connection if exists conn1") - .ok("DROP TEMPORARY CONNECTION IF EXISTS `CONN1`"); - } - - @Test - void testDropTemporarySystemConnection() { - sql("drop temporary system connection conn1") - .ok("DROP TEMPORARY SYSTEM CONNECTION `CONN1`"); - sql("drop temporary system connection if exists conn1") - .ok("DROP TEMPORARY SYSTEM CONNECTION IF EXISTS `CONN1`"); - } - - @Test - void testDropSystemConnection() { - sql("drop ^system^ connection conn1") - .fails( - "(?s)DROP SYSTEM CONNECTION is not supported, " - + "system connections can only be dropped as temporary " - + "connections, you can use DROP TEMPORARY SYSTEM CONNECTION " - + "instead\\..*"); - } - - @Test - void testAlterConnectionSet() { - final String sql = - "alter connection conn1 set ('password' = 'new_password','url' = 'http://new.com')"; - final String expected = - "ALTER CONNECTION `CONN1` SET (\n" - + " 'password' = 'new_password',\n" - + " 'url' = 'http://new.com'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testAlterConnectionSetWithQualifiedName() { - final String sql = "alter connection catalog1.db1.conn1 set ('token' = 'new_token')"; - final String expected = - "ALTER CONNECTION `CATALOG1`.`DB1`.`CONN1` SET (\n" - + " 'token' = 'new_token'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testAlterConnectionRename() { - final String sql = "alter connection conn1 rename to conn2"; - final String expected = "ALTER CONNECTION `CONN1` RENAME TO `CONN2`"; - sql(sql).ok(expected); - } - - @Test - void testAlterConnectionRenameWithQualifiedName() { - final String sql = "alter connection catalog1.db1.conn1 rename to conn2"; - final String expected = "ALTER CONNECTION `CATALOG1`.`DB1`.`CONN1` RENAME TO `CONN2`"; - sql(sql).ok(expected); - } - - @Test - void testAlterConnectionReset() { - final String sql = "alter connection conn1 reset ('password', 'url')"; - final String expected = "ALTER CONNECTION `CONN1` RESET (\n 'password',\n 'url'\n)"; - sql(sql).ok(expected); - } - - @Test - void testAlterConnectionResetWithQualifiedName() { - final String sql = "alter connection catalog1.db1.conn1 reset ('token')"; - final String expected = "ALTER CONNECTION `CATALOG1`.`DB1`.`CONN1` RESET (\n 'token'\n)"; - sql(sql).ok(expected); - } - - @Test - void testAlterConnectionIfExists() { - final String sql = - "alter connection if exists conn1 set ('password' = 'new_password','url' = 'http://new.com')"; - final String expected = - "ALTER CONNECTION IF EXISTS `CONN1` SET (\n" - + " 'password' = 'new_password',\n" - + " 'url' = 'http://new.com'\n" - + ")"; - sql(sql).ok(expected); - } - - @Test - void testAlterConnectionRenameIfExists() { - final String sql = "alter connection if exists conn1 rename to conn2"; - final String expected = "ALTER CONNECTION IF EXISTS `CONN1` RENAME TO `CONN2`"; - sql(sql).ok(expected); - } - - @Test - void testAlterConnectionResetIfExists() { - final String sql = "alter connection if exists conn1 reset ('password', 'url')"; - final String expected = - "ALTER CONNECTION IF EXISTS `CONN1` RESET (\n 'password',\n 'url'\n)"; - sql(sql).ok(expected); - } - - @Test - void testShowConnections() { - sql("show connections").ok("SHOW CONNECTIONS"); - sql("show connections from db1").ok("SHOW CONNECTIONS FROM `DB1`"); - sql("show connections from catalog1.db1").ok("SHOW CONNECTIONS FROM `CATALOG1`.`DB1`"); - sql("show connections in db1").ok("SHOW CONNECTIONS IN `DB1`"); - sql("show connections in catalog1.db1").ok("SHOW CONNECTIONS IN `CATALOG1`.`DB1`"); - } - - @Test - void testShowConnectionsLike() { - sql("show connections like '%conn%'").ok("SHOW CONNECTIONS LIKE '%CONN%'"); - sql("show connections from db1 like 'my_%'").ok("SHOW CONNECTIONS FROM `DB1` LIKE 'MY_%'"); - sql("show connections not like 'temp_%'").ok("SHOW CONNECTIONS NOT LIKE 'TEMP_%'"); - } - - @Test - void testShowCreateConnection() { - sql("show create connection conn1").ok("SHOW CREATE CONNECTION `CONN1`"); - sql("show create connection catalog1.db1.conn1") - .ok("SHOW CREATE CONNECTION `CATALOG1`.`DB1`.`CONN1`"); - } - - @Test - void testDescribeConnection() { - sql("describe connection conn1").ok("DESCRIBE CONNECTION `CONN1`"); - sql("describe connection catalog1.db1.conn1") - .ok("DESCRIBE CONNECTION `CATALOG1`.`DB1`.`CONN1`"); - sql("describe connection extended conn1").ok("DESCRIBE CONNECTION EXTENDED `CONN1`"); - - sql("desc connection conn1").ok("DESCRIBE CONNECTION `CONN1`"); - sql("desc connection catalog1.db1.conn1") - .ok("DESCRIBE CONNECTION `CATALOG1`.`DB1`.`CONN1`"); - sql("desc connection extended catalog1.db1.conn1") - .ok("DESCRIBE CONNECTION EXTENDED `CATALOG1`.`DB1`.`CONN1`"); - } - - /* - * This test was backported from Calcite 1.38 (CALCITE-6266). - * Remove it together with upgrade to Calcite 1.38. - */ - @Test - void testFromValuesWithoutParens() { - sql("select 1 from ^values^('x')") - .fails( - "(?s)Encountered \"values\" at line 1, column 15\\.\n" - + "Was expecting one of:\n" - + " \"LATERAL\" \\.\\.\\.\n" - + " \"TABLE\" \\.\\.\\.\n" - + " \\.\\.\\.\n" - + " \\.\\.\\.\n" - + " \\.\\.\\.\n" - + " \\.\\.\\.\n" - + " \\.\\.\\.\n" - + " \\.\\.\\.\n" - + " \\.\\.\\.\n" - + " \"\\(\" \\.\\.\\.\n.*" - + " \"UNNEST\" \\.\\.\\.\n.*"); - } - - /* - * This test was backported from Calcite 1.38 (CALCITE-6266). - * Remove it together with upgrade to Calcite 1.38. - */ - @Test - void testUnnest() { - sql("select*from unnest(x)").ok("SELECT *\n" + "FROM UNNEST(`X`)"); - sql("select*from unnest(x) AS T").ok("SELECT *\n" + "FROM UNNEST(`X`) AS `T`"); - // UNNEST cannot be first word in query - sql("^unnest^(x)").fails("(?s)Encountered \"unnest\" at.*"); - // UNNEST with more than one argument - final String sql = "select * from dept,\n" + "unnest(dept.employees, dept.managers)"; - final String expected = - "SELECT *\n" + "FROM `DEPT`,\n" + "UNNEST(`DEPT`.`EMPLOYEES`, `DEPT`.`MANAGERS`)"; - sql(sql).ok(expected); - - // LATERAL UNNEST is the same as UNNEST - // (LATERAL is implicit for UNNEST, so the parser just ignores it) - sql("select * from dept, lateral unnest(dept.employees)") - .ok("SELECT *\n" + "FROM `DEPT`,\n" + "UNNEST(`DEPT`.`EMPLOYEES`)"); - sql("select * from dept, unnest(dept.employees)") - .ok("SELECT *\n" + "FROM `DEPT`,\n" + "UNNEST(`DEPT`.`EMPLOYEES`)"); - - // Does not generate extra parentheses around UNNEST because UNNEST is - // a table expression. - final String sql1 = - "" - + "SELECT\n" - + " item.name,\n" - + " relations.*\n" - + "FROM dfs.tmp item\n" - + "JOIN (\n" - + " SELECT * FROM UNNEST(item.related) i(rels)\n" - + ") relations\n" - + "ON TRUE"; - final String expected1 = - "SELECT `ITEM`.`NAME`, `RELATIONS`.*\n" - + "FROM `DFS`.`TMP` AS `ITEM`\n" - + "INNER JOIN (SELECT *\n" - + "FROM UNNEST(`ITEM`.`RELATED`) AS `I` (`RELS`)) AS `RELATIONS` ON TRUE"; - sql(sql1).ok(expected1); - } - - @Test - void testOuterApplyFunctionFails() { - final String sql = "select * from dept outer apply ramp(deptno)^)^"; - sql(sql).withConformance(SqlConformanceEnum.SQL_SERVER_2008) - .fails("(?s).*Encountered \"\\)\" at .*"); - } - - @Test - void testVariantType() { - sql("CREATE TABLE t (\n" + "v variant" + "\n)") - .ok("CREATE TABLE `T` (\n" + " `V` VARIANT\n" + ")"); - - sql("CREATE TABLE t (\n" + "v VARIANT NOT NULL" + "\n)") - .ok("CREATE TABLE `T` (\n" + " `V` VARIANT NOT NULL\n" + ")"); - } - - @Test - void testBitmapType() { - sql("CREATE TABLE t (\n" + "bm bitmap" + "\n)") - .ok("CREATE TABLE `T` (\n" + " `BM` BITMAP\n" + ")"); - - sql("CREATE TABLE t (\n" + "bm bitmap NOT NULL" + "\n)") - .ok("CREATE TABLE `T` (\n" + " `BM` BITMAP NOT NULL\n" + ")"); - - // BITMAP takes no parameters - sql("CREATE TABLE t (\n" + "bm bitmap^(^1)" + "\n)") - .fails("(?s).*Encountered \"\\(\" at line 2, column 10.\n.*"); - - // BITMAP is a reserved keyword and cannot be used as an identifier without escaping - sql("CREATE TABLE t (\n" + "^bitmap^ INT" + "\n)") - .fails("(?s).*Encountered \"bitmap\" at line 2, column 1.\n.*"); - } -} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserMetadataTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserMetadataTest.java new file mode 100644 index 0000000000000..e9d78c88b520f --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserMetadataTest.java @@ -0,0 +1,272 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** SHOW and DESCRIBE metadata parser tests. */ +@Execution(CONCURRENT) +class FlinkSqlParserMetadataTest extends FlinkSqlParserTestBase { + + @Test + void testAlterFunction() { + sql("alter function function1 as 'org.apache.flink.function.function1'") + .ok("ALTER FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1'"); + + sql("alter temporary function function1 as 'org.apache.flink.function.function1'") + .ok( + "ALTER TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1'"); + + sql("alter temporary function function1 as 'org.apache.flink.function.function1' language scala") + .ok( + "ALTER TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE SCALA"); + + sql("alter temporary system function function1 as 'org.apache.flink.function.function1'") + .ok( + "ALTER TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1'"); + + sql("alter temporary system function function1 as 'org.apache.flink.function.function1' language java") + .ok( + "ALTER TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA"); + } + + @Test + void testShowFunctions() { + sql("show functions").ok("SHOW FUNCTIONS"); + sql("show user functions").ok("SHOW USER FUNCTIONS"); + + sql("show functions like '%'").ok("SHOW FUNCTIONS LIKE '%'"); + sql("show functions not like '%'").ok("SHOW FUNCTIONS NOT LIKE '%'"); + sql("show user functions like '%'").ok("SHOW USER FUNCTIONS LIKE '%'"); + sql("show user functions not like '%'").ok("SHOW USER FUNCTIONS NOT LIKE '%'"); + + sql("show functions from db1").ok("SHOW FUNCTIONS FROM `DB1`"); + sql("show user functions from db1").ok("SHOW USER FUNCTIONS FROM `DB1`"); + sql("show functions in db1").ok("SHOW FUNCTIONS IN `DB1`"); + sql("show user functions in db1").ok("SHOW USER FUNCTIONS IN `DB1`"); + + sql("show functions from catalog1.db1").ok("SHOW FUNCTIONS FROM `CATALOG1`.`DB1`"); + sql("show user functions from catalog1.db1") + .ok("SHOW USER FUNCTIONS FROM `CATALOG1`.`DB1`"); + sql("show functions in catalog1.db1").ok("SHOW FUNCTIONS IN `CATALOG1`.`DB1`"); + sql("show user functions in catalog1.db1").ok("SHOW USER FUNCTIONS IN `CATALOG1`.`DB1`"); + + sql("show functions from db1 like '%'").ok("SHOW FUNCTIONS FROM `DB1` LIKE '%'"); + sql("show user functions from db1 like '%'").ok("SHOW USER FUNCTIONS FROM `DB1` LIKE '%'"); + sql("show functions in db1 ilike '%'").ok("SHOW FUNCTIONS IN `DB1` ILIKE '%'"); + sql("show user functions in db1 ilike '%'").ok("SHOW USER FUNCTIONS IN `DB1` ILIKE '%'"); + + sql("show functions from catalog1.db1 ilike '%'") + .ok("SHOW FUNCTIONS FROM `CATALOG1`.`DB1` ILIKE '%'"); + sql("show user functions from catalog1.db1 ilike '%'") + .ok("SHOW USER FUNCTIONS FROM `CATALOG1`.`DB1` ILIKE '%'"); + sql("show functions in catalog1.db1 like '%'") + .ok("SHOW FUNCTIONS IN `CATALOG1`.`DB1` LIKE '%'"); + sql("show user functions in catalog1.db1 like '%'") + .ok("SHOW USER FUNCTIONS IN `CATALOG1`.`DB1` LIKE '%'"); + + sql("show functions from db1 not like '%'").ok("SHOW FUNCTIONS FROM `DB1` NOT LIKE '%'"); + sql("show user functions from db1 not like '%'") + .ok("SHOW USER FUNCTIONS FROM `DB1` NOT LIKE '%'"); + sql("show functions in db1 not ilike '%'").ok("SHOW FUNCTIONS IN `DB1` NOT ILIKE '%'"); + sql("show user functions in db1 not ilike '%'") + .ok("SHOW USER FUNCTIONS IN `DB1` NOT ILIKE '%'"); + + sql("show functions from catalog1.db1 not like '%'") + .ok("SHOW FUNCTIONS FROM `CATALOG1`.`DB1` NOT LIKE '%'"); + sql("show user functions from catalog1.db1 not like '%'") + .ok("SHOW USER FUNCTIONS FROM `CATALOG1`.`DB1` NOT LIKE '%'"); + sql("show functions in catalog1.db1 not ilike '%'") + .ok("SHOW FUNCTIONS IN `CATALOG1`.`DB1` NOT ILIKE '%'"); + sql("show user functions in catalog1.db1 not ilike '%'") + .ok("SHOW USER FUNCTIONS IN `CATALOG1`.`DB1` NOT ILIKE '%'"); + + sql("show functions ^likes^") + .fails("(?s).*Encountered \"likes\" at line 1, column 16.\n.*"); + sql("show functions not ^likes^") + .fails("(?s).*Encountered \"likes\" at line 1, column 20" + ".\n" + ".*"); + sql("show functions ^ilikes^") + .fails("(?s).*Encountered \"ilikes\" at line 1, column 16.\n.*"); + sql("show functions not ^ilikes^") + .fails("(?s).*Encountered \"ilikes\" at line 1, column 20" + ".\n" + ".*"); + } + + @Test + void testShowProcedures() { + sql("show procedures").ok("SHOW PROCEDURES"); + sql("show procedures not like '%'").ok("SHOW PROCEDURES NOT LIKE '%'"); + + sql("show procedures from db1").ok("SHOW PROCEDURES FROM `DB1`"); + sql("show procedures in db1").ok("SHOW PROCEDURES IN `DB1`"); + + sql("show procedures from catalog1.db1").ok("SHOW PROCEDURES FROM `CATALOG1`.`DB1`"); + sql("show procedures in catalog1.db1").ok("SHOW PROCEDURES IN `CATALOG1`.`DB1`"); + + sql("show procedures from db1 like '%'").ok("SHOW PROCEDURES FROM `DB1` LIKE '%'"); + sql("show procedures in db1 ilike '%'").ok("SHOW PROCEDURES IN `DB1` ILIKE '%'"); + + sql("show procedures from catalog1.db1 Ilike '%'") + .ok("SHOW PROCEDURES FROM `CATALOG1`.`DB1` ILIKE '%'"); + sql("show procedures in catalog1.db1 like '%'") + .ok("SHOW PROCEDURES IN `CATALOG1`.`DB1` LIKE '%'"); + + sql("show procedures from db1 not like '%'").ok("SHOW PROCEDURES FROM `DB1` NOT LIKE '%'"); + sql("show procedures in db1 not ilike '%'").ok("SHOW PROCEDURES IN `DB1` NOT ILIKE '%'"); + + sql("show procedures from catalog1.db1 not like '%'") + .ok("SHOW PROCEDURES FROM `CATALOG1`.`DB1` NOT LIKE '%'"); + sql("show procedures in catalog1.db1 not ilike '%'") + .ok("SHOW PROCEDURES IN `CATALOG1`.`DB1` NOT ILIKE '%'"); + + sql("show procedures ^db1^").fails("(?s).*Encountered \"db1\" at line 1, column 17.\n.*"); + sql("show procedures ^catalog1^.db1") + .fails("(?s).*Encountered \"catalog1\" at line 1, column 17.\n.*"); + + sql("show procedures ^search^ db1") + .fails("(?s).*Encountered \"search\" at line 1, column 17.\n.*"); + + sql("show procedures from db1 ^likes^ '%t'") + .fails("(?s).*Encountered \"likes\" at line 1, column 26.\n.*"); + } + + @Test + void testShowTables() { + sql("show tables").ok("SHOW TABLES"); + sql("show tables not like '%'").ok("SHOW TABLES NOT LIKE '%'"); + + sql("show tables from db1").ok("SHOW TABLES FROM `DB1`"); + sql("show tables in db1").ok("SHOW TABLES IN `DB1`"); + + sql("show tables from catalog1.db1").ok("SHOW TABLES FROM `CATALOG1`.`DB1`"); + sql("show tables in catalog1.db1").ok("SHOW TABLES IN `CATALOG1`.`DB1`"); + + sql("show tables from db1 like '%'").ok("SHOW TABLES FROM `DB1` LIKE '%'"); + sql("show tables in db1 like '%'").ok("SHOW TABLES IN `DB1` LIKE '%'"); + + sql("show tables from catalog1.db1 like '%'") + .ok("SHOW TABLES FROM `CATALOG1`.`DB1` LIKE '%'"); + sql("show tables in catalog1.db1 like '%'").ok("SHOW TABLES IN `CATALOG1`.`DB1` LIKE '%'"); + + sql("show tables from db1 not like '%'").ok("SHOW TABLES FROM `DB1` NOT LIKE '%'"); + sql("show tables in db1 not like '%'").ok("SHOW TABLES IN `DB1` NOT LIKE '%'"); + + sql("show tables from catalog1.db1 not like '%'") + .ok("SHOW TABLES FROM `CATALOG1`.`DB1` NOT LIKE '%'"); + sql("show tables in catalog1.db1 not like '%'") + .ok("SHOW TABLES IN `CATALOG1`.`DB1` NOT LIKE '%'"); + + sql("show tables ^db1^").fails("(?s).*Encountered \"db1\" at line 1, column 13.\n.*"); + sql("show tables ^catalog1^.db1") + .fails("(?s).*Encountered \"catalog1\" at line 1, column 13.\n.*"); + + sql("show tables ^search^ db1") + .fails("(?s).*Encountered \"search\" at line 1, column 13.\n.*"); + + sql("show tables from db1 ^likes^ '%t'") + .fails("(?s).*Encountered \"likes\" at line 1, column 22.\n.*"); + } + + @Test + void testShowCreateModel() { + sql("show create model m1").ok("SHOW CREATE MODEL `M1`"); + sql("show create model catalog1.db1.m1").ok("SHOW CREATE MODEL `CATALOG1`.`DB1`.`M1`"); + } + + @Test + void testShowCreateTable() { + sql("show create table tbl").ok("SHOW CREATE TABLE `TBL`"); + sql("show create table catalog1.db1.tbl").ok("SHOW CREATE TABLE `CATALOG1`.`DB1`.`TBL`"); + } + + @Test + void testShowCreateView() { + sql("show create view v1").ok("SHOW CREATE VIEW `V1`"); + sql("show create view db1.v1").ok("SHOW CREATE VIEW `DB1`.`V1`"); + sql("show create view catalog1.db1.v1").ok("SHOW CREATE VIEW `CATALOG1`.`DB1`.`V1`"); + } + + @Test + void testDescribeTable() { + sql("describe tbl").ok("DESCRIBE `TBL`"); + sql("describe catalog1.db1.tbl").ok("DESCRIBE `CATALOG1`.`DB1`.`TBL`"); + sql("describe extended db1").ok("DESCRIBE EXTENDED `DB1`"); + + sql("desc tbl").ok("DESCRIBE `TBL`"); + sql("desc catalog1.db1.tbl").ok("DESCRIBE `CATALOG1`.`DB1`.`TBL`"); + sql("desc extended db1").ok("DESCRIBE EXTENDED `DB1`"); + } + + @Test + void testDescribeModel() { + sql("describe model mdl").ok("DESCRIBE MODEL `MDL`"); + sql("describe model catalog1.db1.mdl").ok("DESCRIBE MODEL `CATALOG1`.`DB1`.`MDL`"); + + sql("desc model mdl").ok("DESCRIBE MODEL `MDL`"); + sql("desc model catalog1.db1.mdl").ok("DESCRIBE MODEL `CATALOG1`.`DB1`.`MDL`"); + } + + @Test + void testDescribeFunction() { + sql("describe function fn").ok("DESCRIBE FUNCTION `FN`"); + sql("describe function catalog1.db1.fn").ok("DESCRIBE FUNCTION `CATALOG1`.`DB1`.`FN`"); + sql("describe function extended fn").ok("DESCRIBE FUNCTION EXTENDED `FN`"); + + sql("desc function fn").ok("DESCRIBE FUNCTION `FN`"); + sql("desc function catalog1.db1.fn").ok("DESCRIBE FUNCTION `CATALOG1`.`DB1`.`FN`"); + sql("desc function extended fn").ok("DESCRIBE FUNCTION EXTENDED `FN`"); + } + + @Test + void testShowColumns() { + sql("show columns from tbl").ok("SHOW COLUMNS FROM `TBL`"); + sql("show columns in tbl").ok("SHOW COLUMNS IN `TBL`"); + + sql("show columns from db1.tbl").ok("SHOW COLUMNS FROM `DB1`.`TBL`"); + sql("show columns in db1.tbl").ok("SHOW COLUMNS IN `DB1`.`TBL`"); + + sql("show columns from catalog1.db1.tbl").ok("SHOW COLUMNS FROM `CATALOG1`.`DB1`.`TBL`"); + sql("show columns in catalog1.db1.tbl").ok("SHOW COLUMNS IN `CATALOG1`.`DB1`.`TBL`"); + + sql("show columns from tbl like '%'").ok("SHOW COLUMNS FROM `TBL` LIKE '%'"); + sql("show columns in tbl like '%'").ok("SHOW COLUMNS IN `TBL` LIKE '%'"); + + sql("show columns from db1.tbl like '%'").ok("SHOW COLUMNS FROM `DB1`.`TBL` LIKE '%'"); + sql("show columns in db1.tbl like '%'").ok("SHOW COLUMNS IN `DB1`.`TBL` LIKE '%'"); + + sql("show columns from catalog1.db1.tbl like '%'") + .ok("SHOW COLUMNS FROM `CATALOG1`.`DB1`.`TBL` LIKE '%'"); + sql("show columns in catalog1.db1.tbl like '%'") + .ok("SHOW COLUMNS IN `CATALOG1`.`DB1`.`TBL` LIKE '%'"); + + sql("show columns from tbl not like '%'").ok("SHOW COLUMNS FROM `TBL` NOT LIKE '%'"); + sql("show columns in tbl not like '%'").ok("SHOW COLUMNS IN `TBL` NOT LIKE '%'"); + + sql("show columns from db1.tbl not like '%'") + .ok("SHOW COLUMNS FROM `DB1`.`TBL` NOT LIKE '%'"); + sql("show columns in db1.tbl not like '%'").ok("SHOW COLUMNS IN `DB1`.`TBL` NOT LIKE '%'"); + + sql("show columns from catalog1.db1.tbl not like '%'") + .ok("SHOW COLUMNS FROM `CATALOG1`.`DB1`.`TBL` NOT LIKE '%'"); + sql("show columns in catalog1.db1.tbl not like '%'") + .ok("SHOW COLUMNS IN `CATALOG1`.`DB1`.`TBL` NOT LIKE '%'"); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserMiscTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserMiscTest.java new file mode 100644 index 0000000000000..73d428ebb622a --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserMiscTest.java @@ -0,0 +1,185 @@ +/* + * 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; + +import org.apache.calcite.sql.validate.SqlConformanceEnum; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** Miscellaneous expression and type parser tests. */ +@Execution(CONCURRENT) +class FlinkSqlParserMiscTest extends FlinkSqlParserTestBase { + + @Test + void testArrayAgg() { + sql("select\n" + + " array_agg(ename respect nulls order by deptno, ename) as c1,\n" + + " array_agg(ename order by deptno, ename desc) as c2,\n" + + " array_agg(distinct ename) as c3,\n" + + " array_agg(ename) as c4\n" + + "from emp group by gender") + .ok( + "SELECT" + + " ARRAY_AGG(`ENAME` ORDER BY `DEPTNO`, `ENAME`) RESPECT NULLS AS `C1`," + + " ARRAY_AGG(`ENAME` ORDER BY `DEPTNO`, `ENAME` DESC) AS `C2`," + + " ARRAY_AGG(DISTINCT `ENAME`) AS `C3`," + + " ARRAY_AGG(`ENAME`) AS `C4`\n" + + "FROM `EMP`\n" + + "GROUP BY `GENDER`"); + } + + @Test + void testCastAsMapType() { + this.expr("cast(a as map)").ok("CAST(`A` AS MAP< INTEGER, INTEGER >)"); + this.expr("cast(a as map)") + .ok("CAST(`A` AS MAP< INTEGER, VARCHAR ARRAY >)"); + this.expr("cast(a as map>)") + .ok("CAST(`A` AS MAP< VARCHAR MULTISET, MAP< INTEGER, INTEGER > >)"); + } + + // Override the test because our ROW field type default is nullable, + // which is different with Calcite. + @Test + void testCastAsRowType() { + final String expr = "cast(a as row(f0 int, f1 varchar))"; + final String expected = "CAST(`A` AS ROW(`F0` INTEGER, `F1` VARCHAR))"; + expr(expr).ok(expected); + + final String expr1 = "cast(a as row(f0 int not null, f1 varchar null))"; + final String expected1 = "CAST(`A` AS ROW(`F0` INTEGER NOT NULL, `F1` VARCHAR))"; + expr(expr1).ok(expected1); + + final String expr2 = + "cast(a as row(f0 row(ff0 int not null, ff1 varchar null) null," + + " f1 timestamp not null))"; + final String expected2 = + "CAST(`A` AS ROW(`F0` ROW(`FF0` INTEGER NOT NULL, `FF1` VARCHAR)," + + " `F1` TIMESTAMP NOT NULL))"; + expr(expr2).ok(expected2); + + final String expr3 = "cast(a as row(f0 bigint not null, f1 decimal null) array)"; + final String expected3 = "CAST(`A` AS ROW(`F0` BIGINT NOT NULL, `F1` DECIMAL) ARRAY)"; + expr(expr3).ok(expected3); + + final String expr4 = "cast(a as row(f0 varchar not null, f1 timestamp null) multiset)"; + final String expected4 = "CAST(`A` AS ROW(`F0` VARCHAR NOT NULL, `F1` TIMESTAMP) MULTISET)"; + expr(expr4).ok(expected4); + } + + /* + * This test was backported from Calcite 1.38 (CALCITE-6266). + * Remove it together with upgrade to Calcite 1.38. + */ + @Test + void testFromValuesWithoutParens() { + sql("select 1 from ^values^('x')") + .fails( + "(?s)Encountered \"values\" at line 1, column 15\\.\n" + + "Was expecting one of:\n" + + " \"LATERAL\" \\.\\.\\.\n" + + " \"TABLE\" \\.\\.\\.\n" + + " \\.\\.\\.\n" + + " \\.\\.\\.\n" + + " \\.\\.\\.\n" + + " \\.\\.\\.\n" + + " \\.\\.\\.\n" + + " \\.\\.\\.\n" + + " \\.\\.\\.\n" + + " \"\\(\" \\.\\.\\.\n.*" + + " \"UNNEST\" \\.\\.\\.\n.*"); + } + + /* + * This test was backported from Calcite 1.38 (CALCITE-6266). + * Remove it together with upgrade to Calcite 1.38. + */ + @Test + void testUnnest() { + sql("select*from unnest(x)").ok("SELECT *\n" + "FROM UNNEST(`X`)"); + sql("select*from unnest(x) AS T").ok("SELECT *\n" + "FROM UNNEST(`X`) AS `T`"); + // UNNEST cannot be first word in query + sql("^unnest^(x)").fails("(?s)Encountered \"unnest\" at.*"); + // UNNEST with more than one argument + final String sql = "select * from dept,\n" + "unnest(dept.employees, dept.managers)"; + final String expected = + "SELECT *\n" + "FROM `DEPT`,\n" + "UNNEST(`DEPT`.`EMPLOYEES`, `DEPT`.`MANAGERS`)"; + sql(sql).ok(expected); + + // LATERAL UNNEST is the same as UNNEST + // (LATERAL is implicit for UNNEST, so the parser just ignores it) + sql("select * from dept, lateral unnest(dept.employees)") + .ok("SELECT *\n" + "FROM `DEPT`,\n" + "UNNEST(`DEPT`.`EMPLOYEES`)"); + sql("select * from dept, unnest(dept.employees)") + .ok("SELECT *\n" + "FROM `DEPT`,\n" + "UNNEST(`DEPT`.`EMPLOYEES`)"); + + // Does not generate extra parentheses around UNNEST because UNNEST is + // a table expression. + final String sql1 = + "" + + "SELECT\n" + + " item.name,\n" + + " relations.*\n" + + "FROM dfs.tmp item\n" + + "JOIN (\n" + + " SELECT * FROM UNNEST(item.related) i(rels)\n" + + ") relations\n" + + "ON TRUE"; + final String expected1 = + "SELECT `ITEM`.`NAME`, `RELATIONS`.*\n" + + "FROM `DFS`.`TMP` AS `ITEM`\n" + + "INNER JOIN (SELECT *\n" + + "FROM UNNEST(`ITEM`.`RELATED`) AS `I` (`RELS`)) AS `RELATIONS` ON TRUE"; + sql(sql1).ok(expected1); + } + + @Test + void testOuterApplyFunctionFails() { + final String sql = "select * from dept outer apply ramp(deptno)^)^"; + sql(sql).withConformance(SqlConformanceEnum.SQL_SERVER_2008) + .fails("(?s).*Encountered \"\\)\" at .*"); + } + + @Test + void testVariantType() { + sql("CREATE TABLE t (\n" + "v variant" + "\n)") + .ok("CREATE TABLE `T` (\n" + " `V` VARIANT\n" + ")"); + + sql("CREATE TABLE t (\n" + "v VARIANT NOT NULL" + "\n)") + .ok("CREATE TABLE `T` (\n" + " `V` VARIANT NOT NULL\n" + ")"); + } + + @Test + void testBitmapType() { + sql("CREATE TABLE t (\n" + "bm bitmap" + "\n)") + .ok("CREATE TABLE `T` (\n" + " `BM` BITMAP\n" + ")"); + + sql("CREATE TABLE t (\n" + "bm bitmap NOT NULL" + "\n)") + .ok("CREATE TABLE `T` (\n" + " `BM` BITMAP NOT NULL\n" + ")"); + + // BITMAP takes no parameters + sql("CREATE TABLE t (\n" + "bm bitmap^(^1)" + "\n)") + .fails("(?s).*Encountered \"\\(\" at line 2, column 10.\n.*"); + + // BITMAP is a reserved keyword and cannot be used as an identifier without escaping + sql("CREATE TABLE t (\n" + "^bitmap^ INT" + "\n)") + .fails("(?s).*Encountered \"bitmap\" at line 2, column 1.\n.*"); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserModelTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserModelTest.java new file mode 100644 index 0000000000000..daf3dfb278949 --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserModelTest.java @@ -0,0 +1,269 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** MODEL parser tests. */ +@Execution(CONCURRENT) +class FlinkSqlParserModelTest extends FlinkSqlParserTestBase { + + @Test + void testShowModels() { + sql("show models").ok("SHOW MODELS"); + sql("show models from db1").ok("SHOW MODELS FROM `DB1`"); + sql("show models from catalog1.db1").ok("SHOW MODELS FROM `CATALOG1`.`DB1`"); + sql("show models in db1").ok("SHOW MODELS IN `DB1`"); + sql("show models in catalog1.db1").ok("SHOW MODELS IN `CATALOG1`.`DB1`"); + } + + @Test + void testDropModel() { + sql("drop model m1").ok("DROP MODEL `M1`"); + sql("drop model db1.m1").ok("DROP MODEL `DB1`.`M1`"); + sql("drop model catalog1.db1.m1").ok("DROP MODEL `CATALOG1`.`DB1`.`M1`"); + } + + @Test + void testDropTemporaryModel() { + sql("drop temporary model m1").ok("DROP TEMPORARY MODEL `M1`"); + sql("drop temporary model if exists m1").ok("DROP TEMPORARY MODEL IF EXISTS `M1`"); + } + + @Test + void testDropModelIfExists() { + sql("drop model if exists catalog1.db1.m1") + .ok("DROP MODEL IF EXISTS `CATALOG1`.`DB1`.`M1`"); + } + + @Test + void testAlterModelSet() { + final String sql = "alter model m1 set ('key1' = 'value1','key2' = 'value2')"; + final String expected = + "ALTER MODEL `M1` SET (\n" + + " 'key1' = 'value1',\n" + + " 'key2' = 'value2'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testAlterModelIfExists() { + final String sql = "alter model if exists m1 set ('key1' = 'value1','key2' = 'value2')"; + final String expected = + "ALTER MODEL IF EXISTS `M1` SET (\n" + + " 'key1' = 'value1',\n" + + " 'key2' = 'value2'\n" + + ")"; + sql(sql).ok(expected); + } + + @Test + void testAlterModelRename() { + final String sql = "alter model m1 rename to m2"; + final String expected = "ALTER MODEL `M1` RENAME TO `M2`"; + sql(sql).ok(expected); + } + + @Test + void testAlterModelRenameIfExists() { + final String sql = "alter model if exists m1 rename to m2"; + final String expected = "ALTER MODEL IF EXISTS `M1` RENAME TO `M2`"; + sql(sql).ok(expected); + } + + @Test + void testAlterModelReset() { + final String sql = "alter model m1 reset ('key1', 'key2')"; + final String expected = "ALTER MODEL `M1` RESET (\n 'key1',\n 'key2'\n)"; + sql(sql).ok(expected); + } + + @Test + void testAlterModelResetIfExists() { + final String sql = "alter model if exists m1 reset ('key1', 'key2')"; + final String expected = "ALTER MODEL IF EXISTS `M1` RESET (\n 'key1',\n 'key2'\n)"; + sql(sql).ok(expected); + } + + @Test + void testCreateModel() { + sql("create model m1\n" + + " INPUT(col1 INT, col2 STRING)\n" + + " OUTPUT(label DOUBLE)\n" + + " COMMENT 'model_comment'\n" + + " WITH (\n" + + " 'key1'='value1',\n" + + " 'key2'='value2'\n" + + " )\n") + .ok( + "CREATE MODEL `M1` INPUT (\n" + + " `COL1` INTEGER,\n" + + " `COL2` STRING\n" + + ") OUTPUT (\n" + + " `LABEL` DOUBLE\n" + + ")\n" + + "COMMENT 'model_comment'" + + "\nWITH (\n" + + " 'key1' = 'value1',\n" + + " 'key2' = 'value2'\n" + + ")"); + } + + @Test + void testCreateModelIfNotExists() { + sql("create model if not exists m1\n" + + " INPUT(col1 INT, col2 STRING)\n" + + " OUTPUT(label DOUBLE)\n" + + " COMMENT 'model_comment'\n" + + " WITH (\n" + + " 'key1'='value1',\n" + + " 'key2'='value2'\n" + + " )\n") + .ok( + "CREATE MODEL IF NOT EXISTS `M1` INPUT (\n" + + " `COL1` INTEGER,\n" + + " `COL2` STRING\n" + + ") OUTPUT (\n" + + " `LABEL` DOUBLE\n" + + ")\n" + + "COMMENT 'model_comment'" + + "\nWITH (\n" + + " 'key1' = 'value1',\n" + + " 'key2' = 'value2'\n" + + ")"); + } + + @Test + void testCreateModelAs() { + sql("create model m1\n" + + " WITH (\n" + + " 'key1'='value1',\n" + + " 'key2'='value2'\n" + + " ) as select f1, f2 from t1\n") + .ok( + "CREATE MODEL `M1`" + + "\nWITH (\n" + + " 'key1' = 'value1',\n" + + " 'key2' = 'value2'\n" + + ")\n" + + "AS\n" + + "SELECT `F1`, `F2`\n" + + "FROM `T1`"); + } + + @Test + void testCreateModelAsIfNotExists() { + sql("create model if not exists m1\n" + + " WITH (\n" + + " 'key1'='value1',\n" + + " 'key2'='value2'\n" + + " ) as select f1, f2 from t1\n") + .ok( + "CREATE MODEL IF NOT EXISTS `M1`" + + "\nWITH (\n" + + " 'key1' = 'value1',\n" + + " 'key2' = 'value2'\n" + + ")\n" + + "AS\n" + + "SELECT `F1`, `F2`\n" + + "FROM `T1`"); + } + + @Test + void testCreateModelAsWithInput() { + sql("create model if not exists m1\n" + + " INPUT(col1 INT, col2 STRING)\n" + + " OUTPUT(label DOUBLE)\n" + + " WITH (\n" + + " 'key1'='value1',\n" + + " 'key2'='value2'\n" + + " ) as select f1, f2 from t1\n") + .ok( + "CREATE MODEL IF NOT EXISTS `M1` INPUT (\n" + + " `COL1` INTEGER,\n" + + " `COL2` STRING\n" + + ") OUTPUT (\n" + + " `LABEL` DOUBLE\n" + + ")" + + "\nWITH (\n" + + " 'key1' = 'value1',\n" + + " 'key2' = 'value2'\n" + + ")\n" + + "AS\n" + + "SELECT `F1`, `F2`\n" + + "FROM `T1`") + .node( + new ValidationMatcher() + .fails( + "CREATE MODEL AS SELECT syntax does not support to specify explicit input columns.")); + } + + @Test + void testCreateModelAsWithOutput() { + sql("create model if not exists m1\n" + + " OUTPUT(label DOUBLE)\n" + + " WITH (\n" + + " 'key1'='value1',\n" + + " 'key2'='value2'\n" + + " ) as select f1, f2 from t1\n") + .ok( + "CREATE MODEL IF NOT EXISTS `M1` OUTPUT (\n" + + " `LABEL` DOUBLE\n" + + ")" + + "\nWITH (\n" + + " 'key1' = 'value1',\n" + + " 'key2' = 'value2'\n" + + ")\n" + + "AS\n" + + "SELECT `F1`, `F2`\n" + + "FROM `T1`") + .node( + new ValidationMatcher() + .fails( + "CREATE MODEL AS SELECT syntax does not support to specify explicit output columns.")); + } + + @Test + void testModelInFunction() { + sql("select * from table(ml_predict(TABLE my_table, MODEL my_model))") + .ok( + "SELECT *\n" + + "FROM TABLE(`ML_PREDICT`((TABLE `MY_TABLE`), MODEL `MY_MODEL`))"); + } + + @Test + void testModelInFunctionWithoutTable() { + sql("select * from func(TABLE my_table, MODEL cat.db.my_model)") + .ok( + "SELECT *\n" + + "FROM TABLE(`FUNC`((TABLE `MY_TABLE`), MODEL `CAT`.`DB`.`MY_MODEL`))"); + } + + @Test + void testModelInFunctionNamedArgs() { + sql("select * from table(ml_predict(INPUT => TABLE my_table, model => MODEL my_model))") + .ok( + "SELECT *\n" + + "FROM TABLE(`ML_PREDICT`(`INPUT` => (TABLE `MY_TABLE`), `MODEL` => (MODEL `MY_MODEL`)))"); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserTestBase.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserTestBase.java new file mode 100644 index 0000000000000..5656cf854cbfa --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserTestBase.java @@ -0,0 +1,79 @@ +/* + * 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; + +import org.apache.flink.sql.parser.error.SqlValidateException; +import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl; + +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.parser.SqlParserFixture; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.hamcrest.TypeSafeDiagnosingMatcher; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** Base class providing shared parser test utilities for Flink SQL parser tests. */ +@Execution(CONCURRENT) +abstract class FlinkSqlParserTestBase { + + public SqlParserFixture fixture() { + return SqlParserFixture.DEFAULT.withConfig( + c -> c.withParserFactory(FlinkSqlParserImpl.FACTORY)); + } + + protected SqlParserFixture sql(String sql) { + return fixture().sql(sql); + } + + protected SqlParserFixture expr(String sql) { + return sql(sql).expression(true); + } + + public static BaseMatcher validated(String validatedSql) { + return new TypeSafeDiagnosingMatcher() { + @Override + protected boolean matchesSafely(SqlNode item, Description mismatchDescription) { + if (item instanceof ExtendedSqlNode) { + try { + ((ExtendedSqlNode) item).validate(); + } catch (SqlValidateException e) { + mismatchDescription.appendText( + "Could not validate the node. Exception: \n"); + mismatchDescription.appendValue(e); + } + + String actual = item.toSqlString(null, true).getSql(); + return actual.equals(validatedSql); + } + mismatchDescription.appendText( + "This matcher can be applied only to ExtendedSqlNode."); + return false; + } + + @Override + public void describeTo(Description description) { + description.appendText( + "The validated node string representation should be equal to: \n"); + description.appendText(validatedSql); + } + }; + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserUnParserTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserUnParserTest.java new file mode 100644 index 0000000000000..4dc1652cf66e1 --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserUnParserTest.java @@ -0,0 +1,117 @@ +/* + * 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; + +import org.apache.calcite.sql.parser.SqlParserFixture; +import org.apache.calcite.sql.parser.SqlParserTest; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** + * Unparser (round-trip) tests for all Flink SQL parser syntax categories. + * + *

Each {@link Nested} inner class extends the corresponding parser test class and overrides + * {@link #fixture()} to use {@link SqlParserTest.UnparsingTesterImpl}, which verifies that parse + * → unparse → re-parse produces consistent results. + */ +@Execution(CONCURRENT) +class FlinkSqlParserUnParserTest { + + @Nested + class CatalogUnParserTest extends FlinkSqlParserCatalogTest { + @Override + public SqlParserFixture fixture() { + return super.fixture().withTester(new SqlParserTest.UnparsingTesterImpl()); + } + } + + @Nested + class MetadataUnParserTest extends FlinkSqlParserMetadataTest { + @Override + public SqlParserFixture fixture() { + return super.fixture().withTester(new SqlParserTest.UnparsingTesterImpl()); + } + } + + @Nested + class CreateTableUnParserTest extends FlinkSqlParserCreateTableTest { + @Override + public SqlParserFixture fixture() { + return super.fixture().withTester(new SqlParserTest.UnparsingTesterImpl()); + } + } + + @Nested + class AlterTableUnParserTest extends FlinkSqlParserAlterTableTest { + @Override + public SqlParserFixture fixture() { + return super.fixture().withTester(new SqlParserTest.UnparsingTesterImpl()); + } + } + + @Nested + class CtasUnParserTest extends FlinkSqlParserCtasTest { + @Override + public SqlParserFixture fixture() { + return super.fixture().withTester(new SqlParserTest.UnparsingTesterImpl()); + } + } + + @Nested + class ViewUnParserTest extends FlinkSqlParserViewTest { + @Override + public SqlParserFixture fixture() { + return super.fixture().withTester(new SqlParserTest.UnparsingTesterImpl()); + } + } + + @Nested + class ModelUnParserTest extends FlinkSqlParserModelTest { + @Override + public SqlParserFixture fixture() { + return super.fixture().withTester(new SqlParserTest.UnparsingTesterImpl()); + } + } + + @Nested + class ConnectionUnParserTest extends FlinkSqlParserConnectionTest { + @Override + public SqlParserFixture fixture() { + return super.fixture().withTester(new SqlParserTest.UnparsingTesterImpl()); + } + } + + @Nested + class ExecuteUnParserTest extends FlinkSqlParserExecuteTest { + @Override + public SqlParserFixture fixture() { + return super.fixture().withTester(new SqlParserTest.UnparsingTesterImpl()); + } + } + + @Nested + class MiscUnParserTest extends FlinkSqlParserMiscTest { + @Override + public SqlParserFixture fixture() { + return super.fixture().withTester(new SqlParserTest.UnparsingTesterImpl()); + } + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserViewTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserViewTest.java new file mode 100644 index 0000000000000..ddd856f72b448 --- /dev/null +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserViewTest.java @@ -0,0 +1,370 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; + +import java.util.List; + +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +/** VIEW, FUNCTION and MODULE parser tests. */ +@Execution(CONCURRENT) +class FlinkSqlParserViewTest extends FlinkSqlParserTestBase { + + @Test + void testCreateView() { + final String sql = "create view v as select col1 from tbl"; + final String expected = "CREATE VIEW `V`\n" + "AS\n" + "SELECT `COL1`\n" + "FROM `TBL`"; + sql(sql).ok(expected); + } + + @Test + void testCreateViewWithInvalidFieldList() { + final String expected = + "(?s).*Encountered \"\\)\" at line 1, column 15.\n" + + "Was expecting one of:\n" + + ".*\n" + + ".*\n" + + ".*\n" + + ".*\n" + + ".*"; + sql("CREATE VIEW V(^)^ AS SELECT * FROM TBL").fails(expected); + } + + @Test + void testCreateViewWithComment() { + final String sql = "create view v COMMENT 'this is a view' as select col1 from tbl"; + final String expected = + "CREATE VIEW `V`\n" + + "COMMENT 'this is a view'\n" + + "AS\n" + + "SELECT `COL1`\n" + + "FROM `TBL`"; + sql(sql).ok(expected); + } + + @Test + void testCreateViewWithFieldNames() { + final String sql = "create view v(col1, col2) as select col3, col4 from tbl"; + final String expected = + "CREATE VIEW `V` (`COL1`, `COL2`)\n" + + "AS\n" + + "SELECT `COL3`, `COL4`\n" + + "FROM `TBL`"; + sql(sql).ok(expected); + } + + @Test + void testCreateViewWithInvalidName() { + final String sql = "create view v(^*^) COMMENT 'this is a view' as select col1 from tbl"; + final String expected = "(?s).*Encountered \"\\*\" at line 1, column 15.*"; + + sql(sql).fails(expected); + } + + @Test + void testCreateTemporaryView() { + final String sql = "create temporary view v as select col1 from tbl"; + final String expected = + "CREATE TEMPORARY VIEW `V`\n" + "AS\n" + "SELECT `COL1`\n" + "FROM `TBL`"; + sql(sql).ok(expected); + } + + @Test + void testCreateTemporaryViewIfNotExists() { + final String sql = "create temporary view if not exists v as select col1 from tbl"; + final String expected = + "CREATE TEMPORARY VIEW IF NOT EXISTS `V`\n" + + "AS\n" + + "SELECT `COL1`\n" + + "FROM `TBL`"; + sql(sql).ok(expected); + } + + @Test + void testCreateViewIfNotExists() { + final String sql = "create view if not exists v as select col1 from tbl"; + final String expected = + "CREATE VIEW IF NOT EXISTS `V`\n" + "AS\n" + "SELECT `COL1`\n" + "FROM `TBL`"; + sql(sql).ok(expected); + } + + @Test + void testDropView() { + final String sql = "DROP VIEW IF EXISTS view_name"; + final String expected = "DROP VIEW IF EXISTS `VIEW_NAME`"; + sql(sql).ok(expected); + } + + @Test + void testDropTemporaryView() { + final String sql = "DROP TEMPORARY VIEW IF EXISTS view_name"; + final String expected = "DROP TEMPORARY VIEW IF EXISTS `VIEW_NAME`"; + sql(sql).ok(expected); + } + + @Test + void testAlterView() { + sql("ALTER VIEW v1 RENAME TO v2").ok("ALTER VIEW `V1` RENAME TO `V2`"); + sql("ALTER VIEW v1 AS SELECT c1, c2 FROM tbl") + .ok("ALTER VIEW `V1`\n" + "AS\n" + "SELECT `C1`, `C2`\n" + "FROM `TBL`"); + } + + @Test + void testShowViews() { + sql("show views").ok("SHOW VIEWS"); + sql("show views not like '%'").ok("SHOW VIEWS NOT LIKE '%'"); + + sql("show views from db1").ok("SHOW VIEWS FROM `DB1`"); + sql("show views in db1").ok("SHOW VIEWS IN `DB1`"); + + sql("show views from catalog1.db1").ok("SHOW VIEWS FROM `CATALOG1`.`DB1`"); + sql("show views in catalog1.db1").ok("SHOW VIEWS IN `CATALOG1`.`DB1`"); + + sql("show views from db1 like '%'").ok("SHOW VIEWS FROM `DB1` LIKE '%'"); + sql("show views in db1 like '%'").ok("SHOW VIEWS IN `DB1` LIKE '%'"); + + sql("show views from catalog1.db1 like '%'") + .ok("SHOW VIEWS FROM `CATALOG1`.`DB1` LIKE '%'"); + sql("show views in catalog1.db1 like '%'").ok("SHOW VIEWS IN `CATALOG1`.`DB1` LIKE '%'"); + + sql("show views from db1 not like '%'").ok("SHOW VIEWS FROM `DB1` NOT LIKE '%'"); + sql("show views in db1 not like '%'").ok("SHOW VIEWS IN `DB1` NOT LIKE '%'"); + + sql("show views from catalog1.db1 not like '%'") + .ok("SHOW VIEWS FROM `CATALOG1`.`DB1` NOT LIKE '%'"); + sql("show views in catalog1.db1 not like '%'") + .ok("SHOW VIEWS IN `CATALOG1`.`DB1` NOT LIKE '%'"); + + sql("show views ^db1^").fails("(?s).*Encountered \"db1\" at line 1, column 12.\n.*"); + sql("show views ^catalog1^.db1") + .fails("(?s).*Encountered \"catalog1\" at line 1, column 12.\n.*"); + + sql("show views ^search^ db1") + .fails("(?s).*Encountered \"search\" at line 1, column 12.\n.*"); + + sql("show views from db1 ^likes^ '%t'") + .fails("(?s).*Encountered \"likes\" at line 1, column 21.\n.*"); + } + + @Test + void testShowPartitions() { + sql("show partitions c1.d1.tbl").ok("SHOW PARTITIONS `C1`.`D1`.`TBL`"); + sql("show partitions tbl partition (p=1)").ok("SHOW PARTITIONS `TBL` PARTITION (`P` = 1)"); + } + + @Test + void testCreateTableWithNakedTableName() { + String sql = "CREATE TABLE tbl1"; + sql(sql).node(new ValidationMatcher()); + } + + @Test + void testCreateViewWithEmptyFields() { + String sql = "CREATE VIEW v1 AS SELECT 1"; + sql(sql).ok("CREATE VIEW `V1`\n" + "AS\n" + "SELECT 1"); + } + + @Test + void testCreateFunction() { + sql("create function catalog1.db1.function1 as 'org.apache.flink.function.function1'") + .ok( + "CREATE FUNCTION `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.flink.function.function1'"); + + sql("create temporary function catalog1.db1.function1 as 'org.apache.flink.function.function1'") + .ok( + "CREATE TEMPORARY FUNCTION `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.flink.function.function1'"); + + sql("create temporary function db1.function1 as 'org.apache.flink.function.function1'") + .ok( + "CREATE TEMPORARY FUNCTION `DB1`.`FUNCTION1` AS 'org.apache.flink.function.function1'"); + + sql("create temporary function function1 as 'org.apache.flink.function.function1'") + .ok( + "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1'"); + + sql("create temporary function if not exists catalog1.db1.function1 as 'org.apache.flink.function.function1'") + .ok( + "CREATE TEMPORARY FUNCTION IF NOT EXISTS `CATALOG1`.`DB1`.`FUNCTION1` AS 'org.apache.flink.function.function1'"); + + sql("create temporary function function1 as 'org.apache.flink.function.function1' language java") + .ok( + "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA"); + + sql("create temporary system function function1 as 'org.apache.flink.function.function1' language scala") + .ok( + "CREATE TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE SCALA"); + + // Temporary system function always belongs to the system and current session. + sql("create temporary system function catalog1^.^db1.function1 as 'org.apache.flink.function.function1'") + .fails("(?s).*Encountered \".\" at.*"); + + sql("create ^system^ function function1 as 'org.apache.flink.function.function1'") + .fails( + "CREATE SYSTEM FUNCTION is not supported, " + + "system functions can only be registered as temporary " + + "functions, you can use CREATE TEMPORARY SYSTEM FUNCTION instead."); + + // test creating functions with either jar or artifact + for (String usageType : List.of("JAR", "ARTIFACT")) { + sql("create temporary function function1 as 'org.apache.flink.function.function1' language java using " + + usageType + + " 'file:///path/to/test.jar'") + .ok( + "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING " + + usageType + + " 'file:///path/to/test.jar'"); + + sql("create temporary function function1 as 'org.apache.flink.function.function1' language scala using " + + usageType + + " '/path/to/test.jar'") + .ok( + "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE SCALA USING " + + usageType + + " '/path/to/test.jar'"); + + sql("create temporary system function function1 as 'org.apache.flink.function.function1' language scala using " + + usageType + + " '/path/to/test.jar'") + .ok( + "CREATE TEMPORARY SYSTEM FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE SCALA USING " + + usageType + + " '/path/to/test.jar'"); + + sql("create function function1 as 'org.apache.flink.function.function1' language java using " + + usageType + + " 'file:///path/to/test.jar', jar 'hdfs:///path/to/test2.jar'") + .ok( + "CREATE FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING " + + usageType + + " 'file:///path/to/test.jar', JAR 'hdfs:///path/to/test2.jar'"); + + sql("create temporary function function1 as 'org.apache.flink.function.function1' language ^sql^ using " + + usageType + + " 'file:///path/to/test.jar'") + .fails( + "CREATE FUNCTION USING JAR/ARTIFACT syntax is not applicable to SQL language."); + + sql("create temporary function function1 as 'org.apache.flink.function.function1' language ^python^ using " + + usageType + + " 'file:///path/to/test.jar'") + .fails( + "CREATE FUNCTION USING JAR/ARTIFACT syntax is not applicable to PYTHON language."); + + sql("create function function1 as 'org.apache.flink.function.function1' language java using " + + usageType + + " 'file:///path/to/test.jar' WITH ('k1' = 'v1', 'k2' = 'v2')") + .ok( + "CREATE FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING " + + usageType + + " 'file:///path/to/test.jar'\nWITH (\n" + + " 'k1' = 'v1',\n" + + " 'k2' = 'v2'\n" + + ")"); + + sql("create temporary function function1 as 'org.apache.flink.function.function1' language java using " + + usageType + + " 'file:///path/to/test.jar' WITH ('k1' = 'v1', 'k2' = 'v2')") + .ok( + "CREATE TEMPORARY FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING " + + usageType + + " 'file:///path/to/test.jar'\nWITH (\n" + + " 'k1' = 'v1',\n" + + " 'k2' = 'v2'\n" + + ")"); + } + + // test mixing jar and artifact keywords + sql("create function function1 as 'org.apache.flink.function.function1' language java using jar 'file:///path/to/test.jar', artifact 'hdfs:///path/to/test2.jar'") + .ok( + "CREATE FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING JAR 'file:///path/to/test.jar', ARTIFACT 'hdfs:///path/to/test2.jar'"); + + sql("create function function1 as 'org.apache.flink.function.function1' language java using artifact 'file:///path/to/test.jar', jar 'hdfs:///path/to/test2.jar'") + .ok( + "CREATE FUNCTION `FUNCTION1` AS 'org.apache.flink.function.function1' LANGUAGE JAVA USING ARTIFACT 'file:///path/to/test.jar', JAR 'hdfs:///path/to/test2.jar'"); + + sql("create temporary function function1 as 'org.apache.flink.function.function1' language java using ^file^ 'file:///path/to/test'") + .fails( + "Encountered \"file\" at line 1, column 98.\n" + + "Was expecting one of:\n" + + " \"ARTIFACT\" ...\n" + + " \"JAR\" ...\n" + + " .*"); + } + + @Test + void testDropTemporaryFunction() { + sql("drop temporary function catalog1.db1.function1") + .ok("DROP TEMPORARY FUNCTION `CATALOG1`.`DB1`.`FUNCTION1`"); + + sql("drop temporary system function catalog1.db1.function1") + .ok("DROP TEMPORARY SYSTEM FUNCTION `CATALOG1`.`DB1`.`FUNCTION1`"); + + sql("drop temporary function if exists catalog1.db1.function1") + .ok("DROP TEMPORARY FUNCTION IF EXISTS `CATALOG1`.`DB1`.`FUNCTION1`"); + + sql("drop temporary system function if exists catalog1.db1.function1") + .ok("DROP TEMPORARY SYSTEM FUNCTION IF EXISTS `CATALOG1`.`DB1`.`FUNCTION1`"); + } + + @Test + void testLoadModule() { + sql("load module core").ok("LOAD MODULE `CORE`"); + + sql("load module dummy with ('k1' = 'v1', 'k2' = 'v2')") + .ok( + "LOAD MODULE `DUMMY`" + + "\nWITH (\n" + + " 'k1' = 'v1',\n" + + " 'k2' = 'v2'\n" + + ")"); + + sql("load module ^'core'^") + .fails("(?s).*Encountered \"\\\\'core\\\\'\" at line 1, column 13.\n.*"); + } + + @Test + void testUnloadModule() { + sql("unload module core").ok("UNLOAD MODULE `CORE`"); + + sql("unload module ^'core'^") + .fails("(?s).*Encountered \"\\\\'core\\\\'\" at line 1, column 15.\n.*"); + } + + @Test + void testUseModules() { + sql("use modules core").ok("USE MODULES `CORE`"); + + sql("use modules x, y, z").ok("USE MODULES `X`, `Y`, `Z`"); + + sql("use modules x^,^").fails("(?s).*Encountered \"\" at line 1, column 14.\n.*"); + + sql("use modules ^'core'^") + .fails("(?s).*Encountered \"\\\\'core\\\\'\" at line 1, column 13.\n.*"); + } + + @Test + void testShowModules() { + sql("show modules").ok("SHOW MODULES"); + + sql("show full modules").ok("SHOW FULL MODULES"); + } +} diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlUnParserTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlUnParserTest.java deleted file mode 100644 index cced9d2e44055..0000000000000 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlUnParserTest.java +++ /dev/null @@ -1,45 +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; - -import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl; - -import org.apache.calcite.sql.parser.SqlParserFixture; -import org.junit.jupiter.api.parallel.Execution; - -import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; - -/** - * Extension to {@link FlinkSqlParserImplTest} that ensures that every expression can un-parse - * successfully. - */ -@Execution(CONCURRENT) -class FlinkSqlUnParserTest extends FlinkSqlParserImplTest { - // ~ Constructors ----------------------------------------------------------- - - public FlinkSqlUnParserTest() {} - - // ~ Methods ---------------------------------------------------------------- - - public SqlParserFixture fixture() { - return super.fixture() - .withTester(new UnparsingTesterImpl()) - .withConfig(c -> c.withParserFactory(FlinkSqlParserImpl.FACTORY)); - } -} diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml index 84cbbdd619db7..3751169541b8f 100644 --- a/tools/maven/suppressions.xml +++ b/tools/maven/suppressions.xml @@ -34,7 +34,6 @@ under the License. -