Sure Yuxia, I just added the support for RTAS statements too.

- Sergio

On Wed, Jun 12, 2024 at 8:22 PM yuxia <luoyu...@alumni.sjtu.edu.cn> wrote:

> Hi, Sergio.
> Thanks for driving the FLIP. Given we also has REPLACE TABLE AS
> Statement[1] and it's almost same with CREATE TABLE AS Statement,
> would you mind also supporting schema definition for REPLACE TABLE AS
> Statement in this FLIP? It'll be a great to align REPLACE TABLE AS Statement
> to CREATE TABLE AS Statement
>
>
> [1]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-303%3A+Support+REPLACE+TABLE+AS+SELECT+statement
>
> Best regards,
> Yuxia
>
> ----- 原始邮件 -----
> 发件人: "Timo Walther" <twal...@apache.org>
> 收件人: "dev" <dev@flink.apache.org>
> 发送时间: 星期三, 2024年 6 月 12日 下午 10:19:14
> 主题: Re: [DISCUSS] FLIP-463: Schema Definition in CREATE TABLE AS Statement
>
> > I just noticed the CREATE TABLE LIKE statement allows the definition
>  > of new columns in the CREATE part. The difference
>  > with this CTAS proposal is that TABLE LIKE appends the new columns at
>  > the end of the schema instead of adding them
>  > at the beginning like this proposal and Mysql do.
>
> This should be fine. The LIKE rather "extends from" the right table.
> Whereas the SELECT in CTAS rather extends the left schema definition.
> Given that "the extended part" is always appended, we could argue that
> the current CTAS behavior in the FLIP is acceptable.
>
>  > If you want to rename a column in the create part, then that column
>  > position must be in the same position as the query column.
>  > I didn't like the Postgres approach because it does not let us add
>  > columns that do not exist in the query schema.
>
> Flink offers similar functionality in INSERT INTO. INSERT INTO also
> allows syntax like: `INSERT INTO (b, c) SELECT * FROM t`. So given that
> our CTAS can be seen as a CREATE TABLE + INSERT INTO. I would adopt
> Jeyhun comment in the FLIP. If users don't want to add additional schema
> parts, the same column reordering should be available as if one would
> write a INSERT INTO.
>
> Regards,
> Timo
>
>
>
>
> On 12.06.24 04:30, Yanquan Lv wrote:
> > Hi Sergio, thanks for driving it, +1 for this.
> >
> > I have some comments:
> > 1. If we have a source table with primary keys and partition keys
> defined,
> > what is the default behavior if PARTITIONED and DISTRIBUTED not specified
> > in the CTAS statement, It should not be inherited by default?
> > 2. I suggest providing a complete syntax that includes table_properties
> > like FLIP-218.
> >
> >
> > Sergio Pena <ser...@confluent.io.invalid> 于2024年6月12日周三 03:54写道:
> >
> >> I just noticed the CREATE TABLE LIKE statement allows the definition of
> new
> >> columns in the CREATE part. The difference
> >> with this CTAS proposal is that TABLE LIKE appends the new columns at
> the
> >> end of the schema instead of adding them
> >> at the beginning like this proposal and Mysql do.
> >>
> >>> create table t1(id int, name string);
> >>>> create table s1(a int, b string) like t1;
> >>>> describe s1;
> >>
> >> +-------------+-----------+----------+--------+
> >>> | Column Name | Data Type | Nullable | Extras |
> >>> +-------------+-----------+----------+--------+
> >>> | id          | INT       | NULL     |        |
> >>> | name        | STRING    | NULL     |        |
> >>> | a           | INT       | NULL     |        |
> >>> | b           | STRING    | NULL     |        |
> >>> +-------------+-----------+----------+--------+
> >>
> >>
> >>
> >> The CREATE TABLE LIKE also does not let the definition of existing
> columns
> >> in the CREATE part. The statement fails
> >> that the column already exists.
> >>
> >>> create table t1(id int, name string);
> >>
> >>> create table s1(id double) like t1;
> >>> A column named 'id' already exists in the base table.
> >>>
> >>
> >> What do you guys think of making it similar to the CREATE TABLE LIKE?
> Seems
> >> the best approach in order to
> >> be compatible with it.
> >>
> >> - Sergio
> >>
> >> On Tue, Jun 11, 2024 at 2:10 PM Sergio Pena <ser...@confluent.io>
> wrote:
> >>
> >>> Thanks Timo for answering Jeyhun questions.
> >>>
> >>> To add info more about your questions Jeyhun. This proposal is not
> >>> handling NULL/NOT_NULL types. I noticed that
> >>> the current CTAS impl. (as Timo said) adds this constraint as part of
> the
> >>> resulting schema. And when defining
> >>> a primary key in the CREATE part, if the resulting schema does not
> have a
> >>> NOT NULL in the column then the CTAS
> >>> will fail. This is similar to the CREATE TABLE LIKE which expects the
> >> LIKE
> >>> table to have a NOT NULL column if
> >>> the user defines a primary key in the CREATE part.
> >>>
> >>>> In some cases, redefining the column types might be redundant,
> >> especially
> >>>> when users dont change the column type. A user just wants to change
> the
> >>>> column name from the SELECT clause. Should we also support this
> >> scenario,
> >>>> similar to postgres?
> >>>
> >>> I looked into Postgres too. Postgres matches the columns based on the
> >>> order defined in the create and select part.
> >>> If you want to rename a column in the create part, then that column
> >>> position must be in the same position as the query column.
> >>> I didn't like the Postgres approach because it does not let us add
> >> columns
> >>> that do not exist in the query schema.
> >>>
> >>> i.e. query has schema (a int, b string), now the `a` column is renamed
> to
> >>> `id` because both are in the same position 0
> >>> `create table s1(id int) as select a, b from t1`;
> >>> results in: [id int, b string]
> >>>
> >>> I think, if users want to rename then they can use a different alias in
> >>> the select part. They could also do explicit casting
> >>> for changing the data types, which now makes it redundant (as you said)
> >> to
> >>> allow redefining the query columns again. But
> >>> perhaps there are cases where explicit casting does not work and just
> >>> defining the column would? i.e. making a nullable
> >>> type to not null? I couldn't make `cast(c1 as int not null)` to work
> for
> >>> instance, but it may work in the create part?
> >>>
> >>>> Could you also mention the casting rules in the FLIP for this case?
> >>>
> >>> I mentioned they're the same as insert/select when doing implicit
> >> casting.
> >>> I will search for more info about the insert/select
> >>> and add the casting rules in the flip..
> >>>
> >>> - Sergio
> >>>
> >>>
> >>> On Tue, Jun 11, 2024 at 12:59 AM Timo Walther <twal...@apache.org>
> >> wrote:
> >>>
> >>>> Hi Sergio,
> >>>>
> >>>> thanks for proposing this FLIP for finalizing the CTAS statement.
> >>>> Adopting the logic from MySQL for deriving and potentially overwriting
> >>>> parts of the schema should be easy to understand for everyone. So +1
> for
> >>>> the FLIP in general.
> >>>>
> >>>>   > How do you handle CTAS statements with SELECT clauses that have
> >>>>   > (implicit or explicit) NULLABLE or NOT NULLABLE columns?
> >>>>
> >>>> @Jeyhun: I don't think there is anything special about this. The
> current
> >>>> CTAS implementation should already cover that. It takes the
> nullability
> >>>> of the column's data type as a constraint into derived schema. Keep in
> >>>> mind that nullability is part of the data type in Flink, not only a
> >>>> constraint on the schema. This decision was made due to Calcite
> >> internals.
> >>>>
> >>>>   > redefining the column types might be redundant, especially
> >>>>   > when users dont change the column type
> >>>>
> >>>> This is indeed a good point. On one hand, I think we should avoid
> >>>> further complicating the syntax. But looking at other vendors [1] this
> >>>> seems indeed a valid use case. If it doesn't cause too many special
> >>>> cases in the parser (and it's look-ahead), I'm fine with supporting a
> >>>> list of column names as well. However, the most important use case
> will
> >>>> be specifying a watermark, metadata columns, or other schema parts
> that
> >>>> are not just columns names.
> >>>>
> >>>> Regards,
> >>>> Timo
> >>>>
> >>>>
> >>>> [1]
> >>>>
> >>>>
> >>
> https://learn.microsoft.com/en-us/sql/t-sql/statements/create-table-as-select-azure-sql-data-warehouse?view=azure-sqldw-latest
> >>>>
> >>>>
> >>>> On 10.06.24 21:37, Jeyhun Karimov wrote:
> >>>>> Hi Sergio,
> >>>>>
> >>>>> Thanks for driving this FLIP. +1 for it.
> >>>>> I have a few questions:
> >>>>>
> >>>>> - How do you handle CTAS statements with SELECT clauses that have
> >>>> (implicit
> >>>>> or explicit) NULLABLE or NOT NULLABLE columns? Could you also mention
> >>>> the
> >>>>> casting rules in the FLIP for this case?
> >>>>> - In some cases, redefining the column types might be redundant,
> >>>> especially
> >>>>> when users dont change the column type. For example, a user just
> wants
> >>>> to
> >>>>> change the column name from the SELECT clause.
> >>>>> Should we also support this scenario, similar to the postgres [1] ?
> >>>>>
> >>>>> Regards,
> >>>>> Jeyhun
> >>>>>
> >>>>>
> >>>>> [1] https://www.postgresql.org/docs/8.1/sql-createtableas.html
> >>>>>
> >>>>> On Mon, Jun 10, 2024 at 6:28 PM Sergio Pena
> >> <ser...@confluent.io.invalid
> >>>>>
> >>>>> wrote:
> >>>>>
> >>>>>> Hi David,
> >>>>>>
> >>>>>> The CTAS feature is already part of Flink (proposed in FLIP-218
> [1]).
> >>>> The
> >>>>>> new FLIP-463 is just to extend the CTAS syntax to allow for adding
> >> new
> >>>>>> columns to
> >>>>>> the created table that are not part of the generated schema. I think
> >>>>>> FLIP-218 [1] was discussed in the mail list somewhere, but I
> couldn't
> >>>> find
> >>>>>> the discussion thread.
> >>>>>> I was hoping it could contain the answers for your questions as
> >> that's
> >>>>>> where CTAS was implemented. There's a user doc [2] for it that may
> >> help
> >>>>>> too.
> >>>>>>
> >>>>>> But, in a nutshell, CTAS is similar to running CREATE and then
> >>>>>> INSERT/SELECT statements. The CTAS will execute a background job
> that
> >>>>>> executes
> >>>>>> the AS SELECT query and then inserts the data into the newly created
> >>>> table.
> >>>>>> So the table will be kept up to date as the source data changes as
> >> you
> >>>>>> said.
> >>>>>>
> >>>>>> Generated columns are from the AS SELECT schema, right? Yes, that's
> >>>> already
> >>>>>> allowed. FLIP-463 will also allow modifying the schema of the sink
> >>>> table
> >>>>>> in case you need to add more columns.
> >>>>>>
> >>>>>> I notice amazon [3] talks of the difference between view and CTAS;
> >> that
> >>>>>>> CTAS persists the content. Is this the approach we are taking? If
> so
> >>>>>> where
> >>>>>>> are we persisting?
> >>>>>>
> >>>>>> This is already supported by FLIP-218 [1]. But yes, the CTAS
> persists
> >>>> the
> >>>>>> content in the new table you specify in the CTAS statement.
> >>>>>>
> >>>>>>
> >>>>>>> is the table read-only, or can we insert/ delete / update into it.
> >> If
> >>>> it
> >>>>>>> is read only how will the inserts , deletes updates fail.
> >>>>>>
> >>>>>> Are there any restrictions on the select ? Can the select be a join,
> >>>>>>> aggregate, windowed?
> >>>>>>
> >>>>>> I notice Azure [2] supports ISNULL. Is there a thought to change the
> >>>>>>> nullability for the CTAS?
> >>>>>>
> >>>>>> Amazon [4] ignores ordered by clauses ? is that the same for this
> >>>> proposal?
> >>>>>>
> >>>>>> I suppose all of this is already supported by FLIP-218. I don't see
> >> any
> >>>>>> restrictions on the query. You can learn more about it in the
> >> FLIP-218
> >>>> [1]
> >>>>>>
> >>>>>> - Sergio
> >>>>>>
> >>>>>> [1]
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=199541185
> >>>>>> [2]
> >>>>>>
> >>>>>>
> >>>>
> >>
> https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/sql/create/#as-select_statement
> >>>>>>
> >>>>>> On Mon, Jun 10, 2024 at 3:10 AM David Radley <
> >> david_rad...@uk.ibm.com>
> >>>>>> wrote:
> >>>>>>
> >>>>>>> Hi Sergio,
> >>>>>>> Sounds good . I am relatively new to this area and had some basic
> >>>>>>> questions:
> >>>>>>>
> >>>>>>> I notice in [1] it talks of materialized views. And CREATE view can
> >>>>>>> already take the AS keyword. It would be useful to me to understand
> >>>> when
> >>>>>> we
> >>>>>>> would use each of these.
> >>>>>>>
> >>>>>>> - I assume the table will be kept up to date as the source data
> >>>> changes
> >>>>>>> like a view.
> >>>>>>> - Are there any restrictions on the select ? Can the select be a
> >> join,
> >>>>>>> aggregate, windowed?
> >>>>>>> - I assume generated columns are allowed?
> >>>>>>> - is the table read-only, or can we insert/ delete / update into
> it.
> >>>> If
> >>>>>> it
> >>>>>>> is read only how will the inserts , deletes updates fail.
> >>>>>>> - I notice Azure [2] supports ISNULL. Is there a thought to change
> >> the
> >>>>>>> nullability for the CTAS?
> >>>>>>> - I notice amazon [3] talks of the difference between view and
> CTAS;
> >>>> that
> >>>>>>> CTAS persists the content. Is this the approach we are taking? If
> so
> >>>>>> where
> >>>>>>> are we persisting?
> >>>>>>> - Amazon [4] ignores ordered by clauses ? is that the same for this
> >>>>>>> proposal?
> >>>>>>>
> >>>>>>> [1]
> >>>>>>>
> >>>>>>
> >>>>
> >>
> https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/concepts/dynamic_tables/
> >>>>>>> [2]
> >>>>>>>
> >>>>>>
> >>>>
> >>
> https://learn.microsoft.com/en-us/azure/synapse-analytics/sql-data-warehouse/sql-data-warehouse-develop-ctas#selectinto-vs-ctas
> >>>>>>> [3]
> >>>>>>>
> >>>>>>
> >>>>
> >>
> https://docs.aws.amazon.com/athena/latest/ug/ctas-considerations-limitations.html#ctas-considerations-limitations-queries-vs-views
> >>>>>>> [4]
> >>>>>>>
> >>>>>>
> >>>>
> >>
> https://docs.aws.amazon.com/athena/latest/ug/ctas-considerations-limitations.html#ctas-considerations-limitations-order-by-ignored
> >>>>>>>
> >>>>>>>
> >>>>>>> Kind regards, David.
> >>>>>>>
> >>>>>>> From: Sergio Pena <ser...@confluent.io.INVALID>
> >>>>>>> Date: Friday, 7 June 2024 at 16:13
> >>>>>>> To: dev@flink.apache.org <dev@flink.apache.org>
> >>>>>>> Subject: [EXTERNAL] [DISCUSS] FLIP-463: Schema Definition in CREATE
> >>>> TABLE
> >>>>>>> AS Statement
> >>>>>>> HI All,
> >>>>>>>
> >>>>>>> I'd like to start a discussion on FLIP-463: Schema Definition in
> >>>> CREATE
> >>>>>>> TABLE AS Statement [1]
> >>>>>>>
> >>>>>>> The proposal extends the CTAS statement to allow users to define
> >> their
> >>>>>> own
> >>>>>>> schema by adding columns, primary and partition keys, and table
> >>>>>>> distribution to the CREATE statement.
> >>>>>>>
> >>>>>>> Any thoughts are welcome.
> >>>>>>>
> >>>>>>> Thanks,
> >>>>>>> - Sergio Pena
> >>>>>>>
> >>>>>>> [1]
> >>>>>>>
> >>>>>>>
> >>>>>>
> >>>>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-463%3A+Schema+Definition+in+CREATE+TABLE+AS+Statement
> >>>>>>>
> >>>>>>> Unless otherwise stated above:
> >>>>>>>
> >>>>>>> IBM United Kingdom Limited
> >>>>>>> Registered in England and Wales with number 741598
> >>>>>>> Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6
> >>>> 3AU
> >>>>>>>
> >>>>>>
> >>>>>
> >>>>
> >>>>
> >>
> >
>

Reply via email to