Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-28 Thread Leonard Xu
Hi, all

I open a new discussion of FLIP-132[1] which based on our consensus on current 
thread.

Let me keep communication in the new thread, please let me know if you have any 
concerns.

Best
Leonard
[1] 
http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-132-Temporal-Table-DDL-td43483.html
 


> 在 2020年7月7日,00:31,Seth Wiesman  写道:
> 
> * I mistyped the rejected_query, it should be
> 
> CREATE VIEW AS post_agg_stream SELECT currencyId, AVG(rate)* as *rate*
> FROM *currency_rates
> 
> CREATE VIEW AS rejected_query
> SELECT
>  ...FROM
>  transactions AS t
>  JOIN post_agg_stream FOR SYSTEM_TIME AS OF t.transactionTime AS r
>  ON r.currency = t.currency
> 
> 
> On Mon, Jul 6, 2020 at 11:29 AM Seth Wiesman  wrote:
> 
>> Hey Leonard,
>> 
>> Agreed, this is a fun discussion!
>> 
>> (1) For support changelog source backed CDC tools, a problem is that can
>>> we use the temporal table as a general source table which may followed by
>>> some aggregation operations,  more accurate is wether the aggregation
>>> operator can use the DELETE record that we just updated the “correct”
>>> operation time to retract a record, maybe not. This will pull us back to
>>> the discussion of operation time VS event time, it’s a real cool but
>>> complicated topic see above discussion from mine and @Jark’s.
>>> 
>> 
>> I fully agree this is a complicated topic, however, I don't think its
>> actually a problem that needs to be solved for the first version of this
>> feature. My proposal is to disallow using upsert streams as temporal tables
>> if an aggregation operation has been applied. Going back to my notion that
>> temporal tables are a tool for performing streaming star schema
>> denormalization, the dimension table in a star schema is rarely aggregated
>> pre-join. In the case of a CDC stream of currency rates joined to
>> transactions, the CDC stream only needs to support filter pushdowns and
>> map-like transformations before being joined. I believe this is a
>> reasonable limitation we can impose that will unblock a large percentage of
>> use cases, and once we better understand the semantics of the correct
>> operation in a retraction the limitation can be removed in future versions
>> while remaining backward compatible.
>> 
>> 
>> 
>> CREATE TABLE currency_rates (
>>  currencyId BIGINT PRIMARY KEY,
>>  rate DECIMAL(10, 2)) WITH (
>> 'connector' = 'kafka',
>> 'format' = 'debezium-json'
>> )
>> *CREATE* TABLE transactions (
>>  currencyId BIGINT,
>>  transactionTime TIMESTAMP(3)) WITH (
>> 
>> )
>> 
>> -- Uner my proposal this query would be supported because the currency_rates
>> 
>> -- table is used in a temporal join without any aggregations having been 
>> applied
>> 
>> CREATE VIEW AS working_query
>> SELECT
>>  ...FROM
>>  transactions AS t
>>  JOIN currency_rates FOR SYSTEM_TIME AS OF t.transactionTime AS r
>>  ON r.currency = t.currencyId
>> 
>> -- However, this query would be rejected by the planner until we determine 
>> the proper time semantics of a retacation
>> 
>> CREATE VIEW AS post_agg_stream SELECT currencyId, AVG(rate)* as *rate* FROM 
>> *currency_rates
>> 
>> CREATE VIEW AS rejected_query
>> SELECT
>>  ...FROM
>>  transactions AS t
>>  JOIN currency_rates FOR SYSTEM_TIME AS OF t.transactionTime AS r
>>  ON r.currency = t.currency
>> 
>> 
>> 
>> (2) For upsert source that defines PRIMARY KEY and may contains multiple
>>> records under a PK, the main problem is the  PK semantic,the multiple
>>> records under a PK broke the unique semantic on a table. We need to walk
>>> around this by (a) Adding another primary key keyword and explain the
>>> upsert semantic (b) Creating temporal table base on a view that is the
>>> deduplicated result of source table[2].
>>> 
>> 
>> This feels like more of a bikeshedding question than a blocker and I look
>> forward to seeing what you come up with!
>> 
>> Seth
>> 
>> On Mon, Jul 6, 2020 at 10:59 AM Benchao Li  wrote:
>> 
>>> Hi everyone,
>>> 
>>> Thanks a lot for the great discussions so far.
>>> 
>>> After reading through the long discussion, I still have one question.
>>> Currently the temporal table function supports both event time and proc
>>> time joining.
>>> If we use "FOR SYSTEM_TIME AS OF" syntax without "TEMPORAL" keyword in
>>> DDL,
>>> does it mean we can only use temporal table function join with event time?
>>> If we can, how do we distinguish it with current temporal table (also
>>> known as dimension table)?
>>> 
>>> Maybe I'm missing something here. Correct me if I'm wrong.
>>> 
>>> Leonard Xu  于2020年7月6日周一 下午11:34写道:
>>> 
 Hi, Seth
 
 Thanks for your explanation of user cases, and you’re wright the look up
 join/table is one kind of temporal table join/table which tracks latest
 snapshot of external  DB-like tables, it's why we proposed use same
 temporal join syntax.
 
 In fact, 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-06 Thread Seth Wiesman
* I mistyped the rejected_query, it should be

CREATE VIEW AS post_agg_stream SELECT currencyId, AVG(rate)* as *rate*
FROM *currency_rates

CREATE VIEW AS rejected_query
SELECT
  ...FROM
  transactions AS t
  JOIN post_agg_stream FOR SYSTEM_TIME AS OF t.transactionTime AS r
  ON r.currency = t.currency


On Mon, Jul 6, 2020 at 11:29 AM Seth Wiesman  wrote:

> Hey Leonard,
>
> Agreed, this is a fun discussion!
>
> (1) For support changelog source backed CDC tools, a problem is that can
>> we use the temporal table as a general source table which may followed by
>> some aggregation operations,  more accurate is wether the aggregation
>> operator can use the DELETE record that we just updated the “correct”
>> operation time to retract a record, maybe not. This will pull us back to
>> the discussion of operation time VS event time, it’s a real cool but
>> complicated topic see above discussion from mine and @Jark’s.
>>
>
> I fully agree this is a complicated topic, however, I don't think its
> actually a problem that needs to be solved for the first version of this
> feature. My proposal is to disallow using upsert streams as temporal tables
> if an aggregation operation has been applied. Going back to my notion that
> temporal tables are a tool for performing streaming star schema
> denormalization, the dimension table in a star schema is rarely aggregated
> pre-join. In the case of a CDC stream of currency rates joined to
> transactions, the CDC stream only needs to support filter pushdowns and
> map-like transformations before being joined. I believe this is a
> reasonable limitation we can impose that will unblock a large percentage of
> use cases, and once we better understand the semantics of the correct
> operation in a retraction the limitation can be removed in future versions
> while remaining backward compatible.
>
>
>
> CREATE TABLE currency_rates (
>   currencyId BIGINT PRIMARY KEY,
>   rate DECIMAL(10, 2)) WITH (
>  'connector' = 'kafka',
>  'format' = 'debezium-json'
> )
> *CREATE* TABLE transactions (
>   currencyId BIGINT,
>   transactionTime TIMESTAMP(3)) WITH (
>
> )
>
> -- Uner my proposal this query would be supported because the currency_rates
>
> -- table is used in a temporal join without any aggregations having been 
> applied
>
> CREATE VIEW AS working_query
> SELECT
>   ...FROM
>   transactions AS t
>   JOIN currency_rates FOR SYSTEM_TIME AS OF t.transactionTime AS r
>   ON r.currency = t.currencyId
>
> -- However, this query would be rejected by the planner until we determine 
> the proper time semantics of a retacation
>
> CREATE VIEW AS post_agg_stream SELECT currencyId, AVG(rate)* as *rate* FROM 
> *currency_rates
>
> CREATE VIEW AS rejected_query
> SELECT
>   ...FROM
>   transactions AS t
>   JOIN currency_rates FOR SYSTEM_TIME AS OF t.transactionTime AS r
>   ON r.currency = t.currency
>
>
>
> (2) For upsert source that defines PRIMARY KEY and may contains multiple
>> records under a PK, the main problem is the  PK semantic,the multiple
>> records under a PK broke the unique semantic on a table. We need to walk
>> around this by (a) Adding another primary key keyword and explain the
>> upsert semantic (b) Creating temporal table base on a view that is the
>> deduplicated result of source table[2].
>>
>
> This feels like more of a bikeshedding question than a blocker and I look
> forward to seeing what you come up with!
>
> Seth
>
> On Mon, Jul 6, 2020 at 10:59 AM Benchao Li  wrote:
>
>> Hi everyone,
>>
>> Thanks a lot for the great discussions so far.
>>
>> After reading through the long discussion, I still have one question.
>> Currently the temporal table function supports both event time and proc
>> time joining.
>> If we use "FOR SYSTEM_TIME AS OF" syntax without "TEMPORAL" keyword in
>> DDL,
>> does it mean we can only use temporal table function join with event time?
>> If we can, how do we distinguish it with current temporal table (also
>> known as dimension table)?
>>
>> Maybe I'm missing something here. Correct me if I'm wrong.
>>
>> Leonard Xu  于2020年7月6日周一 下午11:34写道:
>>
>>> Hi, Seth
>>>
>>> Thanks for your explanation of user cases, and you’re wright the look up
>>> join/table is one kind of temporal table join/table which tracks latest
>>> snapshot of external  DB-like tables, it's why we proposed use same
>>> temporal join syntax.
>>>
>>> In fact, I have invested and checked Debezuim format and Canal format
>>> more these days, and we can obtain the extract DML operation time from
>>> their meta information which comes from DB bin-log.  Although extracting
>>> meta information from record is a part of FLIP-107 scope[1], at least we
>>> have a way to extract the correct operation time. Event we can obtain the
>>> expected operation time, there’re some problems.
>>>
>>> (1) For support changelog source backed CDC tools, a problem is that can
>>> we use the temporal table as a general source table which may followed by
>>> some aggregation operations,  more accurate 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-06 Thread Seth Wiesman
Hey Leonard,

Agreed, this is a fun discussion!

(1) For support changelog source backed CDC tools, a problem is that can we
> use the temporal table as a general source table which may followed by some
> aggregation operations,  more accurate is wether the aggregation operator
> can use the DELETE record that we just updated the “correct” operation time
> to retract a record, maybe not. This will pull us back to the discussion of
> operation time VS event time, it’s a real cool but complicated topic see
> above discussion from mine and @Jark’s.
>

I fully agree this is a complicated topic, however, I don't think its
actually a problem that needs to be solved for the first version of this
feature. My proposal is to disallow using upsert streams as temporal tables
if an aggregation operation has been applied. Going back to my notion that
temporal tables are a tool for performing streaming star schema
denormalization, the dimension table in a star schema is rarely aggregated
pre-join. In the case of a CDC stream of currency rates joined to
transactions, the CDC stream only needs to support filter pushdowns and
map-like transformations before being joined. I believe this is a
reasonable limitation we can impose that will unblock a large percentage of
use cases, and once we better understand the semantics of the correct
operation in a retraction the limitation can be removed in future versions
while remaining backward compatible.



CREATE TABLE currency_rates (
  currencyId BIGINT PRIMARY KEY,
  rate DECIMAL(10, 2)) WITH (
 'connector' = 'kafka',
 'format' = 'debezium-json'
)
*CREATE* TABLE transactions (
  currencyId BIGINT,
  transactionTime TIMESTAMP(3)) WITH (

)

-- Uner my proposal this query would be supported because the currency_rates

-- table is used in a temporal join without any aggregations having been applied

CREATE VIEW AS working_query
SELECT
  ...FROM
  transactions AS t
  JOIN currency_rates FOR SYSTEM_TIME AS OF t.transactionTime AS r
  ON r.currency = t.currencyId

-- However, this query would be rejected by the planner until we
determine the proper time semantics of a retacation

CREATE VIEW AS post_agg_stream SELECT currencyId, AVG(rate)* as *rate*
FROM *currency_rates

CREATE VIEW AS rejected_query
SELECT
  ...FROM
  transactions AS t
  JOIN currency_rates FOR SYSTEM_TIME AS OF t.transactionTime AS r
  ON r.currency = t.currency



(2) For upsert source that defines PRIMARY KEY and may contains multiple
> records under a PK, the main problem is the  PK semantic,the multiple
> records under a PK broke the unique semantic on a table. We need to walk
> around this by (a) Adding another primary key keyword and explain the
> upsert semantic (b) Creating temporal table base on a view that is the
> deduplicated result of source table[2].
>

This feels like more of a bikeshedding question than a blocker and I look
forward to seeing what you come up with!

Seth

On Mon, Jul 6, 2020 at 10:59 AM Benchao Li  wrote:

> Hi everyone,
>
> Thanks a lot for the great discussions so far.
>
> After reading through the long discussion, I still have one question.
> Currently the temporal table function supports both event time and proc
> time joining.
> If we use "FOR SYSTEM_TIME AS OF" syntax without "TEMPORAL" keyword in DDL,
> does it mean we can only use temporal table function join with event time?
> If we can, how do we distinguish it with current temporal table (also
> known as dimension table)?
>
> Maybe I'm missing something here. Correct me if I'm wrong.
>
> Leonard Xu  于2020年7月6日周一 下午11:34写道:
>
>> Hi, Seth
>>
>> Thanks for your explanation of user cases, and you’re wright the look up
>> join/table is one kind of temporal table join/table which tracks latest
>> snapshot of external  DB-like tables, it's why we proposed use same
>> temporal join syntax.
>>
>> In fact, I have invested and checked Debezuim format and Canal format
>> more these days, and we can obtain the extract DML operation time from
>> their meta information which comes from DB bin-log.  Although extracting
>> meta information from record is a part of FLIP-107 scope[1], at least we
>> have a way to extract the correct operation time. Event we can obtain the
>> expected operation time, there’re some problems.
>>
>> (1) For support changelog source backed CDC tools, a problem is that can
>> we use the temporal table as a general source table which may followed by
>> some aggregation operations,  more accurate is wether the aggregation
>> operator can use the DELETE record that we just updated the “correct”
>> operation time to retract a record, maybe not. This will pull us back to
>> the discussion of operation time VS event time, it’s a real cool but
>> complicated topic see above discussion from mine and @Jark’s.
>>
>> (2) For upsert source that defines PRIMARY KEY and may contains multiple
>> records under a PK, the main problem is the  PK semantic,the multiple
>> records under a PK broke the unique semantic on a table. We need 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-06 Thread Benchao Li
Hi everyone,

Thanks a lot for the great discussions so far.

After reading through the long discussion, I still have one question.
Currently the temporal table function supports both event time and proc
time joining.
If we use "FOR SYSTEM_TIME AS OF" syntax without "TEMPORAL" keyword in DDL,
does it mean we can only use temporal table function join with event time?
If we can, how do we distinguish it with current temporal table (also known
as dimension table)?

Maybe I'm missing something here. Correct me if I'm wrong.

Leonard Xu  于2020年7月6日周一 下午11:34写道:

> Hi, Seth
>
> Thanks for your explanation of user cases, and you’re wright the look up
> join/table is one kind of temporal table join/table which tracks latest
> snapshot of external  DB-like tables, it's why we proposed use same
> temporal join syntax.
>
> In fact, I have invested and checked Debezuim format and Canal format more
> these days, and we can obtain the extract DML operation time from their
> meta information which comes from DB bin-log.  Although extracting meta
> information from record is a part of FLIP-107 scope[1], at least we have a
> way to extract the correct operation time. Event we can obtain the expected
> operation time, there’re some problems.
>
> (1) For support changelog source backed CDC tools, a problem is that can
> we use the temporal table as a general source table which may followed by
> some aggregation operations,  more accurate is wether the aggregation
> operator can use the DELETE record that we just updated the “correct”
> operation time to retract a record, maybe not. This will pull us back to
> the discussion of operation time VS event time, it’s a real cool but
> complicated topic see above discussion from mine and @Jark’s.
>
> (2) For upsert source that defines PRIMARY KEY and may contains multiple
> records under a PK, the main problem is the  PK semantic,the multiple
> records under a PK broke the unique semantic on a table. We need to walk
> around this by (a) Adding another primary key keyword and explain the
> upsert semantic (b) Creating temporal table base on a view that is the
> deduplicated result of source table[2].
>
> I’m working on (2), and if we want to support(1)i.e. support DELETE
> entirely, that’s really a big challenge but I also think wright thing for
> long term.
>
> If we decide to do (1), we need import operation time concept firstly, we
> need change the codebase for deal the operation time header in many places
> secondly, and finally explain and tell users how to understand and use
> temporal table.
>
> I’m a little worried about it’s valuable enough, I proposed only support
> (2) because it is a good replacement of current Temporal Table Function and
> will not introduce more concept and works.
>
> Jark, Jingsong, Konstantin, WDYT?
>
>
> Best,
> Leonard Xu
> [1]
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-107%3A+Reading+table+columns+from+different+parts+of+source+records#FLIP107:Readingtablecolumnsfromdifferentpartsofsourcerecords-Accessread-onlymetadatae.g.partition
> <
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-107:+Reading+table+columns+from+different+parts+of+source+records#FLIP107:Readingtablecolumnsfromdifferentpartsofsourcerecords-Accessread-onlymetadatae.g.partition
> >
> [2]
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#deduplication
> <
> https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#deduplication
> >
>
>
> > 在 2020年7月6日,22:02,Seth Wiesman  写道:
> >
> > As an aside, I conceptually view temporal table joins to be semantically
> equivalent to look up table joins. They are just two different ways of
> consuming the same data.
> >
> > Seth
> >
> > On Mon, Jul 6, 2020 at 8:56 AM Seth Wiesman  > wrote:
> > Hi Leonard,
> >
> > Regarding DELETE operations I tend to have the opposite reaction. I
> spend a lot of time working with production Flink users across a large
> number of organizations and to say we don't support temporal tables that
> include DELETEs will be a blocker for adoption. Even organizations that
> claim to never delete rows still occasionally due so per  GDPR requests or
> other regulations.
> >
> > I actually do think users will understand the limitations. Flink today
> has a very clear value proposition around correctness, your results are as
> correct as the input data provided. This does not change under support for
> DELETE records. Flink is providing the most correct results possible based
> on the resolution of the fields as generated by 3rd party systems. As
> Debezium and other CDC libraries become more accurate, so will Flink.
> >
> > Seth
> >
> > On Fri, Jul 3, 2020 at 11:00 PM Leonard Xu  xbjt...@gmail.com>> wrote:
> > Hi, Konstantin
> >
> >> . Would we support a temporal join with a changelog stream with
> >> event time semantics by ignoring DELETE messages or would it be
> completed
> >> unsupported.
> >
> > I don’t know 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-06 Thread Leonard Xu
Hi, Seth

Thanks for your explanation of user cases, and you’re wright the look up 
join/table is one kind of temporal table join/table which tracks latest 
snapshot of external  DB-like tables, it's why we proposed use same temporal 
join syntax.

In fact, I have invested and checked Debezuim format and Canal format more 
these days, and we can obtain the extract DML operation time from their meta 
information which comes from DB bin-log.  Although extracting meta information 
from record is a part of FLIP-107 scope[1], at least we have a way to extract 
the correct operation time. Event we can obtain the expected operation time, 
there’re some problems. 

(1) For support changelog source backed CDC tools, a problem is that can we use 
the temporal table as a general source table which may followed by some 
aggregation operations,  more accurate is wether the aggregation operator can 
use the DELETE record that we just updated the “correct” operation time to 
retract a record, maybe not. This will pull us back to the discussion of 
operation time VS event time, it’s a real cool but complicated topic see above 
discussion from mine and @Jark’s.

(2) For upsert source that defines PRIMARY KEY and may contains multiple 
records under a PK, the main problem is the  PK semantic,the multiple records 
under a PK broke the unique semantic on a table. We need to walk around this by 
(a) Adding another primary key keyword and explain the upsert semantic (b) 
Creating temporal table base on a view that is the deduplicated result of 
source table[2]. 

I’m working on (2), and if we want to support(1)i.e. support DELETE entirely, 
that’s really a big challenge but I also think wright thing for long term.

If we decide to do (1), we need import operation time concept firstly, we need 
change the codebase for deal the operation time header in many places secondly, 
and finally explain and tell users how to understand and use temporal table. 

I’m a little worried about it’s valuable enough, I proposed only support (2) 
because it is a good replacement of current Temporal Table Function and will 
not introduce more concept and works.

Jark, Jingsong, Konstantin, WDYT?


Best,
Leonard Xu   
[1] 
https://cwiki.apache.org/confluence/display/FLINK/FLIP-107%3A+Reading+table+columns+from+different+parts+of+source+records#FLIP107:Readingtablecolumnsfromdifferentpartsofsourcerecords-Accessread-onlymetadatae.g.partition
 

[2] 
https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/queries.html#deduplication
 



