[DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

classic Classic list List threaded Threaded
13 messages Options
Reply | Threaded
Open this post in threaded view
|

[DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Danny Chan
In umbrella task FLINK-10232 we have introduced CREATE TABLE grammar in our new module flink-sql-parser. And we proposed to use computed column to describe the time attribute of process time in the design doc FLINK SQL DDL, so user may create a table with process time attribute as follows:
create table T1(
  a int,
  b bigint,
  c varchar,
  d as PROCTIME,
) with (
  'k1' = 'v1',
  'k2' = 'v2'
);

The column d would be a process time attribute for table T1.

Besides that, computed  columns have several other use cases, such as these [2]:


• Virtual generated columns can be used as a way to simplify and unify queries. A complicated condition can be defined as a generated column and referred to from multiple queries on the table to ensure that all of them use exactly the same condition.
• Stored generated columns can be used as a materialized cache for complicated conditions that are costly to calculate on the fly.
• Generated columns can simulate functional indexes: Use a generated column to define a functional expression and index it. This can be useful for working with columns of types that cannot be indexed directly, such as JSON columns.
• For stored generated columns, the disadvantage of this approach is that values are stored twice; once as the value of the generated column and once in the index.
• If a generated column is indexed, the optimizer recognizes query expressions that match the column definition and uses indexes from the column as appropriate during query execution(Not supported yet).



Computed columns are introduced in SQL-SERVER-2016 [1], MYSQL-5.6 [2] and ORACLE-11g [3].

This is the design doc:
https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing

Any suggestions are appreciated, thanks.

[1] https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
[2] https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
[3] https://oracle-base.com/articles/11g/virtual-columns-11gr1

Best,
Danny Chan
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

qi luo
Fantastic! We're also very interested in this feature.

+Boxiu

On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <[hidden email]> wrote:

> In umbrella task FLINK-10232 we have introduced CREATE TABLE grammar in
> our new module flink-sql-parser. And we proposed to use computed column to
> describe the time attribute of process time in the design doc FLINK SQL
> DDL, so user may create a table with process time attribute as follows:
> create table T1(
>   a int,
>   b bigint,
>   c varchar,
>   d as PROCTIME,
> ) with (
>   'k1' = 'v1',
>   'k2' = 'v2'
> );
>
> The column d would be a process time attribute for table T1.
>
> Besides that, computed  columns have several other use cases, such as
> these [2]:
>
>
> • Virtual generated columns can be used as a way to simplify and unify
> queries. A complicated condition can be defined as a generated column and
> referred to from multiple queries on the table to ensure that all of them
> use exactly the same condition.
> • Stored generated columns can be used as a materialized cache for
> complicated conditions that are costly to calculate on the fly.
> • Generated columns can simulate functional indexes: Use a generated
> column to define a functional expression and index it. This can be useful
> for working with columns of types that cannot be indexed directly, such as
> JSON columns.
> • For stored generated columns, the disadvantage of this approach is that
> values are stored twice; once as the value of the generated column and once
> in the index.
> • If a generated column is indexed, the optimizer recognizes query
> expressions that match the column definition and uses indexes from the
> column as appropriate during query execution(Not supported yet).
>
>
>
> Computed columns are introduced in SQL-SERVER-2016 [1], MYSQL-5.6 [2] and
> ORACLE-11g [3].
>
> This is the design doc:
>
> https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
>
> Any suggestions are appreciated, thanks.
>
> [1]
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
> [2]
> https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
> [3] https://oracle-base.com/articles/11g/virtual-columns-11gr1
>
> Best,
> Danny Chan
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Kurt Young
Hi Danny,

Thanks for preparing this design document. IMO It's a very useful
feature, especially combined with time attribute support to specify
watermark in DDL.

The design doc looks quite good, but I would suggest to reduce the
scope of the first version. Like we don't have to support "STORED"
in the first MVP version, and you can also delete related content in
document to make it more clean and easier to understand.

Best,
Kurt


On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]> wrote:

> Fantastic! We're also very interested in this feature.
>
> +Boxiu
>
> On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <[hidden email]> wrote:
>
> > In umbrella task FLINK-10232 we have introduced CREATE TABLE grammar in
> > our new module flink-sql-parser. And we proposed to use computed column
> to
> > describe the time attribute of process time in the design doc FLINK SQL
> > DDL, so user may create a table with process time attribute as follows:
> > create table T1(
> >   a int,
> >   b bigint,
> >   c varchar,
> >   d as PROCTIME,
> > ) with (
> >   'k1' = 'v1',
> >   'k2' = 'v2'
> > );
> >
> > The column d would be a process time attribute for table T1.
> >
> > Besides that, computed  columns have several other use cases, such as
> > these [2]:
> >
> >
> > • Virtual generated columns can be used as a way to simplify and unify
> > queries. A complicated condition can be defined as a generated column and
> > referred to from multiple queries on the table to ensure that all of them
> > use exactly the same condition.
> > • Stored generated columns can be used as a materialized cache for
> > complicated conditions that are costly to calculate on the fly.
> > • Generated columns can simulate functional indexes: Use a generated
> > column to define a functional expression and index it. This can be useful
> > for working with columns of types that cannot be indexed directly, such
> as
> > JSON columns.
> > • For stored generated columns, the disadvantage of this approach is that
> > values are stored twice; once as the value of the generated column and
> once
> > in the index.
> > • If a generated column is indexed, the optimizer recognizes query
> > expressions that match the column definition and uses indexes from the
> > column as appropriate during query execution(Not supported yet).
> >
> >
> >
> > Computed columns are introduced in SQL-SERVER-2016 [1], MYSQL-5.6 [2] and
> > ORACLE-11g [3].
> >
> > This is the design doc:
> >
> >
> https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
> >
> > Any suggestions are appreciated, thanks.
> >
> > [1]
> >
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
> > [2]
> >
> https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
> > [3] https://oracle-base.com/articles/11g/virtual-columns-11gr1
> >
> > Best,
> > Danny Chan
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Jark Wu-2
+1 to remove “STORED” related content. We can add them when user requires.
Others looks good to me in general.

Thanks,
Jark


> 在 2019年10月24日,14:58,Kurt Young <[hidden email]> 写道:
>
> Hi Danny,
>
> Thanks for preparing this design document. IMO It's a very useful
> feature, especially combined with time attribute support to specify
> watermark in DDL.
>
> The design doc looks quite good, but I would suggest to reduce the
> scope of the first version. Like we don't have to support "STORED"
> in the first MVP version, and you can also delete related content in
> document to make it more clean and easier to understand.
>
> Best,
> Kurt
>
>
> On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]> wrote:
>
>> Fantastic! We're also very interested in this feature.
>>
>> +Boxiu
>>
>> On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <[hidden email]> wrote:
>>
>>> In umbrella task FLINK-10232 we have introduced CREATE TABLE grammar in
>>> our new module flink-sql-parser. And we proposed to use computed column
>> to
>>> describe the time attribute of process time in the design doc FLINK SQL
>>> DDL, so user may create a table with process time attribute as follows:
>>> create table T1(
>>>  a int,
>>>  b bigint,
>>>  c varchar,
>>>  d as PROCTIME,
>>> ) with (
>>>  'k1' = 'v1',
>>>  'k2' = 'v2'
>>> );
>>>
>>> The column d would be a process time attribute for table T1.
>>>
>>> Besides that, computed  columns have several other use cases, such as
>>> these [2]:
>>>
>>>
>>> • Virtual generated columns can be used as a way to simplify and unify
>>> queries. A complicated condition can be defined as a generated column and
>>> referred to from multiple queries on the table to ensure that all of them
>>> use exactly the same condition.
>>> • Stored generated columns can be used as a materialized cache for
>>> complicated conditions that are costly to calculate on the fly.
>>> • Generated columns can simulate functional indexes: Use a generated
>>> column to define a functional expression and index it. This can be useful
>>> for working with columns of types that cannot be indexed directly, such
>> as
>>> JSON columns.
>>> • For stored generated columns, the disadvantage of this approach is that
>>> values are stored twice; once as the value of the generated column and
>> once
>>> in the index.
>>> • If a generated column is indexed, the optimizer recognizes query
>>> expressions that match the column definition and uses indexes from the
>>> column as appropriate during query execution(Not supported yet).
>>>
>>>
>>>
>>> Computed columns are introduced in SQL-SERVER-2016 [1], MYSQL-5.6 [2] and
>>> ORACLE-11g [3].
>>>
>>> This is the design doc:
>>>
>>>
>> https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
>>>
>>> Any suggestions are appreciated, thanks.
>>>
>>> [1]
>>>
>> https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
>>> [2]
>>>
>> https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
>>> [3] https://oracle-base.com/articles/11g/virtual-columns-11gr1
>>>
>>> Best,
>>> Danny Chan
>>>
>>

Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Timo Walther-2
Having an MVP and a limited scope sounds good to me. But I would not
remove the STORED keyword entirely from the document.

