Hi Bowen: Thanks for your feedback. Your opinion convinced me and I just remove the section about catalog create statement and also remove `DBPROPERTIES` `PROPERTIES` from alter DDLs. Open to more comments or votes :) !
Best, Terry Wang > 2019年11月6日 07:22,Bowen Li <bowenl...@gmail.com> 写道: > > Hi Terry, > > I went over the FLIP in detail again. The FLIP mostly LGTM. A couple issues: > > - since we on't plan to support catalog ddl, can you remove them from the > FLIP? > - I found there are some discrepancies in proposed database and table DDLs. > For db ddl, the create db syntax proposes specifying k-v properties > following "WITH". However, alter db ddl comes with a keyword "DBPROPERTIES": > > CREATE DATABASE [ IF NOT EXISTS ] [ catalogName.] dataBaseName [ COMMENT > database_comment ] > [*WITH *( name=value [, name=value]*)] > > > ALTER DATABASE [ catalogName.] dataBaseName SET *DBPROPERTIES* ( > name=value [, name=value]*) > > > IIUIC, are you borrowing syntax from Hive? Note that Hive's db create > ddl comes with "DBPROPERTIES" though - "CREATE (DATABASE|SCHEMA) [IF NOT > EXISTS] database_name ... [*WITH DBPROPERTIES* (k=v, ...)];" [1] > > The same applies to table ddl. The proposed alter table ddl comes with > "SET *PROPERTIES* (...)", however, Flink's existing table create ddl since > 1.9 [2] doesn't have "PROPERTIES" keyword. As opposed to Hive's syntax, > both create and alter table ddl comes with "TBLPROPERTIES" [1]. > > I feel it's better to be consistent among our DDLs. One option is to > just remove the "PROPERTIES" and "DBPROPERTIES" keywords in proposed syntax. > > [1] https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL > [2] > https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/sql.html#specifying-a-ddl > > On Tue, Nov 5, 2019 at 12:54 PM Peter Huang <huangzhenqiu0...@gmail.com> > wrote: > >> +1 for the enhancement. >> >> On Tue, Nov 5, 2019 at 11:04 AM Xuefu Z <usxu...@gmail.com> wrote: >> >>> +1 to the long missing feature in Flink SQL. >>> >>> On Tue, Nov 5, 2019 at 6:32 AM Terry Wang <zjuwa...@gmail.com> wrote: >>> >>>> Hi all, >>>> >>>> I would like to start the vote for FLIP-69[1] which is discussed and >>>> reached consensus in the discussion thread[2]. >>>> >>>> The vote will be open for at least 72 hours. I'll try to close it by >>>> 2019-11-08 14:30 UTC, unless there is an objection or not enough votes. >>>> >>>> [1] >>>> >>> >> https://cwiki.apache.org/confluence/display/FLINK/FLIP+69+-+Flink+SQL+DDL+Enhancement >>>> < >>>> >>> >> https://cwiki.apache.org/confluence/display/FLINK/FLIP+69+-+Flink+SQL+DDL+Enhancement >>>>> >>>> [2] >>>> >>> >> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-69-Flink-SQL-DDL-Enhancement-td33090.html >>>> < >>>> >>> >> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-69-Flink-SQL-DDL-Enhancement-td33090.html >>>>> >>>> Best, >>>> Terry Wang >>>> >>>> >>>> >>>> >>> >>> -- >>> Xuefu Zhang >>> >>> "In Honey We Trust!" >>> >>