You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Jingsong Li <ji...@gmail.com> on 2021/12/28 03:23:11 UTC

Re: [DISCUSS] FLIP-188: Introduce Built-in Dynamic Table Storage

Hi all,

After some experimentation, we felt no problem putting the dynamic
storage outside of flink, and it also allowed us to design the
interface in more depth.

What do you think? If there is no problem, I am asking for PMC's help
here: we want to propose flink-dynamic-storage as a flink subproject,
and we want to build the project under apache.

Best,
Jingsong


On Wed, Nov 24, 2021 at 8:10 PM Jingsong Li <ji...@gmail.com> wrote:
>
> Hi Stephan,
>
> Thanks for your reply.
>
> Data never expires automatically.
>
> If there is a need for data retention, the user can choose one of the
> following options:
> - In the SQL for querying the managed table, users filter the data by themselves
> - Define the time partition, and users can delete the expired
> partition by themselves. (DROP PARTITION ...)
> - In the future version, we will support the "DELETE FROM" statement,
> users can delete the expired data according to the conditions.
>
> So to answer your question:
>
> > Will the VMQ send retractions so that the data will be removed from the table (via compactions)?
>
> The current implementation is not sending retraction, which I think
> theoretically should be sent, currently the user can filter by
> subsequent conditions.
> And yes, the subscriber would not see strictly a correct result. I
> think this is something we can improve for Flink SQL.
>
> > Do we want time retention semantics handled by the compaction?
>
> Currently, no, Data never expires automatically.
>
> > Do we want to declare those types of queries "out of scope" initially?
>
> I think we want users to be able to use three options above to
> accomplish their requirements.
>
> I will update FLIP to make the definition clearer and more explicit.
>
> Best,
> Jingsong
>
> On Wed, Nov 24, 2021 at 5:01 AM Stephan Ewen <ew...@gmail.com> wrote:
> >
> > Thanks for digging into this.
> > Regarding this query:
> >
> > INSERT INTO the_table
> >   SELECT window_end, COUNT(*)
> >     FROM (TUMBLE(TABLE interactions, DESCRIPTOR(ts), INTERVAL '5' MINUTES))
> > GROUP BY window_end
> >   HAVING now() - window_end <= INTERVAL '14' DAYS;
> >
> > I am not sure I understand what the conclusion is on the data retention question, where the continuous streaming SQL query has retention semantics. I think we would need to answer the following questions (I will call the query that computed the managed table the "view materializer query" - VMQ).
> >
> > (1) I guess the VMQ will send no updates for windows beyond the "retention period" is over (14 days), as you said. That makes sense.
> >
> > (2) Will the VMQ send retractions so that the data will be removed from the table (via compactions)?
> >   - if yes, this seems semantically better for users, but it will be expensive to keep the timers for retractions.
> >   - if not, we can still solve this by adding filters to queries against the managed table, as long as these queries are in Flink.
> >   - any subscriber to the changelog stream would not see strictly a correct result if we are not doing the retractions
> >
> > (3) Do we want time retention semantics handled by the compaction?
> >   - if we say that we lazily apply the deletes in the queries that read the managed tables, then we could also age out the old data during compaction.
> >   - that is cheap, but it might be too much of a special case to be very relevant here.
> >
> > (4) Do we want to declare those types of queries "out of scope" initially?
> >   - if yes, how many users are we affecting? (I guess probably not many, but would be good to hear some thoughts from others on this)
> >   - should we simply reject such queries in the optimizer as "not possible to support in managed tables"? I would suggest that, always better to tell users exactly what works and what not, rather than letting them be surprised in the end. Users can still remove the HAVING clause if they want the query to run, and that would be better than if the VMQ just silently ignores those semantics.
> >
> > Thanks,
> > Stephan
> >
>
>
> --
> Best, Jingsong Lee



--
Best, Jingsong Lee

Re: [DISCUSS] FLIP-188: Introduce Built-in Dynamic Table Storage

Posted by Francesco Guardiani <fr...@ververica.com>.
+1 with a separate repo and +1 with the flink-storage name

On Fri, Jan 7, 2022 at 8:40 AM Jingsong Li <ji...@gmail.com> wrote:

> Hi everyone,
>
> Vote for create a separate sub project for FLIP-188 thread is here:
> https://lists.apache.org/thread/wzzhr27cvrh6w107bn464m1m1ycfll1z
>
> Best,
> Jingsong
>
>
> On Fri, Jan 7, 2022 at 3:30 PM Jingsong Li <ji...@gmail.com> wrote:
> >
> > Hi Timo,
> >
> > I think we can consider exposing to DataStream users in the future, if
> > the API definition is clear after.
> > I am fine with `flink-table-store` too.
> > But I tend to prefer shorter and clearer name:
> > `flink-store`.
> >
> > I think I can create a separate thread to vote.
> >
> > Looking forward to your thoughts!
> >
> > Best,
> > Jingsong
> >
> >
> > On Thu, Dec 30, 2021 at 9:48 PM Timo Walther <tw...@apache.org> wrote:
> > >
> > > +1 for a separate repository. And also +1 for finding a good name.
> > >
> > > `flink-warehouse` would be definitely a good marketing name but I agree
> > > that we should not start marketing for code bases. Are we planning to
> > > make this storage also available to DataStream API users? If not, I
> > > would also vote for `flink-managed-table` or better:
> `flink-table-store`
> > >
> > > Thanks,
> > > Timo
> > >
> > >
> > >
> > > On 29.12.21 07:58, Jingsong Li wrote:
> > > > Thanks Till for your suggestions.
> > > >
> > > > Personally, I like flink-warehouse, this is what we want to convey to
> > > > the user, but it indicates a bit too much scope.
> > > >
> > > > How about just calling it flink-store?
> > > > Simply to convey an impression: this is flink's store project,
> > > > providing a built-in store for the flink compute engine, which can be
> > > > used by flink-table as well as flink-datastream.
> > > >
> > > > Best,
> > > > Jingsong
> > > >
> > > > On Tue, Dec 28, 2021 at 5:15 PM Till Rohrmann <tr...@apache.org>
> wrote:
> > > >>
> > > >> Hi Jingsong,
> > > >>
> > > >> I think that developing flink-dynamic-storage as a separate sub
> project is
> > > >> a very good idea since it allows us to move a lot faster and
> decouple
> > > >> releases from Flink. Hence big +1.
> > > >>
> > > >> Do we want to name it flink-dynamic-storage or shall we use a more
> > > >> descriptive name? dynamic-storage sounds a bit generic to me and I
> wouldn't
> > > >> know that this has something to do with letting Flink manage your
> tables
> > > >> and their storage. I don't have a very good idea but maybe we can
> call it
> > > >> flink-managed-tables, flink-warehouse, flink-olap or so.
> > > >>
> > > >> Cheers,
> > > >> Till
> > > >>
> > > >> On Tue, Dec 28, 2021 at 9:49 AM Martijn Visser <
> martijn@ververica.com>
> > > >> wrote:
> > > >>
> > > >>> Hi Jingsong,
> > > >>>
> > > >>> That sounds promising! +1 from my side to continue development
> under
> > > >>> flink-dynamic-storage as a Flink subproject. I think having a more
> in-depth
> > > >>> interface will benefit everyone.
> > > >>>
> > > >>> Best regards,
> > > >>>
> > > >>> Martijn
> > > >>>
> > > >>> On Tue, 28 Dec 2021 at 04:23, Jingsong Li <ji...@gmail.com>
> wrote:
> > > >>>
> > > >>>> Hi all,
> > > >>>>
> > > >>>> After some experimentation, we felt no problem putting the dynamic
> > > >>>> storage outside of flink, and it also allowed us to design the
> > > >>>> interface in more depth.
> > > >>>>
> > > >>>> What do you think? If there is no problem, I am asking for PMC's
> help
> > > >>>> here: we want to propose flink-dynamic-storage as a flink
> subproject,
> > > >>>> and we want to build the project under apache.
> > > >>>>
> > > >>>> Best,
> > > >>>> Jingsong
> > > >>>>
> > > >>>>
> > > >>>> On Wed, Nov 24, 2021 at 8:10 PM Jingsong Li <
> jingsonglee0@gmail.com>
> > > >>>> wrote:
> > > >>>>>
> > > >>>>> Hi Stephan,
> > > >>>>>
> > > >>>>> Thanks for your reply.
> > > >>>>>
> > > >>>>> Data never expires automatically.
> > > >>>>>
> > > >>>>> If there is a need for data retention, the user can choose one
> of the
> > > >>>>> following options:
> > > >>>>> - In the SQL for querying the managed table, users filter the
> data by
> > > >>>> themselves
> > > >>>>> - Define the time partition, and users can delete the expired
> > > >>>>> partition by themselves. (DROP PARTITION ...)
> > > >>>>> - In the future version, we will support the "DELETE FROM"
> statement,
> > > >>>>> users can delete the expired data according to the conditions.
> > > >>>>>
> > > >>>>> So to answer your question:
> > > >>>>>
> > > >>>>>> Will the VMQ send retractions so that the data will be removed
> from
> > > >>>> the table (via compactions)?
> > > >>>>>
> > > >>>>> The current implementation is not sending retraction, which I
> think
> > > >>>>> theoretically should be sent, currently the user can filter by
> > > >>>>> subsequent conditions.
> > > >>>>> And yes, the subscriber would not see strictly a correct result.
> I
> > > >>>>> think this is something we can improve for Flink SQL.
> > > >>>>>
> > > >>>>>> Do we want time retention semantics handled by the compaction?
> > > >>>>>
> > > >>>>> Currently, no, Data never expires automatically.
> > > >>>>>
> > > >>>>>> Do we want to declare those types of queries "out of scope"
> initially?
> > > >>>>>
> > > >>>>> I think we want users to be able to use three options above to
> > > >>>>> accomplish their requirements.
> > > >>>>>
> > > >>>>> I will update FLIP to make the definition clearer and more
> explicit.
> > > >>>>>
> > > >>>>> Best,
> > > >>>>> Jingsong
> > > >>>>>
> > > >>>>> On Wed, Nov 24, 2021 at 5:01 AM Stephan Ewen <
> ewenstephan@gmail.com>
> > > >>>> wrote:
> > > >>>>>>
> > > >>>>>> Thanks for digging into this.
> > > >>>>>> Regarding this query:
> > > >>>>>>
> > > >>>>>> INSERT INTO the_table
> > > >>>>>>    SELECT window_end, COUNT(*)
> > > >>>>>>      FROM (TUMBLE(TABLE interactions, DESCRIPTOR(ts), INTERVAL
> '5'
> > > >>>> MINUTES))
> > > >>>>>> GROUP BY window_end
> > > >>>>>>    HAVING now() - window_end <= INTERVAL '14' DAYS;
> > > >>>>>>
> > > >>>>>> I am not sure I understand what the conclusion is on the data
> > > >>>> retention question, where the continuous streaming SQL query has
> retention
> > > >>>> semantics. I think we would need to answer the following
> questions (I will
> > > >>>> call the query that computed the managed table the "view
> materializer
> > > >>>> query" - VMQ).
> > > >>>>>>
> > > >>>>>> (1) I guess the VMQ will send no updates for windows beyond the
> > > >>>> "retention period" is over (14 days), as you said. That makes
> sense.
> > > >>>>>>
> > > >>>>>> (2) Will the VMQ send retractions so that the data will be
> removed
> > > >>>> from the table (via compactions)?
> > > >>>>>>    - if yes, this seems semantically better for users, but it
> will be
> > > >>>> expensive to keep the timers for retractions.
> > > >>>>>>    - if not, we can still solve this by adding filters to
> queries
> > > >>>> against the managed table, as long as these queries are in Flink.
> > > >>>>>>    - any subscriber to the changelog stream would not see
> strictly a
> > > >>>> correct result if we are not doing the retractions
> > > >>>>>>
> > > >>>>>> (3) Do we want time retention semantics handled by the
> compaction?
> > > >>>>>>    - if we say that we lazily apply the deletes in the queries
> that
> > > >>>> read the managed tables, then we could also age out the old data
> during
> > > >>>> compaction.
> > > >>>>>>    - that is cheap, but it might be too much of a special case
> to be
> > > >>>> very relevant here.
> > > >>>>>>
> > > >>>>>> (4) Do we want to declare those types of queries "out of scope"
> > > >>>> initially?
> > > >>>>>>    - if yes, how many users are we affecting? (I guess probably
> not
> > > >>>> many, but would be good to hear some thoughts from others on this)
> > > >>>>>>    - should we simply reject such queries in the optimizer as
> "not
> > > >>>> possible to support in managed tables"? I would suggest that,
> always better
> > > >>>> to tell users exactly what works and what not, rather than
> letting them be
> > > >>>> surprised in the end. Users can still remove the HAVING clause if
> they want
> > > >>>> the query to run, and that would be better than if the VMQ just
> silently
> > > >>>> ignores those semantics.
> > > >>>>>>
> > > >>>>>> Thanks,
> > > >>>>>> Stephan
> > > >>>>>>
> > > >>>>>
> > > >>>>>
> > > >>>>> --
> > > >>>>> Best, Jingsong Lee
> > > >>>>
> > > >>>>
> > > >>>>
> > > >>>> --
> > > >>>> Best, Jingsong Lee
> > > >>>>
> > > >>>
> > > >
> > > >
> > > >
> > >
> >
> >
> > --
> > Best, Jingsong Lee
>
>
>
> --
> Best, Jingsong Lee
>

Re: [DISCUSS] FLIP-188: Introduce Built-in Dynamic Table Storage

Posted by Jingsong Li <ji...@gmail.com>.
Hi everyone,

Vote for create a separate sub project for FLIP-188 thread is here:
https://lists.apache.org/thread/wzzhr27cvrh6w107bn464m1m1ycfll1z

Best,
Jingsong


On Fri, Jan 7, 2022 at 3:30 PM Jingsong Li <ji...@gmail.com> wrote:
>
> Hi Timo,
>
> I think we can consider exposing to DataStream users in the future, if
> the API definition is clear after.
> I am fine with `flink-table-store` too.
> But I tend to prefer shorter and clearer name:
> `flink-store`.
>
> I think I can create a separate thread to vote.
>
> Looking forward to your thoughts!
>
> Best,
> Jingsong
>
>
> On Thu, Dec 30, 2021 at 9:48 PM Timo Walther <tw...@apache.org> wrote:
> >
> > +1 for a separate repository. And also +1 for finding a good name.
> >
> > `flink-warehouse` would be definitely a good marketing name but I agree
> > that we should not start marketing for code bases. Are we planning to
> > make this storage also available to DataStream API users? If not, I
> > would also vote for `flink-managed-table` or better: `flink-table-store`
> >
> > Thanks,
> > Timo
> >
> >
> >
> > On 29.12.21 07:58, Jingsong Li wrote:
> > > Thanks Till for your suggestions.
> > >
> > > Personally, I like flink-warehouse, this is what we want to convey to
> > > the user, but it indicates a bit too much scope.
> > >
> > > How about just calling it flink-store?
> > > Simply to convey an impression: this is flink's store project,
> > > providing a built-in store for the flink compute engine, which can be
> > > used by flink-table as well as flink-datastream.
> > >
> > > Best,
> > > Jingsong
> > >
> > > On Tue, Dec 28, 2021 at 5:15 PM Till Rohrmann <tr...@apache.org> wrote:
> > >>
> > >> Hi Jingsong,
> > >>
> > >> I think that developing flink-dynamic-storage as a separate sub project is
> > >> a very good idea since it allows us to move a lot faster and decouple
> > >> releases from Flink. Hence big +1.
> > >>
> > >> Do we want to name it flink-dynamic-storage or shall we use a more
> > >> descriptive name? dynamic-storage sounds a bit generic to me and I wouldn't
> > >> know that this has something to do with letting Flink manage your tables
> > >> and their storage. I don't have a very good idea but maybe we can call it
> > >> flink-managed-tables, flink-warehouse, flink-olap or so.
> > >>
> > >> Cheers,
> > >> Till
> > >>
> > >> On Tue, Dec 28, 2021 at 9:49 AM Martijn Visser <ma...@ververica.com>
> > >> wrote:
> > >>
> > >>> Hi Jingsong,
> > >>>
> > >>> That sounds promising! +1 from my side to continue development under
> > >>> flink-dynamic-storage as a Flink subproject. I think having a more in-depth
> > >>> interface will benefit everyone.
> > >>>
> > >>> Best regards,
> > >>>
> > >>> Martijn
> > >>>
> > >>> On Tue, 28 Dec 2021 at 04:23, Jingsong Li <ji...@gmail.com> wrote:
> > >>>
> > >>>> Hi all,
> > >>>>
> > >>>> After some experimentation, we felt no problem putting the dynamic
> > >>>> storage outside of flink, and it also allowed us to design the
> > >>>> interface in more depth.
> > >>>>
> > >>>> What do you think? If there is no problem, I am asking for PMC's help
> > >>>> here: we want to propose flink-dynamic-storage as a flink subproject,
> > >>>> and we want to build the project under apache.
> > >>>>
> > >>>> Best,
> > >>>> Jingsong
> > >>>>
> > >>>>
> > >>>> On Wed, Nov 24, 2021 at 8:10 PM Jingsong Li <ji...@gmail.com>
> > >>>> wrote:
> > >>>>>
> > >>>>> Hi Stephan,
> > >>>>>
> > >>>>> Thanks for your reply.
> > >>>>>
> > >>>>> Data never expires automatically.
> > >>>>>
> > >>>>> If there is a need for data retention, the user can choose one of the
> > >>>>> following options:
> > >>>>> - In the SQL for querying the managed table, users filter the data by
> > >>>> themselves
> > >>>>> - Define the time partition, and users can delete the expired
> > >>>>> partition by themselves. (DROP PARTITION ...)
> > >>>>> - In the future version, we will support the "DELETE FROM" statement,
> > >>>>> users can delete the expired data according to the conditions.
> > >>>>>
> > >>>>> So to answer your question:
> > >>>>>
> > >>>>>> Will the VMQ send retractions so that the data will be removed from
> > >>>> the table (via compactions)?
> > >>>>>
> > >>>>> The current implementation is not sending retraction, which I think
> > >>>>> theoretically should be sent, currently the user can filter by
> > >>>>> subsequent conditions.
> > >>>>> And yes, the subscriber would not see strictly a correct result. I
> > >>>>> think this is something we can improve for Flink SQL.
> > >>>>>
> > >>>>>> Do we want time retention semantics handled by the compaction?
> > >>>>>
> > >>>>> Currently, no, Data never expires automatically.
> > >>>>>
> > >>>>>> Do we want to declare those types of queries "out of scope" initially?
> > >>>>>
> > >>>>> I think we want users to be able to use three options above to
> > >>>>> accomplish their requirements.
> > >>>>>
> > >>>>> I will update FLIP to make the definition clearer and more explicit.
> > >>>>>
> > >>>>> Best,
> > >>>>> Jingsong
> > >>>>>
> > >>>>> On Wed, Nov 24, 2021 at 5:01 AM Stephan Ewen <ew...@gmail.com>
> > >>>> wrote:
> > >>>>>>
> > >>>>>> Thanks for digging into this.
> > >>>>>> Regarding this query:
> > >>>>>>
> > >>>>>> INSERT INTO the_table
> > >>>>>>    SELECT window_end, COUNT(*)
> > >>>>>>      FROM (TUMBLE(TABLE interactions, DESCRIPTOR(ts), INTERVAL '5'
> > >>>> MINUTES))
> > >>>>>> GROUP BY window_end
> > >>>>>>    HAVING now() - window_end <= INTERVAL '14' DAYS;
> > >>>>>>
> > >>>>>> I am not sure I understand what the conclusion is on the data
> > >>>> retention question, where the continuous streaming SQL query has retention
> > >>>> semantics. I think we would need to answer the following questions (I will
> > >>>> call the query that computed the managed table the "view materializer
> > >>>> query" - VMQ).
> > >>>>>>
> > >>>>>> (1) I guess the VMQ will send no updates for windows beyond the
> > >>>> "retention period" is over (14 days), as you said. That makes sense.
> > >>>>>>
> > >>>>>> (2) Will the VMQ send retractions so that the data will be removed
> > >>>> from the table (via compactions)?
> > >>>>>>    - if yes, this seems semantically better for users, but it will be
> > >>>> expensive to keep the timers for retractions.
> > >>>>>>    - if not, we can still solve this by adding filters to queries
> > >>>> against the managed table, as long as these queries are in Flink.
> > >>>>>>    - any subscriber to the changelog stream would not see strictly a
> > >>>> correct result if we are not doing the retractions
> > >>>>>>
> > >>>>>> (3) Do we want time retention semantics handled by the compaction?
> > >>>>>>    - if we say that we lazily apply the deletes in the queries that
> > >>>> read the managed tables, then we could also age out the old data during
> > >>>> compaction.
> > >>>>>>    - that is cheap, but it might be too much of a special case to be
> > >>>> very relevant here.
> > >>>>>>
> > >>>>>> (4) Do we want to declare those types of queries "out of scope"
> > >>>> initially?
> > >>>>>>    - if yes, how many users are we affecting? (I guess probably not
> > >>>> many, but would be good to hear some thoughts from others on this)
> > >>>>>>    - should we simply reject such queries in the optimizer as "not
> > >>>> possible to support in managed tables"? I would suggest that, always better
> > >>>> to tell users exactly what works and what not, rather than letting them be
> > >>>> surprised in the end. Users can still remove the HAVING clause if they want
> > >>>> the query to run, and that would be better than if the VMQ just silently
> > >>>> ignores those semantics.
> > >>>>>>
> > >>>>>> Thanks,
> > >>>>>> Stephan
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>> --
> > >>>>> Best, Jingsong Lee
> > >>>>
> > >>>>
> > >>>>
> > >>>> --
> > >>>> Best, Jingsong Lee
> > >>>>
> > >>>
> > >
> > >
> > >
> >
>
>
> --
> Best, Jingsong Lee



--
Best, Jingsong Lee

Re: [DISCUSS] FLIP-188: Introduce Built-in Dynamic Table Storage

Posted by Jingsong Li <ji...@gmail.com>.
Hi Timo,

I think we can consider exposing to DataStream users in the future, if
the API definition is clear after.
I am fine with `flink-table-store` too.
But I tend to prefer shorter and clearer name:
`flink-store`.

I think I can create a separate thread to vote.

Looking forward to your thoughts!

Best,
Jingsong


On Thu, Dec 30, 2021 at 9:48 PM Timo Walther <tw...@apache.org> wrote:
>
> +1 for a separate repository. And also +1 for finding a good name.
>
> `flink-warehouse` would be definitely a good marketing name but I agree
> that we should not start marketing for code bases. Are we planning to
> make this storage also available to DataStream API users? If not, I
> would also vote for `flink-managed-table` or better: `flink-table-store`
>
> Thanks,
> Timo
>
>
>
> On 29.12.21 07:58, Jingsong Li wrote:
> > Thanks Till for your suggestions.
> >
> > Personally, I like flink-warehouse, this is what we want to convey to
> > the user, but it indicates a bit too much scope.
> >
> > How about just calling it flink-store?
> > Simply to convey an impression: this is flink's store project,
> > providing a built-in store for the flink compute engine, which can be
> > used by flink-table as well as flink-datastream.
> >
> > Best,
> > Jingsong
> >
> > On Tue, Dec 28, 2021 at 5:15 PM Till Rohrmann <tr...@apache.org> wrote:
> >>
> >> Hi Jingsong,
> >>
> >> I think that developing flink-dynamic-storage as a separate sub project is
> >> a very good idea since it allows us to move a lot faster and decouple
> >> releases from Flink. Hence big +1.
> >>
> >> Do we want to name it flink-dynamic-storage or shall we use a more
> >> descriptive name? dynamic-storage sounds a bit generic to me and I wouldn't
> >> know that this has something to do with letting Flink manage your tables
> >> and their storage. I don't have a very good idea but maybe we can call it
> >> flink-managed-tables, flink-warehouse, flink-olap or so.
> >>
> >> Cheers,
> >> Till
> >>
> >> On Tue, Dec 28, 2021 at 9:49 AM Martijn Visser <ma...@ververica.com>
> >> wrote:
> >>
> >>> Hi Jingsong,
> >>>
> >>> That sounds promising! +1 from my side to continue development under
> >>> flink-dynamic-storage as a Flink subproject. I think having a more in-depth
> >>> interface will benefit everyone.
> >>>
> >>> Best regards,
> >>>
> >>> Martijn
> >>>
> >>> On Tue, 28 Dec 2021 at 04:23, Jingsong Li <ji...@gmail.com> wrote:
> >>>
> >>>> Hi all,
> >>>>
> >>>> After some experimentation, we felt no problem putting the dynamic
> >>>> storage outside of flink, and it also allowed us to design the
> >>>> interface in more depth.
> >>>>
> >>>> What do you think? If there is no problem, I am asking for PMC's help
> >>>> here: we want to propose flink-dynamic-storage as a flink subproject,
> >>>> and we want to build the project under apache.
> >>>>
> >>>> Best,
> >>>> Jingsong
> >>>>
> >>>>
> >>>> On Wed, Nov 24, 2021 at 8:10 PM Jingsong Li <ji...@gmail.com>
> >>>> wrote:
> >>>>>
> >>>>> Hi Stephan,
> >>>>>
> >>>>> Thanks for your reply.
> >>>>>
> >>>>> Data never expires automatically.
> >>>>>
> >>>>> If there is a need for data retention, the user can choose one of the
> >>>>> following options:
> >>>>> - In the SQL for querying the managed table, users filter the data by
> >>>> themselves
> >>>>> - Define the time partition, and users can delete the expired
> >>>>> partition by themselves. (DROP PARTITION ...)
> >>>>> - In the future version, we will support the "DELETE FROM" statement,
> >>>>> users can delete the expired data according to the conditions.
> >>>>>
> >>>>> So to answer your question:
> >>>>>
> >>>>>> Will the VMQ send retractions so that the data will be removed from
> >>>> the table (via compactions)?
> >>>>>
> >>>>> The current implementation is not sending retraction, which I think
> >>>>> theoretically should be sent, currently the user can filter by
> >>>>> subsequent conditions.
> >>>>> And yes, the subscriber would not see strictly a correct result. I
> >>>>> think this is something we can improve for Flink SQL.
> >>>>>
> >>>>>> Do we want time retention semantics handled by the compaction?
> >>>>>
> >>>>> Currently, no, Data never expires automatically.
> >>>>>
> >>>>>> Do we want to declare those types of queries "out of scope" initially?
> >>>>>
> >>>>> I think we want users to be able to use three options above to
> >>>>> accomplish their requirements.
> >>>>>
> >>>>> I will update FLIP to make the definition clearer and more explicit.
> >>>>>
> >>>>> Best,
> >>>>> Jingsong
> >>>>>
> >>>>> On Wed, Nov 24, 2021 at 5:01 AM Stephan Ewen <ew...@gmail.com>
> >>>> wrote:
> >>>>>>
> >>>>>> Thanks for digging into this.
> >>>>>> Regarding this query:
> >>>>>>
> >>>>>> INSERT INTO the_table
> >>>>>>    SELECT window_end, COUNT(*)
> >>>>>>      FROM (TUMBLE(TABLE interactions, DESCRIPTOR(ts), INTERVAL '5'
> >>>> MINUTES))
> >>>>>> GROUP BY window_end
> >>>>>>    HAVING now() - window_end <= INTERVAL '14' DAYS;
> >>>>>>
> >>>>>> I am not sure I understand what the conclusion is on the data
> >>>> retention question, where the continuous streaming SQL query has retention
> >>>> semantics. I think we would need to answer the following questions (I will
> >>>> call the query that computed the managed table the "view materializer
> >>>> query" - VMQ).
> >>>>>>
> >>>>>> (1) I guess the VMQ will send no updates for windows beyond the
> >>>> "retention period" is over (14 days), as you said. That makes sense.
> >>>>>>
> >>>>>> (2) Will the VMQ send retractions so that the data will be removed
> >>>> from the table (via compactions)?
> >>>>>>    - if yes, this seems semantically better for users, but it will be
> >>>> expensive to keep the timers for retractions.
> >>>>>>    - if not, we can still solve this by adding filters to queries
> >>>> against the managed table, as long as these queries are in Flink.
> >>>>>>    - any subscriber to the changelog stream would not see strictly a
> >>>> correct result if we are not doing the retractions
> >>>>>>
> >>>>>> (3) Do we want time retention semantics handled by the compaction?
> >>>>>>    - if we say that we lazily apply the deletes in the queries that
> >>>> read the managed tables, then we could also age out the old data during
> >>>> compaction.
> >>>>>>    - that is cheap, but it might be too much of a special case to be
> >>>> very relevant here.
> >>>>>>
> >>>>>> (4) Do we want to declare those types of queries "out of scope"
> >>>> initially?
> >>>>>>    - if yes, how many users are we affecting? (I guess probably not
> >>>> many, but would be good to hear some thoughts from others on this)
> >>>>>>    - should we simply reject such queries in the optimizer as "not
> >>>> possible to support in managed tables"? I would suggest that, always better
> >>>> to tell users exactly what works and what not, rather than letting them be
> >>>> surprised in the end. Users can still remove the HAVING clause if they want
> >>>> the query to run, and that would be better than if the VMQ just silently
> >>>> ignores those semantics.
> >>>>>>
> >>>>>> Thanks,
> >>>>>> Stephan
> >>>>>>
> >>>>>
> >>>>>
> >>>>> --
> >>>>> Best, Jingsong Lee
> >>>>
> >>>>
> >>>>
> >>>> --
> >>>> Best, Jingsong Lee
> >>>>
> >>>
> >
> >
> >
>


--
Best, Jingsong Lee

Re: [DISCUSS] FLIP-188: Introduce Built-in Dynamic Table Storage

Posted by Timo Walther <tw...@apache.org>.
+1 for a separate repository. And also +1 for finding a good name.

`flink-warehouse` would be definitely a good marketing name but I agree 
that we should not start marketing for code bases. Are we planning to 
make this storage also available to DataStream API users? If not, I 
would also vote for `flink-managed-table` or better: `flink-table-store`

Thanks,
Timo



On 29.12.21 07:58, Jingsong Li wrote:
> Thanks Till for your suggestions.
> 
> Personally, I like flink-warehouse, this is what we want to convey to
> the user, but it indicates a bit too much scope.
> 
> How about just calling it flink-store?
> Simply to convey an impression: this is flink's store project,
> providing a built-in store for the flink compute engine, which can be
> used by flink-table as well as flink-datastream.
> 
> Best,
> Jingsong
> 
> On Tue, Dec 28, 2021 at 5:15 PM Till Rohrmann <tr...@apache.org> wrote:
>>
>> Hi Jingsong,
>>
>> I think that developing flink-dynamic-storage as a separate sub project is
>> a very good idea since it allows us to move a lot faster and decouple
>> releases from Flink. Hence big +1.
>>
>> Do we want to name it flink-dynamic-storage or shall we use a more
>> descriptive name? dynamic-storage sounds a bit generic to me and I wouldn't
>> know that this has something to do with letting Flink manage your tables
>> and their storage. I don't have a very good idea but maybe we can call it
>> flink-managed-tables, flink-warehouse, flink-olap or so.
>>
>> Cheers,
>> Till
>>
>> On Tue, Dec 28, 2021 at 9:49 AM Martijn Visser <ma...@ververica.com>
>> wrote:
>>
>>> Hi Jingsong,
>>>
>>> That sounds promising! +1 from my side to continue development under
>>> flink-dynamic-storage as a Flink subproject. I think having a more in-depth
>>> interface will benefit everyone.
>>>
>>> Best regards,
>>>
>>> Martijn
>>>
>>> On Tue, 28 Dec 2021 at 04:23, Jingsong Li <ji...@gmail.com> wrote:
>>>
>>>> Hi all,
>>>>
>>>> After some experimentation, we felt no problem putting the dynamic
>>>> storage outside of flink, and it also allowed us to design the
>>>> interface in more depth.
>>>>
>>>> What do you think? If there is no problem, I am asking for PMC's help
>>>> here: we want to propose flink-dynamic-storage as a flink subproject,
>>>> and we want to build the project under apache.
>>>>
>>>> Best,
>>>> Jingsong
>>>>
>>>>
>>>> On Wed, Nov 24, 2021 at 8:10 PM Jingsong Li <ji...@gmail.com>
>>>> wrote:
>>>>>
>>>>> Hi Stephan,
>>>>>
>>>>> Thanks for your reply.
>>>>>
>>>>> Data never expires automatically.
>>>>>
>>>>> If there is a need for data retention, the user can choose one of the
>>>>> following options:
>>>>> - In the SQL for querying the managed table, users filter the data by
>>>> themselves
>>>>> - Define the time partition, and users can delete the expired
>>>>> partition by themselves. (DROP PARTITION ...)
>>>>> - In the future version, we will support the "DELETE FROM" statement,
>>>>> users can delete the expired data according to the conditions.
>>>>>
>>>>> So to answer your question:
>>>>>
>>>>>> Will the VMQ send retractions so that the data will be removed from
>>>> the table (via compactions)?
>>>>>
>>>>> The current implementation is not sending retraction, which I think
>>>>> theoretically should be sent, currently the user can filter by
>>>>> subsequent conditions.
>>>>> And yes, the subscriber would not see strictly a correct result. I
>>>>> think this is something we can improve for Flink SQL.
>>>>>
>>>>>> Do we want time retention semantics handled by the compaction?
>>>>>
>>>>> Currently, no, Data never expires automatically.
>>>>>
>>>>>> Do we want to declare those types of queries "out of scope" initially?
>>>>>
>>>>> I think we want users to be able to use three options above to
>>>>> accomplish their requirements.
>>>>>
>>>>> I will update FLIP to make the definition clearer and more explicit.
>>>>>
>>>>> Best,
>>>>> Jingsong
>>>>>
>>>>> On Wed, Nov 24, 2021 at 5:01 AM Stephan Ewen <ew...@gmail.com>
>>>> wrote:
>>>>>>
>>>>>> Thanks for digging into this.
>>>>>> Regarding this query:
>>>>>>
>>>>>> INSERT INTO the_table
>>>>>>    SELECT window_end, COUNT(*)
>>>>>>      FROM (TUMBLE(TABLE interactions, DESCRIPTOR(ts), INTERVAL '5'
>>>> MINUTES))
>>>>>> GROUP BY window_end
>>>>>>    HAVING now() - window_end <= INTERVAL '14' DAYS;
>>>>>>
>>>>>> I am not sure I understand what the conclusion is on the data
>>>> retention question, where the continuous streaming SQL query has retention
>>>> semantics. I think we would need to answer the following questions (I will
>>>> call the query that computed the managed table the "view materializer
>>>> query" - VMQ).
>>>>>>
>>>>>> (1) I guess the VMQ will send no updates for windows beyond the
>>>> "retention period" is over (14 days), as you said. That makes sense.
>>>>>>
>>>>>> (2) Will the VMQ send retractions so that the data will be removed
>>>> from the table (via compactions)?
>>>>>>    - if yes, this seems semantically better for users, but it will be
>>>> expensive to keep the timers for retractions.
>>>>>>    - if not, we can still solve this by adding filters to queries
>>>> against the managed table, as long as these queries are in Flink.
>>>>>>    - any subscriber to the changelog stream would not see strictly a
>>>> correct result if we are not doing the retractions
>>>>>>
>>>>>> (3) Do we want time retention semantics handled by the compaction?
>>>>>>    - if we say that we lazily apply the deletes in the queries that
>>>> read the managed tables, then we could also age out the old data during
>>>> compaction.
>>>>>>    - that is cheap, but it might be too much of a special case to be
>>>> very relevant here.
>>>>>>
>>>>>> (4) Do we want to declare those types of queries "out of scope"
>>>> initially?
>>>>>>    - if yes, how many users are we affecting? (I guess probably not
>>>> many, but would be good to hear some thoughts from others on this)
>>>>>>    - should we simply reject such queries in the optimizer as "not
>>>> possible to support in managed tables"? I would suggest that, always better
>>>> to tell users exactly what works and what not, rather than letting them be
>>>> surprised in the end. Users can still remove the HAVING clause if they want
>>>> the query to run, and that would be better than if the VMQ just silently
>>>> ignores those semantics.
>>>>>>
>>>>>> Thanks,
>>>>>> Stephan
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> Best, Jingsong Lee
>>>>
>>>>
>>>>
>>>> --
>>>> Best, Jingsong Lee
>>>>
>>>
> 
> 
> 


Re: [DISCUSS] FLIP-188: Introduce Built-in Dynamic Table Storage

Posted by Jingsong Li <ji...@gmail.com>.
Thanks Till for your suggestions.

Personally, I like flink-warehouse, this is what we want to convey to
the user, but it indicates a bit too much scope.

How about just calling it flink-store?
Simply to convey an impression: this is flink's store project,
providing a built-in store for the flink compute engine, which can be
used by flink-table as well as flink-datastream.

Best,
Jingsong

On Tue, Dec 28, 2021 at 5:15 PM Till Rohrmann <tr...@apache.org> wrote:
>
> Hi Jingsong,
>
> I think that developing flink-dynamic-storage as a separate sub project is
> a very good idea since it allows us to move a lot faster and decouple
> releases from Flink. Hence big +1.
>
> Do we want to name it flink-dynamic-storage or shall we use a more
> descriptive name? dynamic-storage sounds a bit generic to me and I wouldn't
> know that this has something to do with letting Flink manage your tables
> and their storage. I don't have a very good idea but maybe we can call it
> flink-managed-tables, flink-warehouse, flink-olap or so.
>
> Cheers,
> Till
>
> On Tue, Dec 28, 2021 at 9:49 AM Martijn Visser <ma...@ververica.com>
> wrote:
>
> > Hi Jingsong,
> >
> > That sounds promising! +1 from my side to continue development under
> > flink-dynamic-storage as a Flink subproject. I think having a more in-depth
> > interface will benefit everyone.
> >
> > Best regards,
> >
> > Martijn
> >
> > On Tue, 28 Dec 2021 at 04:23, Jingsong Li <ji...@gmail.com> wrote:
> >
> >> Hi all,
> >>
> >> After some experimentation, we felt no problem putting the dynamic
> >> storage outside of flink, and it also allowed us to design the
> >> interface in more depth.
> >>
> >> What do you think? If there is no problem, I am asking for PMC's help
> >> here: we want to propose flink-dynamic-storage as a flink subproject,
> >> and we want to build the project under apache.
> >>
> >> Best,
> >> Jingsong
> >>
> >>
> >> On Wed, Nov 24, 2021 at 8:10 PM Jingsong Li <ji...@gmail.com>
> >> wrote:
> >> >
> >> > Hi Stephan,
> >> >
> >> > Thanks for your reply.
> >> >
> >> > Data never expires automatically.
> >> >
> >> > If there is a need for data retention, the user can choose one of the
> >> > following options:
> >> > - In the SQL for querying the managed table, users filter the data by
> >> themselves
> >> > - Define the time partition, and users can delete the expired
> >> > partition by themselves. (DROP PARTITION ...)
> >> > - In the future version, we will support the "DELETE FROM" statement,
> >> > users can delete the expired data according to the conditions.
> >> >
> >> > So to answer your question:
> >> >
> >> > > Will the VMQ send retractions so that the data will be removed from
> >> the table (via compactions)?
> >> >
> >> > The current implementation is not sending retraction, which I think
> >> > theoretically should be sent, currently the user can filter by
> >> > subsequent conditions.
> >> > And yes, the subscriber would not see strictly a correct result. I
> >> > think this is something we can improve for Flink SQL.
> >> >
> >> > > Do we want time retention semantics handled by the compaction?
> >> >
> >> > Currently, no, Data never expires automatically.
> >> >
> >> > > Do we want to declare those types of queries "out of scope" initially?
> >> >
> >> > I think we want users to be able to use three options above to
> >> > accomplish their requirements.
> >> >
> >> > I will update FLIP to make the definition clearer and more explicit.
> >> >
> >> > Best,
> >> > Jingsong
> >> >
> >> > On Wed, Nov 24, 2021 at 5:01 AM Stephan Ewen <ew...@gmail.com>
> >> wrote:
> >> > >
> >> > > Thanks for digging into this.
> >> > > Regarding this query:
> >> > >
> >> > > INSERT INTO the_table
> >> > >   SELECT window_end, COUNT(*)
> >> > >     FROM (TUMBLE(TABLE interactions, DESCRIPTOR(ts), INTERVAL '5'
> >> MINUTES))
> >> > > GROUP BY window_end
> >> > >   HAVING now() - window_end <= INTERVAL '14' DAYS;
> >> > >
> >> > > I am not sure I understand what the conclusion is on the data
> >> retention question, where the continuous streaming SQL query has retention
> >> semantics. I think we would need to answer the following questions (I will
> >> call the query that computed the managed table the "view materializer
> >> query" - VMQ).
> >> > >
> >> > > (1) I guess the VMQ will send no updates for windows beyond the
> >> "retention period" is over (14 days), as you said. That makes sense.
> >> > >
> >> > > (2) Will the VMQ send retractions so that the data will be removed
> >> from the table (via compactions)?
> >> > >   - if yes, this seems semantically better for users, but it will be
> >> expensive to keep the timers for retractions.
> >> > >   - if not, we can still solve this by adding filters to queries
> >> against the managed table, as long as these queries are in Flink.
> >> > >   - any subscriber to the changelog stream would not see strictly a
> >> correct result if we are not doing the retractions
> >> > >
> >> > > (3) Do we want time retention semantics handled by the compaction?
> >> > >   - if we say that we lazily apply the deletes in the queries that
> >> read the managed tables, then we could also age out the old data during
> >> compaction.
> >> > >   - that is cheap, but it might be too much of a special case to be
> >> very relevant here.
> >> > >
> >> > > (4) Do we want to declare those types of queries "out of scope"
> >> initially?
> >> > >   - if yes, how many users are we affecting? (I guess probably not
> >> many, but would be good to hear some thoughts from others on this)
> >> > >   - should we simply reject such queries in the optimizer as "not
> >> possible to support in managed tables"? I would suggest that, always better
> >> to tell users exactly what works and what not, rather than letting them be
> >> surprised in the end. Users can still remove the HAVING clause if they want
> >> the query to run, and that would be better than if the VMQ just silently
> >> ignores those semantics.
> >> > >
> >> > > Thanks,
> >> > > Stephan
> >> > >
> >> >
> >> >
> >> > --
> >> > Best, Jingsong Lee
> >>
> >>
> >>
> >> --
> >> Best, Jingsong Lee
> >>
> >



-- 
Best, Jingsong Lee

Re: [DISCUSS] FLIP-188: Introduce Built-in Dynamic Table Storage

Posted by Till Rohrmann <tr...@apache.org>.
Hi Jingsong,

I think that developing flink-dynamic-storage as a separate sub project is
a very good idea since it allows us to move a lot faster and decouple
releases from Flink. Hence big +1.

Do we want to name it flink-dynamic-storage or shall we use a more
descriptive name? dynamic-storage sounds a bit generic to me and I wouldn't
know that this has something to do with letting Flink manage your tables
and their storage. I don't have a very good idea but maybe we can call it
flink-managed-tables, flink-warehouse, flink-olap or so.

Cheers,
Till

On Tue, Dec 28, 2021 at 9:49 AM Martijn Visser <ma...@ververica.com>
wrote:

> Hi Jingsong,
>
> That sounds promising! +1 from my side to continue development under
> flink-dynamic-storage as a Flink subproject. I think having a more in-depth
> interface will benefit everyone.
>
> Best regards,
>
> Martijn
>
> On Tue, 28 Dec 2021 at 04:23, Jingsong Li <ji...@gmail.com> wrote:
>
>> Hi all,
>>
>> After some experimentation, we felt no problem putting the dynamic
>> storage outside of flink, and it also allowed us to design the
>> interface in more depth.
>>
>> What do you think? If there is no problem, I am asking for PMC's help
>> here: we want to propose flink-dynamic-storage as a flink subproject,
>> and we want to build the project under apache.
>>
>> Best,
>> Jingsong
>>
>>
>> On Wed, Nov 24, 2021 at 8:10 PM Jingsong Li <ji...@gmail.com>
>> wrote:
>> >
>> > Hi Stephan,
>> >
>> > Thanks for your reply.
>> >
>> > Data never expires automatically.
>> >
>> > If there is a need for data retention, the user can choose one of the
>> > following options:
>> > - In the SQL for querying the managed table, users filter the data by
>> themselves
>> > - Define the time partition, and users can delete the expired
>> > partition by themselves. (DROP PARTITION ...)
>> > - In the future version, we will support the "DELETE FROM" statement,
>> > users can delete the expired data according to the conditions.
>> >
>> > So to answer your question:
>> >
>> > > Will the VMQ send retractions so that the data will be removed from
>> the table (via compactions)?
>> >
>> > The current implementation is not sending retraction, which I think
>> > theoretically should be sent, currently the user can filter by
>> > subsequent conditions.
>> > And yes, the subscriber would not see strictly a correct result. I
>> > think this is something we can improve for Flink SQL.
>> >
>> > > Do we want time retention semantics handled by the compaction?
>> >
>> > Currently, no, Data never expires automatically.
>> >
>> > > Do we want to declare those types of queries "out of scope" initially?
>> >
>> > I think we want users to be able to use three options above to
>> > accomplish their requirements.
>> >
>> > I will update FLIP to make the definition clearer and more explicit.
>> >
>> > Best,
>> > Jingsong
>> >
>> > On Wed, Nov 24, 2021 at 5:01 AM Stephan Ewen <ew...@gmail.com>
>> wrote:
>> > >
>> > > Thanks for digging into this.
>> > > Regarding this query:
>> > >
>> > > INSERT INTO the_table
>> > >   SELECT window_end, COUNT(*)
>> > >     FROM (TUMBLE(TABLE interactions, DESCRIPTOR(ts), INTERVAL '5'
>> MINUTES))
>> > > GROUP BY window_end
>> > >   HAVING now() - window_end <= INTERVAL '14' DAYS;
>> > >
>> > > I am not sure I understand what the conclusion is on the data
>> retention question, where the continuous streaming SQL query has retention
>> semantics. I think we would need to answer the following questions (I will
>> call the query that computed the managed table the "view materializer
>> query" - VMQ).
>> > >
>> > > (1) I guess the VMQ will send no updates for windows beyond the
>> "retention period" is over (14 days), as you said. That makes sense.
>> > >
>> > > (2) Will the VMQ send retractions so that the data will be removed
>> from the table (via compactions)?
>> > >   - if yes, this seems semantically better for users, but it will be
>> expensive to keep the timers for retractions.
>> > >   - if not, we can still solve this by adding filters to queries
>> against the managed table, as long as these queries are in Flink.
>> > >   - any subscriber to the changelog stream would not see strictly a
>> correct result if we are not doing the retractions
>> > >
>> > > (3) Do we want time retention semantics handled by the compaction?
>> > >   - if we say that we lazily apply the deletes in the queries that
>> read the managed tables, then we could also age out the old data during
>> compaction.
>> > >   - that is cheap, but it might be too much of a special case to be
>> very relevant here.
>> > >
>> > > (4) Do we want to declare those types of queries "out of scope"
>> initially?
>> > >   - if yes, how many users are we affecting? (I guess probably not
>> many, but would be good to hear some thoughts from others on this)
>> > >   - should we simply reject such queries in the optimizer as "not
>> possible to support in managed tables"? I would suggest that, always better
>> to tell users exactly what works and what not, rather than letting them be
>> surprised in the end. Users can still remove the HAVING clause if they want
>> the query to run, and that would be better than if the VMQ just silently
>> ignores those semantics.
>> > >
>> > > Thanks,
>> > > Stephan
>> > >
>> >
>> >
>> > --
>> > Best, Jingsong Lee
>>
>>
>>
>> --
>> Best, Jingsong Lee
>>
>

Re: [DISCUSS] FLIP-188: Introduce Built-in Dynamic Table Storage

Posted by Martijn Visser <ma...@ververica.com>.
Hi Jingsong,

That sounds promising! +1 from my side to continue development under
flink-dynamic-storage as a Flink subproject. I think having a more in-depth
interface will benefit everyone.

Best regards,

Martijn

On Tue, 28 Dec 2021 at 04:23, Jingsong Li <ji...@gmail.com> wrote:

> Hi all,
>
> After some experimentation, we felt no problem putting the dynamic
> storage outside of flink, and it also allowed us to design the
> interface in more depth.
>
> What do you think? If there is no problem, I am asking for PMC's help
> here: we want to propose flink-dynamic-storage as a flink subproject,
> and we want to build the project under apache.
>
> Best,
> Jingsong
>
>
> On Wed, Nov 24, 2021 at 8:10 PM Jingsong Li <ji...@gmail.com>
> wrote:
> >
> > Hi Stephan,
> >
> > Thanks for your reply.
> >
> > Data never expires automatically.
> >
> > If there is a need for data retention, the user can choose one of the
> > following options:
> > - In the SQL for querying the managed table, users filter the data by
> themselves
> > - Define the time partition, and users can delete the expired
> > partition by themselves. (DROP PARTITION ...)
> > - In the future version, we will support the "DELETE FROM" statement,
> > users can delete the expired data according to the conditions.
> >
> > So to answer your question:
> >
> > > Will the VMQ send retractions so that the data will be removed from
> the table (via compactions)?
> >
> > The current implementation is not sending retraction, which I think
> > theoretically should be sent, currently the user can filter by
> > subsequent conditions.
> > And yes, the subscriber would not see strictly a correct result. I
> > think this is something we can improve for Flink SQL.
> >
> > > Do we want time retention semantics handled by the compaction?
> >
> > Currently, no, Data never expires automatically.
> >
> > > Do we want to declare those types of queries "out of scope" initially?
> >
> > I think we want users to be able to use three options above to
> > accomplish their requirements.
> >
> > I will update FLIP to make the definition clearer and more explicit.
> >
> > Best,
> > Jingsong
> >
> > On Wed, Nov 24, 2021 at 5:01 AM Stephan Ewen <ew...@gmail.com>
> wrote:
> > >
> > > Thanks for digging into this.
> > > Regarding this query:
> > >
> > > INSERT INTO the_table
> > >   SELECT window_end, COUNT(*)
> > >     FROM (TUMBLE(TABLE interactions, DESCRIPTOR(ts), INTERVAL '5'
> MINUTES))
> > > GROUP BY window_end
> > >   HAVING now() - window_end <= INTERVAL '14' DAYS;
> > >
> > > I am not sure I understand what the conclusion is on the data
> retention question, where the continuous streaming SQL query has retention
> semantics. I think we would need to answer the following questions (I will
> call the query that computed the managed table the "view materializer
> query" - VMQ).
> > >
> > > (1) I guess the VMQ will send no updates for windows beyond the
> "retention period" is over (14 days), as you said. That makes sense.
> > >
> > > (2) Will the VMQ send retractions so that the data will be removed
> from the table (via compactions)?
> > >   - if yes, this seems semantically better for users, but it will be
> expensive to keep the timers for retractions.
> > >   - if not, we can still solve this by adding filters to queries
> against the managed table, as long as these queries are in Flink.
> > >   - any subscriber to the changelog stream would not see strictly a
> correct result if we are not doing the retractions
> > >
> > > (3) Do we want time retention semantics handled by the compaction?
> > >   - if we say that we lazily apply the deletes in the queries that
> read the managed tables, then we could also age out the old data during
> compaction.
> > >   - that is cheap, but it might be too much of a special case to be
> very relevant here.
> > >
> > > (4) Do we want to declare those types of queries "out of scope"
> initially?
> > >   - if yes, how many users are we affecting? (I guess probably not
> many, but would be good to hear some thoughts from others on this)
> > >   - should we simply reject such queries in the optimizer as "not
> possible to support in managed tables"? I would suggest that, always better
> to tell users exactly what works and what not, rather than letting them be
> surprised in the end. Users can still remove the HAVING clause if they want
> the query to run, and that would be better than if the VMQ just silently
> ignores those semantics.
> > >
> > > Thanks,
> > > Stephan
> > >
> >
> >
> > --
> > Best, Jingsong Lee
>
>
>
> --
> Best, Jingsong Lee
>