> 在 2020年7月6日,22:02,Seth Wiesman  写道:
> 
> As an aside, I conceptually view temporal table joins to be semantically 
> equivalent to look up table joins. They are just two different ways of 
> consuming the same data. 
> 
> Seth 
> 
> On Mon, Jul 6, 2020 at 8:56 AM Seth Wiesman  > wrote:
> Hi Leonard, 
> 
> Regarding DELETE operations I tend to have the opposite reaction. I spend a 
> lot of time working with production Flink users across a large number of 
> organizations and to say we don't support temporal tables that include 
> DELETEs will be a blocker for adoption. Even organizations that claim to 
> never delete rows still occasionally due so per  GDPR requests or other 
> regulations.
> 
> I actually do think users will understand the limitations. Flink today has a 
> very clear value proposition around correctness, your results are as correct 
> as the input data provided. This does not change under support for DELETE 
> records. Flink is providing the most correct results possible based on the 
> resolution of the fields as generated by 3rd party systems. As Debezium and 
> other CDC libraries become more accurate, so will Flink. 
> 
> Seth 
> 
> On Fri, Jul 3, 2020 at 11:00 PM Leonard Xu  > wrote:
> Hi, Konstantin
> 
>> . Would we support a temporal join with a changelog stream with
>> event time semantics by ignoring DELETE messages or would it be completed
>> unsupported.
> 
> I don’t know the percentage of this feature in temporal scenarios.
> 
> Comparing to support the approximate event time join by ignoring DELETE 
> message or by extracting an approximate event time for DELET message,  I’m 
> not sure is this acceptable for user even if we have explained the limitation 
> of approximate event time join, I tend to do not support this feature, 
> because we can not ensure the semantic of event time and it may lead an 
> incorrect result for user in some scenarios. 
> 
> If the percentage is highly enough and most user cases can accept the 
> approximate  event time, I'm ok to support it  for usability although it 
> doesn’t 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-06 Thread Seth Wiesman
As an aside, I conceptually view temporal table joins to be semantically
equivalent to look up table joins. They are just two different ways of
consuming the same data.

Seth

On Mon, Jul 6, 2020 at 8:56 AM Seth Wiesman  wrote:

> Hi Leonard,
>
> Regarding DELETE operations I tend to have the opposite reaction. I spend
> a lot of time working with production Flink users across a large number of
> organizations and to say we don't support temporal tables that include
> DELETEs will be a blocker for adoption. Even organizations that claim to
> never delete rows still occasionally due so per  GDPR requests or other
> regulations.
>
> I actually do think users will understand the limitations. Flink today has
> a very clear value proposition around correctness, your results are as
> correct as the input data provided. This does not change under support for
> DELETE records. Flink is providing the most correct results possible based
> on the resolution of the fields as generated by 3rd party systems. As
> Debezium and other CDC libraries become more accurate, so will Flink.
>
> Seth
>
> On Fri, Jul 3, 2020 at 11:00 PM Leonard Xu  wrote:
>
>> Hi, Konstantin
>>
>> . Would we support a temporal join with a changelog stream with
>> event time semantics by ignoring DELETE messages or would it be completed
>> unsupported.
>>
>>
>> I don’t know the percentage of this feature in temporal scenarios.
>>
>> Comparing to support the approximate event time join by ignoring DELETE
>> message or by extracting an approximate event time for DELET message,  I’m
>> not sure is this acceptable for user even if we have explained the
>> limitation of approximate event time join, I tend to do not support this
>> feature, because we can not ensure the semantic of event time and it may
>> lead an incorrect result for user in some scenarios.
>>
>> If the percentage is highly enough and most user cases can accept the
>> approximate  event time, I'm ok to support it  for usability although it
>> doesn’t implements the event time semantic strictly.
>>
>> Cheers,
>> Leonard Xu
>>
>>
>>


Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-06 Thread Seth Wiesman
Hi Leonard,

Regarding DELETE operations I tend to have the opposite reaction. I spend a
lot of time working with production Flink users across a large number of
organizations and to say we don't support temporal tables that include
DELETEs will be a blocker for adoption. Even organizations that claim to
never delete rows still occasionally due so per  GDPR requests or other
regulations.

I actually do think users will understand the limitations. Flink today has
a very clear value proposition around correctness, your results are as
correct as the input data provided. This does not change under support for
DELETE records. Flink is providing the most correct results possible based
on the resolution of the fields as generated by 3rd party systems. As
Debezium and other CDC libraries become more accurate, so will Flink.

Seth

On Fri, Jul 3, 2020 at 11:00 PM Leonard Xu  wrote:

> Hi, Konstantin
>
> . Would we support a temporal join with a changelog stream with
> event time semantics by ignoring DELETE messages or would it be completed
> unsupported.
>
>
> I don’t know the percentage of this feature in temporal scenarios.
>
> Comparing to support the approximate event time join by ignoring DELETE
> message or by extracting an approximate event time for DELET message,  I’m
> not sure is this acceptable for user even if we have explained the
> limitation of approximate event time join, I tend to do not support this
> feature, because we can not ensure the semantic of event time and it may
> lead an incorrect result for user in some scenarios.
>
> If the percentage is highly enough and most user cases can accept the
> approximate  event time, I'm ok to support it  for usability although it
> doesn’t implements the event time semantic strictly.
>
> Cheers,
> Leonard Xu
>
>
>


Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-03 Thread Leonard Xu
Hi, Konstantin

> . Would we support a temporal join with a changelog stream with
> event time semantics by ignoring DELETE messages or would it be completed
> unsupported.

I don’t know the percentage of this feature in temporal scenarios.

Comparing to support the approximate event time join by ignoring DELETE message 
or by extracting an approximate event time for DELET message,  I’m not sure is 
this acceptable for user even if we have explained the limitation of 
approximate event time join, I tend to do not support this feature, because we 
can not ensure the semantic of event time and it may lead an incorrect result 
for user in some scenarios. 

If the percentage is highly enough and most user cases can accept the 
approximate  event time, I'm ok to support it  for usability although it 
doesn’t implements the event time semantic strictly. 

Cheers,
Leonard Xu   
  



Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-03 Thread Konstantin Knauf
Hi Leonard,

Thank you for the summary. I don't fully understand the implications of
(3). Would we support a temporal join with a changelog stream with
event time semantics by ignoring DELETE messages or would it be completed
unsupported. I mean something like the following sequence of statements:

CREATE TABLE currency_rates (
  currencyId BIGINT PRIMARY KEY,
  rate DECIMAL(10, 2)) WITH (
 'connector' = 'kafka',
 'format' = 'debezium-json'
)
*CREATE* TABLE transactions (
  currencyId BIGINT,
  transactionTime TIMESTAMP(3)) WITH (

)


SELECT
  ...FROM
  transactions AS t
  JOIN currency_rates FOR SYSTEM_TIME AS OF t.transactionTime AS r
  ON r.currency = t.currency

Cheers,

Konstantin

On Fri, Jul 3, 2020 at 4:52 PM Leonard Xu  wrote:

> Thanks Jingsong, Jark, Knauf, Seth for sharing your thoughts.
>
> Although we discussed many details about the concept, I think it’s worth
> to clarify the semantic from long term goals. Temporal table concept was
> first imported in SQL:2011, I made some investigation of Temporal Table
> work mechanism in traditional DBMS which implements it like SQL Server[1],
> PostgreSQL[2]
>
> In DBMS, Temporal Table is implemented as a pair of tables, *a current
> table and a history table*.The current table contains the current value
> of each row, the history table contains each previous value for each row.
> Each row contains a time range constructed by RowStartTime and RowEndTime
> to define the period validity of the row. The RowStartTime and RowEndTime
> is changed by DBMS when a DML operation happened, Given a simple temporal
> table in SQL Server to show how it works:
> CREATE TABLE dbo.currency (
> [currency] VARCHAR(10) NOT NULL PRIMARY KEY, [rate] INT, [RowStart]
> DATETIME2 GENERATED ALWAYS AS ROW START, [RowEnd] DATETIME2 GENERATED
> ALWAYS AS ROW END, PERIOD FOR SYSTEM_TIME (RowStart, RowEnd) ) WITH 
> (SYSTEM_VERSIONING
> = ON (HISTORY_TABLE = dbo.currency_History));
>
> 1> select * from currency; // *The initial test data, the RowEndTime is
> the max value of timestamp type* currency rate RowStart RowEnd --
> --- --
> -- Euro 114 2020-06-29 15:06:24.7459246
> -12-31 23:59:59.999 US Dollar 102 2020-06-29 15:06:24.7503288
> -12-31 23:59:59.999 1>* UPDATE dbo.currency SET [rate] = 118
> WHERE currency = 'Euro’*; //* UPDATE **Euro currency* 2> select * from
> *currency_History*; // *The history table increased a record that
> represents the validity period of record (Euro,114)* currency rate
> RowStart RowEnd -- ---
> --
> -- Euro 114 2020-06-29 15:06:24.7459246
> 2020-06-29 15:07:01.1245406 1> *DELETE FROM dbo.currency WHERE currency =
> 'Euro’;* //* DELETE **Euro currency* 1> select * from *currency_History*;
> currency rate RowStart RowEnd // *The history table also increased
> another record that represents the validity period of record (Euro, 118)*
> -- --- --
> -- Euro 114 2020-06-29 15:06:24.7459246
> 2020-06-29 15:07:01.1245406 Euro 118 2020-06-29 15:07:01.1245406 2020-06-29
> 15:07:11.2981995 1> select * from currency; currency rate RowStart RowEnd
> // *Current table only keep the latest value * -- ---
> --
> -- US Dollar 102 2020-06-29
> 15:06:24.7503288 -12-31 23:59:59.999
>
> The history table is very important for history version tracking, pleas
> note the *DELETE* operation also increase a record in history table and
> the record’s RowEndTime is the system time that the DELETE operation
> happened. In one word, temporal table use time range [RowStart, RowEnd) to
> mark period validity, store all versions’ records in history table for
> history tracking, use DBMS operation time to change the RowStart or
> RowEnd.
>
> Back to our Flink World, temporal table with event time attribute works
> well in data source that contains INSERT, UPDATE messages except DELETE 
> currently.
> Let us see what happened in DELETE message scenario(i.e. changelog
> source), both DBMS Temporal Table and other general table can capture data
> change by CDC tools and have same format, I used debezuim to capture a SQL
> server table changes:
>
> 1> select * from currency; currency rate RowStart RowEnd --
> --- --
> -- Euro 118 2020-06-29 15:07:01.1245406
> -12-31 23:59:59.999 US Dollar 102 2020-06-29 15:06:24.7503288
> -12-31 23:59:59.999 1>* DELETE FROM dbo.currency WHERE currency =
> 'Euro’; * //* DELETE **Euro currency* 1> select * from currency_History;
> currency rate RowStart RowEnd -- ---
> --
> -- Euro 118 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-03 Thread Leonard Xu
Thanks Jingsong, Jark, Knauf, Seth for sharing your thoughts.

Although we discussed many details about the concept, I think it’s worth to 
clarify the semantic from long term goals. Temporal table concept was first 
imported in SQL:2011, I made some investigation of Temporal Table work 
mechanism in traditional DBMS which implements it like SQL Server[1], 
PostgreSQL[2]
 
In DBMS, Temporal Table is implemented as a pair of tables, a current table and 
a history table.The current table contains the current value of each row, the 
history table contains each previous value for each row. Each row contains a 
time range constructed by RowStartTime and RowEndTime to define the period 
validity of the row. The RowStartTime and RowEndTime is changed by DBMS when a 
DML operation happened, Given a simple temporal table in SQL Server to show how 
it works:
CREATE TABLE dbo.currency (
[currency]  VARCHAR(10) NOT NULL PRIMARY KEY,
[rate] INT,
[RowStart] DATETIME2 GENERATED ALWAYS AS ROW START,
[RowEnd] DATETIME2 GENERATED ALWAYS AS ROW END,
PERIOD FOR SYSTEM_TIME (RowStart, RowEnd)
) WITH
(SYSTEM_VERSIONING = ON (HISTORY_TABLE = dbo.currency_History));

1> select * from currency;  
 // The initial  test data, the RowEndTime is the max value of 
timestamp type
currency   rateRowStart   RowEnd
-- --- -- 
--
Euro   1142020-06-29 15:06:24.7459246
-12-31 23:59:59.999
US Dollar   1022020-06-29 15:06:24.7503288
-12-31 23:59:59.999
1> UPDATE dbo.currency SET [rate] = 118 WHERE currency = 'Euro’;   // UPDATE 
Euro currency
2> select * from currency_History;  
// The history table increased a record that represents the 
validity period of record (Euro,114)
currency   rateRowStart   RowEnd
-- --- -- 
--
Euro   1142020-06-29 15:06:24.7459246
2020-06-29 15:07:01.1245406
1> DELETE FROM dbo.currency WHERE currency = 'Euro’;   // DELETE Euro 
currency
1> select * from currency_History;
currency   rateRowStart   RowEnd
 // The history table also increased another record that represents the 
validity period of record (Euro, 118)
-- --- -- 
--
Euro   1142020-06-29 15:06:24.7459246
2020-06-29 15:07:01.1245406
Euro   1182020-06-29 15:07:01.1245406
2020-06-29 15:07:11.2981995
1> select * from currency;
currency   rateRowStart   RowEnd
 // Current table only keep the latest value 
-- --- -- 
--
US Dollar   1022020-06-29 15:06:24.7503288
-12-31 23:59:59.999

The history table is very important for history version tracking, pleas note 
the DELETE operation also increase a record in history table and the record’s 
RowEndTime is the system time that the DELETE operation happened. In one word, 
temporal table use time range [RowStart, RowEnd) to mark period validity, store 
all versions’ records in history table for history tracking, use DBMS operation 
time to change the RowStart or RowEnd.

Back to our Flink World, temporal table with event time attribute works well in 
data source that contains INSERT, UPDATE messages except DELETE currently. Let 
us see what happened in DELETE message  scenario(i.e. changelog source), both 
DBMS Temporal Table and other general table can capture data change by CDC 
tools and have same format, I used debezuim to capture a SQL server table 
changes:

1> select * from currency;
currency   rateRowStart   RowEnd
-- --- -- 
--
Euro   1182020-06-29 15:07:01.1245406
-12-31 23:59:59.999
US Dollar  1022020-06-29 15:06:24.7503288 
-12-31 23:59:59.999
1> DELETE FROM dbo.currency WHERE currency = 'Euro’;   // DELETE 
Euro currency
1> select * from currency_History;
currency   rateRowStart   RowEnd
-- --- -- 
--
Euro   

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-02 Thread Seth Wiesman
It is clear there are a lot of edge cases with temporal tables that need to
be carefully thought out. If we go at this problem from the perspective of
what a majority of users need to accomplish in production, I believe there
is a simpler version of this problem we can solve that can be expanded in
the future.

The most important practical use case is denormalizing star schemas. A user
has the main data stream, their fact table, that needs to be processed.
Before applying specific business logic the dimension stream needs to be
joined with one or more dimension streams. The canonical example of this
being joining transactions with currency rates. I'm not saying all this to
be pedantic but to make the point that if we can solve this practical use
case in a way that may be extended in the future I believe that will
already be immensely useful for most users.

In this case, the data is most likely coming from CDC or something that
approximates it and contains a clearly defined event time column.

For this common use case the syntax would only need to support:

   - Single event time column
   - Joining with externally defined Upsert streams. In the first version,
   a stream could only be used as a temporal table if the join was the first
   operation after reading from an external source. We could disallow using
   streams post flink aggregation as temporal tables in the beginning until
   there is a larger consensus of what timestamp to use.

Seth

On Thu, Jul 2, 2020 at 11:49 AM Konstantin Knauf  wrote:

> Hi everyone,
>
> well, this got complicated :) Let me add my thoughts:
>
> * Temporal Table Joins are already quite hard to understand for many users.
> If need be, we should trade off for simplicity.
>
> * The important case is the *event time *temporal join. In my understanding
> processing time temporal joins are comparably easy, no history tracking is
> needed, etc.
>
> * It seems that for regular upsert streams with an event time attribute,
> everyone agrees that it works. There are olny questions about multiple
> event time attributes, which we could in my opinion postpone for future
> work.
>
> * For changelog streams, which specify an event time column explicitly, it
> should be possible to use it for event time temporal tables. I understand
> that deletion can not be handled properly, but we could - for example -
> handle this exactly like an upsert stream, i.e. ignore deletions. This is a
> limitation, but it is at least easy to understand and acceptable for many
> use cases, I believe. Alternatively, one could also use the "ts_ms" for
> deletion, which would always be larger than the event time.
>
> CREATE TABLE currency_rates (
>   id BIGINT,
>   name STRING,
>   rate DECIMAL(10, 5),  time TIMESTAMP(3),   WATERMARK FOR time AS ...)
> WITH (
>  'connector' = 'kafka',
>  ...
>  'format' = 'debezium-json')
>
>
> * For changelog streams without an event time attribute (the more common
> case?), it would be great if we can support temporal table joins based on
> "ts_ms" (in the debezium case). One option could be to "simply" extract
> "ts_ms" and make it possible to use it as an event time column. Then we
> would again be in the above case. Thinking about it, this could even be
> addressed in [1], which is also planned for Flink 1.12 as far as I know. *
> This could look something like:
>
> CREATE TABLE topic_products (
>   id BIGINT,
>   name STRING,
>   description STRING,
>   weight DECIMAL(10, 2),   time TIMESTAMP(3)) WITH (
>  'connector' = 'kafka',
>  ...
>  'format' = 'debezium-json'
>  'timestamp' = 'time' )
>
>
> I hope I roughly understood your concerns and made sense in my comments.
> Looking forward to what you think.
>
> Cheers,
>
> Konstantin
>
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-107%3A+Reading+table+columns+from+different+parts+of+source+records
> *  In this case, the DELETE statement could theoretically actually be
> handled properly, because the "ts_ms" is used throughout.
>
> On Sun, Jun 28, 2020 at 8:05 AM Jingsong Li 
> wrote:
>
> > Thanks for your discussion.
> >
> > Looks like the problem is supporting the versioned temporal table for the
> > changelog source.
> >
> > I want to share more of my thoughts:
> >
> > When I think about changelog sources, I treat it as a view like: "CREATE
> > VIEW changelog_table AS SELECT ... FROM origin_table GROUP BY ..." (Some
> > queries produce changelog records).
> >
> > Does this view support window aggregation? No...
> > Does this view support versioned temporal tables? No...
> >
> > Because both window aggregation and versioned temporal tables require a
> > time attribute.
> >
> > So can we give this view a new time attribute?
> > 1. No, keep it not supported.
> > 2. Using processing time.
> > 3. there is an operation time in this view, something like processing
> time
> > when modifying the origin table. Treat this operation time as rowtime.
> > 4. Introduce a new time attribute concept: operation time. 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-07-02 Thread Konstantin Knauf
Hi everyone,

well, this got complicated :) Let me add my thoughts:

* Temporal Table Joins are already quite hard to understand for many users.
If need be, we should trade off for simplicity.

* The important case is the *event time *temporal join. In my understanding
processing time temporal joins are comparably easy, no history tracking is
needed, etc.

* It seems that for regular upsert streams with an event time attribute,
everyone agrees that it works. There are olny questions about multiple
event time attributes, which we could in my opinion postpone for future
work.

* For changelog streams, which specify an event time column explicitly, it
should be possible to use it for event time temporal tables. I understand
that deletion can not be handled properly, but we could - for example -
handle this exactly like an upsert stream, i.e. ignore deletions. This is a
limitation, but it is at least easy to understand and acceptable for many
use cases, I believe. Alternatively, one could also use the "ts_ms" for
deletion, which would always be larger than the event time.

CREATE TABLE currency_rates (
  id BIGINT,
  name STRING,
  rate DECIMAL(10, 5),  time TIMESTAMP(3),   WATERMARK FOR time AS ...) WITH (
 'connector' = 'kafka',
 ...
 'format' = 'debezium-json')


* For changelog streams without an event time attribute (the more common
case?), it would be great if we can support temporal table joins based on
"ts_ms" (in the debezium case). One option could be to "simply" extract
"ts_ms" and make it possible to use it as an event time column. Then we
would again be in the above case. Thinking about it, this could even be
addressed in [1], which is also planned for Flink 1.12 as far as I know. *
This could look something like:

CREATE TABLE topic_products (
  id BIGINT,
  name STRING,
  description STRING,
  weight DECIMAL(10, 2),   time TIMESTAMP(3)) WITH (
 'connector' = 'kafka',
 ...
 'format' = 'debezium-json'
 'timestamp' = 'time' )


I hope I roughly understood your concerns and made sense in my comments.
Looking forward to what you think.

Cheers,

Konstantin