It shows that we have a long-term vision. Instead of deleting this
content, I would move it to a Outlook/Future Work section.

Regards,
Timo


On 24.10.19 10:55, Jark Wu wrote:

> +1 to remove “STORED” related content. We can add them when user requires.
> Others looks good to me in general.
>
> Thanks,
> Jark
>
>
>> 在 2019年10月24日,14:58,Kurt Young <[hidden email]> 写道:
>>
>> Hi Danny,
>>
>> Thanks for preparing this design document. IMO It's a very useful
>> feature, especially combined with time attribute support to specify
>> watermark in DDL.
>>
>> The design doc looks quite good, but I would suggest to reduce the
>> scope of the first version. Like we don't have to support "STORED"
>> in the first MVP version, and you can also delete related content in
>> document to make it more clean and easier to understand.
>>
>> Best,
>> Kurt
>>
>>
>> On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]> wrote:
>>
>>> Fantastic! We're also very interested in this feature.
>>>
>>> +Boxiu
>>>
>>> On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <[hidden email]> wrote:
>>>
>>>> In umbrella task FLINK-10232 we have introduced CREATE TABLE grammar in
>>>> our new module flink-sql-parser. And we proposed to use computed column
>>> to
>>>> describe the time attribute of process time in the design doc FLINK SQL
>>>> DDL, so user may create a table with process time attribute as follows:
>>>> create table T1(
>>>>   a int,
>>>>   b bigint,
>>>>   c varchar,
>>>>   d as PROCTIME,
>>>> ) with (
>>>>   'k1' = 'v1',
>>>>   'k2' = 'v2'
>>>> );
>>>>
>>>> The column d would be a process time attribute for table T1.
>>>>
>>>> Besides that, computed  columns have several other use cases, such as
>>>> these [2]:
>>>>
>>>>
>>>> • Virtual generated columns can be used as a way to simplify and unify
>>>> queries. A complicated condition can be defined as a generated column and
>>>> referred to from multiple queries on the table to ensure that all of them
>>>> use exactly the same condition.
>>>> • Stored generated columns can be used as a materialized cache for
>>>> complicated conditions that are costly to calculate on the fly.
>>>> • Generated columns can simulate functional indexes: Use a generated
>>>> column to define a functional expression and index it. This can be useful
>>>> for working with columns of types that cannot be indexed directly, such
>>> as
>>>> JSON columns.
>>>> • For stored generated columns, the disadvantage of this approach is that
>>>> values are stored twice; once as the value of the generated column and
>>> once
>>>> in the index.
>>>> • If a generated column is indexed, the optimizer recognizes query
>>>> expressions that match the column definition and uses indexes from the
>>>> column as appropriate during query execution(Not supported yet).
>>>>
>>>>
>>>>
>>>> Computed columns are introduced in SQL-SERVER-2016 [1], MYSQL-5.6 [2] and
>>>> ORACLE-11g [3].
>>>>
>>>> This is the design doc:
>>>>
>>>>
>>> https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
>>>>
>>>> Any suggestions are appreciated, thanks.
>>>>
>>>> [1]
>>>>
>>> https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
>>>> [2]
>>>>
>>> https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
>>>> [3] https://oracle-base.com/articles/11g/virtual-columns-11gr1
>>>>
>>>> Best,
>>>> Danny Chan
>>>>
>>>

Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Kurt Young
+1 to move to a future section. By deleting it I mean remove from
the content describing the current processing procedure.

Best,
Kurt


On Thu, Oct 24, 2019 at 5:01 PM Timo Walther <[hidden email]> wrote:

> Having an MVP and a limited scope sounds good to me. But I would not
> remove the STORED keyword entirely from the document.
>
> It shows that we have a long-term vision. Instead of deleting this
> content, I would move it to a Outlook/Future Work section.
>
> Regards,
> Timo
>
>
> On 24.10.19 10:55, Jark Wu wrote:
> > +1 to remove “STORED” related content. We can add them when user
> requires.
> > Others looks good to me in general.
> >
> > Thanks,
> > Jark
> >
> >
> >> 在 2019年10月24日,14:58,Kurt Young <[hidden email]> 写道:
> >>
> >> Hi Danny,
> >>
> >> Thanks for preparing this design document. IMO It's a very useful
> >> feature, especially combined with time attribute support to specify
> >> watermark in DDL.
> >>
> >> The design doc looks quite good, but I would suggest to reduce the
> >> scope of the first version. Like we don't have to support "STORED"
> >> in the first MVP version, and you can also delete related content in
> >> document to make it more clean and easier to understand.
> >>
> >> Best,
> >> Kurt
> >>
> >>
> >> On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]> wrote:
> >>
> >>> Fantastic! We're also very interested in this feature.
> >>>
> >>> +Boxiu
> >>>
> >>> On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <[hidden email]>
> wrote:
> >>>
> >>>> In umbrella task FLINK-10232 we have introduced CREATE TABLE grammar
> in
> >>>> our new module flink-sql-parser. And we proposed to use computed
> column
> >>> to
> >>>> describe the time attribute of process time in the design doc FLINK
> SQL
> >>>> DDL, so user may create a table with process time attribute as
> follows:
> >>>> create table T1(
> >>>>   a int,
> >>>>   b bigint,
> >>>>   c varchar,
> >>>>   d as PROCTIME,
> >>>> ) with (
> >>>>   'k1' = 'v1',
> >>>>   'k2' = 'v2'
> >>>> );
> >>>>
> >>>> The column d would be a process time attribute for table T1.
> >>>>
> >>>> Besides that, computed  columns have several other use cases, such as
> >>>> these [2]:
> >>>>
> >>>>
> >>>> • Virtual generated columns can be used as a way to simplify and unify
> >>>> queries. A complicated condition can be defined as a generated column
> and
> >>>> referred to from multiple queries on the table to ensure that all of
> them
> >>>> use exactly the same condition.
> >>>> • Stored generated columns can be used as a materialized cache for
> >>>> complicated conditions that are costly to calculate on the fly.
> >>>> • Generated columns can simulate functional indexes: Use a generated
> >>>> column to define a functional expression and index it. This can be
> useful
> >>>> for working with columns of types that cannot be indexed directly,
> such
> >>> as
> >>>> JSON columns.
> >>>> • For stored generated columns, the disadvantage of this approach is
> that
> >>>> values are stored twice; once as the value of the generated column and
> >>> once
> >>>> in the index.
> >>>> • If a generated column is indexed, the optimizer recognizes query
> >>>> expressions that match the column definition and uses indexes from the
> >>>> column as appropriate during query execution(Not supported yet).
> >>>>
> >>>>
> >>>>
> >>>> Computed columns are introduced in SQL-SERVER-2016 [1], MYSQL-5.6 [2]
> and
> >>>> ORACLE-11g [3].
> >>>>
> >>>> This is the design doc:
> >>>>
> >>>>
> >>>
> https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
> >>>>
> >>>> Any suggestions are appreciated, thanks.
> >>>>
> >>>> [1]
> >>>>
> >>>
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
> >>>> [2]
> >>>>
> >>>
> https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
> >>>> [3] https://oracle-base.com/articles/11g/virtual-columns-11gr1
> >>>>
> >>>> Best,
> >>>> Danny Chan
> >>>>
> >>>
>
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Jark Wu-2
Yes. I think it makes sense to move to "Future Work" section.

