Thank you, Fabian! I've created the FLIP-25 page
<https://cwiki.apache.org/confluence/display/FLINK/FLIP-25%3A+Support+User+State+TTL+Natively>
.

To continue our discussion points:
1. I see what you mean now. I totally agree. Since we don't completely know
it now, shall we experiment or prototype a little bit before deciding this?
2. -
3. Adding tags to timers is an option.

Another option I came up with recently, is like this: let *InternalTimerService
*maintains user timers and TTL timers separately. Implementation classes of
InternalTimerService should add two new collections of timers,  e.g.
*Ttl*ProcessingTimeTimersQueue
and *Ttl*EventTimeTimersQueue for HeapInternalTimerService. Upon
InternalTimerService#onProcessingTime() and advanceWatermark(), they will
first iterate through ProcessingTimeTimers and EventTimeTimers (user
timers) and then through *Ttl*ProcessingTimeTimers and *Ttl*EventTimeTimers
(Ttl timers).

We'll also add the following new internal APIs to register Ttl timers:

```
@Internal
public void registerTtlProcessingTimeTimer(N namespace, long time);

@Internal
public void registerTtlEventTimeTimer(N namespace, long time);
```

The biggest advantage, compared to option 1, is that it doesn't impact
existing timer-related checkpoint/savepoint, restore and migrations.

What do you think?  And, any other Flink committers want to chime in for
ideas? I've also documented the above two discussion points to the FLIP
page.

Thanks,
Bowen


On Wed, May 2, 2018 at 5:36 AM, Fabian Hueske <fhue...@gmail.com> wrote:

> Hi Bowen,
>
> 1. The motivation to keep the TTL logic outside of the state backend was
> mainly to avoid state backend custom implementations. If we have a generic
> approach that would work for all state backends, we could try to put the
> logic into a base class like AbstractStateBackend. After all, state cleanup
> is tightly related to the responsibilities of state backends.
> 2. -
> 3. You're right. We should first call the user code before cleaning up.
> The main problem that I see right now is that we have to distinguish
> between user and TTL timers. AFAIK, the timer service does not support
> timer tags (or another method) to distinguish timers.
>
> I've given you the permissions to create and edit wiki pages.
>
> Best, Fabian
>
> 2018-04-30 7:47 GMT+02:00 Bowen Li <bowenl...@gmail.com>:
>
>> Thanks Fabian! Here're my comments inline, and let me know your thoughts.
>>
>> 1. Where should the TTL code reside? In the state backend or in the
>> operator?
>>
>> I believe TTL code should not reside in state backend, because a critical
>> design is that TTL is independent of and transparent to state backends.
>>
>> According to my current knowledge, I think it probably should live with
>> operators in flink-streaming-java.
>>
>>
>> 2. How to get notified about state accesses? I guess this depends on 1.
>>
>> You previously suggested using callbacks. I believe that's the right way
>> to do decoupling.
>>
>>
>> 3. How to avoid conflicts of TTL timers and user timers?
>>
>> User timers might always be invoked first? This is not urgent, shall we
>> bake it for more time and discuss it along the way?
>>
>>
>>
>> Besides, I don't have access to create a FLIP page under
>> https://cwiki.apache.org/confluence/display/FLINK/Flin
>> k+Improvement+Proposals. Can you grant me the proper access?
>>
>> Thanks,
>>
>> Bowen
>>
>>
>>
>>
>> On Tue, Apr 24, 2018 at 2:40 AM, Fabian Hueske <fhue...@gmail.com> wrote:
>>
>>> Hi Bowen,
>>>
>>> Thanks for updating the proposal. This looks pretty good (as I said
>>> before).
>>> There are a few areas, that are not yet fully fleshed out:
>>>
>>> 1. Where should the TTL code reside? In the state backend or in the
>>> operator?
>>> 2. How to get notified about state accesses? I guess this depends on 1.
>>> 3. How to avoid conflicts of TTL timers and user timers?
>>>
>>> @Stefan (in CC) might have some ideas on these issues as well.
>>>
>>> Cheers, Fabian
>>>
>>> 2018-04-22 21:14 GMT+02:00 Bowen <bowenl...@gmail.com>:
>>>
>>>> Hello community,
>>>>
>>>> We've come up with a completely new design for Flink state TTL, documented
>>>> here
>>>> <https://docs.google.com/document/d/1PPwHMDHGWMOO8YGv08BNi8Fqe_h7SjeALRzmW-ZxSfY/edit?usp=sharing>,
>>>> and have run it by a few Flink PMC/committers.
>>>>
>>>> What do you think? We'd love to hear feedbacks from you
>>>>
>>>> Thanks,
>>>> Bowen
>>>>
>>>>
>>>> On Wed, Feb 7, 2018 at 12:53 PM, Fabian Hueske <fhue...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi Bowen,
>>>>>
>>>>> Thanks for the proposal! I think state TTL would be a great feature!
>>>>> Actually, we have implemented this for SQL / Table API [1].
>>>>> I've added a couple of comments to the design doc.
>>>>>
>>>>> In principle, I'm not sure if this functionality should be added to the
>>>>> state backends.
>>>>> We could also use the existing timer service which would have a few
>>>>> nice
>>>>> benefits (see my comments in the docs).
>>>>>
>>>>> Best, Fabian
>>>>>
>>>>> [1]
>>>>> https://ci.apache.org/projects/flink/flink-docs-release-1.4/
>>>>> dev/table/streaming.html#idle-state-retention-time
>>>>>
>>>>> 2018-02-06 8:26 GMT+01:00 Bowen Li <bowenl...@gmail.com>:
>>>>>
>>>>> > Hi guys,
>>>>> >
>>>>> > I want to propose a new FLIP -- FLIP-25 - Support User State TTL
>>>>> Natively
>>>>> > in Flink. This has been one of most handy and most frequently asked
>>>>> > features in Flink community. The jira ticket is FLINK-3089
>>>>> > <https://issues.apache.org/jira/browse/FLINK-3089>.
>>>>> >
>>>>> > I've written a rough design
>>>>> > <https://docs.google.com/document/d/1qiFqtCC80n4oFmmfxBWXrCd37mYKc
>>>>> > uureyEr_nPAvSo/edit#>
>>>>> > doc
>>>>> > <https://docs.google.com/document/d/1qiFqtCC80n4oFmmfxBWXrCd37mYKc
>>>>> > uureyEr_nPAvSo/edit#>,
>>>>> > and developed prototypes for both heap and rocksdb state backends.
>>>>> >
>>>>> > My question is: shall we create a FLIP page for this? Can I be
>>>>> granted the
>>>>> > privileges of creating pages in
>>>>> > https://cwiki.apache.org/confluence/display/FLINK/
>>>>> > Flink+Improvement+Proposals
>>>>> > ?
>>>>> >
>>>>> > Thanks,
>>>>> > Bowen
>>>>> >
>>>>>
>>>>
>>>>
>>>
>>
>

Reply via email to