You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Elias Levy (JIRA)" <ji...@apache.org> on 2017/04/01 18:05:42 UTC

[jira] [Commented] (FLINK-3089) State API Should Support Data Expiration

    [ https://issues.apache.org/jira/browse/FLINK-3089?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15952328#comment-15952328 ] 

Elias Levy commented on FLINK-3089:
-----------------------------------

I gather that {{TimelyFlatMapFunction}} has been supplanted by the {{TimerService}} available via the process function context.  That may be a viable solution in some situation, but it may not be sufficient in others.  

I note that the {{TimerService}} appears to be implemented via {{HeapInternalTimerService}}, which keeps the timers in the heap.  For jobs processing billions of keys with long lived timers (days), keeping them in memory may not be feasible.  

For such jobs, it would also add a massive amount of state that must be snapshotted.

In addition, it is not clear to me whether setting a new timer for a key removes a previous one.  If it doesn't, there may be large amount of spurious timers when you are trying to model a cache with a TTL based on last update.  That said, looking at the code I think a new timer removes previous ones for the key.  Could someone confirm?

It is also less efficient than expiring state in bulk at regular intervals.

> State API Should Support Data Expiration
> ----------------------------------------
>
>                 Key: FLINK-3089
>                 URL: https://issues.apache.org/jira/browse/FLINK-3089
>             Project: Flink
>          Issue Type: New Feature
>          Components: DataStream API, State Backends, Checkpointing
>            Reporter: Niels Basjes
>
> In some usecases (webanalytics) there is a need to have a state per visitor on a website (i.e. keyBy(sessionid) ).
> At some point the visitor simply leaves and no longer creates new events (so a special 'end of session' event will not occur).
> The only way to determine that a visitor has left is by choosing a timeout, like "After 30 minutes no events we consider the visitor 'gone'".
> Only after this (chosen) timeout has expired should we discard this state.
> In the Trigger part of Windows we can set a timer and close/discard this kind of information. But that introduces the buffering effect of the window (which in some scenarios is unwanted).
> What I would like is to be able to set a timeout on a specific OperatorState value which I can update afterwards.
> This makes it possible to create a map function that assigns the right value and that discards the state automatically.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)