Best,
Jark

On Thu, 24 Oct 2019 at 17:11, Kurt Young <[hidden email]> wrote:

> +1 to move to a future section. By deleting it I mean remove from
> the content describing the current processing procedure.
>
> Best,
> Kurt
>
>
> On Thu, Oct 24, 2019 at 5:01 PM Timo Walther <[hidden email]> wrote:
>
> > Having an MVP and a limited scope sounds good to me. But I would not
> > remove the STORED keyword entirely from the document.
> >
> > It shows that we have a long-term vision. Instead of deleting this
> > content, I would move it to a Outlook/Future Work section.
> >
> > Regards,
> > Timo
> >
> >
> > On 24.10.19 10:55, Jark Wu wrote:
> > > +1 to remove “STORED” related content. We can add them when user
> > requires.
> > > Others looks good to me in general.
> > >
> > > Thanks,
> > > Jark
> > >
> > >
> > >> 在 2019年10月24日,14:58,Kurt Young <[hidden email]> 写道:
> > >>
> > >> Hi Danny,
> > >>
> > >> Thanks for preparing this design document. IMO It's a very useful
> > >> feature, especially combined with time attribute support to specify
> > >> watermark in DDL.
> > >>
> > >> The design doc looks quite good, but I would suggest to reduce the
> > >> scope of the first version. Like we don't have to support "STORED"
> > >> in the first MVP version, and you can also delete related content in
> > >> document to make it more clean and easier to understand.
> > >>
> > >> Best,
> > >> Kurt
> > >>
> > >>
> > >> On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]> wrote:
> > >>
> > >>> Fantastic! We're also very interested in this feature.
> > >>>
> > >>> +Boxiu
> > >>>
> > >>> On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <[hidden email]>
> > wrote:
> > >>>
> > >>>> In umbrella task FLINK-10232 we have introduced CREATE TABLE grammar
> > in
> > >>>> our new module flink-sql-parser. And we proposed to use computed
> > column
> > >>> to
> > >>>> describe the time attribute of process time in the design doc FLINK
> > SQL
> > >>>> DDL, so user may create a table with process time attribute as
> > follows:
> > >>>> create table T1(
> > >>>>   a int,
> > >>>>   b bigint,
> > >>>>   c varchar,
> > >>>>   d as PROCTIME,
> > >>>> ) with (
> > >>>>   'k1' = 'v1',
> > >>>>   'k2' = 'v2'
> > >>>> );
> > >>>>
> > >>>> The column d would be a process time attribute for table T1.
> > >>>>
> > >>>> Besides that, computed  columns have several other use cases, such
> as
> > >>>> these [2]:
> > >>>>
> > >>>>
> > >>>> • Virtual generated columns can be used as a way to simplify and
> unify
> > >>>> queries. A complicated condition can be defined as a generated
> column
> > and
> > >>>> referred to from multiple queries on the table to ensure that all of
> > them
> > >>>> use exactly the same condition.
> > >>>> • Stored generated columns can be used as a materialized cache for
> > >>>> complicated conditions that are costly to calculate on the fly.
> > >>>> • Generated columns can simulate functional indexes: Use a generated
> > >>>> column to define a functional expression and index it. This can be
> > useful
> > >>>> for working with columns of types that cannot be indexed directly,
> > such
> > >>> as
> > >>>> JSON columns.
> > >>>> • For stored generated columns, the disadvantage of this approach is
> > that
> > >>>> values are stored twice; once as the value of the generated column
> and
> > >>> once
> > >>>> in the index.
> > >>>> • If a generated column is indexed, the optimizer recognizes query
> > >>>> expressions that match the column definition and uses indexes from
> the
> > >>>> column as appropriate during query execution(Not supported yet).
> > >>>>
> > >>>>
> > >>>>
> > >>>> Computed columns are introduced in SQL-SERVER-2016 [1], MYSQL-5.6
> [2]
> > and
> > >>>> ORACLE-11g [3].
> > >>>>
> > >>>> This is the design doc:
> > >>>>
> > >>>>
> > >>>
> >
> https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
> > >>>>
> > >>>> Any suggestions are appreciated, thanks.
> > >>>>
> > >>>> [1]
> > >>>>
> > >>>
> >
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
> > >>>> [2]
> > >>>>
> > >>>
> >
> https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
> > >>>> [3] https://oracle-base.com/articles/11g/virtual-columns-11gr1
> > >>>>
> > >>>> Best,
> > >>>> Danny Chan
> > >>>>
> > >>>
> >
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Jark Wu-2
If we move "STORED" to future work section, it is still unclear that
whether we should have an consensu on the design of "STORED"?

1) no consensus needed, we just put the design effort in the future part to
continue the work if we want to support it in the future. So the vote on
this FLIP dosen't contain the future work part.
2) consensus needed, we target the MVP part in 1.10 release, and postpone
the "STORED" implementation work to the future release. So the vote on this
FLIP contains the future work part.

IMO, "STORED" keyword makes the design much complex, and I didn't see a
requirement for this yet.
In order to have a consensus on this FLIP soon, I would lean to #1.

What do you think?

Best,
Jark


On Thu, 24 Oct 2019 at 17:25, Jark Wu <[hidden email]> wrote:

