[ 
https://issues.apache.org/jira/browse/BEAM-11034?focusedWorklogId=499499&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-499499
 ]

ASF GitHub Bot logged work on BEAM-11034:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 12/Oct/20 16:19
            Start Date: 12/Oct/20 16:19
    Worklog Time Spent: 10m 
      Work Description: reuvenlax commented on pull request #13032:
URL: https://github.com/apache/beam/pull/13032#issuecomment-707216813


   What onDrain method are you referring to?
   
   On Mon, Oct 12, 2020 at 9:16 AM Kenn Knowles <[email protected]>
   wrote:
   
   > *@kennknowles* commented on this pull request.
   > ------------------------------
   >
   > In
   > 
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java
   > <https://github.com/apache/beam/pull/13032#discussion_r503400133>:
   >
   > >        Instant cleanupTime = earliestAllowableCleanupTime(window, 
windowingStrategy);
   > -      // if DoFn has OnWindowExpiration then set holds for system timer.
   > -      Instant cleanupOutputTimestamp =
   > -          fnSignature.onWindowExpiration() == null ? cleanupTime : 
cleanupTime.minus(1L);
   > -      stepContext.setStateCleanupTimer(
   > -          CLEANUP_TIMER_ID, window, windowCoder, cleanupTime, 
cleanupOutputTimestamp);
   > +      // Set a cleanup timer for state at the end of the window to 
trigger onWindowExpiration and
   > +      // garbage collect state. We avoid doing this for the global window 
if there is no window
   > +      // expiration set as the state will be up when the pipeline 
terminates. Setting the timer
   > +      // leads to a unbounded growth of timers for pipelines with many 
unique keys in the global
   > +      // window.
   > +      if (cleanupTime.isBefore(GlobalWindow.INSTANCE.maxTimestamp())
   >
   > Re-reading the bug and thinking about your comments, can we eliminate this
   > condition and invoke from some onDrain method?
   >
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/beam/pull/13032#pullrequestreview-506754853>,
   > or unsubscribe
   > 
<https://github.com/notifications/unsubscribe-auth/AFAYJVNF2HY2AJWMWTQKBFLSKMTWLANCNFSM4SHHJYVQ>
   > .
   >
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 499499)
    Time Spent: 1h 50m  (was: 1h 40m)

> State garbage collection timers set by Dataflow SimpleParDoFn pile up for the 
> GlobalWindow
> ------------------------------------------------------------------------------------------
>
>                 Key: BEAM-11034
>                 URL: https://issues.apache.org/jira/browse/BEAM-11034
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-dataflow
>            Reporter: Sam Whittle
>            Assignee: Sam Whittle
>            Priority: P2
>          Time Spent: 1h 50m
>  Remaining Estimate: 0h
>
> If the dofn is stateful, garbage collection timers are set for the end of the 
> window plus allowed lateness:
> https://github.com/apache/beam/blob/6fdde4f4eab72b49b10a8bb1cb3be263c5c416b5/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java#L491
> For the global window this ends up setting garbage collection timers that 
> will only fire once the pipeline is drained.  For pipelines that have 
> constantly newly arriving unique stateful keys, and otherwise cleanup their 
> state appropriately when triggering occurs, the # of timers builds up over 
> time.
> Example window and trigger, where the user has the opportunity to clean up 
> state for the key after at most a minute.  However they have no control over 
> the timer set.
> GlobalWindows()
> .triggering(Repeatedly.forever(AfterFirst.of(
> AfterPane.elementCountAtLeast(5000),
> AfterProcessingTime.pastFirstElementInPane().plusDelayOf(Duration.standardMi
> nutes(1))).discardingFiredPanes().withAllowedLateness(Duration.ZERO);



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to