Skip to content

Commit ae4f20e

Browse files
LadyForestwuchong
authored andcommitted
[FLINK-16384][table][sql-client] Improve implementation of 'SHOW CREATE TABLE' statement.
This commit tries to 1. resolve the conflicts 2. revert the changes made on old planner 3. apply spotless formatting 4. fix DDL missing `TEMPORARY` keyword for temporary table 5. display table's full object path as catalog.db.table 6. support displaying the expanded query for view 7. add view test in CatalogTableITCase, and adapt sql client test to the new test framework 8. adapt docs This closes #13011
1 parent a3437d5 commit ae4f20e

File tree

26 files changed

+432
-315
lines changed

26 files changed

+432
-315
lines changed

docs/content.zh/docs/dev/table/sql/show.md

Lines changed: 42 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,14 +28,15 @@ under the License.
2828

2929

3030

31-
SHOW 语句用于列出所有的 catalog,或者列出当前 catalog 中所有的 database,或者列出当前 catalog 和当前 database 的所有表或视图,或者列出当前正在使用的 catalog 和 database, 或者列出当前 catalog 和当前 database 中所有的 function,包括:系统 function 和用户定义的 function,或者仅仅列出当前 catalog 和当前 database 中用户定义的 function,或者列出当前环境所有激活的 module,或者列出当前环境所有加载的 module 及激活状态。
31+
SHOW 语句用于列出所有的 catalog,或者列出当前 catalog 中所有的 database,或者列出当前 catalog 和当前 database 的所有表或视图,或者列出当前正在使用的 catalog 和 database, 或者列出创建指定表的语句,或者列出当前 catalog 和当前 database 中所有的 function,包括:系统 function 和用户定义的 function,或者仅仅列出当前 catalog 和当前 database 中用户定义的 function,或者列出当前环境所有激活的 module,或者列出当前环境所有加载的 module 及激活状态。
3232

3333
目前 Flink SQL 支持下列 SHOW 语句:
3434
- SHOW CATALOGS
3535
- SHOW CURRENT CATALOG
3636
- SHOW DATABASES
3737
- SHOW CURRENT DATABASE
3838
- SHOW TABLES
39+
- SHOW CREATE TABLE
3940
- SHOW VIEWS
4041
- SHOW FUNCTIONS
4142
- SHOW MODULES
@@ -120,6 +121,15 @@ tEnv.executeSql("SHOW TABLES").print();
120121
// | my_table |
121122
// +------------+
122123

124+
// show create table
125+
tEnv.executeSql("SHOW CREATE TABLE my_table").print();
126+
// CREATE TABLE `default_catalog`.`default_db`.`my_table` (
127+
// ...
128+
// ) WITH (
129+
// ...
130+
// )
131+
132+
123133
// create a view
124134
tEnv.executeSql("CREATE VIEW my_view AS ...");
125135
// show views
@@ -201,6 +211,13 @@ tEnv.executeSql("SHOW TABLES").print()
201211
// | my_table |
202212
// +------------+
203213

214+
// show create table
215+
tEnv.executeSql("SHOW CREATE TABLE my_table").print()
216+
// CREATE TABLE `default_catalog`.`default_db`.`my_table` (
217+
// ...
218+
// ) WITH (
219+
// ...
220+
// )
204221
// create a view
205222
tEnv.executeSql("CREATE VIEW my_view AS ...")
206223
// show views
@@ -281,6 +298,13 @@ table_env.execute_sql("SHOW TABLES").print()
281298
# +------------+
282299
# | my_table |
283300
# +------------+
301+
# show create table
302+
table_env.executeSql("SHOW CREATE TABLE my_table").print()
303+
# CREATE TABLE `default_catalog`.`default_db`.`my_table` (
304+
# ...
305+
# ) WITH (
306+
# ...
307+
# )
284308

285309
# create a view
286310
table_env.execute_sql("CREATE VIEW my_view AS ...")
@@ -346,6 +370,13 @@ Flink SQL> CREATE TABLE my_table (...) WITH (...);
346370
Flink SQL> SHOW TABLES;
347371
my_table
348372

373+
Flink SQL> SHOW CREATE TABLE my_table;
374+
CREATE TABLE `default_catalog`.`default_db`.`my_table` (
375+
...
376+
) WITH (
377+
...
378+
)
379+
349380
Flink SQL> CREATE VIEW my_view AS ...;
350381
[INFO] View has been created.
351382