> Yes. I think it makes sense to move to "Future Work" section.
>
> Best,
> Jark
>
> On Thu, 24 Oct 2019 at 17:11, Kurt Young <[hidden email]> wrote:
>
>> +1 to move to a future section. By deleting it I mean remove from
>> the content describing the current processing procedure.
>>
>> Best,
>> Kurt
>>
>>
>> On Thu, Oct 24, 2019 at 5:01 PM Timo Walther <[hidden email]> wrote:
>>
>> > Having an MVP and a limited scope sounds good to me. But I would not
>> > remove the STORED keyword entirely from the document.
>> >
>> > It shows that we have a long-term vision. Instead of deleting this
>> > content, I would move it to a Outlook/Future Work section.
>> >
>> > Regards,
>> > Timo
>> >
>> >
>> > On 24.10.19 10:55, Jark Wu wrote:
>> > > +1 to remove “STORED” related content. We can add them when user
>> > requires.
>> > > Others looks good to me in general.
>> > >
>> > > Thanks,
>> > > Jark
>> > >
>> > >
>> > >> 在 2019年10月24日,14:58,Kurt Young <[hidden email]> 写道:
>> > >>
>> > >> Hi Danny,
>> > >>
>> > >> Thanks for preparing this design document. IMO It's a very useful
>> > >> feature, especially combined with time attribute support to specify
>> > >> watermark in DDL.
>> > >>
>> > >> The design doc looks quite good, but I would suggest to reduce the
>> > >> scope of the first version. Like we don't have to support "STORED"
>> > >> in the first MVP version, and you can also delete related content in
>> > >> document to make it more clean and easier to understand.
>> > >>
>> > >> Best,
>> > >> Kurt
>> > >>
>> > >>
>> > >> On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]> wrote:
>> > >>
>> > >>> Fantastic! We're also very interested in this feature.
>> > >>>
>> > >>> +Boxiu
>> > >>>
>> > >>> On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <[hidden email]>
>> > wrote:
>> > >>>
>> > >>>> In umbrella task FLINK-10232 we have introduced CREATE TABLE
>> grammar
>> > in
>> > >>>> our new module flink-sql-parser. And we proposed to use computed
>> > column
>> > >>> to
>> > >>>> describe the time attribute of process time in the design doc FLINK
>> > SQL
>> > >>>> DDL, so user may create a table with process time attribute as
>> > follows:
>> > >>>> create table T1(
>> > >>>>   a int,
>> > >>>>   b bigint,
>> > >>>>   c varchar,
>> > >>>>   d as PROCTIME,
>> > >>>> ) with (
>> > >>>>   'k1' = 'v1',
>> > >>>>   'k2' = 'v2'
>> > >>>> );
>> > >>>>
>> > >>>> The column d would be a process time attribute for table T1.
>> > >>>>
>> > >>>> Besides that, computed  columns have several other use cases, such
>> as
>> > >>>> these [2]:
>> > >>>>
>> > >>>>
>> > >>>> • Virtual generated columns can be used as a way to simplify and
>> unify
>> > >>>> queries. A complicated condition can be defined as a generated
>> column
>> > and
>> > >>>> referred to from multiple queries on the table to ensure that all
>> of
>> > them
>> > >>>> use exactly the same condition.
>> > >>>> • Stored generated columns can be used as a materialized cache for
>> > >>>> complicated conditions that are costly to calculate on the fly.
>> > >>>> • Generated columns can simulate functional indexes: Use a
>> generated
>> > >>>> column to define a functional expression and index it. This can be
>> > useful
>> > >>>> for working with columns of types that cannot be indexed directly,
>> > such
>> > >>> as
>> > >>>> JSON columns.
>> > >>>> • For stored generated columns, the disadvantage of this approach
>> is
>> > that
>> > >>>> values are stored twice; once as the value of the generated column
>> and
>> > >>> once
>> > >>>> in the index.
>> > >>>> • If a generated column is indexed, the optimizer recognizes query
>> > >>>> expressions that match the column definition and uses indexes from
>> the
>> > >>>> column as appropriate during query execution(Not supported yet).
>> > >>>>
>> > >>>>
>> > >>>>
>> > >>>> Computed columns are introduced in SQL-SERVER-2016 [1], MYSQL-5.6
>> [2]
>> > and
>> > >>>> ORACLE-11g [3].
>> > >>>>
>> > >>>> This is the design doc:
>> > >>>>
>> > >>>>
>> > >>>
>> >
>> https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
>> > >>>>
>> > >>>> Any suggestions are appreciated, thanks.
>> > >>>>
>> > >>>> [1]
>> > >>>>
>> > >>>
>> >
>> https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
>> > >>>> [2]
>> > >>>>
>> > >>>
>> >
>> https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
>> > >>>> [3] https://oracle-base.com/articles/11g/virtual-columns-11gr1
>> > >>>>
>> > >>>> Best,
>> > >>>> Danny Chan
>> > >>>>
>> > >>>
>> >
>> >
>>
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Danny Chan
In reply to this post by Kurt Young
Thanks, I have removed the contents about the “STORED” keyword, we may introduce that in the future if there is requests.

Also thanks all the reviewers for the design doc, I have resolved all the questions/suggestions in the doc at this time.

I will kick off a voting thread shortly as there were no comments in this thread so far, so I would assume there are no objections :)

Best,
Danny Chan
在 2019年10月24日 +0800 PM5:11,Kurt Young <[hidden email]>,写道:

> +1 to move to a future section. By deleting it I mean remove from
> the content describing the current processing procedure.
>
> Best,
> Kurt
>
>
> On Thu, Oct 24, 2019 at 5:01 PM Timo Walther <[hidden email]> wrote:
>
> > Having an MVP and a limited scope sounds good to me. But I would not
> > remove the STORED keyword entirely from the document.
> >
> > It shows that we have a long-term vision. Instead of deleting this
> > content, I would move it to a Outlook/Future Work section.
> >
> > Regards,
> > Timo
> >
> >
> > On 24.10.19 10:55, Jark Wu wrote:
> > > +1 to remove “STORED” related content. We can add them when user
> > requires.
> > > Others looks good to me in general.
> > >
> > > Thanks,
> > > Jark
> > >
> > >
> > > > 在 2019年10月24日,14:58,Kurt Young <[hidden email]> 写道:
> > > >
> > > > Hi Danny,
> > > >
> > > > Thanks for preparing this design document. IMO It's a very useful
> > > > feature, especially combined with time attribute support to specify
> > > > watermark in DDL.
> > > >
> > > > The design doc looks quite good, but I would suggest to reduce the
> > > > scope of the first version. Like we don't have to support "STORED"
> > > > in the first MVP version, and you can also delete related content in
> > > > document to make it more clean and easier to understand.
> > > >
> > > > Best,
> > > > Kurt
> > > >
> > > >
> > > > On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]> wrote:
> > > >
> > > > > Fantastic! We're also very interested in this feature.
> > > > >
> > > > > +Boxiu
> > > > >
> > > > > On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <[hidden email]>
> > wrote:
> > > > >
> > > > > > In umbrella task FLINK-10232 we have introduced CREATE TABLE grammar
> > in
> > > > > > our new module flink-sql-parser. And we proposed to use computed
> > column
> > > > > to
> > > > > > describe the time attribute of process time in the design doc FLINK
> > SQL
> > > > > > DDL, so user may create a table with process time attribute as
> > follows:
> > > > > > create table T1(
> > > > > > a int,
> > > > > > b bigint,
> > > > > > c varchar,
> > > > > > d as PROCTIME,
> > > > > > ) with (
> > > > > > 'k1' = 'v1',
> > > > > > 'k2' = 'v2'
> > > > > > );
> > > > > >
> > > > > > The column d would be a process time attribute for table T1.
> > > > > >
> > > > > > Besides that, computed columns have several other use cases, such as
> > > > > > these [2]:
> > > > > >
> > > > > >
> > > > > > • Virtual generated columns can be used as a way to simplify and unify
> > > > > > queries. A complicated condition can be defined as a generated column
> > and
> > > > > > referred to from multiple queries on the table to ensure that all of
> > them
> > > > > > use exactly the same condition.
> > > > > > • Stored generated columns can be used as a materialized cache for
> > > > > > complicated conditions that are costly to calculate on the fly.
> > > > > > • Generated columns can simulate functional indexes: Use a generated
> > > > > > column to define a functional expression and index it. This can be
> > useful
> > > > > > for working with columns of types that cannot be indexed directly,
> > such
> > > > > as
> > > > > > JSON columns.
> > > > > > • For stored generated columns, the disadvantage of this approach is
> > that
> > > > > > values are stored twice; once as the value of the generated column and
> > > > > once
> > > > > > in the index.
> > > > > > • If a generated column is indexed, the optimizer recognizes query
> > > > > > expressions that match the column definition and uses indexes from the
> > > > > > column as appropriate during query execution(Not supported yet).
> > > > > >
> > > > > >
> > > > > >
> > > > > > Computed columns are introduced in SQL-SERVER-2016 [1], MYSQL-5.6 [2]
> > and
> > > > > > ORACLE-11g [3].
> > > > > >
> > > > > > This is the design doc:
> > > > > >
> > > > > >
> > > > >
> > https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
> > > > > >
> > > > > > Any suggestions are appreciated, thanks.
> > > > > >
> > > > > > [1]
> > > > > >
> > > > >
> > https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
> > > > > > [2]
> > > > > >
> > > > >
> > https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
> > > > > > [3] https://oracle-base.com/articles/11g/virtual-columns-11gr1
> > > > > >
> > > > > > Best,
> > > > > > Danny Chan
> > > > > >
> > > > >
> >
> >
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Kurt Young
The updated doc looks good to me now, +1 to kick off a voting thread.