[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-107%3A+Reading+table+columns+from+different+parts+of+source+records
*  In this case, the DELETE statement could theoretically actually be
handled properly, because the "ts_ms" is used throughout.

On Sun, Jun 28, 2020 at 8:05 AM Jingsong Li  wrote:

> Thanks for your discussion.
>
> Looks like the problem is supporting the versioned temporal table for the
> changelog source.
>
> I want to share more of my thoughts:
>
> When I think about changelog sources, I treat it as a view like: "CREATE
> VIEW changelog_table AS SELECT ... FROM origin_table GROUP BY ..." (Some
> queries produce changelog records).
>
> Does this view support window aggregation? No...
> Does this view support versioned temporal tables? No...
>
> Because both window aggregation and versioned temporal tables require a
> time attribute.
>
> So can we give this view a new time attribute?
> 1. No, keep it not supported.
> 2. Using processing time.
> 3. there is an operation time in this view, something like processing time
> when modifying the origin table. Treat this operation time as rowtime.
> 4. Introduce a new time attribute concept: operation time. Assuming it
> monotonically increases, no watermark.
>
> NOTE: For the versioned temporal table, there is a time-relation between
> these two tables. This time attribute must be something user perceived.
>
> I am slightly +1 for #1 and #2.
> For #1: If users really want to support the versioned temporal table for
> the changelog source. They can change the definition. And make the
> changelog source as a regular table, then they have an operation time field
> in the table schema, they can use this field as a rowtime field.
> For #2: This versioned temporal table is joined using the processing-time
> way, it means we assume records come in a monotonically way, I think it is
> good to match changelog concept.
>
> -1 for #3 and #4.
> It can work, but l think it is hard to understand what is the rowtime
> attribute after "changing" the table.
> And I don't think it is not worth creating another concept for users.
>
> Best,
> Jingsong Lee
>
> On Thu, Jun 25, 2020 at 10:30 PM Jark Wu  wrote:
>
> > Hi all,
> >
> > Thanks Leonard for summarizing our discussion. I want to share more of my
> > thoughts:
> >
> > * rowtime is a column in the its schema, so the rowtime of DELETE event
> is
> > the value of the previous image.
> > * operation time is the time when the DML statements happen in databases,
> > so the operation time of DELETE events is the time when it happens.
> > * rowtime can't be used as operation time for history tracking
> > * operation time can't be used as rowtime (can't apply window on the
> > operation time)
> > * rowtime and operation time are orthogonal concepts and used in
> different
> > scenarios.
> > * operation time implicitly means it is monotonically 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-28 Thread Jingsong Li
Thanks for your discussion.

Looks like the problem is supporting the versioned temporal table for the
changelog source.

I want to share more of my thoughts:

When I think about changelog sources, I treat it as a view like: "CREATE
VIEW changelog_table AS SELECT ... FROM origin_table GROUP BY ..." (Some
queries produce changelog records).

Does this view support window aggregation? No...
Does this view support versioned temporal tables? No...

Because both window aggregation and versioned temporal tables require a
time attribute.

So can we give this view a new time attribute?
1. No, keep it not supported.
2. Using processing time.
3. there is an operation time in this view, something like processing time
when modifying the origin table. Treat this operation time as rowtime.
4. Introduce a new time attribute concept: operation time. Assuming it
monotonically increases, no watermark.

NOTE: For the versioned temporal table, there is a time-relation between
these two tables. This time attribute must be something user perceived.

I am slightly +1 for #1 and #2.
For #1: If users really want to support the versioned temporal table for
the changelog source. They can change the definition. And make the
changelog source as a regular table, then they have an operation time field
in the table schema, they can use this field as a rowtime field.
For #2: This versioned temporal table is joined using the processing-time
way, it means we assume records come in a monotonically way, I think it is
good to match changelog concept.

-1 for #3 and #4.
It can work, but l think it is hard to understand what is the rowtime
attribute after "changing" the table.
And I don't think it is not worth creating another concept for users.

Best,
Jingsong Lee

On Thu, Jun 25, 2020 at 10:30 PM Jark Wu  wrote:

> Hi all,
>
> Thanks Leonard for summarizing our discussion. I want to share more of my
> thoughts:
>
> * rowtime is a column in the its schema, so the rowtime of DELETE event is
> the value of the previous image.
> * operation time is the time when the DML statements happen in databases,
> so the operation time of DELETE events is the time when it happens.
> * rowtime can't be used as operation time for history tracking
> * operation time can't be used as rowtime (can't apply window on the
> operation time)
> * rowtime and operation time are orthogonal concepts and used in different
> scenarios.
> * operation time implicitly means it is monotonically increasing, we don't
> need watermark syntax to specify the out of boundness for it.
>
> ==
> So, conclusion from my side so far:
>
> * watermark/rowtime + primary key + changelog source != versioned temporal
> table
> * operation time + primary key + changelog source == versioned temporal
> table
> * We may need something like 'PERIOD FOR SYSTEM_TIME(op_ts)' to define the
> operation time
>
> ==
> However, there is still a pending question I don't have answer:
>
> Assuming you are doing a MIN aggregate on the operation time, that doesn't
> work because the DELETE/UPDATE_BEFORE doesn't hold
> the previous value of operation time and thus can't retract.
>
> The operation time in fact should be metadata information (just like
> RowKind) which shouldn't be in the shema, and can't be accessed in queries.
> But the PERIOD FOR SYSTEM_TIME syntax is in the schema part and should
> refer to a field in the schema...
>
> ==
>
> Anyway, let's focus on the operation_time vs rowtime problem first. Let me
> know what's your thought!
>
> Best,
> Jark
>
> On Wed, 24 Jun 2020 at 23:49, Leonard Xu  wrote:
>
> > Hi, kurt, Fabian
> >
> > After an offline discussion with Jark, We think that the  'PERIOD FOR
> > SYSTEM_TIME(operation_time)' statement might be needed now. Changelog
> table
> > is superset of insert-only table, use PRIMARY KEY and rowtime may work
> well
> > in insert-only or upsert source but has some problem in changelog table.
> >
> > 'PERIOD FOR SYSTEM_TIME(operation_time)' in a temporal table
> > defines/maintains  the valid time of each row, the rowtime can not play
> the
> > history tracking function well.
> >
> > *# 1.*operation time (version time) *vs* rowtime (watermark)
> >
> > I will take an example to explain. The following changelog records came
> > from database table using debezium tool:
> > { "before":  null
> >   "after":{"currency": "Euro", "rate": 118, "gmt_modified":
> > "12:00:01"},
> >   "op":   "c",  //INSERT
> >   "ts_ms": 1592971201000 // 2020-06-24 12:00:02
> > }
> > { "before": {"currency": "Euro", "rate": 114, "gmt_modified":
> "12:00:05"},
> >   "after":{"currency": "Euro", "rate": 118, "gmt_modified":
> > "12:00:05"},
> >   "op":   "u",  //UPDATE
> >   "ts_ms": 1592971206000 // 2020-06-24 12:00:06
> > }
> >
> > { "before": {"currency": "Euro", 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-25 Thread Jark Wu
Hi all,

Thanks Leonard for summarizing our discussion. I want to share more of my
thoughts:

* rowtime is a column in the its schema, so the rowtime of DELETE event is
the value of the previous image.
* operation time is the time when the DML statements happen in databases,
so the operation time of DELETE events is the time when it happens.
* rowtime can't be used as operation time for history tracking
* operation time can't be used as rowtime (can't apply window on the
operation time)
* rowtime and operation time are orthogonal concepts and used in different
scenarios.
* operation time implicitly means it is monotonically increasing, we don't
need watermark syntax to specify the out of boundness for it.

==
So, conclusion from my side so far:

* watermark/rowtime + primary key + changelog source != versioned temporal
table
* operation time + primary key + changelog source == versioned temporal
table
* We may need something like 'PERIOD FOR SYSTEM_TIME(op_ts)' to define the
operation time

==
However, there is still a pending question I don't have answer:

Assuming you are doing a MIN aggregate on the operation time, that doesn't
work because the DELETE/UPDATE_BEFORE doesn't hold
the previous value of operation time and thus can't retract.

The operation time in fact should be metadata information (just like
RowKind) which shouldn't be in the shema, and can't be accessed in queries.
But the PERIOD FOR SYSTEM_TIME syntax is in the schema part and should
refer to a field in the schema...

==

Anyway, let's focus on the operation_time vs rowtime problem first. Let me
know what's your thought!

Best,
Jark

On Wed, 24 Jun 2020 at 23:49, Leonard Xu  wrote:

> Hi, kurt, Fabian
>
> After an offline discussion with Jark, We think that the  'PERIOD FOR
> SYSTEM_TIME(operation_time)' statement might be needed now. Changelog table
> is superset of insert-only table, use PRIMARY KEY and rowtime may work well
> in insert-only or upsert source but has some problem in changelog table.
>
> 'PERIOD FOR SYSTEM_TIME(operation_time)' in a temporal table
> defines/maintains  the valid time of each row, the rowtime can not play the
> history tracking function well.
>
> *# 1.*operation time (version time) *vs* rowtime (watermark)
>
> I will take an example to explain. The following changelog records came
> from database table using debezium tool:
> { "before":  null
>   "after":{"currency": "Euro", "rate": 118, "gmt_modified":
> "12:00:01"},
>   "op":   "c",  //INSERT
>   "ts_ms": 1592971201000 // 2020-06-24 12:00:02
> }
> { "before": {"currency": "Euro", "rate": 114, "gmt_modified": "12:00:05"},
>   "after":{"currency": "Euro", "rate": 118, "gmt_modified":
> "12:00:05"},
>   "op":   "u",  //UPDATE
>   "ts_ms": 1592971206000 // 2020-06-24 12:00:06
> }
>
> { "before": {"currency": "Euro", "rate": 118, "gmt_modified": "12:00:05"},
>   "after": null,
>   "op":"d",  //DELETE
>   "ts_ms":  159311000  // 2020-06-24 20:00:11
> }
>
> The rowtime should be the "gmt_modified" field that belongs to the
> original record,the "ts_ms" is the the operation time when the DML
> statement happen in the DB. For DELETE changelog record, its "gmt_modified"
> field (12:00:05) can not reflect the real operation time (20:00:11).
>
> In temporal join case, we should maintain the valid time of each row. For
> a DELETE event, we should use the operation time of DELETE as the “end
> time” of the row. That says, the record {"currency": "Euro", "rate": 118}
> is not exist anymore after “20:00:11”, not “12:00:05”.
>
> we would not access the record {"currency": "Euro", "rate": 118,
> "gmt_modified": "12:00:05"} when rowtime is bigger than (12:00:05) if we
> use rowtime to track the history version, because the DELETE changelog
> record also has rowtime (12:00:05) and will clear the record in state. In
> fact, the expected result is that the record expires until (20:00:11) when
> the record is deleted rather than the last update time(20:00:11) in
> materialized state.
>
> From this case, I found rowtime and operation time should be orthogonal in
> temporal table scenario. The operation time should be strictly
> monotonically increasing  (no out of order) and only be used for tracking a
> history version of a changelog table, every history version of changelog
> table equals a database table snapshot due to the stream-table duality.
>
> *# 2.*The semantic of rowtime and watermark on changelog
>
> The rowtime and watermark can also be defined on a changelog table just
> like other source backed queue, Flink supports cascaded window aggregation
> (with ) in SQL like:
> SELECT
>  TUMBLE_ROWTIME(rowtime, INTERVAL '60' SECOND),
>  MAX(rate) AS rate
> FROM (
>SELECT
>   MAX(rate) AS rate,
>   

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-24 Thread Leonard Xu
Hi, kurt, Fabian

After an offline discussion with Jark, We think that the  'PERIOD FOR 
SYSTEM_TIME(operation_time)' statement might be needed now. Changelog table is 
superset of insert-only table, use PRIMARY KEY and rowtime may work well in 
insert-only or upsert source but has some problem in changelog table.

'PERIOD FOR SYSTEM_TIME(operation_time)' in a temporal table defines/maintains  
the valid time of each row, the rowtime can not play the history tracking 
function well.

# 1.operation time (version time) vs rowtime (watermark)

I will take an example to explain. The following changelog records came from 
database table using debezium tool:
{ "before":  null
  "after":{"currency": "Euro", "rate": 118, "gmt_modified": "12:00:01"},
  "op":   "c",  //INSERT
  "ts_ms": 1592971201000 // 2020-06-24 12:00:02
}
{ "before": {"currency": "Euro", "rate": 114, "gmt_modified": "12:00:05"},
  "after":{"currency": "Euro", "rate": 118, "gmt_modified": "12:00:05"},
  "op":   "u",  //UPDATE
  "ts_ms": 1592971206000 // 2020-06-24 12:00:06
}

{ "before": {"currency": "Euro", "rate": 118, "gmt_modified": "12:00:05"},
  "after": null,
  "op":"d",  //DELETE
  "ts_ms":  159311000  // 2020-06-24 20:00:11
}

The rowtime should be the "gmt_modified" field that belongs to the original 
record,the "ts_ms" is the the operation time when the DML statement happen in 
the DB. For DELETE changelog record, its "gmt_modified" field (12:00:05) can 
not reflect the real operation time (20:00:11).

In temporal join case, we should maintain the valid time of each row. For a 
DELETE event, we should use the operation time of DELETE as the “end time” of 
the row. That says, the record {"currency": "Euro", "rate": 118} is not exist 
anymore after “20:00:11”, not “12:00:05”.

we would not access the record {"currency": "Euro", "rate": 118, 
"gmt_modified": "12:00:05"} when rowtime is bigger than (12:00:05) if we use 
rowtime to track the history version, because the DELETE changelog record also 
has rowtime (12:00:05) and will clear the record in state. In fact, the 
expected result is that the record expires until (20:00:11) when the record is 
deleted rather than the last update time(20:00:11) in materialized state.

From this case, I found rowtime and operation time should be orthogonal in 
temporal table scenario. The operation time should be strictly monotonically 
increasing  (no out of order) and only be used for tracking a history version 
of a changelog table, every history version of changelog table equals a 
database table snapshot due to the stream-table duality. 

# 2.The semantic of rowtime and watermark on changelog 

The rowtime and watermark can also be defined on a changelog table just like 
other source backed queue, Flink supports cascaded window aggregation (with ) 
in SQL like:
SELECT
 TUMBLE_ROWTIME(rowtime, INTERVAL '60' SECOND),
 MAX(rate) AS rate
FROM (
   SELECT
  MAX(rate) AS rate,
  TUMBLE_ROWTIME(rowtime, INTERVAL '5' SECOND) AS `rowtime`
   FROM currency
GROUP BY TUMBLE(rowtime, INTERVAL '5' SECOND)
)
GROUP BY TUMBLE(rowtime, INTERVAL '60' SECOND

We can think of the output of the first window aggregation as a changelog 
source of the second window aggregation. There are INSERT/UPDATE/DELETE 
messages and also watermarks in the changelog stream. And the rowtime in the 
changelog stream is the `TUMBLE_ROWTIME` value (just like the `gmt_modified` 
column in DB). 

#  summary
we should use ‘PERIOD FOR SYSTEM_TIME(operation_time) syntax to track history 
version by operation time rather than rowtime in temporal table scenario.
we also support define a rowtime(watermark) on changelog table, but the rowtime 
will not be used to track the history of changelog stream.  


WDYT? please correct me if I am wrong.


Best,
Leonard




> 在 2020年6月24日,11:31,Leonard Xu  写道:
> 
> Hi, everyone
> 
> Thanks Fabian,Kurt for making the multiple version(event time) clear, I also 
> like the 'PERIOD FOR SYSTEM' syntax which supported in SQL standard. I think 
> we can add some explanation of the multiple version support in the future 
> section of FLIP.
> 
> For the PRIMARY KEY semantic, I agree with Jark's point that the semantic 
> should unify both on changelog source and insert-only source.
> 
> Currently, Flink supports PRIMARY KEY after FLIP-87, Flink uses PRIMARY KEY 
> NOT ENFORCED because Flink does not own the data like other DBMS therefore 
> Flink won't validate/enforce the key integrity and only trusts the external 
> systems. It is  expected user and external system/application should make 
> sure no deduplicated records happened when using NOT ENFORCED.
> 
> (a) For PRIMARY KEY NOT ENFORCED semantic on changelog source:
> It means the materialized changelogs (INSERT/UPDATE/DELETE) should be unique 
> on the primary key constraints.Flink assumes messages are in order on the 
> primary key. Flink will use the PRIMARY KEY for some 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-23 Thread Leonard Xu
Hi, everyone

Thanks Fabian,Kurt for making the multiple version(event time) clear, I also 
like the 'PERIOD FOR SYSTEM' syntax which supported in SQL standard. I think we 
can add some explanation of the multiple version support in the future section 
of FLIP.

For the PRIMARY KEY semantic, I agree with Jark's point that the semantic 
should unify both on changelog source and insert-only source.

Currently, Flink supports PRIMARY KEY after FLIP-87, Flink uses PRIMARY KEY NOT 
ENFORCED because Flink does not own the data like other DBMS therefore Flink 
won't validate/enforce the key integrity and only trusts the external systems. 
It is  expected user and external system/application should make sure no 
deduplicated records happened when using NOT ENFORCED.

(a) For PRIMARY KEY NOT ENFORCED semantic on changelog source:
It means the materialized changelogs (INSERT/UPDATE/DELETE) should be unique on 
the primary key constraints.Flink assumes messages are in order on the primary 
key. Flink will use the PRIMARY KEY for some optimization, e.g. use the PRIMARY 
KEY to update the materialized state by key in temporal join operator. 
 
(b) For PRIMARY KEY NOT ENFORCED semantic on insert-only source:
It means records should be unique on the primary key constraints. If there are 
INSERT records with duplicate primary key columns, the result of SQL query 
might be nondeterministic because it broken the PRIMARY KEY constraints.

Cheers,
Leonard


> 在 2020年6月23日,23:35,Fabian Hueske  写道:
> 
> Thanks Kurt,
> 
> Yes, you are right.
> The `PERIOD FOR SYSTEM_TIME` that you linked before corresponds to the
> VERSION clause that I used and would explicitly define the versioning of a
> table.
> I didn't know that the `PERIOD FOR SYSTEM_TIME` cause is already defined by
> the SQL standard.
> I think we would need a slightly different syntax though because (so far)
> the validity of a row is determined by its own timestamp and the timestamp
> of the next row.
> 
> Adding a clause later solves the ambiguity issue for tables with multiple
> event-time attributes.
> However, I'd feel more comfortable having such a cause and an explicit
> definition of the temporal property from the beginning.
> I guess this is a matter of personal preference so I'll go with the
> majority if we decide that every table that has a primary key and an
> event-time attribute should be usable in an event-time temporal table join.
> 
> Thanks, Fabian
> 
> 
> Am Di., 23. Juni 2020 um 16:58 Uhr schrieb Kurt Young :
> 
>> Hi Fabian,
>> 
>> I agree with you that implicitly letting event time to be the version of
>> the table will
>> work in most cases, but not for all. That's the reason I mentioned `PERIOD
>> FOR` [1]
>> syntax in my first email, which is already in sql standard to represent the
>> validity of
>> each row in the table.
>> 
>> If the event time can't be used, or multiple event time are defined, we
>> could still add
>> this syntax in the future.
>> 
>> What do you think?
>> 
>> [1]
>> 
>> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables?view=sql-server-ver15
>> Best,
>> Kurt
>> 
>> 
>> On Tue, Jun 23, 2020 at 9:12 PM Fabian Hueske  wrote:
>> 
>>> Hi everyone,
>>> 
>>> Every table with a primary key and an event-time attribute provides what
>> is
>>> needed for an event-time temporal table join.
>>> I agree that, from a technical point of view, the TEMPORAL keyword is not
>>> required.
>>> 
>>> I'm more sceptical about implicitly deriving the versioning information
>> of
>>> a (temporal) table as the table's only event-time attribute.
>>> In the query
>>> 
>>> SELECT *
>>> FROM orders o, rates r FOR SYSTEM_TIME AS OF o.ordertime
>>> WHERE o.currency = r.currency
>>> 
>>> the syntax of the temporal table join does not explicitly reference the
>>> version of the temporal rates table.
>>> Hence, the system needs a way to derive the version of temporal table.
>>> 
>>> Implicitly using the (only) event-time attribute of a temporal table
>> (rates
>>> in the example above) to identify the right version works in most cases,
>>> but probably not in all.
>>> * What if a table has more than one event-time attribute? (TableSchema is
>>> designed to support multiple watermarks; queries with interval joins
>>> produce tables with multiple event-time attributes, ...)
>>> * What if the table does not have an event-time attribute in its schema
>> but
>>> the version should only be provided as meta data?
>>> 
>>> We could add a clause to define the version of a table, such as:
>>> 
>>> CREATE TABLE rates (
>>>   currency CHAR(3) NOT NULL PRIMARY KEY,
>>>   rate DOUBLE,
>>>   rowtime TIMESTAMP,
>>>   WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE),
>>> VERSION (rowtime)
>>> WITH (...);
>>> 
>>> The presence of a the VERSION clause (or whatever syntax) would
>> explicitly
>>> define the version of a (temporal) table.
>>> It would also render the need for the TEMPORAL keyword superfluous
>> because
>>> there would be 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-23 Thread Fabian Hueske
Thanks Kurt,

Yes, you are right.
The `PERIOD FOR SYSTEM_TIME` that you linked before corresponds to the
VERSION clause that I used and would explicitly define the versioning of a
table.
I didn't know that the `PERIOD FOR SYSTEM_TIME` cause is already defined by
the SQL standard.
I think we would need a slightly different syntax though because (so far)
the validity of a row is determined by its own timestamp and the timestamp
of the next row.

Adding a clause later solves the ambiguity issue for tables with multiple
event-time attributes.
However, I'd feel more comfortable having such a cause and an explicit
definition of the temporal property from the beginning.
I guess this is a matter of personal preference so I'll go with the
majority if we decide that every table that has a primary key and an
event-time attribute should be usable in an event-time temporal table join.

Thanks, Fabian


Am Di., 23. Juni 2020 um 16:58 Uhr schrieb Kurt Young :

> Hi Fabian,
>
> I agree with you that implicitly letting event time to be the version of
> the table will
> work in most cases, but not for all. That's the reason I mentioned `PERIOD
> FOR` [1]
> syntax in my first email, which is already in sql standard to represent the
> validity of
> each row in the table.
>
> If the event time can't be used, or multiple event time are defined, we
> could still add
> this syntax in the future.
>
> What do you think?
>
> [1]
>
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables?view=sql-server-ver15
> Best,
> Kurt
>
>
> On Tue, Jun 23, 2020 at 9:12 PM Fabian Hueske  wrote:
>
> > Hi everyone,
> >
> > Every table with a primary key and an event-time attribute provides what
> is
> > needed for an event-time temporal table join.
> > I agree that, from a technical point of view, the TEMPORAL keyword is not
> > required.
> >
> > I'm more sceptical about implicitly deriving the versioning information
> of
> > a (temporal) table as the table's only event-time attribute.
> > In the query
> >
> > SELECT *
> > FROM orders o, rates r FOR SYSTEM_TIME AS OF o.ordertime
> > WHERE o.currency = r.currency
> >
> > the syntax of the temporal table join does not explicitly reference the
> > version of the temporal rates table.
> > Hence, the system needs a way to derive the version of temporal table.
> >
> > Implicitly using the (only) event-time attribute of a temporal table
> (rates
> > in the example above) to identify the right version works in most cases,
> > but probably not in all.
> > * What if a table has more than one event-time attribute? (TableSchema is
> > designed to support multiple watermarks; queries with interval joins
> > produce tables with multiple event-time attributes, ...)
> > * What if the table does not have an event-time attribute in its schema
> but
> > the version should only be provided as meta data?
> >
> > We could add a clause to define the version of a table, such as:
> >
> > CREATE TABLE rates (
> >currency CHAR(3) NOT NULL PRIMARY KEY,
> >rate DOUBLE,
> >rowtime TIMESTAMP,
> >WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE),
> > VERSION (rowtime)
> > WITH (...);
> >
> > The presence of a the VERSION clause (or whatever syntax) would
> explicitly
> > define the version of a (temporal) table.
> > It would also render the need for the TEMPORAL keyword superfluous
> because
> > there would be another indicator that a table can be used in a temporal
> > table join.
> >
> > I'm OK with not adding the TEMPORAL keyword, but I recommend that we
> think
> > again about the proposed implicit definition of a table's version and how
> > it might limit use in the future.
> >
> > Cheers,
> > Fabian
> >
> > Am Mo., 22. Juni 2020 um 16:14 Uhr schrieb Jark Wu :
> >
> > > I'm also +1 for not adding the TEMPORAL keyword.
> > >
> > > +1 to make the PRIMARY KEY semantic clear for sources.
> > > From my point of view:
> > >
> > > 1) PRIMARY KEY on changelog souruce:
> > > It means that when the changelogs (INSERT/UPDATE/DELETE) are
> > materialized,
> > > the materialized table should be unique on the primary key columns.
> > > Flink assumes messages are in order on the primary key. Flink doesn't
> > > validate/enforces the key integrity, but simply trust it (thus NOT
> > > ENFORCED).
> > > Flink will use the PRIMARY KEY for some optimization, e.g. use the
> > PRIMARY
> > > KEY to update the materilized state by key in temporal join operator.
> > >
> > > 2) PRIMARY KEY on insert-only source:
> > > I prefer to have the same semantic to the batch source and changelog
> > > source, that it implies that records are not duplicate on the primary
> > key.
> > > Flink just simply trust the primary key constraint, and doesn't valid
> it.
> > > If there is duplicate primary keys with INSERT changeflag, then result
> of
> > > Flink query might be wrong.
> > >
> > > If this is a TEMPORAL TABLE FUNCTION scenario, that source emits
> > duplicate
> > > primary keys with INSERT changeflag, when we 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-23 Thread Kurt Young
Hi Fabian,

I agree with you that implicitly letting event time to be the version of
the table will
work in most cases, but not for all. That's the reason I mentioned `PERIOD
FOR` [1]
syntax in my first email, which is already in sql standard to represent the
validity of
each row in the table.

If the event time can't be used, or multiple event time are defined, we
could still add
this syntax in the future.

What do you think?

[1]
https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables?view=sql-server-ver15
Best,
Kurt


On Tue, Jun 23, 2020 at 9:12 PM Fabian Hueske  wrote:

> Hi everyone,
>
> Every table with a primary key and an event-time attribute provides what is
> needed for an event-time temporal table join.
> I agree that, from a technical point of view, the TEMPORAL keyword is not
> required.
>
> I'm more sceptical about implicitly deriving the versioning information of
> a (temporal) table as the table's only event-time attribute.
> In the query
>
> SELECT *
> FROM orders o, rates r FOR SYSTEM_TIME AS OF o.ordertime
> WHERE o.currency = r.currency
>
> the syntax of the temporal table join does not explicitly reference the
> version of the temporal rates table.
> Hence, the system needs a way to derive the version of temporal table.
>
> Implicitly using the (only) event-time attribute of a temporal table (rates
> in the example above) to identify the right version works in most cases,
> but probably not in all.
> * What if a table has more than one event-time attribute? (TableSchema is
> designed to support multiple watermarks; queries with interval joins
> produce tables with multiple event-time attributes, ...)
> * What if the table does not have an event-time attribute in its schema but
> the version should only be provided as meta data?
>
> We could add a clause to define the version of a table, such as:
>
> CREATE TABLE rates (
>currency CHAR(3) NOT NULL PRIMARY KEY,
>rate DOUBLE,
>rowtime TIMESTAMP,
>WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE),
> VERSION (rowtime)
> WITH (...);
>
> The presence of a the VERSION clause (or whatever syntax) would explicitly
> define the version of a (temporal) table.
> It would also render the need for the TEMPORAL keyword superfluous because
> there would be another indicator that a table can be used in a temporal
> table join.
>
> I'm OK with not adding the TEMPORAL keyword, but I recommend that we think
> again about the proposed implicit definition of a table's version and how
> it might limit use in the future.
>
> Cheers,
> Fabian
>
> Am Mo., 22. Juni 2020 um 16:14 Uhr schrieb Jark Wu :
>
> > I'm also +1 for not adding the TEMPORAL keyword.
> >
> > +1 to make the PRIMARY KEY semantic clear for sources.
> > From my point of view:
> >
> > 1) PRIMARY KEY on changelog souruce:
> > It means that when the changelogs (INSERT/UPDATE/DELETE) are
> materialized,
> > the materialized table should be unique on the primary key columns.
> > Flink assumes messages are in order on the primary key. Flink doesn't
> > validate/enforces the key integrity, but simply trust it (thus NOT
> > ENFORCED).
> > Flink will use the PRIMARY KEY for some optimization, e.g. use the
> PRIMARY
> > KEY to update the materilized state by key in temporal join operator.
> >
> > 2) PRIMARY KEY on insert-only source:
> > I prefer to have the same semantic to the batch source and changelog
> > source, that it implies that records are not duplicate on the primary
> key.
> > Flink just simply trust the primary key constraint, and doesn't valid it.
> > If there is duplicate primary keys with INSERT changeflag, then result of
> > Flink query might be wrong.
> >
> > If this is a TEMPORAL TABLE FUNCTION scenario, that source emits
> duplicate
> > primary keys with INSERT changeflag, when we migrate this case to
> temporal
> > table DDL,
> > I think this source should emit INSERT/UPDATE (UPSERT) messages instead
> of
> > INSERT-only messages,  e.g. a Kafka compacted topic source?
> >
> > Best,
> > Jark
> >
> >
> > On Mon, 22 Jun 2020 at 17:04, Konstantin Knauf 
> wrote:
> >
> > > Hi everyone,
> > >
> > > I also agree with Leonard/Kurt's proposal for CREATE TEMPORAL TABLE.
> > >
> > > Best,
> > >
> > > Konstantin
> > >
> > > On Mon, Jun 22, 2020 at 10:53 AM Kurt Young  wrote:
> > >
> > > > I agree with Timo, semantic about primary key needs more thought and
> > > > discussion, especially after FLIP-95 and FLIP-105.
> > > >
> > > > Best,
> > > > Kurt
> > > >
> > > >
> > > > On Mon, Jun 22, 2020 at 4:45 PM Timo Walther 
> > wrote:
> > > >
> > > > > Hi Leonard,
> > > > >
> > > > > thanks for the summary.
> > > > >
> > > > > After reading all of the previous arguments and working on
> FLIP-95. I
> > > > > would also lean towards the conclusion of not adding the TEMPORAL
> > > > keyword.
> > > > >
> > > > > After FLIP-95, what we considered as a CREATE TEMPORAL TABLE can be
> > > > > represented as a CREATE TABLE with PRIMARY KEY and WATERMARK. The
> FOR

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-23 Thread Fabian Hueske
Hi everyone,

Every table with a primary key and an event-time attribute provides what is
needed for an event-time temporal table join.
I agree that, from a technical point of view, the TEMPORAL keyword is not
required.

I'm more sceptical about implicitly deriving the versioning information of
a (temporal) table as the table's only event-time attribute.
In the query

SELECT *
FROM orders o, rates r FOR SYSTEM_TIME AS OF o.ordertime
WHERE o.currency = r.currency

the syntax of the temporal table join does not explicitly reference the
version of the temporal rates table.
Hence, the system needs a way to derive the version of temporal table.

Implicitly using the (only) event-time attribute of a temporal table (rates
in the example above) to identify the right version works in most cases,
but probably not in all.
* What if a table has more than one event-time attribute? (TableSchema is
designed to support multiple watermarks; queries with interval joins
produce tables with multiple event-time attributes, ...)
* What if the table does not have an event-time attribute in its schema but
the version should only be provided as meta data?

We could add a clause to define the version of a table, such as:

CREATE TABLE rates (
   currency CHAR(3) NOT NULL PRIMARY KEY,
   rate DOUBLE,
   rowtime TIMESTAMP,
   WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE),
VERSION (rowtime)
WITH (...);

The presence of a the VERSION clause (or whatever syntax) would explicitly
define the version of a (temporal) table.
It would also render the need for the TEMPORAL keyword superfluous because
there would be another indicator that a table can be used in a temporal
table join.

I'm OK with not adding the TEMPORAL keyword, but I recommend that we think
again about the proposed implicit definition of a table's version and how
it might limit use in the future.

Cheers,
Fabian

Am Mo., 22. Juni 2020 um 16:14 Uhr schrieb Jark Wu :

> I'm also +1 for not adding the TEMPORAL keyword.
>
> +1 to make the PRIMARY KEY semantic clear for sources.
> From my point of view:
>
> 1) PRIMARY KEY on changelog souruce:
> It means that when the changelogs (INSERT/UPDATE/DELETE) are materialized,
> the materialized table should be unique on the primary key columns.
> Flink assumes messages are in order on the primary key. Flink doesn't
> validate/enforces the key integrity, but simply trust it (thus NOT
> ENFORCED).
> Flink will use the PRIMARY KEY for some optimization, e.g. use the PRIMARY
> KEY to update the materilized state by key in temporal join operator.
>
> 2) PRIMARY KEY on insert-only source:
> I prefer to have the same semantic to the batch source and changelog
> source, that it implies that records are not duplicate on the primary key.
> Flink just simply trust the primary key constraint, and doesn't valid it.
> If there is duplicate primary keys with INSERT changeflag, then result of
> Flink query might be wrong.
>
> If this is a TEMPORAL TABLE FUNCTION scenario, that source emits duplicate
> primary keys with INSERT changeflag, when we migrate this case to temporal
> table DDL,
> I think this source should emit INSERT/UPDATE (UPSERT) messages instead of
> INSERT-only messages,  e.g. a Kafka compacted topic source?
>
> Best,
> Jark
>
>
> On Mon, 22 Jun 2020 at 17:04, Konstantin Knauf  wrote:
>
> > Hi everyone,
> >
> > I also agree with Leonard/Kurt's proposal for CREATE TEMPORAL TABLE.
> >
> > Best,
> >
> > Konstantin
> >
> > On Mon, Jun 22, 2020 at 10:53 AM Kurt Young  wrote:
> >
> > > I agree with Timo, semantic about primary key needs more thought and
> > > discussion, especially after FLIP-95 and FLIP-105.
> > >
> > > Best,
> > > Kurt
> > >
> > >
> > > On Mon, Jun 22, 2020 at 4:45 PM Timo Walther 
> wrote:
> > >
> > > > Hi Leonard,
> > > >
> > > > thanks for the summary.
> > > >
> > > > After reading all of the previous arguments and working on FLIP-95. I
> > > > would also lean towards the conclusion of not adding the TEMPORAL
> > > keyword.
> > > >
> > > > After FLIP-95, what we considered as a CREATE TEMPORAL TABLE can be
> > > > represented as a CREATE TABLE with PRIMARY KEY and WATERMARK. The FOR
> > > > SYSTEM_TIME AS OF t would trigger the internal materialization and
> > > > "temporal" logic.
> > > >
> > > > However, we should discuss the meaning of PRIMARY KEY again in this
> > > > case. In a TEMPORAL TABLE scenario, the source would emit duplicate
> > > > primary keys with INSERT changeflag but at different point in time.
> > > > Currently, we require a PRIMARY KEY NOT ENFORCED declaration. The
> > > > changelog semantics of FLIP-95 and FLIP-105 don't work well with a
> > > > primary key declaration.
> > > >
> > > > Regards,
> > > > Timo
> > > >
> > > >
> > > > On 20.06.20 17:08, Leonard Xu wrote:
> > > > > Hi everyone,
> > > > >
> > > > > Thanks for the nice discussion. I’d like to move forward the work,
> > > > please let me simply summarize the main opinion and current
> > divergences.
> > > > >
> > > > > 1. The 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-22 Thread Jark Wu
I'm also +1 for not adding the TEMPORAL keyword.

+1 to make the PRIMARY KEY semantic clear for sources.
>From my point of view:

1) PRIMARY KEY on changelog souruce:
It means that when the changelogs (INSERT/UPDATE/DELETE) are materialized,
the materialized table should be unique on the primary key columns.
Flink assumes messages are in order on the primary key. Flink doesn't
validate/enforces the key integrity, but simply trust it (thus NOT
ENFORCED).
Flink will use the PRIMARY KEY for some optimization, e.g. use the PRIMARY
KEY to update the materilized state by key in temporal join operator.

2) PRIMARY KEY on insert-only source:
I prefer to have the same semantic to the batch source and changelog
source, that it implies that records are not duplicate on the primary key.
Flink just simply trust the primary key constraint, and doesn't valid it.
If there is duplicate primary keys with INSERT changeflag, then result of
Flink query might be wrong.

If this is a TEMPORAL TABLE FUNCTION scenario, that source emits duplicate
primary keys with INSERT changeflag, when we migrate this case to temporal
table DDL,
I think this source should emit INSERT/UPDATE (UPSERT) messages instead of
INSERT-only messages,  e.g. a Kafka compacted topic source?

Best,
Jark


On Mon, 22 Jun 2020 at 17:04, Konstantin Knauf  wrote:

> Hi everyone,
>
> I also agree with Leonard/Kurt's proposal for CREATE TEMPORAL TABLE.
>
> Best,
>
> Konstantin
>
> On Mon, Jun 22, 2020 at 10:53 AM Kurt Young  wrote:
>
> > I agree with Timo, semantic about primary key needs more thought and
> > discussion, especially after FLIP-95 and FLIP-105.
> >
> > Best,
> > Kurt
> >
> >
> > On Mon, Jun 22, 2020 at 4:45 PM Timo Walther  wrote:
> >
> > > Hi Leonard,
> > >
> > > thanks for the summary.
> > >
> > > After reading all of the previous arguments and working on FLIP-95. I
> > > would also lean towards the conclusion of not adding the TEMPORAL
> > keyword.
> > >
> > > After FLIP-95, what we considered as a CREATE TEMPORAL TABLE can be
> > > represented as a CREATE TABLE with PRIMARY KEY and WATERMARK. The FOR
> > > SYSTEM_TIME AS OF t would trigger the internal materialization and
> > > "temporal" logic.
> > >
> > > However, we should discuss the meaning of PRIMARY KEY again in this
> > > case. In a TEMPORAL TABLE scenario, the source would emit duplicate
> > > primary keys with INSERT changeflag but at different point in time.
> > > Currently, we require a PRIMARY KEY NOT ENFORCED declaration. The
> > > changelog semantics of FLIP-95 and FLIP-105 don't work well with a
> > > primary key declaration.
> > >
> > > Regards,
> > > Timo
> > >
> > >
> > > On 20.06.20 17:08, Leonard Xu wrote:
> > > > Hi everyone,
> > > >
> > > > Thanks for the nice discussion. I’d like to move forward the work,
> > > please let me simply summarize the main opinion and current
> divergences.
> > > >
> > > > 1. The agreements have been achieved:
> > > >
> > > > 1.1 The motivation we're discussing temporal table DDL is just for
> > > creating temporal table in pure SQL to replace pre-process temporal
> table
> > > in YAML/Table API for usability.
> > > > 1.2 The reason we use "TEMPORAL" keyword rather than “PERIOD FOR
> > > SYSTEM_TIME” is to make user understand easily.
> > > > 1.3 For append-only table, it can convert to changelog table which
> has
> > > been discussed in FLIP-105, we assume the following temporal table is
> > comes
> > > from changelog (Jark, fabian, Timo).
> > > > 1.4 For temporal join syntax, using "FOR SYSTEM_TIME AS OF x" instead
> > of
> > > the current `LATERAL TABLE(rates(x))`  has come to an agreement(Fabian,
> > > Timo, Seth, Konstantin, Kurt).
> > > >
> > > > 2. The small divergence :
> > > >
> > > > About the definition syntax of the temporal table,
> > > >
> > > > CREATE [TEMPORAL] TABLE rates (
> > > > currency CHAR(3) NOT NULL PRIMARY KEY,
> > > > rate DOUBLE,
> > > > rowtime TIMESTAMP,
> > > > WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE)
> > > > WITH (...);
> > > >
> > > > there is small divergence whether add "TEMPORAL" keyword or not.
> > > >
> > > > 2.1  one opinion is using "CREATE TEMPORAL TABLE" (Timo, Fabian,
> Seth),
> > > the main advantages are:
> > > > (1)"TEMPORAL" keyword is intuitive to indicate the history tracking
> > > semantics.
> > > > (2)"TEMPORAL" keyword illustrates that queries can visit the previous
> > > versions of a table like other DBMS use "PERIOD FOR SYSTEM_TIME"
> keyword.
> > > >
> > > > 2.2 the other is using "CREATE TABLE"(Kurt), the main advantages are:
> > > > (1)Just primary key and time attribute can track previous versions
> of a
> > > table well.
> > > > (2)The temporal behavior is triggered by temporal join syntax rather
> > > than in DDL, all Flink DDL table are dynamic table logically including
> > > temporal table. If we decide to use "TEMPORAL" keyword and treats
> > changelog
> > > as temporal table, other tables backed queue like Kafka should also use
> > > "TEMPORAL" 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-22 Thread Konstantin Knauf
Hi everyone,

I also agree with Leonard/Kurt's proposal for CREATE TEMPORAL TABLE.

Best,

Konstantin

On Mon, Jun 22, 2020 at 10:53 AM Kurt Young  wrote:

> I agree with Timo, semantic about primary key needs more thought and
> discussion, especially after FLIP-95 and FLIP-105.
>
> Best,
> Kurt
>
>
> On Mon, Jun 22, 2020 at 4:45 PM Timo Walther  wrote:
>
> > Hi Leonard,
> >
> > thanks for the summary.
> >
> > After reading all of the previous arguments and working on FLIP-95. I
> > would also lean towards the conclusion of not adding the TEMPORAL
> keyword.
> >
> > After FLIP-95, what we considered as a CREATE TEMPORAL TABLE can be
> > represented as a CREATE TABLE with PRIMARY KEY and WATERMARK. The FOR
> > SYSTEM_TIME AS OF t would trigger the internal materialization and
> > "temporal" logic.
> >
> > However, we should discuss the meaning of PRIMARY KEY again in this
> > case. In a TEMPORAL TABLE scenario, the source would emit duplicate
> > primary keys with INSERT changeflag but at different point in time.
> > Currently, we require a PRIMARY KEY NOT ENFORCED declaration. The
> > changelog semantics of FLIP-95 and FLIP-105 don't work well with a
> > primary key declaration.
> >
> > Regards,
> > Timo
> >
> >
> > On 20.06.20 17:08, Leonard Xu wrote:
> > > Hi everyone,
> > >
> > > Thanks for the nice discussion. I’d like to move forward the work,
> > please let me simply summarize the main opinion and current divergences.
> > >
> > > 1. The agreements have been achieved:
> > >
> > > 1.1 The motivation we're discussing temporal table DDL is just for
> > creating temporal table in pure SQL to replace pre-process temporal table
> > in YAML/Table API for usability.
> > > 1.2 The reason we use "TEMPORAL" keyword rather than “PERIOD FOR
> > SYSTEM_TIME” is to make user understand easily.
> > > 1.3 For append-only table, it can convert to changelog table which has
> > been discussed in FLIP-105, we assume the following temporal table is
> comes
> > from changelog (Jark, fabian, Timo).
> > > 1.4 For temporal join syntax, using "FOR SYSTEM_TIME AS OF x" instead
> of
> > the current `LATERAL TABLE(rates(x))`  has come to an agreement(Fabian,
> > Timo, Seth, Konstantin, Kurt).
> > >
> > > 2. The small divergence :
> > >
> > > About the definition syntax of the temporal table,
> > >
> > > CREATE [TEMPORAL] TABLE rates (
> > > currency CHAR(3) NOT NULL PRIMARY KEY,
> > > rate DOUBLE,
> > > rowtime TIMESTAMP,
> > > WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE)
> > > WITH (...);
> > >
> > > there is small divergence whether add "TEMPORAL" keyword or not.
> > >
> > > 2.1  one opinion is using "CREATE TEMPORAL TABLE" (Timo, Fabian, Seth),
> > the main advantages are:
> > > (1)"TEMPORAL" keyword is intuitive to indicate the history tracking
> > semantics.
> > > (2)"TEMPORAL" keyword illustrates that queries can visit the previous
> > versions of a table like other DBMS use "PERIOD FOR SYSTEM_TIME" keyword.
> > >
> > > 2.2 the other is using "CREATE TABLE"(Kurt), the main advantages are:
> > > (1)Just primary key and time attribute can track previous versions of a
> > table well.
> > > (2)The temporal behavior is triggered by temporal join syntax rather
> > than in DDL, all Flink DDL table are dynamic table logically including
> > temporal table. If we decide to use "TEMPORAL" keyword and treats
> changelog
> > as temporal table, other tables backed queue like Kafka should also use
> > "TEMPORAL" keyword.
> > >
> > >
> > > IMO, the statement “CREATE TEMPORARY TEMPORAL TABLE...” follows with
> 2.1
> > may confuse users much. If we take a second to think about, for
> source/sink
> > table which may backed queue (like kafka) or DB (like MySQL), we did not
> > add any keyword in DDL to specify they are source or sinks, it works
> well.
> > > I think temporal table is the third one,  kafka data source and DB data
> > source can play as a source/sink/temporal table depends on the
> > position/syntax that user put them in the query. The above rates table
> > >  - can be a source table if user put it at `SELECT * FROM rates;`
> > >  - can be a temporal table if user put it at `SELECT * FROM orders
> > JOIN rates FOR SYSTEM_TIME AS OF orders.proctime
> > >   ON orders.currency = rates.currency;`
> > >  - can be sink table if user put is at `INSERT INTO rates SELECT *
> > FROM …; `
> > >  From these cases, we found all tables defined in Flink should be
> > dynamic table logically, the source/sink/temporal role depends on the
> > position/syntax in user’s query.
> > >In fact we have used similar syntax for current lookup table, we
> > didn’t add “LOOKUP" or “TEMPORAL" keyword for lookup table and trigger
> the
> > temporal join from the position/syntax(“FOR SYSTEM_TIME AS OF x") in
> query.
> > >
> > > So, I prefer to resolve the small divergence with “CREATE TABLE” which
> > > (1) is more unified with our source/sink/temporal dynamic table
> > conceptually,
> > > (2) is 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-22 Thread Kurt Young
I agree with Timo, semantic about primary key needs more thought and
discussion, especially after FLIP-95 and FLIP-105.

Best,
Kurt


On Mon, Jun 22, 2020 at 4:45 PM Timo Walther  wrote:

> Hi Leonard,
>
> thanks for the summary.
>
> After reading all of the previous arguments and working on FLIP-95. I
> would also lean towards the conclusion of not adding the TEMPORAL keyword.
>
> After FLIP-95, what we considered as a CREATE TEMPORAL TABLE can be
> represented as a CREATE TABLE with PRIMARY KEY and WATERMARK. The FOR
> SYSTEM_TIME AS OF t would trigger the internal materialization and
> "temporal" logic.
>
> However, we should discuss the meaning of PRIMARY KEY again in this
> case. In a TEMPORAL TABLE scenario, the source would emit duplicate
> primary keys with INSERT changeflag but at different point in time.
> Currently, we require a PRIMARY KEY NOT ENFORCED declaration. The
> changelog semantics of FLIP-95 and FLIP-105 don't work well with a
> primary key declaration.
>
> Regards,
> Timo
>
>
> On 20.06.20 17:08, Leonard Xu wrote:
> > Hi everyone,
> >
> > Thanks for the nice discussion. I’d like to move forward the work,
> please let me simply summarize the main opinion and current divergences.
> >
> > 1. The agreements have been achieved:
> >
> > 1.1 The motivation we're discussing temporal table DDL is just for
> creating temporal table in pure SQL to replace pre-process temporal table
> in YAML/Table API for usability.
> > 1.2 The reason we use "TEMPORAL" keyword rather than “PERIOD FOR
> SYSTEM_TIME” is to make user understand easily.
> > 1.3 For append-only table, it can convert to changelog table which has
> been discussed in FLIP-105, we assume the following temporal table is comes
> from changelog (Jark, fabian, Timo).
> > 1.4 For temporal join syntax, using "FOR SYSTEM_TIME AS OF x" instead of
> the current `LATERAL TABLE(rates(x))`  has come to an agreement(Fabian,
> Timo, Seth, Konstantin, Kurt).
> >
> > 2. The small divergence :
> >
> > About the definition syntax of the temporal table,
> >
> > CREATE [TEMPORAL] TABLE rates (
> > currency CHAR(3) NOT NULL PRIMARY KEY,
> > rate DOUBLE,
> > rowtime TIMESTAMP,
> > WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE)
> > WITH (...);
> >
> > there is small divergence whether add "TEMPORAL" keyword or not.
> >
> > 2.1  one opinion is using "CREATE TEMPORAL TABLE" (Timo, Fabian, Seth),
> the main advantages are:
> > (1)"TEMPORAL" keyword is intuitive to indicate the history tracking
> semantics.
> > (2)"TEMPORAL" keyword illustrates that queries can visit the previous
> versions of a table like other DBMS use "PERIOD FOR SYSTEM_TIME" keyword.
> >
> > 2.2 the other is using "CREATE TABLE"(Kurt), the main advantages are:
> > (1)Just primary key and time attribute can track previous versions of a
> table well.
> > (2)The temporal behavior is triggered by temporal join syntax rather
> than in DDL, all Flink DDL table are dynamic table logically including
> temporal table. If we decide to use "TEMPORAL" keyword and treats changelog
> as temporal table, other tables backed queue like Kafka should also use
> "TEMPORAL" keyword.
> >
> >
> > IMO, the statement “CREATE TEMPORARY TEMPORAL TABLE...” follows with 2.1
> may confuse users much. If we take a second to think about, for source/sink
> table which may backed queue (like kafka) or DB (like MySQL), we did not
> add any keyword in DDL to specify they are source or sinks, it works well.
> > I think temporal table is the third one,  kafka data source and DB data
> source can play as a source/sink/temporal table depends on the
> position/syntax that user put them in the query. The above rates table
> >  - can be a source table if user put it at `SELECT * FROM rates;`
> >  - can be a temporal table if user put it at `SELECT * FROM orders
> JOIN rates FOR SYSTEM_TIME AS OF orders.proctime
> >   ON orders.currency = rates.currency;`
> >  - can be sink table if user put is at `INSERT INTO rates SELECT *
> FROM …; `
> >  From these cases, we found all tables defined in Flink should be
> dynamic table logically, the source/sink/temporal role depends on the
> position/syntax in user’s query.
> >In fact we have used similar syntax for current lookup table, we
> didn’t add “LOOKUP" or “TEMPORAL" keyword for lookup table and trigger the
> temporal join from the position/syntax(“FOR SYSTEM_TIME AS OF x") in query.
> >
> > So, I prefer to resolve the small divergence with “CREATE TABLE” which
> > (1) is more unified with our source/sink/temporal dynamic table
> conceptually,
> > (2) is aligned with current lookup table,
> > (3) also make users learn less keyword.
> >
> > WDYT?
> >
> > Best,
> > Leonard Xu
> >
> >
>
>


Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-22 Thread Timo Walther

Hi Leonard,

thanks for the summary.

After reading all of the previous arguments and working on FLIP-95. I 
would also lean towards the conclusion of not adding the TEMPORAL keyword.


After FLIP-95, what we considered as a CREATE TEMPORAL TABLE can be 
represented as a CREATE TABLE with PRIMARY KEY and WATERMARK. The FOR 
SYSTEM_TIME AS OF t would trigger the internal materialization and 
"temporal" logic.


However, we should discuss the meaning of PRIMARY KEY again in this 
case. In a TEMPORAL TABLE scenario, the source would emit duplicate 
primary keys with INSERT changeflag but at different point in time. 
Currently, we require a PRIMARY KEY NOT ENFORCED declaration. The 
changelog semantics of FLIP-95 and FLIP-105 don't work well with a 
primary key declaration.


Regards,
Timo


On 20.06.20 17:08, Leonard Xu wrote:

Hi everyone,

Thanks for the nice discussion. I’d like to move forward the work, please let 
me simply summarize the main opinion and current divergences.

1. The agreements have been achieved:

1.1 The motivation we're discussing temporal table DDL is just for creating 
temporal table in pure SQL to replace pre-process temporal table in YAML/Table 
API for usability.
1.2 The reason we use "TEMPORAL" keyword rather than “PERIOD FOR SYSTEM_TIME” 
is to make user understand easily.
1.3 For append-only table, it can convert to changelog table which has been 
discussed in FLIP-105, we assume the following temporal table is comes from 
changelog (Jark, fabian, Timo).
1.4 For temporal join syntax, using "FOR SYSTEM_TIME AS OF x" instead of the 
current `LATERAL TABLE(rates(x))`  has come to an agreement(Fabian, Timo, Seth, 
Konstantin, Kurt).

2. The small divergence :

About the definition syntax of the temporal table,

CREATE [TEMPORAL] TABLE rates (
currency CHAR(3) NOT NULL PRIMARY KEY,
rate DOUBLE,
rowtime TIMESTAMP,
WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE)
WITH (...);

there is small divergence whether add "TEMPORAL" keyword or not.

2.1  one opinion is using "CREATE TEMPORAL TABLE" (Timo, Fabian, Seth), the 
main advantages are:
(1)"TEMPORAL" keyword is intuitive to indicate the history tracking semantics.
(2)"TEMPORAL" keyword illustrates that queries can visit the previous versions of a table 
like other DBMS use "PERIOD FOR SYSTEM_TIME" keyword.

2.2 the other is using "CREATE TABLE"(Kurt), the main advantages are:
(1)Just primary key and time attribute can track previous versions of a table 
well.
(2)The temporal behavior is triggered by temporal join syntax rather than in DDL, all Flink DDL 
table are dynamic table logically including temporal table. If we decide to use 
"TEMPORAL" keyword and treats changelog as temporal table, other tables backed queue like 
Kafka should also use "TEMPORAL" keyword.


IMO, the statement “CREATE TEMPORARY TEMPORAL TABLE...” follows with 2.1 may 
confuse users much. If we take a second to think about, for source/sink table 
which may backed queue (like kafka) or DB (like MySQL), we did not add any 
keyword in DDL to specify they are source or sinks, it works well.
I think temporal table is the third one,  kafka data source and DB data source 
can play as a source/sink/temporal table depends on the position/syntax that 
user put them in the query. The above rates table
 - can be a source table if user put it at `SELECT * FROM rates;`
 - can be a temporal table if user put it at `SELECT * FROM orders JOIN 
rates FOR SYSTEM_TIME AS OF orders.proctime
  ON orders.currency = rates.currency;`
 - can be sink table if user put is at `INSERT INTO rates SELECT * FROM …; `
 From these cases, we found all tables defined in Flink should be dynamic table 
logically, the source/sink/temporal role depends on the position/syntax in 
user’s query.
   In fact we have used similar syntax for current lookup table, we didn’t add 
“LOOKUP" or “TEMPORAL" keyword for lookup table and trigger the temporal join from 
the position/syntax(“FOR SYSTEM_TIME AS OF x") in query.

So, I prefer to resolve the small divergence with “CREATE TABLE” which
(1) is more unified with our source/sink/temporal dynamic table conceptually,
(2) is aligned with current lookup table,
(3) also make users learn less keyword.

WDYT?

Best,
Leonard Xu






Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-20 Thread Leonard Xu
Hi everyone,

Thanks for the nice discussion. I’d like to move forward the work, please let 
me simply summarize the main opinion and current divergences.

1. The agreements have been achieved:

1.1 The motivation we're discussing temporal table DDL is just for creating 
temporal table in pure SQL to replace pre-process temporal table in YAML/Table 
API for usability.
1.2 The reason we use "TEMPORAL" keyword rather than “PERIOD FOR SYSTEM_TIME” 
is to make user understand easily.
1.3 For append-only table, it can convert to changelog table which has been 
discussed in FLIP-105, we assume the following temporal table is comes from 
changelog (Jark, fabian, Timo).
1.4 For temporal join syntax, using "FOR SYSTEM_TIME AS OF x" instead of the 
current `LATERAL TABLE(rates(x))`  has come to an agreement(Fabian, Timo, Seth, 
Konstantin, Kurt).

2. The small divergence :

About the definition syntax of the temporal table,

CREATE [TEMPORAL] TABLE rates (
   currency CHAR(3) NOT NULL PRIMARY KEY,
   rate DOUBLE,
   rowtime TIMESTAMP,
   WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE)
WITH (...);

there is small divergence whether add "TEMPORAL" keyword or not.

2.1  one opinion is using "CREATE TEMPORAL TABLE" (Timo, Fabian, Seth), the 
main advantages are:
(1)"TEMPORAL" keyword is intuitive to indicate the history tracking semantics.
(2)"TEMPORAL" keyword illustrates that queries can visit the previous versions 
of a table like other DBMS use "PERIOD FOR SYSTEM_TIME" keyword.

2.2 the other is using "CREATE TABLE"(Kurt), the main advantages are:
(1)Just primary key and time attribute can track previous versions of a table 
well.
(2)The temporal behavior is triggered by temporal join syntax rather than in 
DDL, all Flink DDL table are dynamic table logically including temporal table. 
If we decide to use "TEMPORAL" keyword and treats changelog as temporal table, 
other tables backed queue like Kafka should also use "TEMPORAL" keyword.


IMO, the statement “CREATE TEMPORARY TEMPORAL TABLE...” follows with 2.1 may 
confuse users much. If we take a second to think about, for source/sink table 
which may backed queue (like kafka) or DB (like MySQL), we did not add any 
keyword in DDL to specify they are source or sinks, it works well.
I think temporal table is the third one,  kafka data source and DB data source 
can play as a source/sink/temporal table depends on the position/syntax that 
user put them in the query. The above rates table
- can be a source table if user put it at `SELECT * FROM rates;`
- can be a temporal table if user put it at `SELECT * FROM orders JOIN 
rates FOR SYSTEM_TIME AS OF orders.proctime  
 ON orders.currency = rates.currency;`
- can be sink table if user put is at `INSERT INTO rates SELECT * FROM …; `
From these cases, we found all tables defined in Flink should be dynamic table 
logically, the source/sink/temporal role depends on the position/syntax in 
user’s query. 
  In fact we have used similar syntax for current lookup table, we didn’t 
add “LOOKUP" or “TEMPORAL" keyword for lookup table and trigger the temporal 
join from the position/syntax(“FOR SYSTEM_TIME AS OF x") in query.

So, I prefer to resolve the small divergence with “CREATE TABLE” which
(1) is more unified with our source/sink/temporal dynamic table conceptually,
(2) is aligned with current lookup table,
(3) also make users learn less keyword.

WDYT?

Best,
Leonard Xu



Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-13 Thread Kurt Young
Thanks for sharing your opinion. I can see there are some very small
divergences we had through your description. I think it would be a good
idea to first discuss these first.

Let's first put aside table version for now, and only discuss about whether
a DDL table should be treated as a DMBS style table to Flink or as
a changelog of such table. I would say both, but the later one will be the
majority case.

IMO this slight difference has not been distinguished clear enough in the
past. The biggest reason behind this is we only supported "append-only"
table as source in the past. If we take a second and think about the
append-only table, you will find out there is not much differences between
"treating it as a DMBS style table" and "treating it as a table's
changelog", Because no matter you see it from any of these two angles, you
will
see exactly the same thing. That's why we don't need to distinguish them
clearly and most of the things are just worked fine.

Things have been changed since we introducd FLIP-95 and FLIP-105. With
these two FLIPs, we are able to interpret the binlog like messages from
the source, and start to emit append / update / delete messages from
source. I would say the balance has been leaned to the changelog side
of these two angles we faced. It doesn't make much sense that a DBMS's
table itself is having some kind of update and delete messages.

Although the balance has been broke, but the two situations still exist.
Because some kind of tables are still DBMS's style table, e.g, table from
MySQL,
table from HBase. I don't have a strong opinion about how to distinguish
them, TEMPORAL keyword seems fine to me. But if we introduce this keyword,
we need to have a decision about whether an unbounded queue backed table
is TEMPORAL or not. IMO, it is a TEMPORAL table, because this
looks more like a changelog than a DBMS table to me. If this is the case,
I'm afraid that we need to put this keyword to most of the tables user has
been declared, e.g. almost all kafka tables.

But before we really decide what we should do now, I'm also curious to hear
about your opinion about the small divergence I described above.

Best,
Kurt


On Thu, May 14, 2020 at 1:27 AM Fabian Hueske  wrote:

> I think Flink should behave similar to other DBMSs.
>
> Other DBMS do not allow to query the history of a table, even though the
> DBMS has seen all changes of the table (as transactions or directly as a
> changelog if the table was replicated) and recorded them in its log.
> You need to declare a table as TEMPORAL to be able to look up previous
> versions.
>
> Flink is in a very similar situation.
> Even though we've see the physical data of all changes and could also
> store it for some time, I think we should only allow queries against
> previous versions of a table (with FOR SYSTEM_TIME AS OF) if the table was
> defined as TEMPORAL.
>
> IMO this is not about having the data to return a previous version of a
> table (other DBMS have the data as well), it's whether the user should tell
> the system to allow access to the table's history or not.
> As I said before, we could of course declare that all tables are
> automatically temporal and versioned on the only event-time attribute (what
> if there would be more than one?), but I personally don't like such
> implicit conventions.
> I don't have a concrete proposal for a syntax to declare the version
> attribute of a table, but I agree that the "PERIOD FOR SYSTEM_TIME" syntax
> doesn't look very suitable for our purposes.
> I'm sure we can come up with a better syntax for this.
>
> Best, Fabian
>
> Am Sa., 9. Mai 2020 um 03:57 Uhr schrieb Kurt Young :
>
>> All tables being described by Flink's DDL are dynamic tables. But dynamic
>> table is more like a logical concept, but not physical things.
>> Physically, dynamic table has two different forms, one is a materialized
>> table which changes over time (e.g. Database table, HBase table),
>> another form is stream which represents change logs, and they are
>> typically stored in message queue (e.g, Kafka). For the later one, I think
>> the records already representing the history of the dynamic table based
>> on stream-table duality.
>>
>> So regarding to:
>> > Of course we could define that Flink implicitly tracks the (recent,
>> i.e., within watermark bounds) history of all dynamic tables.
>> I don't think this is Flink implicitly tracking the history of the
>> dynamic table, but the physical data of the table is already the history
>> itself. What Flink
>> did is read the history out, and organize them to be prepared for further
>> operations.
>>
>> I agree with another implicit convention I took though, which treats the
>> event time as the version of the dynamic table. Strictly speaking,
>> we should use another syntax "PERIOD FOR SYSTEM_TIME" [1] to indicate the
>> version of the table. I've been thinking about this for quite a bit,
>> it turns out that this semantic is too similar with Flink's event time.
>> 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-13 Thread Fabian Hueske
I think Flink should behave similar to other DBMSs.

Other DBMS do not allow to query the history of a table, even though the
DBMS has seen all changes of the table (as transactions or directly as a
changelog if the table was replicated) and recorded them in its log.
You need to declare a table as TEMPORAL to be able to look up previous
versions.

Flink is in a very similar situation.
Even though we've see the physical data of all changes and could also store
it for some time, I think we should only allow queries against previous
versions of a table (with FOR SYSTEM_TIME AS OF) if the table was defined
as TEMPORAL.

IMO this is not about having the data to return a previous version of a
table (other DBMS have the data as well), it's whether the user should tell
the system to allow access to the table's history or not.
As I said before, we could of course declare that all tables are
automatically temporal and versioned on the only event-time attribute (what
if there would be more than one?), but I personally don't like such
implicit conventions.
I don't have a concrete proposal for a syntax to declare the version
attribute of a table, but I agree that the "PERIOD FOR SYSTEM_TIME" syntax
doesn't look very suitable for our purposes.
I'm sure we can come up with a better syntax for this.

Best, Fabian

Am Sa., 9. Mai 2020 um 03:57 Uhr schrieb Kurt Young :

> All tables being described by Flink's DDL are dynamic tables. But dynamic
> table is more like a logical concept, but not physical things.
> Physically, dynamic table has two different forms, one is a materialized
> table which changes over time (e.g. Database table, HBase table),
> another form is stream which represents change logs, and they are
> typically stored in message queue (e.g, Kafka). For the later one, I think
> the records already representing the history of the dynamic table based on
> stream-table duality.
>
> So regarding to:
> > Of course we could define that Flink implicitly tracks the (recent,
> i.e., within watermark bounds) history of all dynamic tables.
> I don't think this is Flink implicitly tracking the history of the dynamic
> table, but the physical data of the table is already the history itself.
> What Flink
> did is read the history out, and organize them to be prepared for further
> operations.
>
> I agree with another implicit convention I took though, which treats the
> event time as the version of the dynamic table. Strictly speaking,
> we should use another syntax "PERIOD FOR SYSTEM_TIME" [1] to indicate the
> version of the table. I've been thinking about this for quite a bit,
> it turns out that this semantic is too similar with Flink's event time. It
> will cause more trouble for users to understand what does this mean if
> we treat event time and this "PERIOD FOR SYSTEM_TIME" differently. And I'm
> also afraid that we will introduce lots of bugs because not all
> the developers will understand this easily.
> [1]
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables?view=sql-server-ver15
>
> Best,
> Kurt
>
>
> On Sat, May 9, 2020 at 5:32 AM Fabian Hueske  wrote:
>
>> I think we need the TEMPORAL TABLE syntax because they are conceptually
>> more than just regular tables.
>> In a addition to being a table that always holds the latest values (and
>> can thereby serve as input to a continuous query), the system also needs to
>> track the history of such a table to be able to serve different versions of
>> the table (as requested by FOR SYSTEM_TIME AS OF).
>>
>> Of course we could define that Flink implicitly tracks the (recent, i.e.,
>> within watermark bounds) history of all dynamic tables.
>> However, there's one more thing the system needs to know to be able to
>> correctly evaluate FOR SYSTEM_TIME AS OF x, namely which time attribute to
>> use as version of the temporal table.
>> IMO it would be good to make this explicit, especially if there is a plan
>> to eventually support support multiple event-time attributes / watermarks
>> on a table.
>> Just using the only event time attribute would be a bit too much
>> convention magic for my taste (others might of course have a different
>> opinion on this subject).
>>
>> So I agree with Kurt that we don't necessarily need the TEMPORAL TABLE
>> statement if we agree on a few implicit conventions (implicit history table
>> + implicit versioning attribute).
>> I'm not a big fan of such conventions and think it's better to make such
>> things explicit.
>>
>> For temporal joins with processing time semantics, we can use regular
>> dynamic tables without declaring them as TEMPORAL since we don't need a
>> history table to derive the current version.
>> AFAIK, these are already the semantics we use for LookupTableSource.
>>
>> Regarding the question of append-only tables and temporal tables, I'd
>> like to share some more thoughts.
>> As I said above, a temporal table consists of a regular dynamic table A
>> that holds the latest version and a table 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-08 Thread Kurt Young
All tables being described by Flink's DDL are dynamic tables. But dynamic
table is more like a logical concept, but not physical things.
Physically, dynamic table has two different forms, one is a materialized
table which changes over time (e.g. Database table, HBase table),
another form is stream which represents change logs, and they are typically
stored in message queue (e.g, Kafka). For the later one, I think
the records already representing the history of the dynamic table based on
stream-table duality.

So regarding to:
> Of course we could define that Flink implicitly tracks the (recent, i.e.,
within watermark bounds) history of all dynamic tables.
I don't think this is Flink implicitly tracking the history of the dynamic
table, but the physical data of the table is already the history itself.
What Flink
did is read the history out, and organize them to be prepared for further
operations.

I agree with another implicit convention I took though, which treats the
event time as the version of the dynamic table. Strictly speaking,
we should use another syntax "PERIOD FOR SYSTEM_TIME" [1] to indicate the
version of the table. I've been thinking about this for quite a bit,
it turns out that this semantic is too similar with Flink's event time. It
will cause more trouble for users to understand what does this mean if
we treat event time and this "PERIOD FOR SYSTEM_TIME" differently. And I'm
also afraid that we will introduce lots of bugs because not all
the developers will understand this easily.
[1]
https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables?view=sql-server-ver15

Best,
Kurt


On Sat, May 9, 2020 at 5:32 AM Fabian Hueske  wrote:

> I think we need the TEMPORAL TABLE syntax because they are conceptually
> more than just regular tables.
> In a addition to being a table that always holds the latest values (and
> can thereby serve as input to a continuous query), the system also needs to
> track the history of such a table to be able to serve different versions of
> the table (as requested by FOR SYSTEM_TIME AS OF).
>
> Of course we could define that Flink implicitly tracks the (recent, i.e.,
> within watermark bounds) history of all dynamic tables.
> However, there's one more thing the system needs to know to be able to
> correctly evaluate FOR SYSTEM_TIME AS OF x, namely which time attribute to
> use as version of the temporal table.
> IMO it would be good to make this explicit, especially if there is a plan
> to eventually support support multiple event-time attributes / watermarks
> on a table.
> Just using the only event time attribute would be a bit too much
> convention magic for my taste (others might of course have a different
> opinion on this subject).
>
> So I agree with Kurt that we don't necessarily need the TEMPORAL TABLE
> statement if we agree on a few implicit conventions (implicit history table
> + implicit versioning attribute).
> I'm not a big fan of such conventions and think it's better to make such
> things explicit.
>
> For temporal joins with processing time semantics, we can use regular
> dynamic tables without declaring them as TEMPORAL since we don't need a
> history table to derive the current version.
> AFAIK, these are already the semantics we use for LookupTableSource.
>
> Regarding the question of append-only tables and temporal tables, I'd like
> to share some more thoughts.
> As I said above, a temporal table consists of a regular dynamic table A
> that holds the latest version and a table H that holds the history of A.
> 1) When defining a temporal table based on a regular dynamic table (with a
> primary key), we provide A and the Flink automatically maintains H (bounded
> by watermarks)
> 2) When defining a temporal table based on an append-only table, Flink
> ingests H and we use the temporal table function to turn it into a dynamic
> table with a primary key, i.e., into A. This conversion could also be done
> during ingestion by treating the append-only stream as an upsert changelog
> and converting it into a dynamic table with PK and as Table A (just in case
> 1).
>
> As Jark said "converting append-only table into changelog table" was moved
> to future work.
> Until then, we could only define TEMPORAL TABLE on a table that is derived
> from a proper changelog stream with a specific encoding.
> The TEMPORAL VIEW would be a shortcut which would allow us to perform the
> conversion in Flink SQL (and not within the connector) and defining the
> temporal properties on the result of the view.
>
> Cheers,
> Fabian
>
>
>
> Am Fr., 8. Mai 2020 um 08:29 Uhr schrieb Kurt Young :
>
>> I might missed something but why we need a new "TEMPORAL TABLE" syntax?
>>
>> According to Fabian's first mail:
>>
>> > Hence, the requirements for a temporal table are:
>> > * The temporal table has a primary key / unique attribute
>> > * The temporal table has a time-attribute that defines the start of the
>> > validity interval of a row (processing time or 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-08 Thread Fabian Hueske
I think we need the TEMPORAL TABLE syntax because they are conceptually
more than just regular tables.
In a addition to being a table that always holds the latest values (and can
thereby serve as input to a continuous query), the system also needs to
track the history of such a table to be able to serve different versions of
the table (as requested by FOR SYSTEM_TIME AS OF).

Of course we could define that Flink implicitly tracks the (recent, i.e.,
within watermark bounds) history of all dynamic tables.
However, there's one more thing the system needs to know to be able to
correctly evaluate FOR SYSTEM_TIME AS OF x, namely which time attribute to
use as version of the temporal table.
IMO it would be good to make this explicit, especially if there is a plan
to eventually support support multiple event-time attributes / watermarks
on a table.
Just using the only event time attribute would be a bit too much convention
magic for my taste (others might of course have a different opinion on this
subject).

So I agree with Kurt that we don't necessarily need the TEMPORAL TABLE
statement if we agree on a few implicit conventions (implicit history table
+ implicit versioning attribute).
I'm not a big fan of such conventions and think it's better to make such
things explicit.

For temporal joins with processing time semantics, we can use regular
dynamic tables without declaring them as TEMPORAL since we don't need a
history table to derive the current version.
AFAIK, these are already the semantics we use for LookupTableSource.

Regarding the question of append-only tables and temporal tables, I'd like
to share some more thoughts.
As I said above, a temporal table consists of a regular dynamic table A
that holds the latest version and a table H that holds the history of A.
1) When defining a temporal table based on a regular dynamic table (with a
primary key), we provide A and the Flink automatically maintains H (bounded
by watermarks)
2) When defining a temporal table based on an append-only table, Flink
ingests H and we use the temporal table function to turn it into a dynamic
table with a primary key, i.e., into A. This conversion could also be done
during ingestion by treating the append-only stream as an upsert changelog
and converting it into a dynamic table with PK and as Table A (just in case
1).

As Jark said "converting append-only table into changelog table" was moved
to future work.
Until then, we could only define TEMPORAL TABLE on a table that is derived
from a proper changelog stream with a specific encoding.
The TEMPORAL VIEW would be a shortcut which would allow us to perform the
conversion in Flink SQL (and not within the connector) and defining the
temporal properties on the result of the view.

Cheers,
Fabian



Am Fr., 8. Mai 2020 um 08:29 Uhr schrieb Kurt Young :

> I might missed something but why we need a new "TEMPORAL TABLE" syntax?
>
> According to Fabian's first mail:
>
> > Hence, the requirements for a temporal table are:
> > * The temporal table has a primary key / unique attribute
> > * The temporal table has a time-attribute that defines the start of the
> > validity interval of a row (processing time or event time)
> > * The system knows that the history of the table is tracked and can infer
> > how to look up a version.
>
> I think primary key plus proper event time attribute is already
> sufficient. So a join query looks like:
>
> "Fact join Dim FOR SYSTEM_TIME AS OF Fact.some_event_time ON Fact.id =
> Dim.id"
>
> would means for every record belong to Fact, use Fact.some_event_time as
> Dim's version (which
> will only keep all records from Dim table with event time less or equal
> to Fact.some_event_time, and
> keep only one record for each primary key).
>
> The temporal behavior is actually triggered by the join syntax "FOR
> SYSTEM_TIME AS OF Fact.some_event_time"
> but not the DDL description.
>
> Best,
> Kurt
>
>
> On Fri, May 8, 2020 at 10:51 AM Jark Wu  wrote:
>
>> Hi,
>>
>> I agree what Fabian said above.
>> Besides, IMO, (3) is in a lower priority and will involve much more
>> things.
>> It makes sense to me to do it in two-phase.
>>
>> Regarding to (3), the key point to convert an append-only table into
>> changelog table is that the framework should know the operation type,
>> so we introduced a special CREATE VIEW syntax to do it in the
>> documentation
>> [1]. Here is an example:
>>
>> -- my_binlog table is registered as an append-only table
>> CREATE TABLE my_binlog (
>>   before ROW<...>,
>>   after ROW<...>,
>>   op STRING,
>>   op_ms TIMESTAMP(3)
>> ) WITH (
>>   'connector.type' = 'kafka',
>>   ...
>> );
>>
>> -- interpret my_binlog as a changelog on the op_type and id key
>> CREATE VIEW my_table AS
>>   SELECT
>> after.*
>>   FROM my_binlog
>>   CHANGELOG OPERATION BY op
>>   UPDATE KEY BY (id);
>>
>> -- my_table will materialize the insert/delete/update changes
>> -- if we have 4 records in dbz that
>> -- a create for 1004
>> -- an update for 1004
>> -- a 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-08 Thread Kurt Young
I might missed something but why we need a new "TEMPORAL TABLE" syntax?

According to Fabian's first mail:

> Hence, the requirements for a temporal table are:
> * The temporal table has a primary key / unique attribute
> * The temporal table has a time-attribute that defines the start of the
> validity interval of a row (processing time or event time)
> * The system knows that the history of the table is tracked and can infer
> how to look up a version.

I think primary key plus proper event time attribute is already sufficient.
So a join query looks like:

"Fact join Dim FOR SYSTEM_TIME AS OF Fact.some_event_time ON Fact.id =
Dim.id"

would means for every record belong to Fact, use Fact.some_event_time as
Dim's version (which
will only keep all records from Dim table with event time less or equal
to Fact.some_event_time, and
keep only one record for each primary key).

The temporal behavior is actually triggered by the join syntax "FOR
SYSTEM_TIME AS OF Fact.some_event_time"
but not the DDL description.

Best,
Kurt


On Fri, May 8, 2020 at 10:51 AM Jark Wu  wrote:

> Hi,
>
> I agree what Fabian said above.
> Besides, IMO, (3) is in a lower priority and will involve much more things.
> It makes sense to me to do it in two-phase.
>
> Regarding to (3), the key point to convert an append-only table into
> changelog table is that the framework should know the operation type,
> so we introduced a special CREATE VIEW syntax to do it in the documentation
> [1]. Here is an example:
>
> -- my_binlog table is registered as an append-only table
> CREATE TABLE my_binlog (
>   before ROW<...>,
>   after ROW<...>,
>   op STRING,
>   op_ms TIMESTAMP(3)
> ) WITH (
>   'connector.type' = 'kafka',
>   ...
> );
>
> -- interpret my_binlog as a changelog on the op_type and id key
> CREATE VIEW my_table AS
>   SELECT
> after.*
>   FROM my_binlog
>   CHANGELOG OPERATION BY op
>   UPDATE KEY BY (id);
>
> -- my_table will materialize the insert/delete/update changes
> -- if we have 4 records in dbz that
> -- a create for 1004
> -- an update for 1004
> -- a create for 1005
> -- a delete for 1004
> > SELECT COUNT(*) FROM my_table;
> +---+
> |  COUNT(*) |
> +---+
> | 1 |
> +---+
>
> Best,
> Jark
>
> [1]:
>
> https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.sz656g8mb2wb
>
>
> On Fri, 8 May 2020 at 00:24, Fabian Hueske  wrote:
>
> > Thanks for the summary Konstantin.
> > I think you got all points right.
> >
> > IMO, the way forward would be to work on a FLIP to define
> > * the concept of temporal tables,
> > * how to feed them from retraction tables
> > * how to feed them from append-only tables
> > * their specification with CREATE TEMPORAL TABLE,
> > * how to use temporal tables in temporal table joins
> > * how (if at all) to use temporal tables in other types of queries
> >
> > We would keep the LATERAL TABLE syntax because it used for regular
> > table-valued functions.
> > However, we would probably remove the TemporalTableFunction (which is a
> > built-in table-valued function) after we deprecated it for a while.
> >
> > Cheers, Fabian
> >
> > Am Do., 7. Mai 2020 um 18:03 Uhr schrieb Konstantin Knauf <
> > kna...@apache.org>:
> >
> >> Hi everyone,
> >>
> >> Thanks everyone for joining the discussion on this. Please let me
> >> summarize
> >> what I have understood so far.
> >>
> >> 1) For joining an append-only table and a temporal table the syntax the
> >> "FOR
> >> SYSTEM_TIME AS OF " seems to be preferred (Fabian, Timo,
> >> Seth).
> >>
> >> 2) To define a temporal table based on a changelog stream from an
> external
> >> system CREATE TEMPORAL TABLE (as suggested by Timo/Fabian) could be
> used.
> >> 3) In order to also support temporal tables derived from an append-only
> >> stream, we either need to support TEMPORAL VIEW (as mentioned by Fabian)
> >> or
> >> need to have a way to convert an append-only table into a changelog
> table
> >> (briefly discussed in [1]). It is not completely clear to me how a
> >> temporal
> >> table based on an append-only table would be with the syntax proposed in
> >> [1] and 2). @Jark Wu  could you elaborate a bit on
> >> that?
> >>
> >> How do we move forward with this?
> >>
> >> * It seems that a two-phased approach (1 + 2 now, 3 later) makes sense.
> >> What do you think? * If we proceed like this, what would this mean for
> the
> >> current syntax of LATERAL TABLE? Would we keep it? Would we eventually
> >> deprecate and drop it? Since only after 3) we would be on par with the
> >> current temporal table function join, I assume, we could only drop it
> >> thereafter.
> >>
> >> Thanks, Konstantin
> >>
> >> [1]
> >>
> >>
> https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.kduaw9moein6
> >>
> >>
> >> On Sat, Apr 18, 2020 at 3:07 PM Jark Wu  wrote:
> >>
> >> > Hi Fabian,
> >> >
> >> > Just to clarify a little bit, we decided to move the "converting
> >> > 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-07 Thread Jark Wu
Hi,

I agree what Fabian said above.
Besides, IMO, (3) is in a lower priority and will involve much more things.
It makes sense to me to do it in two-phase.

Regarding to (3), the key point to convert an append-only table into
changelog table is that the framework should know the operation type,
so we introduced a special CREATE VIEW syntax to do it in the documentation
[1]. Here is an example:

-- my_binlog table is registered as an append-only table
CREATE TABLE my_binlog (
  before ROW<...>,
  after ROW<...>,
  op STRING,
  op_ms TIMESTAMP(3)
) WITH (
  'connector.type' = 'kafka',
  ...
);

-- interpret my_binlog as a changelog on the op_type and id key
CREATE VIEW my_table AS
  SELECT
after.*
  FROM my_binlog
  CHANGELOG OPERATION BY op
  UPDATE KEY BY (id);

-- my_table will materialize the insert/delete/update changes
-- if we have 4 records in dbz that
-- a create for 1004
-- an update for 1004
-- a create for 1005
-- a delete for 1004
> SELECT COUNT(*) FROM my_table;
+---+
|  COUNT(*) |
+---+
| 1 |
+---+

Best,
Jark

[1]:
https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.sz656g8mb2wb


On Fri, 8 May 2020 at 00:24, Fabian Hueske  wrote:

> Thanks for the summary Konstantin.
> I think you got all points right.
>
> IMO, the way forward would be to work on a FLIP to define
> * the concept of temporal tables,
> * how to feed them from retraction tables
> * how to feed them from append-only tables
> * their specification with CREATE TEMPORAL TABLE,
> * how to use temporal tables in temporal table joins
> * how (if at all) to use temporal tables in other types of queries
>
> We would keep the LATERAL TABLE syntax because it used for regular
> table-valued functions.
> However, we would probably remove the TemporalTableFunction (which is a
> built-in table-valued function) after we deprecated it for a while.
>
> Cheers, Fabian
>
> Am Do., 7. Mai 2020 um 18:03 Uhr schrieb Konstantin Knauf <
> kna...@apache.org>:
>
>> Hi everyone,
>>
>> Thanks everyone for joining the discussion on this. Please let me
>> summarize
>> what I have understood so far.
>>
>> 1) For joining an append-only table and a temporal table the syntax the
>> "FOR
>> SYSTEM_TIME AS OF " seems to be preferred (Fabian, Timo,
>> Seth).
>>
>> 2) To define a temporal table based on a changelog stream from an external
>> system CREATE TEMPORAL TABLE (as suggested by Timo/Fabian) could be used.
>> 3) In order to also support temporal tables derived from an append-only
>> stream, we either need to support TEMPORAL VIEW (as mentioned by Fabian)
>> or
>> need to have a way to convert an append-only table into a changelog table
>> (briefly discussed in [1]). It is not completely clear to me how a
>> temporal
>> table based on an append-only table would be with the syntax proposed in
>> [1] and 2). @Jark Wu  could you elaborate a bit on
>> that?
>>
>> How do we move forward with this?
>>
>> * It seems that a two-phased approach (1 + 2 now, 3 later) makes sense.
>> What do you think? * If we proceed like this, what would this mean for the
>> current syntax of LATERAL TABLE? Would we keep it? Would we eventually
>> deprecate and drop it? Since only after 3) we would be on par with the
>> current temporal table function join, I assume, we could only drop it
>> thereafter.
>>
>> Thanks, Konstantin
>>
>> [1]
>>
>> https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.kduaw9moein6
>>
>>
>> On Sat, Apr 18, 2020 at 3:07 PM Jark Wu  wrote:
>>
>> > Hi Fabian,
>> >
>> > Just to clarify a little bit, we decided to move the "converting
>> > append-only table into changelog table" into future work.
>> > So FLIP-105 only introduced some CDC formats (debezium) and new
>> TableSource
>> > interfaces proposed in FLIP-95.
>> > I should have started a new FLIP for the new CDC formats and keep
>> FLIP-105
>> > as it is to avoid the confusion, sorry about that.
>> >
>> > Best,
>> > Jark
>> >
>> >
>> > On Sat, 18 Apr 2020 at 00:35, Fabian Hueske  wrote:
>> >
>> > > Thanks Jark!
>> > >
>> > > I certainly need to read up on FLIP-105 (and I'll try to adjust my
>> > > terminology to changelog table from now on ;-) )
>> > > If FLIP-105 addresses the issue of converting an append-only table
>> into a
>> > > changelog table that upserts on primary key (basically what the VIEW
>> > > definition in my first email did),
>> > > TEMPORAL VIEWs become much less important.
>> > > In that case, we would be well served with TEMPORAL TABLE and TEMPORAL
>> > VIEW
>> > > would be a nice-to-have feature for some later time.
>> > >
>> > > Cheers, Fabian
>> > >
>> > >
>> > >
>> > >
>> > >
>> > >
>> > > Am Fr., 17. Apr. 2020 um 18:13 Uhr schrieb Jark Wu > >:
>> > >
>> > > > Hi Fabian,
>> > > >
>> > > > I think converting an append-only table into temporal table contains
>> > two
>> > > > things:
>> > > > (1) converting append-only table into changelog table (or retraction
>> > 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-07 Thread Fabian Hueske
Thanks for the summary Konstantin.
I think you got all points right.

IMO, the way forward would be to work on a FLIP to define
* the concept of temporal tables,
* how to feed them from retraction tables
* how to feed them from append-only tables
* their specification with CREATE TEMPORAL TABLE,
* how to use temporal tables in temporal table joins
* how (if at all) to use temporal tables in other types of queries

We would keep the LATERAL TABLE syntax because it used for regular
table-valued functions.
However, we would probably remove the TemporalTableFunction (which is a
built-in table-valued function) after we deprecated it for a while.

Cheers, Fabian

Am Do., 7. Mai 2020 um 18:03 Uhr schrieb Konstantin Knauf :

> Hi everyone,
>
> Thanks everyone for joining the discussion on this. Please let me summarize
> what I have understood so far.
>
> 1) For joining an append-only table and a temporal table the syntax the
> "FOR
> SYSTEM_TIME AS OF " seems to be preferred (Fabian, Timo,
> Seth).
>
> 2) To define a temporal table based on a changelog stream from an external
> system CREATE TEMPORAL TABLE (as suggested by Timo/Fabian) could be used.
> 3) In order to also support temporal tables derived from an append-only
> stream, we either need to support TEMPORAL VIEW (as mentioned by Fabian) or
> need to have a way to convert an append-only table into a changelog table
> (briefly discussed in [1]). It is not completely clear to me how a temporal
> table based on an append-only table would be with the syntax proposed in
> [1] and 2). @Jark Wu  could you elaborate a bit on that?
>
> How do we move forward with this?
>
> * It seems that a two-phased approach (1 + 2 now, 3 later) makes sense.
> What do you think? * If we proceed like this, what would this mean for the
> current syntax of LATERAL TABLE? Would we keep it? Would we eventually
> deprecate and drop it? Since only after 3) we would be on par with the
> current temporal table function join, I assume, we could only drop it
> thereafter.
>
> Thanks, Konstantin
>
> [1]
>
> https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.kduaw9moein6
>
>
> On Sat, Apr 18, 2020 at 3:07 PM Jark Wu  wrote:
>
> > Hi Fabian,
> >
> > Just to clarify a little bit, we decided to move the "converting
> > append-only table into changelog table" into future work.
> > So FLIP-105 only introduced some CDC formats (debezium) and new
> TableSource
> > interfaces proposed in FLIP-95.
> > I should have started a new FLIP for the new CDC formats and keep
> FLIP-105
> > as it is to avoid the confusion, sorry about that.
> >
> > Best,
> > Jark
> >
> >
> > On Sat, 18 Apr 2020 at 00:35, Fabian Hueske  wrote:
> >
> > > Thanks Jark!
> > >
> > > I certainly need to read up on FLIP-105 (and I'll try to adjust my
> > > terminology to changelog table from now on ;-) )
> > > If FLIP-105 addresses the issue of converting an append-only table
> into a
> > > changelog table that upserts on primary key (basically what the VIEW
> > > definition in my first email did),
> > > TEMPORAL VIEWs become much less important.
> > > In that case, we would be well served with TEMPORAL TABLE and TEMPORAL
> > VIEW
> > > would be a nice-to-have feature for some later time.
> > >
> > > Cheers, Fabian
> > >
> > >
> > >
> > >
> > >
> > >
> > > Am Fr., 17. Apr. 2020 um 18:13 Uhr schrieb Jark Wu :
> > >
> > > > Hi Fabian,
> > > >
> > > > I think converting an append-only table into temporal table contains
> > two
> > > > things:
> > > > (1) converting append-only table into changelog table (or retraction
> > > table
> > > > as you said)
> > > > (2) define the converted changelog table (maybe is a view now) as
> > > temporal
> > > > (or history tracked).
> > > >
> > > > The first thing is also mentioned and discussed in FLIP-105 design
> > draft
> > > > [1] which proposed a syntax
> > > > to convert the append-only table into a changelog table.
> > > >
> > > > I think TEMPORAL TABLE is quite straightforward and simple, and can
> > > satisfy
> > > > most existing changelog
> > > > data with popular CDC formats. TEMPORAL VIEW is flexible but will
> > involve
> > > > more SQL codes. I think
> > > > we can support them both.
> > > >
> > > > Best,
> > > > Jark
> > > >
> > > > [1]:
> > > >
> > > >
> > >
> >
> https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.sz656g8mb2wb
> > > >
> > > > On Fri, 17 Apr 2020 at 23:52, Fabian Hueske 
> wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > I agree with most of what Timo said.
> > > > >
> > > > > The TEMPORAL keyword (which unfortunately might be easily confused
> > with
> > > > > TEMPORARY...) looks very intuitive and I think using the only time
> > > > > attribute for versioning would be a good choice.
> > > > >
> > > > > However, TEMPORAL TABLE on retraction tables do not solve the full
> > > > problem.
> > > > > I believe there will be also cases where we need to derive a
> temporal
> > > > table
> 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-07 Thread Konstantin Knauf
Hi everyone,

Thanks everyone for joining the discussion on this. Please let me summarize
what I have understood so far.

1) For joining an append-only table and a temporal table the syntax the "FOR
SYSTEM_TIME AS OF " seems to be preferred (Fabian, Timo,
Seth).

2) To define a temporal table based on a changelog stream from an external
system CREATE TEMPORAL TABLE (as suggested by Timo/Fabian) could be used.
3) In order to also support temporal tables derived from an append-only
stream, we either need to support TEMPORAL VIEW (as mentioned by Fabian) or
need to have a way to convert an append-only table into a changelog table
(briefly discussed in [1]). It is not completely clear to me how a temporal
table based on an append-only table would be with the syntax proposed in
[1] and 2). @Jark Wu  could you elaborate a bit on that?

How do we move forward with this?

* It seems that a two-phased approach (1 + 2 now, 3 later) makes sense.
What do you think? * If we proceed like this, what would this mean for the
current syntax of LATERAL TABLE? Would we keep it? Would we eventually
deprecate and drop it? Since only after 3) we would be on par with the
current temporal table function join, I assume, we could only drop it
thereafter.

Thanks, Konstantin

[1]
https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.kduaw9moein6


On Sat, Apr 18, 2020 at 3:07 PM Jark Wu  wrote:

> Hi Fabian,
>
> Just to clarify a little bit, we decided to move the "converting
> append-only table into changelog table" into future work.
> So FLIP-105 only introduced some CDC formats (debezium) and new TableSource
> interfaces proposed in FLIP-95.
> I should have started a new FLIP for the new CDC formats and keep FLIP-105
> as it is to avoid the confusion, sorry about that.
>
> Best,
> Jark
>
>
> On Sat, 18 Apr 2020 at 00:35, Fabian Hueske  wrote:
>
> > Thanks Jark!
> >
> > I certainly need to read up on FLIP-105 (and I'll try to adjust my
> > terminology to changelog table from now on ;-) )
> > If FLIP-105 addresses the issue of converting an append-only table into a
> > changelog table that upserts on primary key (basically what the VIEW
> > definition in my first email did),
> > TEMPORAL VIEWs become much less important.
> > In that case, we would be well served with TEMPORAL TABLE and TEMPORAL
> VIEW
> > would be a nice-to-have feature for some later time.
> >
> > Cheers, Fabian
> >
> >
> >
> >
> >
> >
> > Am Fr., 17. Apr. 2020 um 18:13 Uhr schrieb Jark Wu :
> >
> > > Hi Fabian,
> > >
> > > I think converting an append-only table into temporal table contains
> two
> > > things:
> > > (1) converting append-only table into changelog table (or retraction
> > table
> > > as you said)
> > > (2) define the converted changelog table (maybe is a view now) as
> > temporal
> > > (or history tracked).
> > >
> > > The first thing is also mentioned and discussed in FLIP-105 design
> draft
> > > [1] which proposed a syntax
> > > to convert the append-only table into a changelog table.
> > >
> > > I think TEMPORAL TABLE is quite straightforward and simple, and can
> > satisfy
> > > most existing changelog
> > > data with popular CDC formats. TEMPORAL VIEW is flexible but will
> involve
> > > more SQL codes. I think
> > > we can support them both.
> > >
> > > Best,
> > > Jark
> > >
> > > [1]:
> > >
> > >
> >
> https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.sz656g8mb2wb
> > >
> > > On Fri, 17 Apr 2020 at 23:52, Fabian Hueske  wrote:
> > >
> > > > Hi,
> > > >
> > > > I agree with most of what Timo said.
> > > >
> > > > The TEMPORAL keyword (which unfortunately might be easily confused
> with
> > > > TEMPORARY...) looks very intuitive and I think using the only time
> > > > attribute for versioning would be a good choice.
> > > >
> > > > However, TEMPORAL TABLE on retraction tables do not solve the full
> > > problem.
> > > > I believe there will be also cases where we need to derive a temporal
> > > table
> > > > from an append only table (what TemporalTableFunctions do right now).
> > > > I think the best choice for this would be TEMPORAL VIEW but as I
> > > explained,
> > > > it might be a longer way until this can be supported.
> > > > TEMPORAL VIEW would also address the problem of preprocessing.
> > > >
> > > > > Regarding retraction table with a primary key and a time-attribute:
> > > > > These semantics are still unclear to me. Can retractions only occur
> > > > > within watermarks? Or are they also used for representing late
> > updates?
> > > >
> > > > Time attributes and retraction streams are a challenging topic that I
> > > > haven't completely understood yet.
> > > > So far we treated time attributes always as part of the data.
> > > > In combination with retractions, it seems that they become metadata
> > that
> > > > specifies when a change was done.
> > > > I think this is different from treating time attributes as regular
> > data.
> > > >
> > 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-04-18 Thread Jark Wu
Hi Fabian,

Just to clarify a little bit, we decided to move the "converting
append-only table into changelog table" into future work.
So FLIP-105 only introduced some CDC formats (debezium) and new TableSource
interfaces proposed in FLIP-95.
I should have started a new FLIP for the new CDC formats and keep FLIP-105
as it is to avoid the confusion, sorry about that.

Best,
Jark


On Sat, 18 Apr 2020 at 00:35, Fabian Hueske  wrote:

> Thanks Jark!
>
> I certainly need to read up on FLIP-105 (and I'll try to adjust my
> terminology to changelog table from now on ;-) )
> If FLIP-105 addresses the issue of converting an append-only table into a
> changelog table that upserts on primary key (basically what the VIEW
> definition in my first email did),
> TEMPORAL VIEWs become much less important.
> In that case, we would be well served with TEMPORAL TABLE and TEMPORAL VIEW
> would be a nice-to-have feature for some later time.
>
> Cheers, Fabian
>
>
>
>
>
>
> Am Fr., 17. Apr. 2020 um 18:13 Uhr schrieb Jark Wu :
>
> > Hi Fabian,
> >
> > I think converting an append-only table into temporal table contains two
> > things:
> > (1) converting append-only table into changelog table (or retraction
> table
> > as you said)
> > (2) define the converted changelog table (maybe is a view now) as
> temporal
> > (or history tracked).
> >
> > The first thing is also mentioned and discussed in FLIP-105 design draft
> > [1] which proposed a syntax
> > to convert the append-only table into a changelog table.
> >
> > I think TEMPORAL TABLE is quite straightforward and simple, and can
> satisfy
> > most existing changelog
> > data with popular CDC formats. TEMPORAL VIEW is flexible but will involve
> > more SQL codes. I think
> > we can support them both.
> >
> > Best,
> > Jark
> >
> > [1]:
> >
> >
> https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.sz656g8mb2wb
> >
> > On Fri, 17 Apr 2020 at 23:52, Fabian Hueske  wrote:
> >
> > > Hi,
> > >
> > > I agree with most of what Timo said.
> > >
> > > The TEMPORAL keyword (which unfortunately might be easily confused with
> > > TEMPORARY...) looks very intuitive and I think using the only time
> > > attribute for versioning would be a good choice.
> > >
> > > However, TEMPORAL TABLE on retraction tables do not solve the full
> > problem.
> > > I believe there will be also cases where we need to derive a temporal
> > table
> > > from an append only table (what TemporalTableFunctions do right now).
> > > I think the best choice for this would be TEMPORAL VIEW but as I
> > explained,
> > > it might be a longer way until this can be supported.
> > > TEMPORAL VIEW would also address the problem of preprocessing.
> > >
> > > > Regarding retraction table with a primary key and a time-attribute:
> > > > These semantics are still unclear to me. Can retractions only occur
> > > > within watermarks? Or are they also used for representing late
> updates?
> > >
> > > Time attributes and retraction streams are a challenging topic that I
> > > haven't completely understood yet.
> > > So far we treated time attributes always as part of the data.
> > > In combination with retractions, it seems that they become metadata
> that
> > > specifies when a change was done.
> > > I think this is different from treating time attributes as regular
> data.
> > >
> > > Cheers, Fabian
> > >
> > >
> > > Am Fr., 17. Apr. 2020 um 17:23 Uhr schrieb Seth Wiesman <
> > > sjwies...@gmail.com
> > > >:
> > >
> > > > I really like the TEMPORAL keyword, I find it very intuitive.
> > > >
> > > > The down side of this approach would be that an additional
> > preprocessing
> > > > > step would not be possible anymore because there is no preceding
> > view.
> > > > >
> > > >
> > > >  Yes and no. My understanding is we are not talking about making any
> > > > changes to how temporal tables are defined in the table api. Since
> you
> > > > cannot currently define temporal table functions in pure SQL
> > > applications,
> > > > but only pre-register them in YAML, you can't do any pre-processing
> as
> > it
> > > > stands today. Preprocessing may be a generally useful feature, I'm
> not
> > > > sure, but this syntax does not lose us anything in pure SQL
> > applications.
> > > >
> > > > These semantics are still unclear to me. Can retractions only occur
> > > > > within watermarks? Or are they also used for representing late
> > updates?
> > > > >
> > > >
> > > > I do not know the SQL standard well enough to give a principled
> > response
> > > to
> > > > this question. However, in my observation of production workloads,
> > users
> > > of
> > > > temporal table functions are doing so to denormalize star schemas
> > before
> > > > performing further transformations and aggregations and expect the
> > output
> > > > to be an append stream. With the ongoing work to better support
> > > changelogs,
> > > > the need for users to understand the differences in append vs upsert
> in
> > > > their 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-04-17 Thread Fabian Hueske
Thanks Jark!

I certainly need to read up on FLIP-105 (and I'll try to adjust my
terminology to changelog table from now on ;-) )
If FLIP-105 addresses the issue of converting an append-only table into a
changelog table that upserts on primary key (basically what the VIEW
definition in my first email did),
TEMPORAL VIEWs become much less important.
In that case, we would be well served with TEMPORAL TABLE and TEMPORAL VIEW
would be a nice-to-have feature for some later time.

Cheers, Fabian






Am Fr., 17. Apr. 2020 um 18:13 Uhr schrieb Jark Wu :

> Hi Fabian,
>
> I think converting an append-only table into temporal table contains two
> things:
> (1) converting append-only table into changelog table (or retraction table
> as you said)
> (2) define the converted changelog table (maybe is a view now) as temporal
> (or history tracked).
>
> The first thing is also mentioned and discussed in FLIP-105 design draft
> [1] which proposed a syntax
> to convert the append-only table into a changelog table.
>
> I think TEMPORAL TABLE is quite straightforward and simple, and can satisfy
> most existing changelog
> data with popular CDC formats. TEMPORAL VIEW is flexible but will involve
> more SQL codes. I think
> we can support them both.
>
> Best,
> Jark
>
> [1]:
>
> https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.sz656g8mb2wb
>
> On Fri, 17 Apr 2020 at 23:52, Fabian Hueske  wrote:
>
> > Hi,
> >
> > I agree with most of what Timo said.
> >
> > The TEMPORAL keyword (which unfortunately might be easily confused with
> > TEMPORARY...) looks very intuitive and I think using the only time
> > attribute for versioning would be a good choice.
> >
> > However, TEMPORAL TABLE on retraction tables do not solve the full
> problem.
> > I believe there will be also cases where we need to derive a temporal
> table
> > from an append only table (what TemporalTableFunctions do right now).
> > I think the best choice for this would be TEMPORAL VIEW but as I
> explained,
> > it might be a longer way until this can be supported.
> > TEMPORAL VIEW would also address the problem of preprocessing.
> >
> > > Regarding retraction table with a primary key and a time-attribute:
> > > These semantics are still unclear to me. Can retractions only occur
> > > within watermarks? Or are they also used for representing late updates?
> >
> > Time attributes and retraction streams are a challenging topic that I
> > haven't completely understood yet.
> > So far we treated time attributes always as part of the data.
> > In combination with retractions, it seems that they become metadata that
> > specifies when a change was done.
> > I think this is different from treating time attributes as regular data.
> >
> > Cheers, Fabian
> >
> >
> > Am Fr., 17. Apr. 2020 um 17:23 Uhr schrieb Seth Wiesman <
> > sjwies...@gmail.com
> > >:
> >
> > > I really like the TEMPORAL keyword, I find it very intuitive.
> > >
> > > The down side of this approach would be that an additional
> preprocessing
> > > > step would not be possible anymore because there is no preceding
> view.
> > > >
> > >
> > >  Yes and no. My understanding is we are not talking about making any
> > > changes to how temporal tables are defined in the table api. Since you
> > > cannot currently define temporal table functions in pure SQL
> > applications,
> > > but only pre-register them in YAML, you can't do any pre-processing as
> it
> > > stands today. Preprocessing may be a generally useful feature, I'm not
> > > sure, but this syntax does not lose us anything in pure SQL
> applications.
> > >
> > > These semantics are still unclear to me. Can retractions only occur
> > > > within watermarks? Or are they also used for representing late
> updates?
> > > >
> > >
> > > I do not know the SQL standard well enough to give a principled
> response
> > to
> > > this question. However, in my observation of production workloads,
> users
> > of
> > > temporal table functions are doing so to denormalize star schemas
> before
> > > performing further transformations and aggregations and expect the
> output
> > > to be an append stream. With the ongoing work to better support
> > changelogs,
> > > the need for users to understand the differences in append vs upsert in
> > > their query may be diminishing but everyone else on this thread can
> > better
> > > speak to that.
> > >
> > > Seth
> > >
> > > On Fri, Apr 17, 2020 at 10:03 AM Timo Walther 
> > wrote:
> > >
> > > > Hi Fabian,
> > > >
> > > > thank you very much for this great summary!
> > > >
> > > > I wasn't aware of the Polymorphic Table Functions standard. This is a
> > > > very interesting topic that we should definitely consider in the
> > future.
> > > > Maybe this could also help us in defining tables more dynamically
> > within
> > > > a query. It could help solving problems as discussed in FLIP-113.
> > > >
> > > > Regarding joining:
> > > >
> > > > IMO we should aim for "FOR SYSTEM_TIME AS OF 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-04-17 Thread Jark Wu
Hi Fabian,

I think converting an append-only table into temporal table contains two
things:
(1) converting append-only table into changelog table (or retraction table
as you said)
(2) define the converted changelog table (maybe is a view now) as temporal
(or history tracked).

The first thing is also mentioned and discussed in FLIP-105 design draft
[1] which proposed a syntax
to convert the append-only table into a changelog table.

I think TEMPORAL TABLE is quite straightforward and simple, and can satisfy
most existing changelog
data with popular CDC formats. TEMPORAL VIEW is flexible but will involve
more SQL codes. I think
we can support them both.

Best,
Jark

[1]:
https://docs.google.com/document/d/1onyIUUdWAHfr_Yd5nZOE7SOExBc6TiW5C4LiL5FrjtQ/edit#heading=h.sz656g8mb2wb

On Fri, 17 Apr 2020 at 23:52, Fabian Hueske  wrote:

> Hi,
>
> I agree with most of what Timo said.
>
> The TEMPORAL keyword (which unfortunately might be easily confused with
> TEMPORARY...) looks very intuitive and I think using the only time
> attribute for versioning would be a good choice.
>
> However, TEMPORAL TABLE on retraction tables do not solve the full problem.
> I believe there will be also cases where we need to derive a temporal table
> from an append only table (what TemporalTableFunctions do right now).
> I think the best choice for this would be TEMPORAL VIEW but as I explained,
> it might be a longer way until this can be supported.
> TEMPORAL VIEW would also address the problem of preprocessing.
>
> > Regarding retraction table with a primary key and a time-attribute:
> > These semantics are still unclear to me. Can retractions only occur
> > within watermarks? Or are they also used for representing late updates?
>
> Time attributes and retraction streams are a challenging topic that I
> haven't completely understood yet.
> So far we treated time attributes always as part of the data.
> In combination with retractions, it seems that they become metadata that
> specifies when a change was done.
> I think this is different from treating time attributes as regular data.
>
> Cheers, Fabian
>
>
> Am Fr., 17. Apr. 2020 um 17:23 Uhr schrieb Seth Wiesman <
> sjwies...@gmail.com
> >:
>
> > I really like the TEMPORAL keyword, I find it very intuitive.
> >
> > The down side of this approach would be that an additional preprocessing
> > > step would not be possible anymore because there is no preceding view.
> > >
> >
> >  Yes and no. My understanding is we are not talking about making any
> > changes to how temporal tables are defined in the table api. Since you
> > cannot currently define temporal table functions in pure SQL
> applications,
> > but only pre-register them in YAML, you can't do any pre-processing as it
> > stands today. Preprocessing may be a generally useful feature, I'm not
> > sure, but this syntax does not lose us anything in pure SQL applications.
> >
> > These semantics are still unclear to me. Can retractions only occur
> > > within watermarks? Or are they also used for representing late updates?
> > >
> >
> > I do not know the SQL standard well enough to give a principled response
> to
> > this question. However, in my observation of production workloads, users
> of
> > temporal table functions are doing so to denormalize star schemas before
> > performing further transformations and aggregations and expect the output
> > to be an append stream. With the ongoing work to better support
> changelogs,
> > the need for users to understand the differences in append vs upsert in
> > their query may be diminishing but everyone else on this thread can
> better
> > speak to that.
> >
> > Seth
> >
> > On Fri, Apr 17, 2020 at 10:03 AM Timo Walther 
> wrote:
> >
> > > Hi Fabian,
> > >
> > > thank you very much for this great summary!
> > >
> > > I wasn't aware of the Polymorphic Table Functions standard. This is a
> > > very interesting topic that we should definitely consider in the
> future.
> > > Maybe this could also help us in defining tables more dynamically
> within
> > > a query. It could help solving problems as discussed in FLIP-113.
> > >
> > > Regarding joining:
> > >
> > > IMO we should aim for "FOR SYSTEM_TIME AS OF x" instead of the current
> > > `LATERAL TABLE(rates(x))` syntax. A function that also behaves like a
> > > table and needs this special `LATERAL` keyword during joining is not
> > > very intuitive. The PTF could be used once they are fully supported by
> > > Calcite and we have the big picture how to also use them for other
> > > time-based operations (windows?, joins?).
> > >
> > > Regarding how represent a temporal table:
> > >
> > > I think that our current DDL, current LookupTableSource and temporal
> > > tables can fit nicely together.
> > >
> > > How about we simply introduce an additional keyword `TEMPORAL` to
> > > indicate history tracking semantics? I think this is the minimal
> > > invasive solution:
> > >
> > > CREATE TEMPORAL TABLE rates (
> > >currency CHAR(3) NOT 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-04-17 Thread Fabian Hueske
Hi,

I agree with most of what Timo said.

The TEMPORAL keyword (which unfortunately might be easily confused with
TEMPORARY...) looks very intuitive and I think using the only time
attribute for versioning would be a good choice.

However, TEMPORAL TABLE on retraction tables do not solve the full problem.
I believe there will be also cases where we need to derive a temporal table
from an append only table (what TemporalTableFunctions do right now).
I think the best choice for this would be TEMPORAL VIEW but as I explained,
it might be a longer way until this can be supported.
TEMPORAL VIEW would also address the problem of preprocessing.

> Regarding retraction table with a primary key and a time-attribute:
> These semantics are still unclear to me. Can retractions only occur
> within watermarks? Or are they also used for representing late updates?

Time attributes and retraction streams are a challenging topic that I
haven't completely understood yet.
So far we treated time attributes always as part of the data.
In combination with retractions, it seems that they become metadata that
specifies when a change was done.
I think this is different from treating time attributes as regular data.

Cheers, Fabian


Am Fr., 17. Apr. 2020 um 17:23 Uhr schrieb Seth Wiesman :

> I really like the TEMPORAL keyword, I find it very intuitive.
>
> The down side of this approach would be that an additional preprocessing
> > step would not be possible anymore because there is no preceding view.
> >
>
>  Yes and no. My understanding is we are not talking about making any
> changes to how temporal tables are defined in the table api. Since you
> cannot currently define temporal table functions in pure SQL applications,
> but only pre-register them in YAML, you can't do any pre-processing as it
> stands today. Preprocessing may be a generally useful feature, I'm not
> sure, but this syntax does not lose us anything in pure SQL applications.
>
> These semantics are still unclear to me. Can retractions only occur
> > within watermarks? Or are they also used for representing late updates?
> >
>
> I do not know the SQL standard well enough to give a principled response to
> this question. However, in my observation of production workloads, users of
> temporal table functions are doing so to denormalize star schemas before
> performing further transformations and aggregations and expect the output
> to be an append stream. With the ongoing work to better support changelogs,
> the need for users to understand the differences in append vs upsert in
> their query may be diminishing but everyone else on this thread can better
> speak to that.
>
> Seth
>
> On Fri, Apr 17, 2020 at 10:03 AM Timo Walther  wrote:
>
> > Hi Fabian,
> >
> > thank you very much for this great summary!
> >
> > I wasn't aware of the Polymorphic Table Functions standard. This is a
> > very interesting topic that we should definitely consider in the future.
> > Maybe this could also help us in defining tables more dynamically within
> > a query. It could help solving problems as discussed in FLIP-113.
> >
> > Regarding joining:
> >
> > IMO we should aim for "FOR SYSTEM_TIME AS OF x" instead of the current
> > `LATERAL TABLE(rates(x))` syntax. A function that also behaves like a
> > table and needs this special `LATERAL` keyword during joining is not
> > very intuitive. The PTF could be used once they are fully supported by
> > Calcite and we have the big picture how to also use them for other
> > time-based operations (windows?, joins?).
> >
> > Regarding how represent a temporal table:
> >
> > I think that our current DDL, current LookupTableSource and temporal
> > tables can fit nicely together.
> >
> > How about we simply introduce an additional keyword `TEMPORAL` to
> > indicate history tracking semantics? I think this is the minimal
> > invasive solution:
> >
> > CREATE TEMPORAL TABLE rates (
> >currency CHAR(3) NOT NULL PRIMARY KEY,
> >rate DOUBLE,
> >rowtime TIMESTAMP,
> >WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE)
> > WITH (...);
> >
> > - The primary key would be defined by the DDL.
> > - The available time attribute would be defined by the DDL. Either as
> > the only time attribute of the table or we introduce a special
> > constraint similar to `PRIMARY KEY`.
> >
> > The down side of this approach would be that an additional preprocessing
> > step would not be possible anymore because there is no preceding view.
> >
> > The `TEMPORAL` semantic can be stored in the properties of the table
> > when writing to a catalog. We do the same for watermarks and computed
> > columns.
> >
> > Without a `TEMPORAL` keyword, a `FOR SYSTEM_TIME AS OF x` would only
> > work on processing time by a lookup into the external system or on
> > event-time by using the time semantics that the external system supports.
> >
> > Regarding retraction table with a primary key and a time-attribute:
> >
> > These semantics are still unclear to me. Can 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-04-17 Thread Seth Wiesman
I really like the TEMPORAL keyword, I find it very intuitive.

The down side of this approach would be that an additional preprocessing
> step would not be possible anymore because there is no preceding view.
>

 Yes and no. My understanding is we are not talking about making any
changes to how temporal tables are defined in the table api. Since you
cannot currently define temporal table functions in pure SQL applications,
but only pre-register them in YAML, you can't do any pre-processing as it
stands today. Preprocessing may be a generally useful feature, I'm not
sure, but this syntax does not lose us anything in pure SQL applications.

These semantics are still unclear to me. Can retractions only occur
> within watermarks? Or are they also used for representing late updates?
>

I do not know the SQL standard well enough to give a principled response to
this question. However, in my observation of production workloads, users of
temporal table functions are doing so to denormalize star schemas before
performing further transformations and aggregations and expect the output
to be an append stream. With the ongoing work to better support changelogs,
the need for users to understand the differences in append vs upsert in
their query may be diminishing but everyone else on this thread can better
speak to that.

Seth

On Fri, Apr 17, 2020 at 10:03 AM Timo Walther  wrote:

> Hi Fabian,
>
> thank you very much for this great summary!
>
> I wasn't aware of the Polymorphic Table Functions standard. This is a
> very interesting topic that we should definitely consider in the future.
> Maybe this could also help us in defining tables more dynamically within
> a query. It could help solving problems as discussed in FLIP-113.
>
> Regarding joining:
>
> IMO we should aim for "FOR SYSTEM_TIME AS OF x" instead of the current
> `LATERAL TABLE(rates(x))` syntax. A function that also behaves like a
> table and needs this special `LATERAL` keyword during joining is not
> very intuitive. The PTF could be used once they are fully supported by
> Calcite and we have the big picture how to also use them for other
> time-based operations (windows?, joins?).
>
> Regarding how represent a temporal table:
>
> I think that our current DDL, current LookupTableSource and temporal
> tables can fit nicely together.
>
> How about we simply introduce an additional keyword `TEMPORAL` to
> indicate history tracking semantics? I think this is the minimal
> invasive solution:
>
> CREATE TEMPORAL TABLE rates (
>currency CHAR(3) NOT NULL PRIMARY KEY,
>rate DOUBLE,
>rowtime TIMESTAMP,
>WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE)
> WITH (...);
>
> - The primary key would be defined by the DDL.
> - The available time attribute would be defined by the DDL. Either as
> the only time attribute of the table or we introduce a special
> constraint similar to `PRIMARY KEY`.
>
> The down side of this approach would be that an additional preprocessing
> step would not be possible anymore because there is no preceding view.
>
> The `TEMPORAL` semantic can be stored in the properties of the table
> when writing to a catalog. We do the same for watermarks and computed
> columns.
>
> Without a `TEMPORAL` keyword, a `FOR SYSTEM_TIME AS OF x` would only
> work on processing time by a lookup into the external system or on
> event-time by using the time semantics that the external system supports.
>
> Regarding retraction table with a primary key and a time-attribute:
>
> These semantics are still unclear to me. Can retractions only occur
> within watermarks? Or are they also used for representing late updates?
>
> Regards,
> Timo
>
>
> On 17.04.20 14:34, Fabian Hueske wrote:
> > Hi all,
> >
> > First of all, I appologize for the text wall that's following... ;-)
> >
> > A temporal table join joins an append-only table and a temporal table.
> > The question about how to represent a temporal table join boils down to
> two
> > questions:
> >
> > 1) How to represent a temporal table
> > 2) How to specify the join of an append-only table and a temporal table
> >
> > I'll discuss these points separately.
> >
> > # 1 How to represent a temporal table
> >
> > A temporal table is a table that can be looked up with a time parameter
> and
> > which returns the rows of the table at that point in time / for that
> > version.
> > In order to be able to (conceptually) look up previous versions, a
> temporal
> > table must be (conceptually) backed by a history table that tracks all
> > previous versions (see SqlServer docs [1]).
> > In the context of our join, we added another restriction namely that the
> > table must have a primary key, i.e., there is only one row for each
> version
> > for each unique key.
> >
> > Hence, the requirements for a temporal table are:
> > * The temporal table has a primary key / unique attribute
> > * The temporal table has a time-attribute that defines the start of the
> > validity interval of a row (processing time or 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-04-17 Thread Fabian Hueske
Hi all,

First of all, I appologize for the text wall that's following... ;-)

A temporal table join joins an append-only table and a temporal table.
The question about how to represent a temporal table join boils down to two
questions:

1) How to represent a temporal table
2) How to specify the join of an append-only table and a temporal table

I'll discuss these points separately.

# 1 How to represent a temporal table

A temporal table is a table that can be looked up with a time parameter and
which returns the rows of the table at that point in time / for that
version.
In order to be able to (conceptually) look up previous versions, a temporal
table must be (conceptually) backed by a history table that tracks all
previous versions (see SqlServer docs [1]).
In the context of our join, we added another restriction namely that the
table must have a primary key, i.e., there is only one row for each version
for each unique key.

Hence, the requirements for a temporal table are:
* The temporal table has a primary key / unique attribute
* The temporal table has a time-attribute that defines the start of the
validity interval of a row (processing time or event time)
* The system knows that the history of the table is tracked and can infer
how to look up a version.

There are two possible types of input from which we want to create temporal
tables (that I'm aware of):

* append-only tables, i.e., tables that contain the full change history
* retraction tables, i.e., tables that are updating and do not remember the
history.

There are a few ways to do this:

## 1.1 Defining a VIEW on an append-only table with a time attribute.

The following view definition results in a view that provides the latest
rate for each currency.

CREATE VIEW rates AS
SELECT
  currency, MAX(rate) as rate, MAX(rowtime) as rowtime
FROM rates_history rh1
WHERE
  rh1.rowtime = (
SELECT max(rowtime)
FROM rates_history rh2
WHERE rh2.curreny = rh1.currency)
GROUP BY currency
WITH (
  'historytracking' = 'true',
  'historytracking.starttime' = 'rowtime');

However, we also need to tell the system to track the history of all
changes of the view in order to be able to look it up.
That's what the properties in the WITH clause are for (inspired by
SqlServer's TEMPORAL TABLE DDL syntax).
Note that this is *not* a syntax proposal but only meant to show which
information is needed.
This view allows to look up any version of the "rates" view.

In addition to designing and implementing the DDL syntax for views that
support temporal lookups, the optimizer would need to understand the
semantics of the view definition in depth.
Among other things it needs to understand that the MAX() aggregation on the
time-attribute preserves its watermark alignment.
AFAIK, this is not the case at the moment (the time attribute would be
converted into a regular TIMESTAMP and lose it's time attribute properties)

## 1.2 A retraction table with a primary key and a time-attribute.

On paper it looks like such a table would automatically qualify as a
time-versioned table because it completely fulfills the requirements.
However, I don't think we can use it *as is* as a temporal table if we want
to have clean semantics.
The problem here is the "lost history" of the retraction table. The dynamic
table that is defined on the retraction stream only stores the latest
version (even though it sees all versions).
Conceptually, a temporal table look up the version of the table at any
point in time because it is backed by a history table.
If this information is not available, we cannot have a semantically clean
definition of the join IMO.

Therefore we should define the table in a way that the system knows that
the history is tracked.
In MSSQL uses a syntax similar to this one

CREATE TABLE rates (
currency CHAR(3) NOT NULL PRIMARY KEY,
rate DOUBLE,
rowtime TIMESTAMP,
WATERMARK FOR rowtime AS rowtime - INTERVAL '5' MINUTE)
WITH (
  'historytracking' = 'true',
  'historytracking.starttime' = 'rowtime');

The 'historytracking' properties would decare that the table tracks its
history and also specify the attribute (rowtime) that is used for
versioning.

## 1.3 Registering a TableFunction that takes an append-only table with
time attribute

The TableFunction requires a few parameters:
* the source table from which to derive the temporal table
* the key attribute on which the versions of the source table should be
computed
* the time attribute that defines the versions
* a lookup timestamp for the version of that is returned.

The reason why we chose the TableFunction approach over the VIEW approach
so far were:
* It is easier for the optimizer to identify a build-in table function than
to analyze and reason about a generic VIEW.
* We would need to make the optimizer a lot smarter to infer all the
properties from the generic VIEW definition that we need for a temporal
table join.
* Passing a parameter to a function is a known thing, passing a parameter
to a 

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-04-16 Thread Jark Wu
Hi Konstantin,

Thanks for bringing this discussion. I think temporal join is a very
important feature and should be exposed to pure SQL users.
And I already received many requirements like this.
However, my concern is that how to properly support this feature in SQL.
Introducing a DDL syntax for Temporal Table Function is one way, but maybe
not the best one.

The most important reason is that the underlying of temporal table function
is exactly a changelog stream.
The temporal join is actually temporal joining a fact stream with the
changelog stream on processing time or event time.
We will soon support to create a changelog source using DDL once FLIP-95
and FLIP-105 is finished.
At that time, we can have a simple DDL to create changelog source like this;

CREATE TABLE rate_changelog (
  currency STRING,
  rate DECIMAL
) WITH (
  'connector' = 'kafka',
  'topic' = 'rate_binlog',
  'properties.bootstrap.servers' = 'localhost:9092',
  'format' = 'debezium-json'
);

In the meanwhile, we already have a SQL standard temporal join syntax [1],
i.e. the "A JOIN B FOR SYSTEM_TIME AS OF ..".
It is currently used as dimension table lookup join, but the semantic is
the same to the "temporal table function join"[2].
I'm in favor of "FOR SYSTEM_TIME AS OF" because it is more nature
becuase the definition of B is a *table* not a *table function*,
and the syntax is included in SQL standard.

So once we have the ability to define "rate_changelog" table, then we can
use the following query to temporal join the changelog on processing time.

SELECT *
FROM orders JOIN rate_changelog FOR SYSTEM_TIME AS OF orders.proctime
ON orders.currency = rate_changelog.currency;

In a nutshell, once FLIP-95 and FLIP-105 is ready, we can easily to support
"temporal join on changelogs" without introducing new syntax.
IMO, introducing a DDL syntax for Temporal Table Function looks like not an
easy way and may have repetitive work.

Best,
Jark

[1]:
https://ci.apache.org/projects/flink/flink-docs-master/dev/table/streaming/joins.html#join-with-a-temporal-table
[2]:
https://ci.apache.org/projects/flink/flink-docs-master/dev/table/streaming/joins.html#join-with-a-temporal-table-function





On Thu, 16 Apr 2020 at 23:04, Benchao Li  wrote:

> Hi Konstantin,
>
> Thanks for bringing up this discussion. +1 for the idea.
> We have met this in our company too, and I planned to support it recently
> in our internal branch.
>
> regarding to your questions,
> 1) I think it might be more a table/view than function, just like Temporal
> Table (which is also known as
> dimension table). Maybe we need a DDL like CREATE VIEW and plus some
> additional settings.
> 2) If we design the DDL for it like view, then maybe temporary is ok
> enough.
>
> Konstantin Knauf  于2020年4月16日周四 下午8:16写道:
>
> > Hi everyone,
> >
> > it would be very useful if temporal tables could be created  via DDL.
> > Currently, users either need to do this in the Table API or in the
> > environment file of the Flink CLI, which both require the user to switch
> > the context of the SQL CLI/Editor. I recently created a ticket for this
> > request [1].
> >
> > I see two main questions:
> >
> > 1) What would be the DDL syntax? A Temporal Table is on the one hand a
> view
> > and on the other a function depending on how you look at it.
> >
> > 2) Would this temporal table view/function be stored in the catalog or
> only
> > be temporary?
> >
> > I personally do not have much experience in this area of Flink, so I am
> > looking forward to hearing your thoughts on this.
> >
> > Best,
> >
> > Konstantin
> >
> > [1] https://issues.apache.org/jira/browse/FLINK-16824
> >
> > --
> >
> > Konstantin Knauf
> >
>
>
> --
>
> Benchao Li
> School of Electronics Engineering and Computer Science, Peking University
> Tel:+86-15650713730
> Email: libenc...@gmail.com; libenc...@pku.edu.cn
>


Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-04-16 Thread Benchao Li
Hi Konstantin,

Thanks for bringing up this discussion. +1 for the idea.
We have met this in our company too, and I planned to support it recently
in our internal branch.

regarding to your questions,
1) I think it might be more a table/view than function, just like Temporal
Table (which is also known as
dimension table). Maybe we need a DDL like CREATE VIEW and plus some
additional settings.
2) If we design the DDL for it like view, then maybe temporary is ok
enough.

Konstantin Knauf  于2020年4月16日周四 下午8:16写道:

> Hi everyone,
>
> it would be very useful if temporal tables could be created  via DDL.
> Currently, users either need to do this in the Table API or in the
> environment file of the Flink CLI, which both require the user to switch
> the context of the SQL CLI/Editor. I recently created a ticket for this
> request [1].
>
> I see two main questions:
>
> 1) What would be the DDL syntax? A Temporal Table is on the one hand a view
> and on the other a function depending on how you look at it.
>
> 2) Would this temporal table view/function be stored in the catalog or only
> be temporary?
>
> I personally do not have much experience in this area of Flink, so I am
> looking forward to hearing your thoughts on this.
>
> Best,
>
> Konstantin
>
> [1] https://issues.apache.org/jira/browse/FLINK-16824
>
> --
>
> Konstantin Knauf
>


-- 

Benchao Li
School of Electronics Engineering and Computer Science, Peking University
Tel:+86-15650713730
Email: libenc...@gmail.com; libenc...@pku.edu.cn