You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Aljoscha Krettek (JIRA)" <ji...@apache.org> on 2016/05/30 12:38:12 UTC

[jira] [Updated] (FLINK-3714) Add Support for "Allowed Lateness"

     [ https://issues.apache.org/jira/browse/FLINK-3714?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Aljoscha Krettek updated FLINK-3714:
------------------------------------
    Description: 
As mentioned in https://docs.google.com/document/d/1Xp-YBf87vLTduYSivgqWVEMjYUmkA-hyb4muX3KRl08/edit# we should add support for an allowed lateness setting.

This includes several things:

 - API for setting allowed lateness
 - Dropping of late elements 
 - Garbage collection of windows state/timers

Depending on whether the {{WindowAssigner}} assigns windows based on event time or processing time we have to adjust the GC behavior. For event-time windows "allowed lateness" makes sense and we should garbage collect after this expires. For processing-time windows "allowed lateness" does not make sense and we should always GC window state/timers at the end timestamp of a processing-time window. I think that we need a method for this on {{WindowAssigner}} that allows to differentiate between event-time windows and processing-time windows: {{boolean WindowAssigner.isEventTime()}}.


  was:
As mentioned in https://docs.google.com/document/d/1Xp-YBf87vLTduYSivgqWVEMjYUmkA-hyb4muX3KRl08/edit# we should add support for an allowed lateness setting.

This includes several things:

 - API for setting allowed lateness
 - Dropping of late elements 
 - Garbage collection of event-time windows 

Lateness only makes sense for event-time windows. So we also have to figure out what the API for this should look like and especially what should happen with the "stream-time characteristic" switch. For example in this:

{code}
env.setStreamTimeCharacteristic(ProcessingTime)
...
DataStream in = ...

result = in
  .keyBy()
  .timeWindow()
  .allowedLateness()
  .apply()
{code}

I think the setting can be silently ignored when doing processing-time windowing.



> Add Support for "Allowed Lateness"
> ----------------------------------
>
>                 Key: FLINK-3714
>                 URL: https://issues.apache.org/jira/browse/FLINK-3714
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Streaming
>            Reporter: Aljoscha Krettek
>            Assignee: Kostas Kloudas
>
> As mentioned in https://docs.google.com/document/d/1Xp-YBf87vLTduYSivgqWVEMjYUmkA-hyb4muX3KRl08/edit# we should add support for an allowed lateness setting.
> This includes several things:
>  - API for setting allowed lateness
>  - Dropping of late elements 
>  - Garbage collection of windows state/timers
> Depending on whether the {{WindowAssigner}} assigns windows based on event time or processing time we have to adjust the GC behavior. For event-time windows "allowed lateness" makes sense and we should garbage collect after this expires. For processing-time windows "allowed lateness" does not make sense and we should always GC window state/timers at the end timestamp of a processing-time window. I think that we need a method for this on {{WindowAssigner}} that allows to differentiate between event-time windows and processing-time windows: {{boolean WindowAssigner.isEventTime()}}.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)