Best,
Kurt


On Fri, Oct 25, 2019 at 8:46 PM Danny Chan <[hidden email]> wrote:

> Thanks, I have removed the contents about the “STORED” keyword, we may
> introduce that in the future if there is requests.
>
> Also thanks all the reviewers for the design doc, I have resolved all the
> questions/suggestions in the doc at this time.
>
> I will kick off a voting thread shortly as there were no comments in this
> thread so far, so I would assume there are no objections :)
>
> Best,
> Danny Chan
> 在 2019年10月24日 +0800 PM5:11,Kurt Young <[hidden email]>,写道:
> > +1 to move to a future section. By deleting it I mean remove from
> > the content describing the current processing procedure.
> >
> > Best,
> > Kurt
> >
> >
> > On Thu, Oct 24, 2019 at 5:01 PM Timo Walther <[hidden email]> wrote:
> >
> > > Having an MVP and a limited scope sounds good to me. But I would not
> > > remove the STORED keyword entirely from the document.
> > >
> > > It shows that we have a long-term vision. Instead of deleting this
> > > content, I would move it to a Outlook/Future Work section.
> > >
> > > Regards,
> > > Timo
> > >
> > >
> > > On 24.10.19 10:55, Jark Wu wrote:
> > > > +1 to remove “STORED” related content. We can add them when user
> > > requires.
> > > > Others looks good to me in general.
> > > >
> > > > Thanks,
> > > > Jark
> > > >
> > > >
> > > > > 在 2019年10月24日,14:58,Kurt Young <[hidden email]> 写道:
> > > > >
> > > > > Hi Danny,
> > > > >
> > > > > Thanks for preparing this design document. IMO It's a very useful
> > > > > feature, especially combined with time attribute support to specify
> > > > > watermark in DDL.
> > > > >
> > > > > The design doc looks quite good, but I would suggest to reduce the
> > > > > scope of the first version. Like we don't have to support "STORED"
> > > > > in the first MVP version, and you can also delete related content
> in
> > > > > document to make it more clean and easier to understand.
> > > > >
> > > > > Best,
> > > > > Kurt
> > > > >
> > > > >
> > > > > On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]> wrote:
> > > > >
> > > > > > Fantastic! We're also very interested in this feature.
> > > > > >
> > > > > > +Boxiu
> > > > > >
> > > > > > On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <
> [hidden email]>
> > > wrote:
> > > > > >
> > > > > > > In umbrella task FLINK-10232 we have introduced CREATE TABLE
> grammar
> > > in
> > > > > > > our new module flink-sql-parser. And we proposed to use
> computed
> > > column
> > > > > > to
> > > > > > > describe the time attribute of process time in the design doc
> FLINK
> > > SQL
> > > > > > > DDL, so user may create a table with process time attribute as
> > > follows:
> > > > > > > create table T1(
> > > > > > > a int,
> > > > > > > b bigint,
> > > > > > > c varchar,
> > > > > > > d as PROCTIME,
> > > > > > > ) with (
> > > > > > > 'k1' = 'v1',
> > > > > > > 'k2' = 'v2'
> > > > > > > );
> > > > > > >
> > > > > > > The column d would be a process time attribute for table T1.
> > > > > > >
> > > > > > > Besides that, computed columns have several other use cases,
> such as
> > > > > > > these [2]:
> > > > > > >
> > > > > > >
> > > > > > > • Virtual generated columns can be used as a way to simplify
> and unify
> > > > > > > queries. A complicated condition can be defined as a generated
> column
> > > and
> > > > > > > referred to from multiple queries on the table to ensure that
> all of
> > > them
> > > > > > > use exactly the same condition.
> > > > > > > • Stored generated columns can be used as a materialized cache
> for
> > > > > > > complicated conditions that are costly to calculate on the fly.
> > > > > > > • Generated columns can simulate functional indexes: Use a
> generated
> > > > > > > column to define a functional expression and index it. This
> can be
> > > useful
> > > > > > > for working with columns of types that cannot be indexed
> directly,
> > > such
> > > > > > as
> > > > > > > JSON columns.
> > > > > > > • For stored generated columns, the disadvantage of this
> approach is
> > > that
> > > > > > > values are stored twice; once as the value of the generated
> column and
> > > > > > once
> > > > > > > in the index.
> > > > > > > • If a generated column is indexed, the optimizer recognizes
> query
> > > > > > > expressions that match the column definition and uses indexes
> from the
> > > > > > > column as appropriate during query execution(Not supported
> yet).
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > Computed columns are introduced in SQL-SERVER-2016 [1],
> MYSQL-5.6 [2]
> > > and
> > > > > > > ORACLE-11g [3].
> > > > > > >
> > > > > > > This is the design doc:
> > > > > > >
> > > > > > >
> > > > > >
> > >
> https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
> > > > > > >
> > > > > > > Any suggestions are appreciated, thanks.
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > >
> > >
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
> > > > > > > [2]
> > > > > > >
> > > > > >
> > >
> https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
> > > > > > > [3] https://oracle-base.com/articles/11g/virtual-columns-11gr1
> > > > > > >
> > > > > > > Best,
> > > > > > > Danny Chan
> > > > > > >
> > > > > >
> > >
> > >
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Jark Wu-2
Hi Danny,

Thanks for updating the documentation, I really like this design. But I
have two more suggestions:

1) As "STORED" keyword is not in the MVP, I would like to remove the
optional "[VIRTUAL]" keyword as well.
Because this optional keyword is useless for now, all the computed columns
are virtual as default.
If we add this optional keyword in the syntax, users may have the feeling
that the default is the
opposite mode "STORED" (the optional "[NOT NULL]" keyword works in this
way), however the
default is virtual which may surprise users.