@@ -428,6 +459,16 @@ SHOW TABLES
428459

429460
展示当前 catalog 和当前 database 中所有的表。
430461

462+
## SHOW CREATE TABLE
463+
464+
```sql
465+
SHOW CREATE TABLE [catalog_name.][db_name.]table_name
466+
```
467+
468+
展示创建指定表的 create 语句。
469+
470+
<span class="label label-danger">Attention</span> 目前 `SHOW CREATE TABLE` 只支持通过 Flink SQL DDL 创建的表。
471+
431472
## SHOW VIEWS
432473

433474
```sql

docs/content/docs/dev/table/sql/show.md

Lines changed: 43 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,14 +26,15 @@ under the License.
2626

2727
# SHOW Statements
2828

29-
SHOW statements are used to list all catalogs, or list all databases in the current catalog, or list all tables/views in the current catalog and the current database, or show current catalog and database, or list all functions including system functions and user-defined functions in the current catalog and current database, or list only user-defined functions in the current catalog and current database, or list enabled module names, or list all loaded modules with enabled status in the current session.
29+
SHOW statements are used to list all catalogs, or list all databases in the current catalog, or list all tables/views in the current catalog and the current database, or show current catalog and database, or show create statement for specified table, or list all functions including system functions and user-defined functions in the current catalog and current database, or list only user-defined functions in the current catalog and current database, or list enabled module names, or list all loaded modules with enabled status in the current session.
3030

3131
Flink SQL supports the following SHOW statements for now:
3232
- SHOW CATALOGS
3333
- SHOW CURRENT CATALOG
3434
- SHOW DATABASES
3535
- SHOW CURRENT DATABASE
3636
- SHOW TABLES
37+
- SHOW CREATE TABLE
3738
- SHOW VIEWS
3839
- SHOW FUNCTIONS
3940
- SHOW MODULES
@@ -119,6 +120,15 @@ tEnv.executeSql("SHOW TABLES").print();
119120
// | my_table |
120121
// +------------+
121122

123+
// show create table
124+
tEnv.executeSql("SHOW CREATE TABLE my_table").print();
125+
// CREATE TABLE `default_catalog`.`default_db`.`my_table` (
126+
// ...
127+
// ) WITH (
128+
// ...
129+
// )
130+
131+
122132
// create a view
123133
tEnv.executeSql("CREATE VIEW my_view AS ...");
124134
// show views
@@ -200,6 +210,13 @@ tEnv.executeSql("SHOW TABLES").print()
200210
// | my_table |
201211
// +------------+
202212

213+
// show create table
214+
tEnv.executeSql("SHOW CREATE TABLE my_table").print()
215+
// CREATE TABLE `default_catalog`.`default_db`.`my_table` (
216+
// ...
217+
// ) WITH (
218+
// ...
219+
// )
203220
// create a view
204221
tEnv.executeSql("CREATE VIEW my_view AS ...")
205222
// show views
@@ -280,6 +297,13 @@ table_env.execute_sql("SHOW TABLES").print()
280297
# +------------+
281298
# | my_table |
282299
# +------------+
300+
# show create table
301+
table_env.executeSql("SHOW CREATE TABLE my_table").print()
302+
# CREATE TABLE `default_catalog`.`default_db`.`my_table` (
303+
# ...
304+
# ) WITH (
305+
# ...
306+
# )
283307

284308
# create a view
285309
table_env.execute_sql("CREATE VIEW my_view AS ...")
@@ -345,6 +369,13 @@ Flink SQL> CREATE TABLE my_table (...) WITH (...);
345369
Flink SQL> SHOW TABLES;
346370
my_table
347371

372+
Flink SQL> SHOW CREATE TABLE my_table;
373+
CREATE TABLE `default_catalog`.`default_db`.`my_table` (
374+
...
375+
) WITH (
376+
...
377+
)
378+
348379
Flink SQL> CREATE VIEW my_view AS ...;
349380
[INFO] View has been created.
350381

@@ -427,6 +458,17 @@ SHOW TABLES
427458

428459
Show all tables in the current catalog and the current database.
429460

461+
462+
## SHOW CREATE TABLE
463+
464+
```sql
465+
SHOW CREATE TABLE
466+
```
467+
468+
Show create table statement for specified table.
469+
470+
<span class="label label-danger">Attention</span> Currently `SHOW CREATE TABLE` only supports table that is created by Flink SQL DDL.
471+
430472
## SHOW VIEWS
431473

432474
```sql

