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

ASF GitHub Bot commented on FLINK-8384:
---------------------------------------

Github user dyanarose commented on the issue:

    https://github.com/apache/flink/pull/5295
  
    I'm not the biggest fan of unchecked casts, but testing this in our POC 
environment casting the existing EventTimeTrigger to a typed Trigger<T, W> is 
working. So if the unchecked cast is acceptable, that would get rid of all 
changes required to Event/ProcessingTimeTrigger
    
    On your second point. (sorry if I mis-attribute this based on github 
profile name) I believe you had mentioned breaking this out into new classes 
when I first brought this up on the mailing list 
https://lists.apache.org/thread.html/6ceb094460bca8e9731e9e1dc0bb479f407f769458bff30c412adf78@%3Cdev.flink.apache.org%3E
    
    Now that you see it in action, do you feel it would be better off as an 
addition to the existing session window classes?
    
    The way I see it is, if I put withGapExtractor() on the existing classes, 
without adding type information to them, then the extract() method on 
SessionWindowGapExtractor will need to have the signature of extract(Object 
input) leaving the implementer to have to cast to the input type.
    
    I have to admit it feels strange to me that the Window assigners all drop 
input type information. But that would mean that these new typed assigners 
would be the odd ones out.


> Session Window Assigner with Dynamic Gaps
> -----------------------------------------
>
>                 Key: FLINK-8384
>                 URL: https://issues.apache.org/jira/browse/FLINK-8384
>             Project: Flink
>          Issue Type: Improvement
>          Components: Streaming
>            Reporter: Dyana Rose
>            Priority: Minor
>
> *Reason for Improvement*
> Currently both Session Window assigners only allow a static inactivity gap. 
> Given the following scenario, this is too restrictive:
> * Given a stream of IoT events from many device types
> * Assume each device type could have a different inactivity gap
> * Assume each device type gap could change while sessions are in flight
> By allowing dynamic inactivity gaps, the correct gap can be determined in the 
> [assignWindows 
> function|https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/assigners/EventTimeSessionWindows.java#L59]
>  by passing the element currently under consideration, the timestamp, and the 
> context to a user defined function. This eliminates the need to create 
> unwieldy work arounds if you only have static gaps.
> Dynamic Session Window gaps should be available for both Event Time and 
> Processing Time streams.
> (short preliminary discussion: 
> https://lists.apache.org/thread.html/08a011c0039e826343e9be0b1bb4e0000cfc2e12976bc65f8a43ee88@%3Cdev.flink.apache.org%3E)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to