2) Not introduce `GenerationStrategy` class for now.
I think the design of `GenerationStrategy` is really good. However, the
same reason as above,
"STORED" is not introduced in the MVP. Then this class is useless (all
computed columns are virtual).
We can introduce this class once we support "STORED". And I think this is a
safer way than rushing
to expose some APIs users don't need yet.

What do you think?

Best,
Jark





On Fri, 25 Oct 2019 at 21:11, Kurt Young <[hidden email]> wrote:

> The updated doc looks good to me now, +1 to kick off a voting thread.
>
> Best,
> Kurt
>
>
> On Fri, Oct 25, 2019 at 8:46 PM Danny Chan <[hidden email]> wrote:
>
> > Thanks, I have removed the contents about the “STORED” keyword, we may
> > introduce that in the future if there is requests.
> >
> > Also thanks all the reviewers for the design doc, I have resolved all the
> > questions/suggestions in the doc at this time.
> >
> > I will kick off a voting thread shortly as there were no comments in this
> > thread so far, so I would assume there are no objections :)
> >
> > Best,
> > Danny Chan
> > 在 2019年10月24日 +0800 PM5:11,Kurt Young <[hidden email]>,写道:
> > > +1 to move to a future section. By deleting it I mean remove from
> > > the content describing the current processing procedure.
> > >
> > > Best,
> > > Kurt
> > >
> > >
> > > On Thu, Oct 24, 2019 at 5:01 PM Timo Walther <[hidden email]>
> wrote:
> > >
> > > > Having an MVP and a limited scope sounds good to me. But I would not
> > > > remove the STORED keyword entirely from the document.
> > > >
> > > > It shows that we have a long-term vision. Instead of deleting this
> > > > content, I would move it to a Outlook/Future Work section.
> > > >
> > > > Regards,
> > > > Timo
> > > >
> > > >
> > > > On 24.10.19 10:55, Jark Wu wrote:
> > > > > +1 to remove “STORED” related content. We can add them when user
> > > > requires.
> > > > > Others looks good to me in general.
> > > > >
> > > > > Thanks,
> > > > > Jark
> > > > >
> > > > >
> > > > > > 在 2019年10月24日,14:58,Kurt Young <[hidden email]> 写道:
> > > > > >
> > > > > > Hi Danny,
> > > > > >
> > > > > > Thanks for preparing this design document. IMO It's a very useful
> > > > > > feature, especially combined with time attribute support to
> specify
> > > > > > watermark in DDL.
> > > > > >
> > > > > > The design doc looks quite good, but I would suggest to reduce
> the
> > > > > > scope of the first version. Like we don't have to support
> "STORED"
> > > > > > in the first MVP version, and you can also delete related content
> > in
> > > > > > document to make it more clean and easier to understand.
> > > > > >
> > > > > > Best,
> > > > > > Kurt
> > > > > >
> > > > > >
> > > > > > On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]>
> wrote:
> > > > > >
> > > > > > > Fantastic! We're also very interested in this feature.
> > > > > > >
> > > > > > > +Boxiu
> > > > > > >
> > > > > > > On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <
> > [hidden email]>
> > > > wrote:
> > > > > > >
> > > > > > > > In umbrella task FLINK-10232 we have introduced CREATE TABLE
> > grammar
> > > > in
> > > > > > > > our new module flink-sql-parser. And we proposed to use
> > computed
> > > > column
> > > > > > > to
> > > > > > > > describe the time attribute of process time in the design doc
> > FLINK
> > > > SQL
> > > > > > > > DDL, so user may create a table with process time attribute
> as
> > > > follows:
> > > > > > > > create table T1(
> > > > > > > > a int,
> > > > > > > > b bigint,
> > > > > > > > c varchar,
> > > > > > > > d as PROCTIME,
> > > > > > > > ) with (
> > > > > > > > 'k1' = 'v1',
> > > > > > > > 'k2' = 'v2'
> > > > > > > > );
> > > > > > > >
> > > > > > > > The column d would be a process time attribute for table T1.
> > > > > > > >
> > > > > > > > Besides that, computed columns have several other use cases,
> > such as
> > > > > > > > these [2]:
> > > > > > > >
> > > > > > > >
> > > > > > > > • Virtual generated columns can be used as a way to simplify
> > and unify
> > > > > > > > queries. A complicated condition can be defined as a
> generated
> > column
> > > > and
> > > > > > > > referred to from multiple queries on the table to ensure that
> > all of
> > > > them
> > > > > > > > use exactly the same condition.
> > > > > > > > • Stored generated columns can be used as a materialized
> cache
> > for
> > > > > > > > complicated conditions that are costly to calculate on the
> fly.
> > > > > > > > • Generated columns can simulate functional indexes: Use a
> > generated
> > > > > > > > column to define a functional expression and index it. This
> > can be
> > > > useful
> > > > > > > > for working with columns of types that cannot be indexed
> > directly,
> > > > such
> > > > > > > as
> > > > > > > > JSON columns.
> > > > > > > > • For stored generated columns, the disadvantage of this
> > approach is
> > > > that
> > > > > > > > values are stored twice; once as the value of the generated
> > column and
> > > > > > > once
> > > > > > > > in the index.
> > > > > > > > • If a generated column is indexed, the optimizer recognizes
> > query
> > > > > > > > expressions that match the column definition and uses indexes
> > from the
> > > > > > > > column as appropriate during query execution(Not supported
> > yet).
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > Computed columns are introduced in SQL-SERVER-2016 [1],
> > MYSQL-5.6 [2]
> > > > and
> > > > > > > > ORACLE-11g [3].
> > > > > > > >
> > > > > > > > This is the design doc:
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > >
> >
> https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
> > > > > > > >
> > > > > > > > Any suggestions are appreciated, thanks.
> > > > > > > >
> > > > > > > > [1]
> > > > > > > >
> > > > > > >
> > > >
> >
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
> > > > > > > > [2]
> > > > > > > >
> > > > > > >
> > > >
> >
> https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
> > > > > > > > [3]
> https://oracle-base.com/articles/11g/virtual-columns-11gr1
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Danny Chan
> > > > > > > >
> > > > > > >
> > > >
> > > >
> >
>
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Danny Chan
Thanks for the review Jark, I agree both point 1 and 2 and already updated the design doc.

In the latest POC code, i think we can simplify the design by removing the VIRTUAL keyword and the GenerationStrategy, we can introduce them later when we add in support for STORED.

Best,
Danny Chan
在 2019年10月25日 +0800 PM11:18,Jark Wu <[hidden email]>,写道:

> Hi Danny,
>
> Thanks for updating the documentation, I really like this design. But I
> have two more suggestions:
>
> 1) As "STORED" keyword is not in the MVP, I would like to remove the
> optional "[VIRTUAL]" keyword as well.
> Because this optional keyword is useless for now, all the computed columns
> are virtual as default.
> If we add this optional keyword in the syntax, users may have the feeling
> that the default is the
> opposite mode "STORED" (the optional "[NOT NULL]" keyword works in this
> way), however the
> default is virtual which may surprise users.
>
> 2) Not introduce `GenerationStrategy` class for now.
> I think the design of `GenerationStrategy` is really good. However, the
> same reason as above,
> "STORED" is not introduced in the MVP. Then this class is useless (all
> computed columns are virtual).
> We can introduce this class once we support "STORED". And I think this is a
> safer way than rushing
> to expose some APIs users don't need yet.
>
> What do you think?
>
> Best,
> Jark
>
>
>
>
>
> On Fri, 25 Oct 2019 at 21:11, Kurt Young <[hidden email]> wrote:
>
> > The updated doc looks good to me now, +1 to kick off a voting thread.
> >
> > Best,
> > Kurt
> >
> >
> > On Fri, Oct 25, 2019 at 8:46 PM Danny Chan <[hidden email]> wrote:
> >
> > > Thanks, I have removed the contents about the “STORED” keyword, we may
> > > introduce that in the future if there is requests.
> > >
> > > Also thanks all the reviewers for the design doc, I have resolved all the
> > > questions/suggestions in the doc at this time.
> > >
> > > I will kick off a voting thread shortly as there were no comments in this
> > > thread so far, so I would assume there are no objections :)
> > >
> > > Best,
> > > Danny Chan
> > > 在 2019年10月24日 +0800 PM5:11,Kurt Young <[hidden email]>,写道:
> > > > +1 to move to a future section. By deleting it I mean remove from
> > > > the content describing the current processing procedure.
> > > >
> > > > Best,
> > > > Kurt
> > > >
> > > >
> > > > On Thu, Oct 24, 2019 at 5:01 PM Timo Walther <[hidden email]>
> > wrote:
> > > >
> > > > > Having an MVP and a limited scope sounds good to me. But I would not
> > > > > remove the STORED keyword entirely from the document.
> > > > >
> > > > > It shows that we have a long-term vision. Instead of deleting this
> > > > > content, I would move it to a Outlook/Future Work section.
> > > > >
> > > > > Regards,
> > > > > Timo
> > > > >
> > > > >
> > > > > On 24.10.19 10:55, Jark Wu wrote:
> > > > > > +1 to remove “STORED” related content. We can add them when user
> > > > > requires.
> > > > > > Others looks good to me in general.
> > > > > >
> > > > > > Thanks,
> > > > > > Jark
> > > > > >
> > > > > >
> > > > > > > 在 2019年10月24日,14:58,Kurt Young <[hidden email]> 写道:
> > > > > > >
> > > > > > > Hi Danny,
> > > > > > >
> > > > > > > Thanks for preparing this design document. IMO It's a very useful
> > > > > > > feature, especially combined with time attribute support to
> > specify
> > > > > > > watermark in DDL.
> > > > > > >
> > > > > > > The design doc looks quite good, but I would suggest to reduce
> > the
> > > > > > > scope of the first version. Like we don't have to support
> > "STORED"
> > > > > > > in the first MVP version, and you can also delete related content
> > > in
> > > > > > > document to make it more clean and easier to understand.
> > > > > > >
> > > > > > > Best,
> > > > > > > Kurt
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]>
> > wrote:
> > > > > > >
> > > > > > > > Fantastic! We're also very interested in this feature.
> > > > > > > >
> > > > > > > > +Boxiu
> > > > > > > >
> > > > > > > > On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <
> > > [hidden email]>
> > > > > wrote:
> > > > > > > >
> > > > > > > > > In umbrella task FLINK-10232 we have introduced CREATE TABLE
> > > grammar
> > > > > in
> > > > > > > > > our new module flink-sql-parser. And we proposed to use
> > > computed
> > > > > column
> > > > > > > > to
> > > > > > > > > describe the time attribute of process time in the design doc
> > > FLINK
> > > > > SQL
> > > > > > > > > DDL, so user may create a table with process time attribute
> > as
> > > > > follows:
> > > > > > > > > create table T1(
> > > > > > > > > a int,
> > > > > > > > > b bigint,
> > > > > > > > > c varchar,
> > > > > > > > > d as PROCTIME,
> > > > > > > > > ) with (
> > > > > > > > > 'k1' = 'v1',
> > > > > > > > > 'k2' = 'v2'
> > > > > > > > > );
> > > > > > > > >
> > > > > > > > > The column d would be a process time attribute for table T1.
> > > > > > > > >
> > > > > > > > > Besides that, computed columns have several other use cases,
> > > such as
> > > > > > > > > these [2]:
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > • Virtual generated columns can be used as a way to simplify
> > > and unify
> > > > > > > > > queries. A complicated condition can be defined as a
> > generated
> > > column
> > > > > and
> > > > > > > > > referred to from multiple queries on the table to ensure that
> > > all of
> > > > > them
> > > > > > > > > use exactly the same condition.
> > > > > > > > > • Stored generated columns can be used as a materialized
> > cache
> > > for
> > > > > > > > > complicated conditions that are costly to calculate on the
> > fly.
> > > > > > > > > • Generated columns can simulate functional indexes: Use a
> > > generated
> > > > > > > > > column to define a functional expression and index it. This
> > > can be
> > > > > useful
> > > > > > > > > for working with columns of types that cannot be indexed
> > > directly,
> > > > > such
> > > > > > > > as
> > > > > > > > > JSON columns.
> > > > > > > > > • For stored generated columns, the disadvantage of this
> > > approach is
> > > > > that
> > > > > > > > > values are stored twice; once as the value of the generated
> > > column and
> > > > > > > > once
> > > > > > > > > in the index.
> > > > > > > > > • If a generated column is indexed, the optimizer recognizes
> > > query
> > > > > > > > > expressions that match the column definition and uses indexes
> > > from the
> > > > > > > > > column as appropriate during query execution(Not supported
> > > yet).
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > Computed columns are introduced in SQL-SERVER-2016 [1],
> > > MYSQL-5.6 [2]
> > > > > and
> > > > > > > > > ORACLE-11g [3].
> > > > > > > > >
> > > > > > > > > This is the design doc:
> > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > >
> > >
> > https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
> > > > > > > > >
> > > > > > > > > Any suggestions are appreciated, thanks.
> > > > > > > > >
> > > > > > > > > [1]
> > > > > > > > >
> > > > > > > >
> > > > >
> > >
> > https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
> > > > > > > > > [2]
> > > > > > > > >
> > > > > > > >
> > > > >
> > >
> > https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
> > > > > > > > > [3]
> > https://oracle-base.com/articles/11g/virtual-columns-11gr1
> > > > > > > > >
> > > > > > > > > Best,
> > > > > > > > > Danny Chan
> > > > > > > > >
> > > > > > > >
> > > > >
> > > > >
> > >
> >
Reply | Threaded
Open this post in threaded view
|

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

Jark Wu-2
Thank you for the updating. The design doc looks pretty good to me now.

+1 to start a vote.

Best,
Jark

On Mon, 28 Oct 2019 at 11:10, Danny Chan <[hidden email]> wrote:

> Thanks for the review Jark, I agree both point 1 and 2 and already updated
> the design doc.
>
> In the latest POC code, i think we can simplify the design by removing the
> VIRTUAL keyword and the GenerationStrategy, we can introduce them later
> when we add in support for STORED.
>
> Best,
> Danny Chan
> 在 2019年10月25日 +0800 PM11:18,Jark Wu <[hidden email]>,写道:
> > Hi Danny,
> >
> > Thanks for updating the documentation, I really like this design. But I
> > have two more suggestions:
> >
> > 1) As "STORED" keyword is not in the MVP, I would like to remove the
> > optional "[VIRTUAL]" keyword as well.
> > Because this optional keyword is useless for now, all the computed
> columns
> > are virtual as default.
> > If we add this optional keyword in the syntax, users may have the feeling
> > that the default is the
> > opposite mode "STORED" (the optional "[NOT NULL]" keyword works in this
> > way), however the
> > default is virtual which may surprise users.
> >
> > 2) Not introduce `GenerationStrategy` class for now.
> > I think the design of `GenerationStrategy` is really good. However, the
> > same reason as above,
> > "STORED" is not introduced in the MVP. Then this class is useless (all
> > computed columns are virtual).
> > We can introduce this class once we support "STORED". And I think this
> is a
> > safer way than rushing
> > to expose some APIs users don't need yet.
> >
> > What do you think?
> >
> > Best,
> > Jark
> >
> >
> >
> >
> >
> > On Fri, 25 Oct 2019 at 21:11, Kurt Young <[hidden email]> wrote:
> >
> > > The updated doc looks good to me now, +1 to kick off a voting thread.
> > >
> > > Best,
> > > Kurt
> > >
> > >
> > > On Fri, Oct 25, 2019 at 8:46 PM Danny Chan <[hidden email]>
> wrote:
> > >
> > > > Thanks, I have removed the contents about the “STORED” keyword, we
> may
> > > > introduce that in the future if there is requests.
> > > >
> > > > Also thanks all the reviewers for the design doc, I have resolved
> all the
> > > > questions/suggestions in the doc at this time.
> > > >
> > > > I will kick off a voting thread shortly as there were no comments in
> this
> > > > thread so far, so I would assume there are no objections :)
> > > >
> > > > Best,
> > > > Danny Chan
> > > > 在 2019年10月24日 +0800 PM5:11,Kurt Young <[hidden email]>,写道:
> > > > > +1 to move to a future section. By deleting it I mean remove from
> > > > > the content describing the current processing procedure.
> > > > >
> > > > > Best,
> > > > > Kurt
> > > > >
> > > > >
> > > > > On Thu, Oct 24, 2019 at 5:01 PM Timo Walther <[hidden email]>
> > > wrote:
> > > > >
> > > > > > Having an MVP and a limited scope sounds good to me. But I would
> not
> > > > > > remove the STORED keyword entirely from the document.
> > > > > >
> > > > > > It shows that we have a long-term vision. Instead of deleting
> this
> > > > > > content, I would move it to a Outlook/Future Work section.
> > > > > >
> > > > > > Regards,
> > > > > > Timo
> > > > > >
> > > > > >
> > > > > > On 24.10.19 10:55, Jark Wu wrote:
> > > > > > > +1 to remove “STORED” related content. We can add them when
> user
> > > > > > requires.
> > > > > > > Others looks good to me in general.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Jark
> > > > > > >
> > > > > > >
> > > > > > > > 在 2019年10月24日,14:58,Kurt Young <[hidden email]> 写道:
> > > > > > > >
> > > > > > > > Hi Danny,
> > > > > > > >
> > > > > > > > Thanks for preparing this design document. IMO It's a very
> useful
> > > > > > > > feature, especially combined with time attribute support to
> > > specify
> > > > > > > > watermark in DDL.
> > > > > > > >
> > > > > > > > The design doc looks quite good, but I would suggest to
> reduce
> > > the
> > > > > > > > scope of the first version. Like we don't have to support
> > > "STORED"
> > > > > > > > in the first MVP version, and you can also delete related
> content
> > > > in
> > > > > > > > document to make it more clean and easier to understand.
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Kurt
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, Sep 17, 2019 at 9:18 PM Qi Luo <[hidden email]>
> > > wrote:
> > > > > > > >
> > > > > > > > > Fantastic! We're also very interested in this feature.
> > > > > > > > >
> > > > > > > > > +Boxiu
> > > > > > > > >
> > > > > > > > > On Tue, Sep 17, 2019 at 11:31 AM Danny Chan <
> > > > [hidden email]>
> > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > In umbrella task FLINK-10232 we have introduced CREATE
> TABLE
> > > > grammar
> > > > > > in
> > > > > > > > > > our new module flink-sql-parser. And we proposed to use
> > > > computed
> > > > > > column
> > > > > > > > > to
> > > > > > > > > > describe the time attribute of process time in the
> design doc
> > > > FLINK
> > > > > > SQL
> > > > > > > > > > DDL, so user may create a table with process time
> attribute
> > > as
> > > > > > follows:
> > > > > > > > > > create table T1(
> > > > > > > > > > a int,
> > > > > > > > > > b bigint,
> > > > > > > > > > c varchar,
> > > > > > > > > > d as PROCTIME,
> > > > > > > > > > ) with (
> > > > > > > > > > 'k1' = 'v1',
> > > > > > > > > > 'k2' = 'v2'
> > > > > > > > > > );
> > > > > > > > > >
> > > > > > > > > > The column d would be a process time attribute for table
> T1.
> > > > > > > > > >
> > > > > > > > > > Besides that, computed columns have several other use
> cases,
> > > > such as
> > > > > > > > > > these [2]:
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > • Virtual generated columns can be used as a way to
> simplify
> > > > and unify
> > > > > > > > > > queries. A complicated condition can be defined as a
> > > generated
> > > > column
> > > > > > and
> > > > > > > > > > referred to from multiple queries on the table to ensure
> that
> > > > all of
> > > > > > them
> > > > > > > > > > use exactly the same condition.
> > > > > > > > > > • Stored generated columns can be used as a materialized
> > > cache
> > > > for
> > > > > > > > > > complicated conditions that are costly to calculate on
> the
> > > fly.
> > > > > > > > > > • Generated columns can simulate functional indexes: Use
> a
> > > > generated
> > > > > > > > > > column to define a functional expression and index it.
> This
> > > > can be
> > > > > > useful
> > > > > > > > > > for working with columns of types that cannot be indexed
> > > > directly,
> > > > > > such
> > > > > > > > > as
> > > > > > > > > > JSON columns.
> > > > > > > > > > • For stored generated columns, the disadvantage of this
> > > > approach is
> > > > > > that
> > > > > > > > > > values are stored twice; once as the value of the
> generated
> > > > column and
> > > > > > > > > once
> > > > > > > > > > in the index.
> > > > > > > > > > • If a generated column is indexed, the optimizer
> recognizes
> > > > query
> > > > > > > > > > expressions that match the column definition and uses
> indexes
> > > > from the
> > > > > > > > > > column as appropriate during query execution(Not
> supported
> > > > yet).
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > Computed columns are introduced in SQL-SERVER-2016 [1],
> > > > MYSQL-5.6 [2]
> > > > > > and
> > > > > > > > > > ORACLE-11g [3].
> > > > > > > > > >
> > > > > > > > > > This is the design doc:
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > >
> > > >
> > >
> https://docs.google.com/document/d/110TseRtTCphxETPY7uhiHpu-dph3NEesh3mYKtJ7QOY/edit?usp=sharing
> > > > > > > > > >
> > > > > > > > > > Any suggestions are appreciated, thanks.
> > > > > > > > > >
> > > > > > > > > > [1]
> > > > > > > > > >
> > > > > > > > >
> > > > > >
> > > >
> > >
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/specify-computed-columns-in-a-table?view=sql-server-2016
> > > > > > > > > > [2]
> > > > > > > > > >
> > > > > > > > >
> > > > > >
> > > >
> > >
> https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html
> > > > > > > > > > [3]
> > > https://oracle-base.com/articles/11g/virtual-columns-11gr1
> > > > > > > > > >
> > > > > > > > > > Best,
> > > > > > > > > > Danny Chan
> > > > > > > > > >
> > > > > > > > >
> > > > > >
> > > > > >
> > > >
> > >
>