docs/dev/table/hive/hive_dialect.md

Whitespace-only changes.

docs/dev/table/hive/hive_dialect.zh.md

Whitespace-only changes.

docs/dev/table/sql/show.md

Whitespace-only changes.

docs/dev/table/sql/show.zh.md

Whitespace-only changes.

flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@
3434
import org.apache.flink.table.operations.ModifyOperation;
3535
import org.apache.flink.table.operations.Operation;
3636
import org.apache.flink.table.operations.QueryOperation;
37+
import org.apache.flink.table.operations.ShowCreateTableOperation;
3738
import org.apache.flink.table.operations.UnloadModuleOperation;
3839
import org.apache.flink.table.operations.UseOperation;
3940
import org.apache.flink.table.operations.command.ClearOperation;
@@ -419,6 +420,9 @@ private void callOperation(Operation operation, ExecutionMode mode) {
419420
} else if (operation instanceof EndStatementSetOperation) {
420421
// END
421422
callEndStatementSet();
423+
} else if (operation instanceof ShowCreateTableOperation) {
424+
// SHOW CREATE TABLE
425+
callShowCreateTable((ShowCreateTableOperation) operation);
422426
} else {
423427
// fallback to default implementation
424428
executeOperation(operation);
@@ -532,6 +536,14 @@ private void callInserts(List<ModifyOperation> operations) {
532536
}
533537

534538
public void callExplain(ExplainOperation operation) {
539+
printRawContent(operation);
540+
}
541+
542+
public void callShowCreateTable(ShowCreateTableOperation operation) {
543+
printRawContent(operation);
544+
}
545+
546+
public void printRawContent(Operation operation) {
535547
TableResult tableResult = executor.executeOperation(sessionId, operation);
536548
// show raw content instead of tableau style
537549
final String explanation =

flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java

Whitespace-only changes.

flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/SqlCommandParserTest.java

Whitespace-only changes.

flink-table/flink-sql-client/src/test/resources/sql/table.q

Lines changed: 44 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -73,6 +73,22 @@ show tables;
7373
1 row in set
7474
!ok
7575
76+
# test SHOW CREATE TABLE
77+
show create table orders;
78+
CREATE TABLE `default_catalog`.`default_database`.`orders` (
79+
`user` BIGINT NOT NULL,
80+
`product` VARCHAR(32),
81+
`amount` INT,
82+
`ts` TIMESTAMP(3),
83+
`ptime` AS PROCTIME(),
84+
WATERMARK FOR `ts` AS `ts` - INTERVAL '1' SECOND,
85+
CONSTRAINT `PK_3599338` PRIMARY KEY (`user`) NOT ENFORCED
86+
) WITH (
87+
'connector' = 'datagen'
88+
)
89+
90+
!ok
91+
7692
# ==========================================================================
7793
# test alter table
7894
# ==========================================================================
@@ -117,6 +133,22 @@ desc orders2;
117133
5 rows in set
118134
!ok
119135
136+
# test SHOW CREATE TABLE
137+
show create table orders2;
138+
CREATE TABLE `default_catalog`.`default_database`.`orders2` (
139+
`user` BIGINT NOT NULL,
140+
`product` VARCHAR(32),
141+
`amount` INT,
142+
`ts` TIMESTAMP(3),
143+
`ptime` AS PROCTIME(),
144+
WATERMARK FOR `ts` AS `ts` - INTERVAL '1' SECOND,
145+
CONSTRAINT `PK_3599338` PRIMARY KEY (`user`) NOT ENFORCED
146+
) WITH (
147+
'connector' = 'kafka'
148+
)
149+
150+
!ok
151+
120152
# ==========================================================================
121153
# test drop table
122154
# ==========================================================================
@@ -165,6 +197,18 @@ show tables;
165197
1 row in set
166198
!ok
167199
200+
# SHOW CREATE TABLE for temporary table
201+
show create table tbl1;
202+
CREATE TEMPORARY TABLE `default_catalog`.`default_database`.`tbl1` (
203+
`user` BIGINT NOT NULL,
204+
`product` VARCHAR(32),
205+
`amount` INT
206+
) WITH (
207+
'connector' = 'datagen'
208+
)
209+
210+
!ok
211+
168212
drop temporary table tbl1;
169213
[INFO] Execute statement succeed.
170214
!info

0 commit comments

Comments
 (0)