You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Konstantin Knauf <kn...@apache.org> on 2020/05/07 16:03:16 UTC

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

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 <time-attribute>" 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 <im...@gmail.com> 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 <im...@gmail.com> 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 <fh...@gmail.com> 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 <im...@gmail.com>:
> >
> > > 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 <fh...@gmail.com> 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 <
> > > > sjwiesman@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 <tw...@apache.org>
> > > > 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 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 VIEW not so much.
> > > > > > > * Users would need to specify the VIEW exactly correct, such
> that
> > > it
> > > > > can
> > > > > > be
> > > > > > > used as a temporal table. Look at 1.1 why this is not trivial.
> > > > > > >
> > > > > > > There is two ways to use a TableFunction:
> > > > > > >
> > > > > > > ### 1.3.1 Built-in and pre-registered function that is
> > > parameterized
> > > > in
> > > > > > the
> > > > > > > SQL query
> > > > > > >
> > > > > > > Here, we do not need to do anything to register the function.
> We
> > > > simply
> > > > > > use
> > > > > > > it in the query (see example in 2.2 below)
> > > > > > >
> > > > > > > ### 1.3.2 Parameterize function when it is registered in the
> > > catalog
> > > > > > (with
> > > > > > > a provided Java implementation)
> > > > > > >
> > > > > > > This is the approach, we've used so far. In the Table API, the
> > > > function
> > > > > > is
> > > > > > > first parameterized and created and then registered:
> > > > > > > We would need a DDL syntax to parameterize UDFs on
> registration.
> > > > > > > I don't want to propose a syntax here, but just to get an idea
> it
> > > > might
> > > > > > > look like this:
> > > > > > >
> > > > > > > CREATE FUNCTION rates AS
> > > > > > > 'org.apache.flink.table.udfs.TemporalTableFunction' WITH
> > ('table' =
> > > > > > > 'rates_history', 'key' = 'cur', 'time' = 'rowtime')
> > > > > > >
> > > > > > > Right now, the Flink Catalog interface does not have the
> > > > functionality
> > > > > to
> > > > > > > store such parameters and would need some hacks to properly
> > create
> > > > > > properly
> > > > > > > parameterize function instances.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > # 2 Defining a join of an append-only table and a temporal
> table
> > > > > > >
> > > > > > > The append-only table needs to have a time-attribute
> (processing
> > > time
> > > > > or
> > > > > > > event time, but same as the temporal table).
> > > > > > > The join then needs to specify two things:
> > > > > > > * an equality predicate that includes the primary key of the
> > > temporal
> > > > > > table
> > > > > > > * declare the time attribute of the append-only table as the
> time
> > > as
> > > > of
> > > > > > > which to look up the temporal table, i.e, get the version of
> the
> > > > > temporal
> > > > > > > table that is valid for the timestamp of the current row from
> the
> > > > > > > append-only table
> > > > > > >
> > > > > > > The tricky part (from a syntax point of view) is to specify the
> > > > lookup
> > > > > > > time.
> > > > > > >
> > > > > > > ## 2.1 the temporal table is a regular table or view (see
> > > approaches
> > > > > 1.1
> > > > > > > and 1.2 above)
> > > > > > >
> > > > > > > In this case we can use the "FOR SYSTEM_TIME AS OF x" clause as
> > > > > follows:
> > > > > > >
> > > > > > > SELECT *
> > > > > > > FROM orders o, rates r FOR SYSTEM_TIME AS OF o.ordertime
> > > > > > > WHERE o.currency = r.currency
> > > > > > >
> > > > > > > IMO, this is a great syntax and the one we should strive for.
> > > > > > > We would need to bend the rules of the SQL standard which only
> > > > allows x
> > > > > > in
> > > > > > > "FOR SYSTEM_TIME AS OF x" to be a constant and the table on
> which
> > > it
> > > > is
> > > > > > > applied usually needs to be a specific type (not sure if views
> > are
> > > > > > > supported), but I guess this is fine.
> > > > > > > NOTE: the "FOR SYSTEM_TIME AS OF x" is already supported for
> > > > > LookupTable
> > > > > > > Joins if x is a processing time attribute [2].
> > > > > > >
> > > > > > > ## 2.2 the temporal table is a TableFunction and parameterized
> in
> > > the
> > > > > > query
> > > > > > > (see 1.3.1 above)
> > > > > > >
> > > > > > > SELECT *
> > > > > > > FROM orders o,
> > > > > > >    TEMPORAL_TABLE(
> > > > > > >      table => TABLE(rates_history),
> > > > > > >      key => DESCRIPTOR(currency),
> > > > > > >      time => DESCRIPTOR(rowtime)) r
> > > > > > >    ON o.currency = r.currency
> > > > > > >
> > > > > > > The function "TEMPORAL_TABLE" is built-in and nothing was
> > > registered
> > > > in
> > > > > > the
> > > > > > > catalog (except the rates_history table).
> > > > > > > In fact this is valid SQL:2016 syntax and called Polymorphic
> > Table
> > > > > > > Functions. Have a look here [3].
> > > > > > >
> > > > > > > ## 2.3 the temporal table is a TableFunction that was
> > parameterized
> > > > > > during
> > > > > > > registration (see 1.3.2 above)
> > > > > > >
> > > > > > > This is what we have at the momement.
> > > > > > >
> > > > > > > SELECT *
> > > > > > > FROM orders o,
> > > > > > >    LATERAL TABLE (rates(o.ordertime))
> > > > > > >    ON o.currency = r.currency
> > > > > > >
> > > > > > > The TableFunction "rates" was registered in the catalog and
> > > > > parameterized
> > > > > > > to the "rates_history" append-only table, the key was set to
> > > > > "currency",
> > > > > > > and the time attribute was declared.
> > > > > > >
> > > > > > > # SUMMARY
> > > > > > >
> > > > > > > IMO we should in the long run aim to define temporal tables
> > either
> > > as
> > > > > > > upsert retraction tables and views on append-only tables and
> join
> > > > them
> > > > > > > using the "FOR SYSTEM_TIME AS OF x" syntax.
> > > > > > > I guess it is debatable whether we need to decare to track
> > history
> > > > for
> > > > > > > these tables (which we don't actually do) or if we do it by
> > > > convention
> > > > > if
> > > > > > > the table has a time attribute.
> > > > > > > It should be (relatively) easy to get this to work for
> retraction
> > > > > tables
> > > > > > > which will be supported soon.
> > > > > > > It will be more work for views because we need to improve the
> > time
> > > > > > > attribute handling with MAX() aggregations.
> > > > > > > The "FOR SYSTEM_TIME AS OF x" is already supported for
> > > > > LookupTableSources
> > > > > > > and would "only" need to be adapted to work on temporal tables.
> > > > > > >
> > > > > > > Registering parameterized TableFunctions in the catalog seems
> > like
> > > > > quite
> > > > > > a
> > > > > > > bit of work. We need new DDL syntax, extend the catalog and
> > > function
> > > > > > > instantiation. This won't be easy, IMO.
> > > > > > > If we only support them as TEMPORARY FUNCTION which are not
> > > > registered
> > > > > in
> > > > > > > the catalog it will be easier. The question is whether it is
> > worth
> > > > the
> > > > > > > effort if we decide for the other approach.
> > > > > > >
> > > > > > > Using TableFunctions that are parameterized in the query will
> > > require
> > > > > to
> > > > > > > extend the Calcite parser and framework to support Polymorphic
> > > Table
> > > > > > > Functions.
> > > > > > > However, there might already some work be done there, because
> > AFAIK
> > > > > > Apache
> > > > > > > Beam aims to support this syntax for windowing functions as
> > > described
> > > > > in
> > > > > > > the "One SQL to rule them all" paper [4].
> > > > > > > It might be the fastest and fully SQL standard compliant way.
> > > > > > >
> > > > > > > Cheers,
> > > > > > > Fabian
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables
> > > > > > > [2]
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/dev/table/streaming/joins.html#usage-1
> > > > > > > [3]
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://standards.iso.org/ittf/PubliclyAvailableStandards/c069776_ISO_IEC_TR_19075-7_2017.zip
> > > > > > > [4] https://arxiv.org/abs/1905.12133
> > > > > > >
> > > > > > > Am Fr., 17. Apr. 2020 um 06:37 Uhr schrieb Jark Wu <
> > > imjark@gmail.com
> > > > >:
> > > > > > >
> > > > > > >> 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 <libenchao@gmail.com
> >
> > > > 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 <kn...@apache.org> 于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: libenchao@gmail.com; libenchao@pku.edu.cn
> > > > > > >>>
> > > > > > >>
> > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


-- 

Konstantin Knauf

https://twitter.com/snntrable

https://github.com/knaufk

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

Posted by Seth Wiesman <sj...@gmail.com>.
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 <kn...@apache.org> 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 <ji...@gmail.com>
> 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 <im...@gmail.com> 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 <xb...@gmail.com> 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":  1593000011000  // 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*
> > > >
> > > >    1. we should use ‘PERIOD FOR SYSTEM_TIME(operation_time) syntax to
> > > >    track history version by operation time rather than rowtime in
> > > temporal
> > > >    table scenario.
> > > >    2. 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 <xb...@gmail.com> 写道:
> > > >
> > > > 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 <fh...@gmail.com> 写道:
> > > >
> > > > 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 <
> > ykt836@gmail.com
> > > >:
> > > >
> > > > 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 <fh...@gmail.com>
> > 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 <imjark@gmail.com
> >:
> > > >
> > > > 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 <kn...@apache.org>
> > > >
> > > > 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 <yk...@gmail.com>
> > > >
> > > > 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 <tw...@apache.org>
> > > >
> > > > 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
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > > --
> > > >
> > > > Konstantin Knauf
> > > >
> > > > https://twitter.com/snntrable
> > > >
> > > > https://github.com/knaufk
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >
> > >
> >
> >
> > --
> > Best, Jingsong Lee
> >
>
>
> --
>
> Konstantin Knauf
>
> https://twitter.com/snntrable
>
> https://github.com/knaufk
>

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

Posted by Konstantin Knauf <kn...@apache.org>.
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 <ji...@gmail.com> 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 <im...@gmail.com> 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 <xb...@gmail.com> 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":  1593000011000  // 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*
> > >
> > >    1. we should use ‘PERIOD FOR SYSTEM_TIME(operation_time) syntax to
> > >    track history version by operation time rather than rowtime in
> > temporal
> > >    table scenario.
> > >    2. 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 <xb...@gmail.com> 写道:
> > >
> > > 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 <fh...@gmail.com> 写道:
> > >
> > > 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 <
> ykt836@gmail.com
> > >:
> > >
> > > 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 <fh...@gmail.com>
> 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 <im...@gmail.com>:
> > >
> > > 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 <kn...@apache.org>
> > >
> > > 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 <yk...@gmail.com>
> > >
> > > 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 <tw...@apache.org>
> > >
> > > 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
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> > > --
> > >
> > > Konstantin Knauf
> > >
> > > https://twitter.com/snntrable
> > >
> > > https://github.com/knaufk
> > >
> > >
> > >
> > >
> > >
> > >
> > >
> >
>
>
> --
> Best, Jingsong Lee
>


-- 

Konstantin Knauf

https://twitter.com/snntrable

https://github.com/knaufk

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

Posted by Leonard Xu <xb...@gmail.com>.
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 <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 <sj...@gmail.com> 写道:
> 
> * 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 <sj...@gmail.com> 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 <li...@apache.org> 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 <xb...@gmail.com> 于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 <sj...@gmail.com> 写道:
>>>>> 
>>>>> 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 <sjwiesman@gmail.com
>>>> <ma...@gmail.com>> 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 <xbjtdcq@gmail.com <mailto:
>>>> xbjtdcq@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 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
>>>>> 
>>>>> 
>>>> 
>>>> 
>>> 
>>> --
>>> 
>>> Best,
>>> Benchao Li
>>> 
>> 


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

Posted by Seth Wiesman <sj...@gmail.com>.
* 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 <sj...@gmail.com> 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 <li...@apache.org> 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 <xb...@gmail.com> 于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 <sj...@gmail.com> 写道:
>>> >
>>> > 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 <sjwiesman@gmail.com
>>> <ma...@gmail.com>> 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 <xbjtdcq@gmail.com <mailto:
>>> xbjtdcq@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 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
>>> >
>>> >
>>>
>>>
>>
>> --
>>
>> Best,
>> Benchao Li
>>
>

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

Posted by Seth Wiesman <sj...@gmail.com>.
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 <li...@apache.org> 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 <xb...@gmail.com> 于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 <sj...@gmail.com> 写道:
>> >
>> > 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 <sjwiesman@gmail.com
>> <ma...@gmail.com>> 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 <xbjtdcq@gmail.com <mailto:
>> xbjtdcq@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 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
>> >
>> >
>>
>>
>
> --
>
> Best,
> Benchao Li
>

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

Posted by Benchao Li <li...@apache.org>.
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 <xb...@gmail.com> 于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 <sj...@gmail.com> 写道:
> >
> > 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 <sjwiesman@gmail.com
> <ma...@gmail.com>> 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 <xbjtdcq@gmail.com <mailto:
> xbjtdcq@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 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
> >
> >
>
>

-- 

Best,
Benchao Li

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

Posted by Leonard Xu <xb...@gmail.com>.
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 <sj...@gmail.com> 写道:
> 
> 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 <sjwiesman@gmail.com <ma...@gmail.com>> 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 <xbjtdcq@gmail.com <ma...@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 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

Posted by Seth Wiesman <sj...@gmail.com>.
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 <sj...@gmail.com> 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 <xb...@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 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

Posted by Seth Wiesman <sj...@gmail.com>.
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 <xb...@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 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

Posted by Leonard Xu <xb...@gmail.com>.
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

Posted by Konstantin Knauf <kn...@apache.org>.
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 <xb...@gmail.com> 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
> 9999-12-31 23:59:59.9999999 US Dollar 102 2020-06-29 15:06:24.7503288
> 9999-12-31 23:59:59.9999999 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 9999-12-31 23:59:59.9999999
>
> 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
> 9999-12-31 23:59:59.9999999 US Dollar 102 2020-06-29 15:06:24.7503288
> 9999-12-31 23:59:59.9999999 1>* DELETE FROM dbo.currency WHERE currency =
> 'Euro’; * //* DELETE **Euro currency* 1> select * from currency_History;
> currency rate RowStart RowEnd ---------- -----------
> --------------------------------------
> -------------------------------------- Euro 118 2020-06-29 15:07:01.1245406 *2020-06-29
> 15:07:11.2981995* { // *The DELETE record produced by CDC tools(both
> debezuim and canal are same)* "before": { "currency": "Euro", "rate":
> 118, "RowStart": 1593443221124540600, //2020-06-29 15:07:01.1245406
> "RowEnd": -4852116231933722724 //9999-12-31 23:59:59.9999999 }, "after":
> null, "op": "d”, // DELETE operation "ts_ms": 15934432361354, //*2020-06-29
> 15:07:16.354, the ’ts_ms’ value is bigger than the record delete operation
> time(**2020-06-29 15:07:11.2981995**)*
> * "transaction": null* }
>
> The main problem is that the *DELETE* record only contains current table
> message which  does not contain the expected RowEnd (*2020-06-29
> 15:07:11.2981995*) in history table. Without the exact RowEndTime, it’s
> impossible to obtain exact previous version of temporal table in Flink,
> `ts_ms` filed in CDC record is an approximate time of RowEndTime but  it
> depends on CDC tool status and can not equal the RowEndTime from semantics
> angle.
>
> Current Temporal Table Function supports:
> (1) Define temporal table backed upsert data source with process time
> (2) Define temporal table backed upsert data source with event time
> I think  the proposed Temporal table currently could support:
> (1) Define temporal table backed upsert(include delete) data source with
> process time
> (2) Define temporal table backed upsert  data source with event time
> (3) Do not support define temporal table backed  data source that
> contains DELETE message with event time. Because most CDC tools can not
> obtain the exact DELETE operation  time currently, the “ts_ms” field from
> meta is just an approximate time which will break event time semantics.
> And we can support it when CDC tools have the ability to
> obtain/extract the DML operation time.
>
> And this has get consensus from me, Jingsong, Jark and Kurt after offline
> discuss, the opinions from Knauf and Seth looks like same with us.
>
> I’ll prepare a design doc for temporal table, thanks everyone involving
> and please let me know if you have any concern.
>
> Best,
> Leonard Xu
> [1]
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables?view=sql-server-ver15
> [2] https://pgxn.org/dist/temporal_tables/
>
>
>

-- 

Konstantin Knauf

https://twitter.com/snntrable

https://github.com/knaufk

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

Posted by Leonard Xu <xb...@gmail.com>.
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            9999-12-31 23:59:59.9999999
    US Dollar       102            2020-06-29 15:06:24.7503288            9999-12-31 23:59:59.9999999
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            9999-12-31 23:59:59.9999999

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            9999-12-31 23:59:59.9999999
    US Dollar          102            2020-06-29 15:06:24.7503288         9999-12-31 23:59:59.9999999
1> DELETE FROM dbo.currency WHERE currency = 'Euro’;               // DELETE Euro currency
1> select * from currency_History;
    currency   rate        RowStart                               RowEnd
    ---------- ----------- -------------------------------------- --------------------------------------
    Euro               118            2020-06-29 15:07:01.1245406            2020-06-29 15:07:11.2981995

{ // The DELETE record produced by  CDC tools(both debezuim and canal are same) 
    "before": {
      "currency": "Euro",
      "rate": 118,
      "RowStart": 1593443221124540600,  //2020-06-29 15:07:01.1245406
      "RowEnd": -4852116231933722724   //9999-12-31 23:59:59.9999999
    },
    "after": null,
    "op": "d”,  // DELETE operation
    "ts_ms": 15934432361354,                //2020-06-29 15:07:16.354, the ’ts_ms’ value is bigger than the record delete operation time(2020-06-29 15:07:11.2981995)
    "transaction": null
}

The main problem is that the DELETE record only contains current table message which  does not contain the expected RowEnd (2020-06-29 15:07:11.2981995) in history table. Without the exact RowEndTime, it’s impossible to obtain exact previous version of temporal table in Flink, `ts_ms` filed in CDC record is an approximate time of RowEndTime but  it depends on CDC tool status and can not equal the RowEndTime from semantics angle.

Current Temporal Table Function supports:
	(1) Define temporal table backed upsert data source with process time
	(2) Define temporal table backed upsert data source with event time
I think  the proposed Temporal table currently could support:
	(1) Define temporal table backed upsert(include delete) data source with process time
	(2) Define temporal table backed upsert  data source with event time
        (3) Do not support define temporal table backed  data source that contains DELETE message with event time. Because most CDC tools can not obtain the exact DELETE operation  time currently, the “ts_ms” field from meta is just an approximate time which will break event time semantics. And we can support it when CDC tools have the ability to
obtain/extract the DML operation time. 
 
And this has get consensus from me, Jingsong, Jark and Kurt after offline discuss, the opinions from Knauf and Seth looks like same with us. 

I’ll prepare a design doc for temporal table, thanks everyone involving and please let me know if you have any concern.

Best,
Leonard Xu
[1] https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables?view=sql-server-ver15 <https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables?view=sql-server-ver15> 
[2] https://pgxn.org/dist/temporal_tables/ <https://pgxn.org/dist/temporal_tables/> 

 


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

Posted by Jingsong Li <ji...@gmail.com>.
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 <im...@gmail.com> 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 <xb...@gmail.com> 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":  1593000011000  // 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*
> >
> >    1. we should use ‘PERIOD FOR SYSTEM_TIME(operation_time) syntax to
> >    track history version by operation time rather than rowtime in
> temporal
> >    table scenario.
> >    2. 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 <xb...@gmail.com> 写道:
> >
> > 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 <fh...@gmail.com> 写道:
> >
> > 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 <ykt836@gmail.com
> >:
> >
> > 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 <fh...@gmail.com> 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 <im...@gmail.com>:
> >
> > 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 <kn...@apache.org>
> >
> > 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 <yk...@gmail.com>
> >
> > 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 <tw...@apache.org>
> >
> > 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
> >
> >
> >
> >
> >
> >
> >
> > --
> >
> > Konstantin Knauf
> >
> > https://twitter.com/snntrable
> >
> > https://github.com/knaufk
> >
> >
> >
> >
> >
> >
> >
>


-- 
Best, Jingsong Lee

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

Posted by Jark Wu <im...@gmail.com>.
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 <xb...@gmail.com> 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":  1593000011000  // 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*
>
>    1. we should use ‘PERIOD FOR SYSTEM_TIME(operation_time) syntax to
>    track history version by operation time rather than rowtime in temporal
>    table scenario.
>    2. 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 <xb...@gmail.com> 写道:
>
> 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 <fh...@gmail.com> 写道:
>
> 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 <yk...@gmail.com>:
>
> 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 <fh...@gmail.com> 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 <im...@gmail.com>:
>
> 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 <kn...@apache.org>
>
> 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 <yk...@gmail.com>
>
> 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 <tw...@apache.org>
>
> 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
>
>
>
>
>
>
>
> --
>
> Konstantin Knauf
>
> https://twitter.com/snntrable
>
> https://github.com/knaufk
>
>
>
>
>
>
>

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

Posted by Leonard Xu <xb...@gmail.com>.
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":  1593000011000  // 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 <xb...@gmail.com> 写道:
> 
> 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 <fhueske@gmail.com <ma...@gmail.com>> 写道:
>> 
>> 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 <ykt836@gmail.com <ma...@gmail.com>>:
>> 
>>> 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 <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 <fh...@gmail.com> 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 <im...@gmail.com>:
>>>> 
>>>>> 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 <kn...@apache.org>
>>>> 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 <yk...@gmail.com>
>>> 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 <tw...@apache.org>
>>>>> 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
>>>>>>>>> 
>>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> --
>>>>>> 
>>>>>> Konstantin Knauf
>>>>>> 
>>>>>> https://twitter.com/snntrable
>>>>>> 
>>>>>> https://github.com/knaufk
>>>>>> 
>>>>> 
>>>> 
>>> 
> 


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

Posted by Leonard Xu <xb...@gmail.com>.
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 <fh...@gmail.com> 写道:
> 
> 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 <yk...@gmail.com>:
> 
>> 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 <fh...@gmail.com> 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 <im...@gmail.com>:
>>> 
>>>> 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 <kn...@apache.org>
>>> 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 <yk...@gmail.com>
>> 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 <tw...@apache.org>
>>>> 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
>>>>>>>> 
>>>>>>>> 
>>>>>>> 
>>>>>>> 
>>>>>> 
>>>>> 
>>>>> 
>>>>> --
>>>>> 
>>>>> Konstantin Knauf
>>>>> 
>>>>> https://twitter.com/snntrable
>>>>> 
>>>>> https://github.com/knaufk
>>>>> 
>>>> 
>>> 
>> 


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

Posted by Fabian Hueske <fh...@gmail.com>.
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 <yk...@gmail.com>:

> 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 <fh...@gmail.com> 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 <im...@gmail.com>:
> >
> > > 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 <kn...@apache.org>
> > 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 <yk...@gmail.com>
> 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 <tw...@apache.org>
> > > 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
> > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > >
> > > > Konstantin Knauf
> > > >
> > > > https://twitter.com/snntrable
> > > >
> > > > https://github.com/knaufk
> > > >
> > >
> >
>

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

Posted by Kurt Young <yk...@gmail.com>.
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 <fh...@gmail.com> 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 <im...@gmail.com>:
>
> > 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 <kn...@apache.org>
> 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 <yk...@gmail.com> 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 <tw...@apache.org>
> > 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
> > > > > >
> > > > > >
> > > > >
> > > > >
> > > >
> > >
> > >
> > > --
> > >
> > > Konstantin Knauf
> > >
> > > https://twitter.com/snntrable
> > >
> > > https://github.com/knaufk
> > >
> >
>

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

Posted by Fabian Hueske <fh...@gmail.com>.
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 <im...@gmail.com>:

> 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 <kn...@apache.org> 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 <yk...@gmail.com> 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 <tw...@apache.org>
> 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
> > > > >
> > > > >
> > > >
> > > >
> > >
> >
> >
> > --
> >
> > Konstantin Knauf
> >
> > https://twitter.com/snntrable
> >
> > https://github.com/knaufk
> >
>

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

Posted by Jark Wu <im...@gmail.com>.
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 <kn...@apache.org> 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 <yk...@gmail.com> 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 <tw...@apache.org> 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
> > > >
> > > >
> > >
> > >
> >
>
>
> --
>
> Konstantin Knauf
>
> https://twitter.com/snntrable
>
> https://github.com/knaufk
>

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

Posted by Konstantin Knauf <kn...@apache.org>.
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 <yk...@gmail.com> 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 <tw...@apache.org> 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
> > >
> > >
> >
> >
>


-- 

Konstantin Knauf

https://twitter.com/snntrable

https://github.com/knaufk

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

Posted by Kurt Young <yk...@gmail.com>.
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 <tw...@apache.org> 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

Posted by Timo Walther <tw...@apache.org>.
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

Posted by Leonard Xu <xb...@gmail.com>.
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

Posted by Kurt Young <yk...@gmail.com>.
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 <fh...@gmail.com> 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 <yk...@gmail.com>:
>
>> 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 <fh...@gmail.com> 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 <yk...@gmail.com>:
>>>
>>>> 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 <im...@gmail.com> 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 <fh...@gmail.com> 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 <
>>>>> > knaufk@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 <time-attribute>" 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 <im...@gmail.com> 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 <im...@gmail.com> 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 <fh...@gmail.com>
>>>>> 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 <
>>>>> imjark@gmail.com
>>>>> >> >:
>>>>> >> > >
>>>>> >> > > > 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 <
>>>>> fhueske@gmail.com>
>>>>> >> 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 <
>>>>> >> > > > > sjwiesman@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 <
>>>>> >> twalthr@apache.org>
>>>>> >> > > > > 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 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 VIEW not so much.
>>>>> >> > > > > > > > * Users would need to specify the VIEW exactly
>>>>> correct, such
>>>>> >> > that
>>>>> >> > > > it
>>>>> >> > > > > > can
>>>>> >> > > > > > > be
>>>>> >> > > > > > > > used as a temporal table. Look at 1.1 why this is not
>>>>> >> trivial.
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > There is two ways to use a TableFunction:
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > ### 1.3.1 Built-in and pre-registered function that is
>>>>> >> > > > parameterized
>>>>> >> > > > > in
>>>>> >> > > > > > > the
>>>>> >> > > > > > > > SQL query
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > Here, we do not need to do anything to register the
>>>>> >> function.
>>>>> >> > We
>>>>> >> > > > > simply
>>>>> >> > > > > > > use
>>>>> >> > > > > > > > it in the query (see example in 2.2 below)
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > ### 1.3.2 Parameterize function when it is registered
>>>>> in the
>>>>> >> > > > catalog
>>>>> >> > > > > > > (with
>>>>> >> > > > > > > > a provided Java implementation)
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > This is the approach, we've used so far. In the Table
>>>>> API,
>>>>> >> the
>>>>> >> > > > > function
>>>>> >> > > > > > > is
>>>>> >> > > > > > > > first parameterized and created and then registered:
>>>>> >> > > > > > > > We would need a DDL syntax to parameterize UDFs on
>>>>> >> > registration.
>>>>> >> > > > > > > > I don't want to propose a syntax here, but just to
>>>>> get an
>>>>> >> idea
>>>>> >> > it
>>>>> >> > > > > might
>>>>> >> > > > > > > > look like this:
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > CREATE FUNCTION rates AS
>>>>> >> > > > > > > > 'org.apache.flink.table.udfs.TemporalTableFunction'
>>>>> WITH
>>>>> >> > > ('table' =
>>>>> >> > > > > > > > 'rates_history', 'key' = 'cur', 'time' = 'rowtime')
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > Right now, the Flink Catalog interface does not have
>>>>> the
>>>>> >> > > > > functionality
>>>>> >> > > > > > to
>>>>> >> > > > > > > > store such parameters and would need some hacks to
>>>>> properly
>>>>> >> > > create
>>>>> >> > > > > > > properly
>>>>> >> > > > > > > > parameterize function instances.
>>>>> >> > > > > > > >
>>>>> >> > > > > > > >
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > # 2 Defining a join of an append-only table and a
>>>>> temporal
>>>>> >> > table
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > The append-only table needs to have a time-attribute
>>>>> >> > (processing
>>>>> >> > > > time
>>>>> >> > > > > > or
>>>>> >> > > > > > > > event time, but same as the temporal table).
>>>>> >> > > > > > > > The join then needs to specify two things:
>>>>> >> > > > > > > > * an equality predicate that includes the primary key
>>>>> of the
>>>>> >> > > > temporal
>>>>> >> > > > > > > table
>>>>> >> > > > > > > > * declare the time attribute of the append-only table
>>>>> as the
>>>>> >> > time
>>>>> >> > > > as
>>>>> >> > > > > of
>>>>> >> > > > > > > > which to look up the temporal table, i.e, get the
>>>>> version of
>>>>> >> > the
>>>>> >> > > > > > temporal
>>>>> >> > > > > > > > table that is valid for the timestamp of the current
>>>>> row
>>>>> >> from
>>>>> >> > the
>>>>> >> > > > > > > > append-only table
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > The tricky part (from a syntax point of view) is to
>>>>> specify
>>>>> >> the
>>>>> >> > > > > lookup
>>>>> >> > > > > > > > time.
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > ## 2.1 the temporal table is a regular table or view
>>>>> (see
>>>>> >> > > > approaches
>>>>> >> > > > > > 1.1
>>>>> >> > > > > > > > and 1.2 above)
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > In this case we can use the "FOR SYSTEM_TIME AS OF x"
>>>>> >> clause as
>>>>> >> > > > > > follows:
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > SELECT *
>>>>> >> > > > > > > > FROM orders o, rates r FOR SYSTEM_TIME AS OF
>>>>> o.ordertime
>>>>> >> > > > > > > > WHERE o.currency = r.currency
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > IMO, this is a great syntax and the one we should
>>>>> strive
>>>>> >> for.
>>>>> >> > > > > > > > We would need to bend the rules of the SQL standard
>>>>> which
>>>>> >> only
>>>>> >> > > > > allows x
>>>>> >> > > > > > > in
>>>>> >> > > > > > > > "FOR SYSTEM_TIME AS OF x" to be a constant and the
>>>>> table on
>>>>> >> > which
>>>>> >> > > > it
>>>>> >> > > > > is
>>>>> >> > > > > > > > applied usually needs to be a specific type (not sure
>>>>> if
>>>>> >> views
>>>>> >> > > are
>>>>> >> > > > > > > > supported), but I guess this is fine.
>>>>> >> > > > > > > > NOTE: the "FOR SYSTEM_TIME AS OF x" is already
>>>>> supported for
>>>>> >> > > > > > LookupTable
>>>>> >> > > > > > > > Joins if x is a processing time attribute [2].
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > ## 2.2 the temporal table is a TableFunction and
>>>>> >> parameterized
>>>>> >> > in
>>>>> >> > > > the
>>>>> >> > > > > > > query
>>>>> >> > > > > > > > (see 1.3.1 above)
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > SELECT *
>>>>> >> > > > > > > > FROM orders o,
>>>>> >> > > > > > > >    TEMPORAL_TABLE(
>>>>> >> > > > > > > >      table => TABLE(rates_history),
>>>>> >> > > > > > > >      key => DESCRIPTOR(currency),
>>>>> >> > > > > > > >      time => DESCRIPTOR(rowtime)) r
>>>>> >> > > > > > > >    ON o.currency = r.currency
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > The function "TEMPORAL_TABLE" is built-in and nothing
>>>>> was
>>>>> >> > > > registered
>>>>> >> > > > > in
>>>>> >> > > > > > > the
>>>>> >> > > > > > > > catalog (except the rates_history table).
>>>>> >> > > > > > > > In fact this is valid SQL:2016 syntax and called
>>>>> Polymorphic
>>>>> >> > > Table
>>>>> >> > > > > > > > Functions. Have a look here [3].
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > ## 2.3 the temporal table is a TableFunction that was
>>>>> >> > > parameterized
>>>>> >> > > > > > > during
>>>>> >> > > > > > > > registration (see 1.3.2 above)
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > This is what we have at the momement.
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > SELECT *
>>>>> >> > > > > > > > FROM orders o,
>>>>> >> > > > > > > >    LATERAL TABLE (rates(o.ordertime))
>>>>> >> > > > > > > >    ON o.currency = r.currency
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > The TableFunction "rates" was registered in the
>>>>> catalog and
>>>>> >> > > > > > parameterized
>>>>> >> > > > > > > > to the "rates_history" append-only table, the key was
>>>>> set to
>>>>> >> > > > > > "currency",
>>>>> >> > > > > > > > and the time attribute was declared.
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > # SUMMARY
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > IMO we should in the long run aim to define temporal
>>>>> tables
>>>>> >> > > either
>>>>> >> > > > as
>>>>> >> > > > > > > > upsert retraction tables and views on append-only
>>>>> tables and
>>>>> >> > join
>>>>> >> > > > > them
>>>>> >> > > > > > > > using the "FOR SYSTEM_TIME AS OF x" syntax.
>>>>> >> > > > > > > > I guess it is debatable whether we need to decare to
>>>>> track
>>>>> >> > > history
>>>>> >> > > > > for
>>>>> >> > > > > > > > these tables (which we don't actually do) or if we do
>>>>> it by
>>>>> >> > > > > convention
>>>>> >> > > > > > if
>>>>> >> > > > > > > > the table has a time attribute.
>>>>> >> > > > > > > > It should be (relatively) easy to get this to work for
>>>>> >> > retraction
>>>>> >> > > > > > tables
>>>>> >> > > > > > > > which will be supported soon.
>>>>> >> > > > > > > > It will be more work for views because we need to
>>>>> improve
>>>>> >> the
>>>>> >> > > time
>>>>> >> > > > > > > > attribute handling with MAX() aggregations.
>>>>> >> > > > > > > > The "FOR SYSTEM_TIME AS OF x" is already supported for
>>>>> >> > > > > > LookupTableSources
>>>>> >> > > > > > > > and would "only" need to be adapted to work on
>>>>> temporal
>>>>> >> tables.
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > Registering parameterized TableFunctions in the
>>>>> catalog
>>>>> >> seems
>>>>> >> > > like
>>>>> >> > > > > > quite
>>>>> >> > > > > > > a
>>>>> >> > > > > > > > bit of work. We need new DDL syntax, extend the
>>>>> catalog and
>>>>> >> > > > function
>>>>> >> > > > > > > > instantiation. This won't be easy, IMO.
>>>>> >> > > > > > > > If we only support them as TEMPORARY FUNCTION which
>>>>> are not
>>>>> >> > > > > registered
>>>>> >> > > > > > in
>>>>> >> > > > > > > > the catalog it will be easier. The question is
>>>>> whether it is
>>>>> >> > > worth
>>>>> >> > > > > the
>>>>> >> > > > > > > > effort if we decide for the other approach.
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > Using TableFunctions that are parameterized in the
>>>>> query
>>>>> >> will
>>>>> >> > > > require
>>>>> >> > > > > > to
>>>>> >> > > > > > > > extend the Calcite parser and framework to support
>>>>> >> Polymorphic
>>>>> >> > > > Table
>>>>> >> > > > > > > > Functions.
>>>>> >> > > > > > > > However, there might already some work be done there,
>>>>> >> because
>>>>> >> > > AFAIK
>>>>> >> > > > > > > Apache
>>>>> >> > > > > > > > Beam aims to support this syntax for windowing
>>>>> functions as
>>>>> >> > > > described
>>>>> >> > > > > > in
>>>>> >> > > > > > > > the "One SQL to rule them all" paper [4].
>>>>> >> > > > > > > > It might be the fastest and fully SQL standard
>>>>> compliant
>>>>> >> way.
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > Cheers,
>>>>> >> > > > > > > > Fabian
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > [1]
>>>>> >> > > > > > > >
>>>>> >> > > > > > >
>>>>> >> > > > > >
>>>>> >> > > > >
>>>>> >> > > >
>>>>> >> > >
>>>>> >> >
>>>>> >>
>>>>> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables
>>>>> >> > > > > > > > [2]
>>>>> >> > > > > > > >
>>>>> >> > > > > > >
>>>>> >> > > > > >
>>>>> >> > > > >
>>>>> >> > > >
>>>>> >> > >
>>>>> >> >
>>>>> >>
>>>>> https://ci.apache.org/projects/flink/flink-docs-release-1.10/dev/table/streaming/joins.html#usage-1
>>>>> >> > > > > > > > [3]
>>>>> >> > > > > > > >
>>>>> >> > > > > > >
>>>>> >> > > > > >
>>>>> >> > > > >
>>>>> >> > > >
>>>>> >> > >
>>>>> >> >
>>>>> >>
>>>>> https://standards.iso.org/ittf/PubliclyAvailableStandards/c069776_ISO_IEC_TR_19075-7_2017.zip
>>>>> >> > > > > > > > [4] https://arxiv.org/abs/1905.12133
>>>>> >> > > > > > > >
>>>>> >> > > > > > > > Am Fr., 17. Apr. 2020 um 06:37 Uhr schrieb Jark Wu <
>>>>> >> > > > imjark@gmail.com
>>>>> >> > > > > >:
>>>>> >> > > > > > > >
>>>>> >> > > > > > > >> 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 <
>>>>> >> libenchao@gmail.com
>>>>> >> > >
>>>>> >> > > > > 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 <kn...@apache.org> 于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: libenchao@gmail.com; libenchao@pku.edu.cn
>>>>> >> > > > > > > >>>
>>>>> >> > > > > > > >>
>>>>> >> > > > > > > >
>>>>> >> > > > > > >
>>>>> >> > > > > > >
>>>>> >> > > > > >
>>>>> >> > > > >
>>>>> >> > > >
>>>>> >> > >
>>>>> >> >
>>>>> >>
>>>>> >>
>>>>> >> --
>>>>> >>
>>>>> >> Konstantin Knauf
>>>>> >>
>>>>> >> https://twitter.com/snntrable
>>>>> >>
>>>>> >> https://github.com/knaufk
>>>>> >>
>>>>> >
>>>>>
>>>>

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

Posted by Fabian Hueske <fh...@gmail.com>.
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 <yk...@gmail.com>:

> 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 <fh...@gmail.com> 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 <yk...@gmail.com>:
>>
>>> 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 <im...@gmail.com> 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 <fh...@gmail.com> 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 <
>>>> > knaufk@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 <time-attribute>" 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 <im...@gmail.com> 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 <im...@gmail.com> 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 <fh...@gmail.com>
>>>> 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 <
>>>> imjark@gmail.com
>>>> >> >:
>>>> >> > >
>>>> >> > > > 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 <fhueske@gmail.com
>>>> >
>>>> >> 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 <
>>>> >> > > > > sjwiesman@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 <
>>>> >> twalthr@apache.org>
>>>> >> > > > > 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 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 VIEW not so much.
>>>> >> > > > > > > > * Users would need to specify the VIEW exactly
>>>> correct, such
>>>> >> > that
>>>> >> > > > it
>>>> >> > > > > > can
>>>> >> > > > > > > be
>>>> >> > > > > > > > used as a temporal table. Look at 1.1 why this is not
>>>> >> trivial.
>>>> >> > > > > > > >
>>>> >> > > > > > > > There is two ways to use a TableFunction:
>>>> >> > > > > > > >
>>>> >> > > > > > > > ### 1.3.1 Built-in and pre-registered function that is
>>>> >> > > > parameterized
>>>> >> > > > > in
>>>> >> > > > > > > the
>>>> >> > > > > > > > SQL query
>>>> >> > > > > > > >
>>>> >> > > > > > > > Here, we do not need to do anything to register the
>>>> >> function.
>>>> >> > We
>>>> >> > > > > simply
>>>> >> > > > > > > use
>>>> >> > > > > > > > it in the query (see example in 2.2 below)
>>>> >> > > > > > > >
>>>> >> > > > > > > > ### 1.3.2 Parameterize function when it is registered
>>>> in the
>>>> >> > > > catalog
>>>> >> > > > > > > (with
>>>> >> > > > > > > > a provided Java implementation)
>>>> >> > > > > > > >
>>>> >> > > > > > > > This is the approach, we've used so far. In the Table
>>>> API,
>>>> >> the
>>>> >> > > > > function
>>>> >> > > > > > > is
>>>> >> > > > > > > > first parameterized and created and then registered:
>>>> >> > > > > > > > We would need a DDL syntax to parameterize UDFs on
>>>> >> > registration.
>>>> >> > > > > > > > I don't want to propose a syntax here, but just to get
>>>> an
>>>> >> idea
>>>> >> > it
>>>> >> > > > > might
>>>> >> > > > > > > > look like this:
>>>> >> > > > > > > >
>>>> >> > > > > > > > CREATE FUNCTION rates AS
>>>> >> > > > > > > > 'org.apache.flink.table.udfs.TemporalTableFunction'
>>>> WITH
>>>> >> > > ('table' =
>>>> >> > > > > > > > 'rates_history', 'key' = 'cur', 'time' = 'rowtime')
>>>> >> > > > > > > >
>>>> >> > > > > > > > Right now, the Flink Catalog interface does not have
>>>> the
>>>> >> > > > > functionality
>>>> >> > > > > > to
>>>> >> > > > > > > > store such parameters and would need some hacks to
>>>> properly
>>>> >> > > create
>>>> >> > > > > > > properly
>>>> >> > > > > > > > parameterize function instances.
>>>> >> > > > > > > >
>>>> >> > > > > > > >
>>>> >> > > > > > > >
>>>> >> > > > > > > > # 2 Defining a join of an append-only table and a
>>>> temporal
>>>> >> > table
>>>> >> > > > > > > >
>>>> >> > > > > > > > The append-only table needs to have a time-attribute
>>>> >> > (processing
>>>> >> > > > time
>>>> >> > > > > > or
>>>> >> > > > > > > > event time, but same as the temporal table).
>>>> >> > > > > > > > The join then needs to specify two things:
>>>> >> > > > > > > > * an equality predicate that includes the primary key
>>>> of the
>>>> >> > > > temporal
>>>> >> > > > > > > table
>>>> >> > > > > > > > * declare the time attribute of the append-only table
>>>> as the
>>>> >> > time
>>>> >> > > > as
>>>> >> > > > > of
>>>> >> > > > > > > > which to look up the temporal table, i.e, get the
>>>> version of
>>>> >> > the
>>>> >> > > > > > temporal
>>>> >> > > > > > > > table that is valid for the timestamp of the current
>>>> row
>>>> >> from
>>>> >> > the
>>>> >> > > > > > > > append-only table
>>>> >> > > > > > > >
>>>> >> > > > > > > > The tricky part (from a syntax point of view) is to
>>>> specify
>>>> >> the
>>>> >> > > > > lookup
>>>> >> > > > > > > > time.
>>>> >> > > > > > > >
>>>> >> > > > > > > > ## 2.1 the temporal table is a regular table or view
>>>> (see
>>>> >> > > > approaches
>>>> >> > > > > > 1.1
>>>> >> > > > > > > > and 1.2 above)
>>>> >> > > > > > > >
>>>> >> > > > > > > > In this case we can use the "FOR SYSTEM_TIME AS OF x"
>>>> >> clause as
>>>> >> > > > > > follows:
>>>> >> > > > > > > >
>>>> >> > > > > > > > SELECT *
>>>> >> > > > > > > > FROM orders o, rates r FOR SYSTEM_TIME AS OF
>>>> o.ordertime
>>>> >> > > > > > > > WHERE o.currency = r.currency
>>>> >> > > > > > > >
>>>> >> > > > > > > > IMO, this is a great syntax and the one we should
>>>> strive
>>>> >> for.
>>>> >> > > > > > > > We would need to bend the rules of the SQL standard
>>>> which
>>>> >> only
>>>> >> > > > > allows x
>>>> >> > > > > > > in
>>>> >> > > > > > > > "FOR SYSTEM_TIME AS OF x" to be a constant and the
>>>> table on
>>>> >> > which
>>>> >> > > > it
>>>> >> > > > > is
>>>> >> > > > > > > > applied usually needs to be a specific type (not sure
>>>> if
>>>> >> views
>>>> >> > > are
>>>> >> > > > > > > > supported), but I guess this is fine.
>>>> >> > > > > > > > NOTE: the "FOR SYSTEM_TIME AS OF x" is already
>>>> supported for
>>>> >> > > > > > LookupTable
>>>> >> > > > > > > > Joins if x is a processing time attribute [2].
>>>> >> > > > > > > >
>>>> >> > > > > > > > ## 2.2 the temporal table is a TableFunction and
>>>> >> parameterized
>>>> >> > in
>>>> >> > > > the
>>>> >> > > > > > > query
>>>> >> > > > > > > > (see 1.3.1 above)
>>>> >> > > > > > > >
>>>> >> > > > > > > > SELECT *
>>>> >> > > > > > > > FROM orders o,
>>>> >> > > > > > > >    TEMPORAL_TABLE(
>>>> >> > > > > > > >      table => TABLE(rates_history),
>>>> >> > > > > > > >      key => DESCRIPTOR(currency),
>>>> >> > > > > > > >      time => DESCRIPTOR(rowtime)) r
>>>> >> > > > > > > >    ON o.currency = r.currency
>>>> >> > > > > > > >
>>>> >> > > > > > > > The function "TEMPORAL_TABLE" is built-in and nothing
>>>> was
>>>> >> > > > registered
>>>> >> > > > > in
>>>> >> > > > > > > the
>>>> >> > > > > > > > catalog (except the rates_history table).
>>>> >> > > > > > > > In fact this is valid SQL:2016 syntax and called
>>>> Polymorphic
>>>> >> > > Table
>>>> >> > > > > > > > Functions. Have a look here [3].
>>>> >> > > > > > > >
>>>> >> > > > > > > > ## 2.3 the temporal table is a TableFunction that was
>>>> >> > > parameterized
>>>> >> > > > > > > during
>>>> >> > > > > > > > registration (see 1.3.2 above)
>>>> >> > > > > > > >
>>>> >> > > > > > > > This is what we have at the momement.
>>>> >> > > > > > > >
>>>> >> > > > > > > > SELECT *
>>>> >> > > > > > > > FROM orders o,
>>>> >> > > > > > > >    LATERAL TABLE (rates(o.ordertime))
>>>> >> > > > > > > >    ON o.currency = r.currency
>>>> >> > > > > > > >
>>>> >> > > > > > > > The TableFunction "rates" was registered in the
>>>> catalog and
>>>> >> > > > > > parameterized
>>>> >> > > > > > > > to the "rates_history" append-only table, the key was
>>>> set to
>>>> >> > > > > > "currency",
>>>> >> > > > > > > > and the time attribute was declared.
>>>> >> > > > > > > >
>>>> >> > > > > > > > # SUMMARY
>>>> >> > > > > > > >
>>>> >> > > > > > > > IMO we should in the long run aim to define temporal
>>>> tables
>>>> >> > > either
>>>> >> > > > as
>>>> >> > > > > > > > upsert retraction tables and views on append-only
>>>> tables and
>>>> >> > join
>>>> >> > > > > them
>>>> >> > > > > > > > using the "FOR SYSTEM_TIME AS OF x" syntax.
>>>> >> > > > > > > > I guess it is debatable whether we need to decare to
>>>> track
>>>> >> > > history
>>>> >> > > > > for
>>>> >> > > > > > > > these tables (which we don't actually do) or if we do
>>>> it by
>>>> >> > > > > convention
>>>> >> > > > > > if
>>>> >> > > > > > > > the table has a time attribute.
>>>> >> > > > > > > > It should be (relatively) easy to get this to work for
>>>> >> > retraction
>>>> >> > > > > > tables
>>>> >> > > > > > > > which will be supported soon.
>>>> >> > > > > > > > It will be more work for views because we need to
>>>> improve
>>>> >> the
>>>> >> > > time
>>>> >> > > > > > > > attribute handling with MAX() aggregations.
>>>> >> > > > > > > > The "FOR SYSTEM_TIME AS OF x" is already supported for
>>>> >> > > > > > LookupTableSources
>>>> >> > > > > > > > and would "only" need to be adapted to work on temporal
>>>> >> tables.
>>>> >> > > > > > > >
>>>> >> > > > > > > > Registering parameterized TableFunctions in the catalog
>>>> >> seems
>>>> >> > > like
>>>> >> > > > > > quite
>>>> >> > > > > > > a
>>>> >> > > > > > > > bit of work. We need new DDL syntax, extend the
>>>> catalog and
>>>> >> > > > function
>>>> >> > > > > > > > instantiation. This won't be easy, IMO.
>>>> >> > > > > > > > If we only support them as TEMPORARY FUNCTION which
>>>> are not
>>>> >> > > > > registered
>>>> >> > > > > > in
>>>> >> > > > > > > > the catalog it will be easier. The question is whether
>>>> it is
>>>> >> > > worth
>>>> >> > > > > the
>>>> >> > > > > > > > effort if we decide for the other approach.
>>>> >> > > > > > > >
>>>> >> > > > > > > > Using TableFunctions that are parameterized in the
>>>> query
>>>> >> will
>>>> >> > > > require
>>>> >> > > > > > to
>>>> >> > > > > > > > extend the Calcite parser and framework to support
>>>> >> Polymorphic
>>>> >> > > > Table
>>>> >> > > > > > > > Functions.
>>>> >> > > > > > > > However, there might already some work be done there,
>>>> >> because
>>>> >> > > AFAIK
>>>> >> > > > > > > Apache
>>>> >> > > > > > > > Beam aims to support this syntax for windowing
>>>> functions as
>>>> >> > > > described
>>>> >> > > > > > in
>>>> >> > > > > > > > the "One SQL to rule them all" paper [4].
>>>> >> > > > > > > > It might be the fastest and fully SQL standard
>>>> compliant
>>>> >> way.
>>>> >> > > > > > > >
>>>> >> > > > > > > > Cheers,
>>>> >> > > > > > > > Fabian
>>>> >> > > > > > > >
>>>> >> > > > > > > > [1]
>>>> >> > > > > > > >
>>>> >> > > > > > >
>>>> >> > > > > >
>>>> >> > > > >
>>>> >> > > >
>>>> >> > >
>>>> >> >
>>>> >>
>>>> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables
>>>> >> > > > > > > > [2]
>>>> >> > > > > > > >
>>>> >> > > > > > >
>>>> >> > > > > >
>>>> >> > > > >
>>>> >> > > >
>>>> >> > >
>>>> >> >
>>>> >>
>>>> https://ci.apache.org/projects/flink/flink-docs-release-1.10/dev/table/streaming/joins.html#usage-1
>>>> >> > > > > > > > [3]
>>>> >> > > > > > > >
>>>> >> > > > > > >
>>>> >> > > > > >
>>>> >> > > > >
>>>> >> > > >
>>>> >> > >
>>>> >> >
>>>> >>
>>>> https://standards.iso.org/ittf/PubliclyAvailableStandards/c069776_ISO_IEC_TR_19075-7_2017.zip
>>>> >> > > > > > > > [4] https://arxiv.org/abs/1905.12133
>>>> >> > > > > > > >
>>>> >> > > > > > > > Am Fr., 17. Apr. 2020 um 06:37 Uhr schrieb Jark Wu <
>>>> >> > > > imjark@gmail.com
>>>> >> > > > > >:
>>>> >> > > > > > > >
>>>> >> > > > > > > >> 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 <
>>>> >> libenchao@gmail.com
>>>> >> > >
>>>> >> > > > > 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 <kn...@apache.org> 于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: libenchao@gmail.com; libenchao@pku.edu.cn
>>>> >> > > > > > > >>>
>>>> >> > > > > > > >>
>>>> >> > > > > > > >
>>>> >> > > > > > >
>>>> >> > > > > > >
>>>> >> > > > > >
>>>> >> > > > >
>>>> >> > > >
>>>> >> > >
>>>> >> >
>>>> >>
>>>> >>
>>>> >> --
>>>> >>
>>>> >> Konstantin Knauf
>>>> >>
>>>> >> https://twitter.com/snntrable
>>>> >>
>>>> >> https://github.com/knaufk
>>>> >>
>>>> >
>>>>
>>>

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

Posted by Kurt Young <yk...@gmail.com>.
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 <fh...@gmail.com> 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 <yk...@gmail.com>:
>
>> 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 <im...@gmail.com> 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 <fh...@gmail.com> 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 <
>>> > knaufk@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 <time-attribute>" 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 <im...@gmail.com> 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 <im...@gmail.com> 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 <fh...@gmail.com>
>>> 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 <
>>> imjark@gmail.com
>>> >> >:
>>> >> > >
>>> >> > > > 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 <fh...@gmail.com>
>>> >> 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 <
>>> >> > > > > sjwiesman@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 <
>>> >> twalthr@apache.org>
>>> >> > > > > 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 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 VIEW not so much.
>>> >> > > > > > > > * Users would need to specify the VIEW exactly correct,
>>> such
>>> >> > that
>>> >> > > > it
>>> >> > > > > > can
>>> >> > > > > > > be
>>> >> > > > > > > > used as a temporal table. Look at 1.1 why this is not
>>> >> trivial.
>>> >> > > > > > > >
>>> >> > > > > > > > There is two ways to use a TableFunction:
>>> >> > > > > > > >
>>> >> > > > > > > > ### 1.3.1 Built-in and pre-registered function that is
>>> >> > > > parameterized
>>> >> > > > > in
>>> >> > > > > > > the
>>> >> > > > > > > > SQL query
>>> >> > > > > > > >
>>> >> > > > > > > > Here, we do not need to do anything to register the
>>> >> function.
>>> >> > We
>>> >> > > > > simply
>>> >> > > > > > > use
>>> >> > > > > > > > it in the query (see example in 2.2 below)
>>> >> > > > > > > >
>>> >> > > > > > > > ### 1.3.2 Parameterize function when it is registered
>>> in the
>>> >> > > > catalog
>>> >> > > > > > > (with
>>> >> > > > > > > > a provided Java implementation)
>>> >> > > > > > > >
>>> >> > > > > > > > This is the approach, we've used so far. In the Table
>>> API,
>>> >> the
>>> >> > > > > function
>>> >> > > > > > > is
>>> >> > > > > > > > first parameterized and created and then registered:
>>> >> > > > > > > > We would need a DDL syntax to parameterize UDFs on
>>> >> > registration.
>>> >> > > > > > > > I don't want to propose a syntax here, but just to get
>>> an
>>> >> idea
>>> >> > it
>>> >> > > > > might
>>> >> > > > > > > > look like this:
>>> >> > > > > > > >
>>> >> > > > > > > > CREATE FUNCTION rates AS
>>> >> > > > > > > > 'org.apache.flink.table.udfs.TemporalTableFunction' WITH
>>> >> > > ('table' =
>>> >> > > > > > > > 'rates_history', 'key' = 'cur', 'time' = 'rowtime')
>>> >> > > > > > > >
>>> >> > > > > > > > Right now, the Flink Catalog interface does not have the
>>> >> > > > > functionality
>>> >> > > > > > to
>>> >> > > > > > > > store such parameters and would need some hacks to
>>> properly
>>> >> > > create
>>> >> > > > > > > properly
>>> >> > > > > > > > parameterize function instances.
>>> >> > > > > > > >
>>> >> > > > > > > >
>>> >> > > > > > > >
>>> >> > > > > > > > # 2 Defining a join of an append-only table and a
>>> temporal
>>> >> > table
>>> >> > > > > > > >
>>> >> > > > > > > > The append-only table needs to have a time-attribute
>>> >> > (processing
>>> >> > > > time
>>> >> > > > > > or
>>> >> > > > > > > > event time, but same as the temporal table).
>>> >> > > > > > > > The join then needs to specify two things:
>>> >> > > > > > > > * an equality predicate that includes the primary key
>>> of the
>>> >> > > > temporal
>>> >> > > > > > > table
>>> >> > > > > > > > * declare the time attribute of the append-only table
>>> as the
>>> >> > time
>>> >> > > > as
>>> >> > > > > of
>>> >> > > > > > > > which to look up the temporal table, i.e, get the
>>> version of
>>> >> > the
>>> >> > > > > > temporal
>>> >> > > > > > > > table that is valid for the timestamp of the current row
>>> >> from
>>> >> > the
>>> >> > > > > > > > append-only table
>>> >> > > > > > > >
>>> >> > > > > > > > The tricky part (from a syntax point of view) is to
>>> specify
>>> >> the
>>> >> > > > > lookup
>>> >> > > > > > > > time.
>>> >> > > > > > > >
>>> >> > > > > > > > ## 2.1 the temporal table is a regular table or view
>>> (see
>>> >> > > > approaches
>>> >> > > > > > 1.1
>>> >> > > > > > > > and 1.2 above)
>>> >> > > > > > > >
>>> >> > > > > > > > In this case we can use the "FOR SYSTEM_TIME AS OF x"
>>> >> clause as
>>> >> > > > > > follows:
>>> >> > > > > > > >
>>> >> > > > > > > > SELECT *
>>> >> > > > > > > > FROM orders o, rates r FOR SYSTEM_TIME AS OF o.ordertime
>>> >> > > > > > > > WHERE o.currency = r.currency
>>> >> > > > > > > >
>>> >> > > > > > > > IMO, this is a great syntax and the one we should strive
>>> >> for.
>>> >> > > > > > > > We would need to bend the rules of the SQL standard
>>> which
>>> >> only
>>> >> > > > > allows x
>>> >> > > > > > > in
>>> >> > > > > > > > "FOR SYSTEM_TIME AS OF x" to be a constant and the
>>> table on
>>> >> > which
>>> >> > > > it
>>> >> > > > > is
>>> >> > > > > > > > applied usually needs to be a specific type (not sure if
>>> >> views
>>> >> > > are
>>> >> > > > > > > > supported), but I guess this is fine.
>>> >> > > > > > > > NOTE: the "FOR SYSTEM_TIME AS OF x" is already
>>> supported for
>>> >> > > > > > LookupTable
>>> >> > > > > > > > Joins if x is a processing time attribute [2].
>>> >> > > > > > > >
>>> >> > > > > > > > ## 2.2 the temporal table is a TableFunction and
>>> >> parameterized
>>> >> > in
>>> >> > > > the
>>> >> > > > > > > query
>>> >> > > > > > > > (see 1.3.1 above)
>>> >> > > > > > > >
>>> >> > > > > > > > SELECT *
>>> >> > > > > > > > FROM orders o,
>>> >> > > > > > > >    TEMPORAL_TABLE(
>>> >> > > > > > > >      table => TABLE(rates_history),
>>> >> > > > > > > >      key => DESCRIPTOR(currency),
>>> >> > > > > > > >      time => DESCRIPTOR(rowtime)) r
>>> >> > > > > > > >    ON o.currency = r.currency
>>> >> > > > > > > >
>>> >> > > > > > > > The function "TEMPORAL_TABLE" is built-in and nothing
>>> was
>>> >> > > > registered
>>> >> > > > > in
>>> >> > > > > > > the
>>> >> > > > > > > > catalog (except the rates_history table).
>>> >> > > > > > > > In fact this is valid SQL:2016 syntax and called
>>> Polymorphic
>>> >> > > Table
>>> >> > > > > > > > Functions. Have a look here [3].
>>> >> > > > > > > >
>>> >> > > > > > > > ## 2.3 the temporal table is a TableFunction that was
>>> >> > > parameterized
>>> >> > > > > > > during
>>> >> > > > > > > > registration (see 1.3.2 above)
>>> >> > > > > > > >
>>> >> > > > > > > > This is what we have at the momement.
>>> >> > > > > > > >
>>> >> > > > > > > > SELECT *
>>> >> > > > > > > > FROM orders o,
>>> >> > > > > > > >    LATERAL TABLE (rates(o.ordertime))
>>> >> > > > > > > >    ON o.currency = r.currency
>>> >> > > > > > > >
>>> >> > > > > > > > The TableFunction "rates" was registered in the catalog
>>> and
>>> >> > > > > > parameterized
>>> >> > > > > > > > to the "rates_history" append-only table, the key was
>>> set to
>>> >> > > > > > "currency",
>>> >> > > > > > > > and the time attribute was declared.
>>> >> > > > > > > >
>>> >> > > > > > > > # SUMMARY
>>> >> > > > > > > >
>>> >> > > > > > > > IMO we should in the long run aim to define temporal
>>> tables
>>> >> > > either
>>> >> > > > as
>>> >> > > > > > > > upsert retraction tables and views on append-only
>>> tables and
>>> >> > join
>>> >> > > > > them
>>> >> > > > > > > > using the "FOR SYSTEM_TIME AS OF x" syntax.
>>> >> > > > > > > > I guess it is debatable whether we need to decare to
>>> track
>>> >> > > history
>>> >> > > > > for
>>> >> > > > > > > > these tables (which we don't actually do) or if we do
>>> it by
>>> >> > > > > convention
>>> >> > > > > > if
>>> >> > > > > > > > the table has a time attribute.
>>> >> > > > > > > > It should be (relatively) easy to get this to work for
>>> >> > retraction
>>> >> > > > > > tables
>>> >> > > > > > > > which will be supported soon.
>>> >> > > > > > > > It will be more work for views because we need to
>>> improve
>>> >> the
>>> >> > > time
>>> >> > > > > > > > attribute handling with MAX() aggregations.
>>> >> > > > > > > > The "FOR SYSTEM_TIME AS OF x" is already supported for
>>> >> > > > > > LookupTableSources
>>> >> > > > > > > > and would "only" need to be adapted to work on temporal
>>> >> tables.
>>> >> > > > > > > >
>>> >> > > > > > > > Registering parameterized TableFunctions in the catalog
>>> >> seems
>>> >> > > like
>>> >> > > > > > quite
>>> >> > > > > > > a
>>> >> > > > > > > > bit of work. We need new DDL syntax, extend the catalog
>>> and
>>> >> > > > function
>>> >> > > > > > > > instantiation. This won't be easy, IMO.
>>> >> > > > > > > > If we only support them as TEMPORARY FUNCTION which are
>>> not
>>> >> > > > > registered
>>> >> > > > > > in
>>> >> > > > > > > > the catalog it will be easier. The question is whether
>>> it is
>>> >> > > worth
>>> >> > > > > the
>>> >> > > > > > > > effort if we decide for the other approach.
>>> >> > > > > > > >
>>> >> > > > > > > > Using TableFunctions that are parameterized in the query
>>> >> will
>>> >> > > > require
>>> >> > > > > > to
>>> >> > > > > > > > extend the Calcite parser and framework to support
>>> >> Polymorphic
>>> >> > > > Table
>>> >> > > > > > > > Functions.
>>> >> > > > > > > > However, there might already some work be done there,
>>> >> because
>>> >> > > AFAIK
>>> >> > > > > > > Apache
>>> >> > > > > > > > Beam aims to support this syntax for windowing
>>> functions as
>>> >> > > > described
>>> >> > > > > > in
>>> >> > > > > > > > the "One SQL to rule them all" paper [4].
>>> >> > > > > > > > It might be the fastest and fully SQL standard compliant
>>> >> way.
>>> >> > > > > > > >
>>> >> > > > > > > > Cheers,
>>> >> > > > > > > > Fabian
>>> >> > > > > > > >
>>> >> > > > > > > > [1]
>>> >> > > > > > > >
>>> >> > > > > > >
>>> >> > > > > >
>>> >> > > > >
>>> >> > > >
>>> >> > >
>>> >> >
>>> >>
>>> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables
>>> >> > > > > > > > [2]
>>> >> > > > > > > >
>>> >> > > > > > >
>>> >> > > > > >
>>> >> > > > >
>>> >> > > >
>>> >> > >
>>> >> >
>>> >>
>>> https://ci.apache.org/projects/flink/flink-docs-release-1.10/dev/table/streaming/joins.html#usage-1
>>> >> > > > > > > > [3]
>>> >> > > > > > > >
>>> >> > > > > > >
>>> >> > > > > >
>>> >> > > > >
>>> >> > > >
>>> >> > >
>>> >> >
>>> >>
>>> https://standards.iso.org/ittf/PubliclyAvailableStandards/c069776_ISO_IEC_TR_19075-7_2017.zip
>>> >> > > > > > > > [4] https://arxiv.org/abs/1905.12133
>>> >> > > > > > > >
>>> >> > > > > > > > Am Fr., 17. Apr. 2020 um 06:37 Uhr schrieb Jark Wu <
>>> >> > > > imjark@gmail.com
>>> >> > > > > >:
>>> >> > > > > > > >
>>> >> > > > > > > >> 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 <
>>> >> libenchao@gmail.com
>>> >> > >
>>> >> > > > > 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 <kn...@apache.org> 于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: libenchao@gmail.com; libenchao@pku.edu.cn
>>> >> > > > > > > >>>
>>> >> > > > > > > >>
>>> >> > > > > > > >
>>> >> > > > > > >
>>> >> > > > > > >
>>> >> > > > > >
>>> >> > > > >
>>> >> > > >
>>> >> > >
>>> >> >
>>> >>
>>> >>
>>> >> --
>>> >>
>>> >> Konstantin Knauf
>>> >>
>>> >> https://twitter.com/snntrable
>>> >>
>>> >> https://github.com/knaufk
>>> >>
>>> >
>>>
>>

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

Posted by Fabian Hueske <fh...@gmail.com>.
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 <yk...@gmail.com>:

> 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 <im...@gmail.com> 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 <fh...@gmail.com> 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 <
>> > knaufk@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 <time-attribute>" 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 <im...@gmail.com> 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 <im...@gmail.com> 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 <fh...@gmail.com>
>> 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 <
>> imjark@gmail.com
>> >> >:
>> >> > >
>> >> > > > 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 <fh...@gmail.com>
>> >> 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 <
>> >> > > > > sjwiesman@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 <
>> >> twalthr@apache.org>
>> >> > > > > 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 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 VIEW not so much.
>> >> > > > > > > > * Users would need to specify the VIEW exactly correct,
>> such
>> >> > that
>> >> > > > it
>> >> > > > > > can
>> >> > > > > > > be
>> >> > > > > > > > used as a temporal table. Look at 1.1 why this is not
>> >> trivial.
>> >> > > > > > > >
>> >> > > > > > > > There is two ways to use a TableFunction:
>> >> > > > > > > >
>> >> > > > > > > > ### 1.3.1 Built-in and pre-registered function that is
>> >> > > > parameterized
>> >> > > > > in
>> >> > > > > > > the
>> >> > > > > > > > SQL query
>> >> > > > > > > >
>> >> > > > > > > > Here, we do not need to do anything to register the
>> >> function.
>> >> > We
>> >> > > > > simply
>> >> > > > > > > use
>> >> > > > > > > > it in the query (see example in 2.2 below)
>> >> > > > > > > >
>> >> > > > > > > > ### 1.3.2 Parameterize function when it is registered in
>> the
>> >> > > > catalog
>> >> > > > > > > (with
>> >> > > > > > > > a provided Java implementation)
>> >> > > > > > > >
>> >> > > > > > > > This is the approach, we've used so far. In the Table
>> API,
>> >> the
>> >> > > > > function
>> >> > > > > > > is
>> >> > > > > > > > first parameterized and created and then registered:
>> >> > > > > > > > We would need a DDL syntax to parameterize UDFs on
>> >> > registration.
>> >> > > > > > > > I don't want to propose a syntax here, but just to get an
>> >> idea
>> >> > it
>> >> > > > > might
>> >> > > > > > > > look like this:
>> >> > > > > > > >
>> >> > > > > > > > CREATE FUNCTION rates AS
>> >> > > > > > > > 'org.apache.flink.table.udfs.TemporalTableFunction' WITH
>> >> > > ('table' =
>> >> > > > > > > > 'rates_history', 'key' = 'cur', 'time' = 'rowtime')
>> >> > > > > > > >
>> >> > > > > > > > Right now, the Flink Catalog interface does not have the
>> >> > > > > functionality
>> >> > > > > > to
>> >> > > > > > > > store such parameters and would need some hacks to
>> properly
>> >> > > create
>> >> > > > > > > properly
>> >> > > > > > > > parameterize function instances.
>> >> > > > > > > >
>> >> > > > > > > >
>> >> > > > > > > >
>> >> > > > > > > > # 2 Defining a join of an append-only table and a
>> temporal
>> >> > table
>> >> > > > > > > >
>> >> > > > > > > > The append-only table needs to have a time-attribute
>> >> > (processing
>> >> > > > time
>> >> > > > > > or
>> >> > > > > > > > event time, but same as the temporal table).
>> >> > > > > > > > The join then needs to specify two things:
>> >> > > > > > > > * an equality predicate that includes the primary key of
>> the
>> >> > > > temporal
>> >> > > > > > > table
>> >> > > > > > > > * declare the time attribute of the append-only table as
>> the
>> >> > time
>> >> > > > as
>> >> > > > > of
>> >> > > > > > > > which to look up the temporal table, i.e, get the
>> version of
>> >> > the
>> >> > > > > > temporal
>> >> > > > > > > > table that is valid for the timestamp of the current row
>> >> from
>> >> > the
>> >> > > > > > > > append-only table
>> >> > > > > > > >
>> >> > > > > > > > The tricky part (from a syntax point of view) is to
>> specify
>> >> the
>> >> > > > > lookup
>> >> > > > > > > > time.
>> >> > > > > > > >
>> >> > > > > > > > ## 2.1 the temporal table is a regular table or view (see
>> >> > > > approaches
>> >> > > > > > 1.1
>> >> > > > > > > > and 1.2 above)
>> >> > > > > > > >
>> >> > > > > > > > In this case we can use the "FOR SYSTEM_TIME AS OF x"
>> >> clause as
>> >> > > > > > follows:
>> >> > > > > > > >
>> >> > > > > > > > SELECT *
>> >> > > > > > > > FROM orders o, rates r FOR SYSTEM_TIME AS OF o.ordertime
>> >> > > > > > > > WHERE o.currency = r.currency
>> >> > > > > > > >
>> >> > > > > > > > IMO, this is a great syntax and the one we should strive
>> >> for.
>> >> > > > > > > > We would need to bend the rules of the SQL standard which
>> >> only
>> >> > > > > allows x
>> >> > > > > > > in
>> >> > > > > > > > "FOR SYSTEM_TIME AS OF x" to be a constant and the table
>> on
>> >> > which
>> >> > > > it
>> >> > > > > is
>> >> > > > > > > > applied usually needs to be a specific type (not sure if
>> >> views
>> >> > > are
>> >> > > > > > > > supported), but I guess this is fine.
>> >> > > > > > > > NOTE: the "FOR SYSTEM_TIME AS OF x" is already supported
>> for
>> >> > > > > > LookupTable
>> >> > > > > > > > Joins if x is a processing time attribute [2].
>> >> > > > > > > >
>> >> > > > > > > > ## 2.2 the temporal table is a TableFunction and
>> >> parameterized
>> >> > in
>> >> > > > the
>> >> > > > > > > query
>> >> > > > > > > > (see 1.3.1 above)
>> >> > > > > > > >
>> >> > > > > > > > SELECT *
>> >> > > > > > > > FROM orders o,
>> >> > > > > > > >    TEMPORAL_TABLE(
>> >> > > > > > > >      table => TABLE(rates_history),
>> >> > > > > > > >      key => DESCRIPTOR(currency),
>> >> > > > > > > >      time => DESCRIPTOR(rowtime)) r
>> >> > > > > > > >    ON o.currency = r.currency
>> >> > > > > > > >
>> >> > > > > > > > The function "TEMPORAL_TABLE" is built-in and nothing was
>> >> > > > registered
>> >> > > > > in
>> >> > > > > > > the
>> >> > > > > > > > catalog (except the rates_history table).
>> >> > > > > > > > In fact this is valid SQL:2016 syntax and called
>> Polymorphic
>> >> > > Table
>> >> > > > > > > > Functions. Have a look here [3].
>> >> > > > > > > >
>> >> > > > > > > > ## 2.3 the temporal table is a TableFunction that was
>> >> > > parameterized
>> >> > > > > > > during
>> >> > > > > > > > registration (see 1.3.2 above)
>> >> > > > > > > >
>> >> > > > > > > > This is what we have at the momement.
>> >> > > > > > > >
>> >> > > > > > > > SELECT *
>> >> > > > > > > > FROM orders o,
>> >> > > > > > > >    LATERAL TABLE (rates(o.ordertime))
>> >> > > > > > > >    ON o.currency = r.currency
>> >> > > > > > > >
>> >> > > > > > > > The TableFunction "rates" was registered in the catalog
>> and
>> >> > > > > > parameterized
>> >> > > > > > > > to the "rates_history" append-only table, the key was
>> set to
>> >> > > > > > "currency",
>> >> > > > > > > > and the time attribute was declared.
>> >> > > > > > > >
>> >> > > > > > > > # SUMMARY
>> >> > > > > > > >
>> >> > > > > > > > IMO we should in the long run aim to define temporal
>> tables
>> >> > > either
>> >> > > > as
>> >> > > > > > > > upsert retraction tables and views on append-only tables
>> and
>> >> > join
>> >> > > > > them
>> >> > > > > > > > using the "FOR SYSTEM_TIME AS OF x" syntax.
>> >> > > > > > > > I guess it is debatable whether we need to decare to
>> track
>> >> > > history
>> >> > > > > for
>> >> > > > > > > > these tables (which we don't actually do) or if we do it
>> by
>> >> > > > > convention
>> >> > > > > > if
>> >> > > > > > > > the table has a time attribute.
>> >> > > > > > > > It should be (relatively) easy to get this to work for
>> >> > retraction
>> >> > > > > > tables
>> >> > > > > > > > which will be supported soon.
>> >> > > > > > > > It will be more work for views because we need to improve
>> >> the
>> >> > > time
>> >> > > > > > > > attribute handling with MAX() aggregations.
>> >> > > > > > > > The "FOR SYSTEM_TIME AS OF x" is already supported for
>> >> > > > > > LookupTableSources
>> >> > > > > > > > and would "only" need to be adapted to work on temporal
>> >> tables.
>> >> > > > > > > >
>> >> > > > > > > > Registering parameterized TableFunctions in the catalog
>> >> seems
>> >> > > like
>> >> > > > > > quite
>> >> > > > > > > a
>> >> > > > > > > > bit of work. We need new DDL syntax, extend the catalog
>> and
>> >> > > > function
>> >> > > > > > > > instantiation. This won't be easy, IMO.
>> >> > > > > > > > If we only support them as TEMPORARY FUNCTION which are
>> not
>> >> > > > > registered
>> >> > > > > > in
>> >> > > > > > > > the catalog it will be easier. The question is whether
>> it is
>> >> > > worth
>> >> > > > > the
>> >> > > > > > > > effort if we decide for the other approach.
>> >> > > > > > > >
>> >> > > > > > > > Using TableFunctions that are parameterized in the query
>> >> will
>> >> > > > require
>> >> > > > > > to
>> >> > > > > > > > extend the Calcite parser and framework to support
>> >> Polymorphic
>> >> > > > Table
>> >> > > > > > > > Functions.
>> >> > > > > > > > However, there might already some work be done there,
>> >> because
>> >> > > AFAIK
>> >> > > > > > > Apache
>> >> > > > > > > > Beam aims to support this syntax for windowing functions
>> as
>> >> > > > described
>> >> > > > > > in
>> >> > > > > > > > the "One SQL to rule them all" paper [4].
>> >> > > > > > > > It might be the fastest and fully SQL standard compliant
>> >> way.
>> >> > > > > > > >
>> >> > > > > > > > Cheers,
>> >> > > > > > > > Fabian
>> >> > > > > > > >
>> >> > > > > > > > [1]
>> >> > > > > > > >
>> >> > > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables
>> >> > > > > > > > [2]
>> >> > > > > > > >
>> >> > > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> https://ci.apache.org/projects/flink/flink-docs-release-1.10/dev/table/streaming/joins.html#usage-1
>> >> > > > > > > > [3]
>> >> > > > > > > >
>> >> > > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> https://standards.iso.org/ittf/PubliclyAvailableStandards/c069776_ISO_IEC_TR_19075-7_2017.zip
>> >> > > > > > > > [4] https://arxiv.org/abs/1905.12133
>> >> > > > > > > >
>> >> > > > > > > > Am Fr., 17. Apr. 2020 um 06:37 Uhr schrieb Jark Wu <
>> >> > > > imjark@gmail.com
>> >> > > > > >:
>> >> > > > > > > >
>> >> > > > > > > >> 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 <
>> >> libenchao@gmail.com
>> >> > >
>> >> > > > > 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 <kn...@apache.org> 于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: libenchao@gmail.com; libenchao@pku.edu.cn
>> >> > > > > > > >>>
>> >> > > > > > > >>
>> >> > > > > > > >
>> >> > > > > > >
>> >> > > > > > >
>> >> > > > > >
>> >> > > > >
>> >> > > >
>> >> > >
>> >> >
>> >>
>> >>
>> >> --
>> >>
>> >> Konstantin Knauf
>> >>
>> >> https://twitter.com/snntrable
>> >>
>> >> https://github.com/knaufk
>> >>
>> >
>>
>

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

Posted by Kurt Young <yk...@gmail.com>.
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 <im...@gmail.com> 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 <fh...@gmail.com> 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 <
> > knaufk@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 <time-attribute>" 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 <im...@gmail.com> 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 <im...@gmail.com> 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 <fh...@gmail.com>
> 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 <
> imjark@gmail.com
> >> >:
> >> > >
> >> > > > 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 <fh...@gmail.com>
> >> 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 <
> >> > > > > sjwiesman@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 <
> >> twalthr@apache.org>
> >> > > > > 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 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 VIEW not so much.
> >> > > > > > > > * Users would need to specify the VIEW exactly correct,
> such
> >> > that
> >> > > > it
> >> > > > > > can
> >> > > > > > > be
> >> > > > > > > > used as a temporal table. Look at 1.1 why this is not
> >> trivial.
> >> > > > > > > >
> >> > > > > > > > There is two ways to use a TableFunction:
> >> > > > > > > >
> >> > > > > > > > ### 1.3.1 Built-in and pre-registered function that is
> >> > > > parameterized
> >> > > > > in
> >> > > > > > > the
> >> > > > > > > > SQL query
> >> > > > > > > >
> >> > > > > > > > Here, we do not need to do anything to register the
> >> function.
> >> > We
> >> > > > > simply
> >> > > > > > > use
> >> > > > > > > > it in the query (see example in 2.2 below)
> >> > > > > > > >
> >> > > > > > > > ### 1.3.2 Parameterize function when it is registered in
> the
> >> > > > catalog
> >> > > > > > > (with
> >> > > > > > > > a provided Java implementation)
> >> > > > > > > >
> >> > > > > > > > This is the approach, we've used so far. In the Table API,
> >> the
> >> > > > > function
> >> > > > > > > is
> >> > > > > > > > first parameterized and created and then registered:
> >> > > > > > > > We would need a DDL syntax to parameterize UDFs on
> >> > registration.
> >> > > > > > > > I don't want to propose a syntax here, but just to get an
> >> idea
> >> > it
> >> > > > > might
> >> > > > > > > > look like this:
> >> > > > > > > >
> >> > > > > > > > CREATE FUNCTION rates AS
> >> > > > > > > > 'org.apache.flink.table.udfs.TemporalTableFunction' WITH
> >> > > ('table' =
> >> > > > > > > > 'rates_history', 'key' = 'cur', 'time' = 'rowtime')
> >> > > > > > > >
> >> > > > > > > > Right now, the Flink Catalog interface does not have the
> >> > > > > functionality
> >> > > > > > to
> >> > > > > > > > store such parameters and would need some hacks to
> properly
> >> > > create
> >> > > > > > > properly
> >> > > > > > > > parameterize function instances.
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > >
> >> > > > > > > > # 2 Defining a join of an append-only table and a temporal
> >> > table
> >> > > > > > > >
> >> > > > > > > > The append-only table needs to have a time-attribute
> >> > (processing
> >> > > > time
> >> > > > > > or
> >> > > > > > > > event time, but same as the temporal table).
> >> > > > > > > > The join then needs to specify two things:
> >> > > > > > > > * an equality predicate that includes the primary key of
> the
> >> > > > temporal
> >> > > > > > > table
> >> > > > > > > > * declare the time attribute of the append-only table as
> the
> >> > time
> >> > > > as
> >> > > > > of
> >> > > > > > > > which to look up the temporal table, i.e, get the version
> of
> >> > the
> >> > > > > > temporal
> >> > > > > > > > table that is valid for the timestamp of the current row
> >> from
> >> > the
> >> > > > > > > > append-only table
> >> > > > > > > >
> >> > > > > > > > The tricky part (from a syntax point of view) is to
> specify
> >> the
> >> > > > > lookup
> >> > > > > > > > time.
> >> > > > > > > >
> >> > > > > > > > ## 2.1 the temporal table is a regular table or view (see
> >> > > > approaches
> >> > > > > > 1.1
> >> > > > > > > > and 1.2 above)
> >> > > > > > > >
> >> > > > > > > > In this case we can use the "FOR SYSTEM_TIME AS OF x"
> >> clause as
> >> > > > > > follows:
> >> > > > > > > >
> >> > > > > > > > SELECT *
> >> > > > > > > > FROM orders o, rates r FOR SYSTEM_TIME AS OF o.ordertime
> >> > > > > > > > WHERE o.currency = r.currency
> >> > > > > > > >
> >> > > > > > > > IMO, this is a great syntax and the one we should strive
> >> for.
> >> > > > > > > > We would need to bend the rules of the SQL standard which
> >> only
> >> > > > > allows x
> >> > > > > > > in
> >> > > > > > > > "FOR SYSTEM_TIME AS OF x" to be a constant and the table
> on
> >> > which
> >> > > > it
> >> > > > > is
> >> > > > > > > > applied usually needs to be a specific type (not sure if
> >> views
> >> > > are
> >> > > > > > > > supported), but I guess this is fine.
> >> > > > > > > > NOTE: the "FOR SYSTEM_TIME AS OF x" is already supported
> for
> >> > > > > > LookupTable
> >> > > > > > > > Joins if x is a processing time attribute [2].
> >> > > > > > > >
> >> > > > > > > > ## 2.2 the temporal table is a TableFunction and
> >> parameterized
> >> > in
> >> > > > the
> >> > > > > > > query
> >> > > > > > > > (see 1.3.1 above)
> >> > > > > > > >
> >> > > > > > > > SELECT *
> >> > > > > > > > FROM orders o,
> >> > > > > > > >    TEMPORAL_TABLE(
> >> > > > > > > >      table => TABLE(rates_history),
> >> > > > > > > >      key => DESCRIPTOR(currency),
> >> > > > > > > >      time => DESCRIPTOR(rowtime)) r
> >> > > > > > > >    ON o.currency = r.currency
> >> > > > > > > >
> >> > > > > > > > The function "TEMPORAL_TABLE" is built-in and nothing was
> >> > > > registered
> >> > > > > in
> >> > > > > > > the
> >> > > > > > > > catalog (except the rates_history table).
> >> > > > > > > > In fact this is valid SQL:2016 syntax and called
> Polymorphic
> >> > > Table
> >> > > > > > > > Functions. Have a look here [3].
> >> > > > > > > >
> >> > > > > > > > ## 2.3 the temporal table is a TableFunction that was
> >> > > parameterized
> >> > > > > > > during
> >> > > > > > > > registration (see 1.3.2 above)
> >> > > > > > > >
> >> > > > > > > > This is what we have at the momement.
> >> > > > > > > >
> >> > > > > > > > SELECT *
> >> > > > > > > > FROM orders o,
> >> > > > > > > >    LATERAL TABLE (rates(o.ordertime))
> >> > > > > > > >    ON o.currency = r.currency
> >> > > > > > > >
> >> > > > > > > > The TableFunction "rates" was registered in the catalog
> and
> >> > > > > > parameterized
> >> > > > > > > > to the "rates_history" append-only table, the key was set
> to
> >> > > > > > "currency",
> >> > > > > > > > and the time attribute was declared.
> >> > > > > > > >
> >> > > > > > > > # SUMMARY
> >> > > > > > > >
> >> > > > > > > > IMO we should in the long run aim to define temporal
> tables
> >> > > either
> >> > > > as
> >> > > > > > > > upsert retraction tables and views on append-only tables
> and
> >> > join
> >> > > > > them
> >> > > > > > > > using the "FOR SYSTEM_TIME AS OF x" syntax.
> >> > > > > > > > I guess it is debatable whether we need to decare to track
> >> > > history
> >> > > > > for
> >> > > > > > > > these tables (which we don't actually do) or if we do it
> by
> >> > > > > convention
> >> > > > > > if
> >> > > > > > > > the table has a time attribute.
> >> > > > > > > > It should be (relatively) easy to get this to work for
> >> > retraction
> >> > > > > > tables
> >> > > > > > > > which will be supported soon.
> >> > > > > > > > It will be more work for views because we need to improve
> >> the
> >> > > time
> >> > > > > > > > attribute handling with MAX() aggregations.
> >> > > > > > > > The "FOR SYSTEM_TIME AS OF x" is already supported for
> >> > > > > > LookupTableSources
> >> > > > > > > > and would "only" need to be adapted to work on temporal
> >> tables.
> >> > > > > > > >
> >> > > > > > > > Registering parameterized TableFunctions in the catalog
> >> seems
> >> > > like
> >> > > > > > quite
> >> > > > > > > a
> >> > > > > > > > bit of work. We need new DDL syntax, extend the catalog
> and
> >> > > > function
> >> > > > > > > > instantiation. This won't be easy, IMO.
> >> > > > > > > > If we only support them as TEMPORARY FUNCTION which are
> not
> >> > > > > registered
> >> > > > > > in
> >> > > > > > > > the catalog it will be easier. The question is whether it
> is
> >> > > worth
> >> > > > > the
> >> > > > > > > > effort if we decide for the other approach.
> >> > > > > > > >
> >> > > > > > > > Using TableFunctions that are parameterized in the query
> >> will
> >> > > > require
> >> > > > > > to
> >> > > > > > > > extend the Calcite parser and framework to support
> >> Polymorphic
> >> > > > Table
> >> > > > > > > > Functions.
> >> > > > > > > > However, there might already some work be done there,
> >> because
> >> > > AFAIK
> >> > > > > > > Apache
> >> > > > > > > > Beam aims to support this syntax for windowing functions
> as
> >> > > > described
> >> > > > > > in
> >> > > > > > > > the "One SQL to rule them all" paper [4].
> >> > > > > > > > It might be the fastest and fully SQL standard compliant
> >> way.
> >> > > > > > > >
> >> > > > > > > > Cheers,
> >> > > > > > > > Fabian
> >> > > > > > > >
> >> > > > > > > > [1]
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables
> >> > > > > > > > [2]
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/dev/table/streaming/joins.html#usage-1
> >> > > > > > > > [3]
> >> > > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> https://standards.iso.org/ittf/PubliclyAvailableStandards/c069776_ISO_IEC_TR_19075-7_2017.zip
> >> > > > > > > > [4] https://arxiv.org/abs/1905.12133
> >> > > > > > > >
> >> > > > > > > > Am Fr., 17. Apr. 2020 um 06:37 Uhr schrieb Jark Wu <
> >> > > > imjark@gmail.com
> >> > > > > >:
> >> > > > > > > >
> >> > > > > > > >> 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 <
> >> libenchao@gmail.com
> >> > >
> >> > > > > 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 <kn...@apache.org> 于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: libenchao@gmail.com; libenchao@pku.edu.cn
> >> > > > > > > >>>
> >> > > > > > > >>
> >> > > > > > > >
> >> > > > > > >
> >> > > > > > >
> >> > > > > >
> >> > > > >
> >> > > >
> >> > >
> >> >
> >>
> >>
> >> --
> >>
> >> Konstantin Knauf
> >>
> >> https://twitter.com/snntrable
> >>
> >> https://github.com/knaufk
> >>
> >
>

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

Posted by Jark Wu <im...@gmail.com>.
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 <fh...@gmail.com> 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 <
> knaufk@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 <time-attribute>" 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 <im...@gmail.com> 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 <im...@gmail.com> 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 <fh...@gmail.com> 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 <imjark@gmail.com
>> >:
>> > >
>> > > > 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 <fh...@gmail.com>
>> 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 <
>> > > > > sjwiesman@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 <
>> twalthr@apache.org>
>> > > > > 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 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 VIEW not so much.
>> > > > > > > > * Users would need to specify the VIEW exactly correct, such
>> > that
>> > > > it
>> > > > > > can
>> > > > > > > be
>> > > > > > > > used as a temporal table. Look at 1.1 why this is not
>> trivial.
>> > > > > > > >
>> > > > > > > > There is two ways to use a TableFunction:
>> > > > > > > >
>> > > > > > > > ### 1.3.1 Built-in and pre-registered function that is
>> > > > parameterized
>> > > > > in
>> > > > > > > the
>> > > > > > > > SQL query
>> > > > > > > >
>> > > > > > > > Here, we do not need to do anything to register the
>> function.
>> > We
>> > > > > simply
>> > > > > > > use
>> > > > > > > > it in the query (see example in 2.2 below)
>> > > > > > > >
>> > > > > > > > ### 1.3.2 Parameterize function when it is registered in the
>> > > > catalog
>> > > > > > > (with
>> > > > > > > > a provided Java implementation)
>> > > > > > > >
>> > > > > > > > This is the approach, we've used so far. In the Table API,
>> the
>> > > > > function
>> > > > > > > is
>> > > > > > > > first parameterized and created and then registered:
>> > > > > > > > We would need a DDL syntax to parameterize UDFs on
>> > registration.
>> > > > > > > > I don't want to propose a syntax here, but just to get an
>> idea
>> > it
>> > > > > might
>> > > > > > > > look like this:
>> > > > > > > >
>> > > > > > > > CREATE FUNCTION rates AS
>> > > > > > > > 'org.apache.flink.table.udfs.TemporalTableFunction' WITH
>> > > ('table' =
>> > > > > > > > 'rates_history', 'key' = 'cur', 'time' = 'rowtime')
>> > > > > > > >
>> > > > > > > > Right now, the Flink Catalog interface does not have the
>> > > > > functionality
>> > > > > > to
>> > > > > > > > store such parameters and would need some hacks to properly
>> > > create
>> > > > > > > properly
>> > > > > > > > parameterize function instances.
>> > > > > > > >
>> > > > > > > >
>> > > > > > > >
>> > > > > > > > # 2 Defining a join of an append-only table and a temporal
>> > table
>> > > > > > > >
>> > > > > > > > The append-only table needs to have a time-attribute
>> > (processing
>> > > > time
>> > > > > > or
>> > > > > > > > event time, but same as the temporal table).
>> > > > > > > > The join then needs to specify two things:
>> > > > > > > > * an equality predicate that includes the primary key of the
>> > > > temporal
>> > > > > > > table
>> > > > > > > > * declare the time attribute of the append-only table as the
>> > time
>> > > > as
>> > > > > of
>> > > > > > > > which to look up the temporal table, i.e, get the version of
>> > the
>> > > > > > temporal
>> > > > > > > > table that is valid for the timestamp of the current row
>> from
>> > the
>> > > > > > > > append-only table
>> > > > > > > >
>> > > > > > > > The tricky part (from a syntax point of view) is to specify
>> the
>> > > > > lookup
>> > > > > > > > time.
>> > > > > > > >
>> > > > > > > > ## 2.1 the temporal table is a regular table or view (see
>> > > > approaches
>> > > > > > 1.1
>> > > > > > > > and 1.2 above)
>> > > > > > > >
>> > > > > > > > In this case we can use the "FOR SYSTEM_TIME AS OF x"
>> clause as
>> > > > > > follows:
>> > > > > > > >
>> > > > > > > > SELECT *
>> > > > > > > > FROM orders o, rates r FOR SYSTEM_TIME AS OF o.ordertime
>> > > > > > > > WHERE o.currency = r.currency
>> > > > > > > >
>> > > > > > > > IMO, this is a great syntax and the one we should strive
>> for.
>> > > > > > > > We would need to bend the rules of the SQL standard which
>> only
>> > > > > allows x
>> > > > > > > in
>> > > > > > > > "FOR SYSTEM_TIME AS OF x" to be a constant and the table on
>> > which
>> > > > it
>> > > > > is
>> > > > > > > > applied usually needs to be a specific type (not sure if
>> views
>> > > are
>> > > > > > > > supported), but I guess this is fine.
>> > > > > > > > NOTE: the "FOR SYSTEM_TIME AS OF x" is already supported for
>> > > > > > LookupTable
>> > > > > > > > Joins if x is a processing time attribute [2].
>> > > > > > > >
>> > > > > > > > ## 2.2 the temporal table is a TableFunction and
>> parameterized
>> > in
>> > > > the
>> > > > > > > query
>> > > > > > > > (see 1.3.1 above)
>> > > > > > > >
>> > > > > > > > SELECT *
>> > > > > > > > FROM orders o,
>> > > > > > > >    TEMPORAL_TABLE(
>> > > > > > > >      table => TABLE(rates_history),
>> > > > > > > >      key => DESCRIPTOR(currency),
>> > > > > > > >      time => DESCRIPTOR(rowtime)) r
>> > > > > > > >    ON o.currency = r.currency
>> > > > > > > >
>> > > > > > > > The function "TEMPORAL_TABLE" is built-in and nothing was
>> > > > registered
>> > > > > in
>> > > > > > > the
>> > > > > > > > catalog (except the rates_history table).
>> > > > > > > > In fact this is valid SQL:2016 syntax and called Polymorphic
>> > > Table
>> > > > > > > > Functions. Have a look here [3].
>> > > > > > > >
>> > > > > > > > ## 2.3 the temporal table is a TableFunction that was
>> > > parameterized
>> > > > > > > during
>> > > > > > > > registration (see 1.3.2 above)
>> > > > > > > >
>> > > > > > > > This is what we have at the momement.
>> > > > > > > >
>> > > > > > > > SELECT *
>> > > > > > > > FROM orders o,
>> > > > > > > >    LATERAL TABLE (rates(o.ordertime))
>> > > > > > > >    ON o.currency = r.currency
>> > > > > > > >
>> > > > > > > > The TableFunction "rates" was registered in the catalog and
>> > > > > > parameterized
>> > > > > > > > to the "rates_history" append-only table, the key was set to
>> > > > > > "currency",
>> > > > > > > > and the time attribute was declared.
>> > > > > > > >
>> > > > > > > > # SUMMARY
>> > > > > > > >
>> > > > > > > > IMO we should in the long run aim to define temporal tables
>> > > either
>> > > > as
>> > > > > > > > upsert retraction tables and views on append-only tables and
>> > join
>> > > > > them
>> > > > > > > > using the "FOR SYSTEM_TIME AS OF x" syntax.
>> > > > > > > > I guess it is debatable whether we need to decare to track
>> > > history
>> > > > > for
>> > > > > > > > these tables (which we don't actually do) or if we do it by
>> > > > > convention
>> > > > > > if
>> > > > > > > > the table has a time attribute.
>> > > > > > > > It should be (relatively) easy to get this to work for
>> > retraction
>> > > > > > tables
>> > > > > > > > which will be supported soon.
>> > > > > > > > It will be more work for views because we need to improve
>> the
>> > > time
>> > > > > > > > attribute handling with MAX() aggregations.
>> > > > > > > > The "FOR SYSTEM_TIME AS OF x" is already supported for
>> > > > > > LookupTableSources
>> > > > > > > > and would "only" need to be adapted to work on temporal
>> tables.
>> > > > > > > >
>> > > > > > > > Registering parameterized TableFunctions in the catalog
>> seems
>> > > like
>> > > > > > quite
>> > > > > > > a
>> > > > > > > > bit of work. We need new DDL syntax, extend the catalog and
>> > > > function
>> > > > > > > > instantiation. This won't be easy, IMO.
>> > > > > > > > If we only support them as TEMPORARY FUNCTION which are not
>> > > > > registered
>> > > > > > in
>> > > > > > > > the catalog it will be easier. The question is whether it is
>> > > worth
>> > > > > the
>> > > > > > > > effort if we decide for the other approach.
>> > > > > > > >
>> > > > > > > > Using TableFunctions that are parameterized in the query
>> will
>> > > > require
>> > > > > > to
>> > > > > > > > extend the Calcite parser and framework to support
>> Polymorphic
>> > > > Table
>> > > > > > > > Functions.
>> > > > > > > > However, there might already some work be done there,
>> because
>> > > AFAIK
>> > > > > > > Apache
>> > > > > > > > Beam aims to support this syntax for windowing functions as
>> > > > described
>> > > > > > in
>> > > > > > > > the "One SQL to rule them all" paper [4].
>> > > > > > > > It might be the fastest and fully SQL standard compliant
>> way.
>> > > > > > > >
>> > > > > > > > Cheers,
>> > > > > > > > Fabian
>> > > > > > > >
>> > > > > > > > [1]
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables
>> > > > > > > > [2]
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://ci.apache.org/projects/flink/flink-docs-release-1.10/dev/table/streaming/joins.html#usage-1
>> > > > > > > > [3]
>> > > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>> https://standards.iso.org/ittf/PubliclyAvailableStandards/c069776_ISO_IEC_TR_19075-7_2017.zip
>> > > > > > > > [4] https://arxiv.org/abs/1905.12133
>> > > > > > > >
>> > > > > > > > Am Fr., 17. Apr. 2020 um 06:37 Uhr schrieb Jark Wu <
>> > > > imjark@gmail.com
>> > > > > >:
>> > > > > > > >
>> > > > > > > >> 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 <
>> libenchao@gmail.com
>> > >
>> > > > > 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 <kn...@apache.org> 于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: libenchao@gmail.com; libenchao@pku.edu.cn
>> > > > > > > >>>
>> > > > > > > >>
>> > > > > > > >
>> > > > > > >
>> > > > > > >
>> > > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>>
>> --
>>
>> Konstantin Knauf
>>
>> https://twitter.com/snntrable
>>
>> https://github.com/knaufk
>>
>

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

Posted by Fabian Hueske <fh...@gmail.com>.
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 <knaufk@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 <time-attribute>" 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 <im...@gmail.com> 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 <im...@gmail.com> 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 <fh...@gmail.com> 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 <im...@gmail.com>:
> > >
> > > > 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 <fh...@gmail.com>
> 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 <
> > > > > sjwiesman@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 <
> twalthr@apache.org>
> > > > > 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 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 VIEW not so much.
> > > > > > > > * Users would need to specify the VIEW exactly correct, such
> > that
> > > > it
> > > > > > can
> > > > > > > be
> > > > > > > > used as a temporal table. Look at 1.1 why this is not
> trivial.
> > > > > > > >
> > > > > > > > There is two ways to use a TableFunction:
> > > > > > > >
> > > > > > > > ### 1.3.1 Built-in and pre-registered function that is
> > > > parameterized
> > > > > in
> > > > > > > the
> > > > > > > > SQL query
> > > > > > > >
> > > > > > > > Here, we do not need to do anything to register the function.
> > We
> > > > > simply
> > > > > > > use
> > > > > > > > it in the query (see example in 2.2 below)
> > > > > > > >
> > > > > > > > ### 1.3.2 Parameterize function when it is registered in the
> > > > catalog
> > > > > > > (with
> > > > > > > > a provided Java implementation)
> > > > > > > >
> > > > > > > > This is the approach, we've used so far. In the Table API,
> the
> > > > > function
> > > > > > > is
> > > > > > > > first parameterized and created and then registered:
> > > > > > > > We would need a DDL syntax to parameterize UDFs on
> > registration.
> > > > > > > > I don't want to propose a syntax here, but just to get an
> idea
> > it
> > > > > might
> > > > > > > > look like this:
> > > > > > > >
> > > > > > > > CREATE FUNCTION rates AS
> > > > > > > > 'org.apache.flink.table.udfs.TemporalTableFunction' WITH
> > > ('table' =
> > > > > > > > 'rates_history', 'key' = 'cur', 'time' = 'rowtime')
> > > > > > > >
> > > > > > > > Right now, the Flink Catalog interface does not have the
> > > > > functionality
> > > > > > to
> > > > > > > > store such parameters and would need some hacks to properly
> > > create
> > > > > > > properly
> > > > > > > > parameterize function instances.
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > # 2 Defining a join of an append-only table and a temporal
> > table
> > > > > > > >
> > > > > > > > The append-only table needs to have a time-attribute
> > (processing
> > > > time
> > > > > > or
> > > > > > > > event time, but same as the temporal table).
> > > > > > > > The join then needs to specify two things:
> > > > > > > > * an equality predicate that includes the primary key of the
> > > > temporal
> > > > > > > table
> > > > > > > > * declare the time attribute of the append-only table as the
> > time
> > > > as
> > > > > of
> > > > > > > > which to look up the temporal table, i.e, get the version of
> > the
> > > > > > temporal
> > > > > > > > table that is valid for the timestamp of the current row from
> > the
> > > > > > > > append-only table
> > > > > > > >
> > > > > > > > The tricky part (from a syntax point of view) is to specify
> the
> > > > > lookup
> > > > > > > > time.
> > > > > > > >
> > > > > > > > ## 2.1 the temporal table is a regular table or view (see
> > > > approaches
> > > > > > 1.1
> > > > > > > > and 1.2 above)
> > > > > > > >
> > > > > > > > In this case we can use the "FOR SYSTEM_TIME AS OF x" clause
> as
> > > > > > follows:
> > > > > > > >
> > > > > > > > SELECT *
> > > > > > > > FROM orders o, rates r FOR SYSTEM_TIME AS OF o.ordertime
> > > > > > > > WHERE o.currency = r.currency
> > > > > > > >
> > > > > > > > IMO, this is a great syntax and the one we should strive for.
> > > > > > > > We would need to bend the rules of the SQL standard which
> only
> > > > > allows x
> > > > > > > in
> > > > > > > > "FOR SYSTEM_TIME AS OF x" to be a constant and the table on
> > which
> > > > it
> > > > > is
> > > > > > > > applied usually needs to be a specific type (not sure if
> views
> > > are
> > > > > > > > supported), but I guess this is fine.
> > > > > > > > NOTE: the "FOR SYSTEM_TIME AS OF x" is already supported for
> > > > > > LookupTable
> > > > > > > > Joins if x is a processing time attribute [2].
> > > > > > > >
> > > > > > > > ## 2.2 the temporal table is a TableFunction and
> parameterized
> > in
> > > > the
> > > > > > > query
> > > > > > > > (see 1.3.1 above)
> > > > > > > >
> > > > > > > > SELECT *
> > > > > > > > FROM orders o,
> > > > > > > >    TEMPORAL_TABLE(
> > > > > > > >      table => TABLE(rates_history),
> > > > > > > >      key => DESCRIPTOR(currency),
> > > > > > > >      time => DESCRIPTOR(rowtime)) r
> > > > > > > >    ON o.currency = r.currency
> > > > > > > >
> > > > > > > > The function "TEMPORAL_TABLE" is built-in and nothing was
> > > > registered
> > > > > in
> > > > > > > the
> > > > > > > > catalog (except the rates_history table).
> > > > > > > > In fact this is valid SQL:2016 syntax and called Polymorphic
> > > Table
> > > > > > > > Functions. Have a look here [3].
> > > > > > > >
> > > > > > > > ## 2.3 the temporal table is a TableFunction that was
> > > parameterized
> > > > > > > during
> > > > > > > > registration (see 1.3.2 above)
> > > > > > > >
> > > > > > > > This is what we have at the momement.
> > > > > > > >
> > > > > > > > SELECT *
> > > > > > > > FROM orders o,
> > > > > > > >    LATERAL TABLE (rates(o.ordertime))
> > > > > > > >    ON o.currency = r.currency
> > > > > > > >
> > > > > > > > The TableFunction "rates" was registered in the catalog and
> > > > > > parameterized
> > > > > > > > to the "rates_history" append-only table, the key was set to
> > > > > > "currency",
> > > > > > > > and the time attribute was declared.
> > > > > > > >
> > > > > > > > # SUMMARY
> > > > > > > >
> > > > > > > > IMO we should in the long run aim to define temporal tables
> > > either
> > > > as
> > > > > > > > upsert retraction tables and views on append-only tables and
> > join
> > > > > them
> > > > > > > > using the "FOR SYSTEM_TIME AS OF x" syntax.
> > > > > > > > I guess it is debatable whether we need to decare to track
> > > history
> > > > > for
> > > > > > > > these tables (which we don't actually do) or if we do it by
> > > > > convention
> > > > > > if
> > > > > > > > the table has a time attribute.
> > > > > > > > It should be (relatively) easy to get this to work for
> > retraction
> > > > > > tables
> > > > > > > > which will be supported soon.
> > > > > > > > It will be more work for views because we need to improve the
> > > time
> > > > > > > > attribute handling with MAX() aggregations.
> > > > > > > > The "FOR SYSTEM_TIME AS OF x" is already supported for
> > > > > > LookupTableSources
> > > > > > > > and would "only" need to be adapted to work on temporal
> tables.
> > > > > > > >
> > > > > > > > Registering parameterized TableFunctions in the catalog seems
> > > like
> > > > > > quite
> > > > > > > a
> > > > > > > > bit of work. We need new DDL syntax, extend the catalog and
> > > > function
> > > > > > > > instantiation. This won't be easy, IMO.
> > > > > > > > If we only support them as TEMPORARY FUNCTION which are not
> > > > > registered
> > > > > > in
> > > > > > > > the catalog it will be easier. The question is whether it is
> > > worth
> > > > > the
> > > > > > > > effort if we decide for the other approach.
> > > > > > > >
> > > > > > > > Using TableFunctions that are parameterized in the query will
> > > > require
> > > > > > to
> > > > > > > > extend the Calcite parser and framework to support
> Polymorphic
> > > > Table
> > > > > > > > Functions.
> > > > > > > > However, there might already some work be done there, because
> > > AFAIK
> > > > > > > Apache
> > > > > > > > Beam aims to support this syntax for windowing functions as
> > > > described
> > > > > > in
> > > > > > > > the "One SQL to rule them all" paper [4].
> > > > > > > > It might be the fastest and fully SQL standard compliant way.
> > > > > > > >
> > > > > > > > Cheers,
> > > > > > > > Fabian
> > > > > > > >
> > > > > > > > [1]
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://docs.microsoft.com/en-us/sql/relational-databases/tables/temporal-tables
> > > > > > > > [2]
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/dev/table/streaming/joins.html#usage-1
> > > > > > > > [3]
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://standards.iso.org/ittf/PubliclyAvailableStandards/c069776_ISO_IEC_TR_19075-7_2017.zip
> > > > > > > > [4] https://arxiv.org/abs/1905.12133
> > > > > > > >
> > > > > > > > Am Fr., 17. Apr. 2020 um 06:37 Uhr schrieb Jark Wu <
> > > > imjark@gmail.com
> > > > > >:
> > > > > > > >
> > > > > > > >> 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 <
> libenchao@gmail.com
> > >
> > > > > 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 <kn...@apache.org> 于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: libenchao@gmail.com; libenchao@pku.edu.cn
> > > > > > > >>>
> > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
> --
>
> Konstantin Knauf
>
> https://twitter.com/snntrable
>
> https://github.com/knaufk
>