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

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

                Author: ASF GitHub Bot
            Created on: 12/Oct/20 18:39
            Start Date: 12/Oct/20 18:39
    Worklog Time Spent: 10m 
      Work Description: scwhittle commented on a change in pull request #13032:
URL: https://github.com/apache/beam/pull/13032#discussion_r503470326



##########
File path: 
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java
##########
@@ -486,14 +487,23 @@ private void processTimers(
     for (W window : windowsToCleanup) {
       // The stepContext is the thing that know if it is batch or streaming, 
hence
       // whether state needs to be cleaned up or will simply be discarded so 
the
-      // timer can be ignored
-
+      // timer can be ignored.
       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())

Review comment:
       I'm not sure I understand your suggestion. 
   If we eliminate this condition, we are back to always setting timers which 
ends up accumulating timers for the global window.  If we eliminate setting the 
timer if the user did not specify onWindowExpiration, we are removing the 
automatic cleanup of windowed user-state for the non global window.
   
   Are you are concerned about setting the timers if onWindowExpiration was 
requested for the global window? That is true, but at least the user can work 
around that by removing the onWindowExpiration annotation and modifying their 
code. Currently there is no mechanism in the dataflow streaming backend to scan 
and delivering an onDrain notification for all keys with user state, though 
that would be possible to add.  Any onDrain mechanism currently would have to 
be powered by timers set for the end of time, which would have the same issues 
as the cleanup timer.
   
   
   




----------------------------------------------------------------
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: 499574)
    Time Spent: 2h  (was: 1h 50m)

> 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
>             Fix For: 2.25.0
>
>          Time Spent: 2h
>  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