[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17067014#comment-17067014 ] Reuven Lax commented on BEAM-9557: -- pr/11226 sent for review > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > Time Spent: 10m > Remaining Estimate: 0h > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17066892#comment-17066892 ] Reuven Lax commented on BEAM-9557: -- ok, this seems simple. I'll send out a PR. > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17066643#comment-17066643 ] Steve Niemitz commented on BEAM-9557: - ah ok, yeah I agree! > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17066364#comment-17066364 ] Reuven Lax commented on BEAM-9557: -- Ah yes, I was referring to processing-time timers. For event-time timers, I believe that we need to maintain the invariant that outputTimestamp <= timerTimestamp < window_end + allowed_lateness > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17066280#comment-17066280 ] Steve Niemitz commented on BEAM-9557: - The default case will work as you mentioned, but if the validation is on output time, I could set an event time timer for eg: given a window end = 03:00:00 outputTimestamp = 02:59:59 target = 03:05:00 The timer will hold the watermark correctly, but the state cleanup timer will fire at 03:00:00 (since it fires off the input watermark) and clean up the state in that window, then my timer will fire 5 minutes later and observe up an empty state. The current validation catches this and will reject the timer. > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17066023#comment-17066023 ] Reuven Lax commented on BEAM-9557: -- I'm not sure why the simple solution I proposed does not work. For event-time timer, the default hold will stay exactly where it was. For processing-time timers, I think the commit referenced contained a typo, and that typo caused the bug; the default hold should always be inside the current window. > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17066010#comment-17066010 ] Rui Wang commented on BEAM-9557: I understand people have their schedules. But this Jira is marked as blocker for 2.20.0 release. Could relevant people have a conclusion soon for wether this Jira should be a blocker or propose a fix? > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17065220#comment-17065220 ] Steve Niemitz commented on BEAM-9557: - I had actually tried that (for other reasons) in the past. The problem is that the state cleanup timers fire based on the input watermark, so unless you're careful you can set an (event time) timer that fires after state is cleaned up. I guess you could do this anyways with processing time timers, so maybe it'd be ok to allow? It's just a little more unexpected w/ event time. I think the best solution is to respect the watermark hold only if it is set (for processing time timers), otherwise to default back to the existing behavior. > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17065112#comment-17065112 ] Reuven Lax commented on BEAM-9557: -- This looks like it might be a type bug? I think that checkArgument(!target.isAfter(windowExpiry)) should actually have been checkArgument(!outputTimestamp.isAfter(windowExpiry), > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17065090#comment-17065090 ] Steve Niemitz commented on BEAM-9557: - imo this should be a blocker for 2.20. It breaks most use of processing time timers, including built-in transforms like Deduplicate. Is this a regression from a previous release? *Yes* Is this a new feature or related to a new feature? Kind of? *It's an enhancement to an existing feature.* What percentage of users would be impacted by this issue if it is not fixed? *Unknown* Would it be possible for the impacted users to skip this version? *Unknown*. Personally I know I won't skip the version, but just revert the commit above in our own fork. Many other users likely won't be so happy to do so. > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17065071#comment-17065071 ] Rui Wang commented on BEAM-9557: If you still want to make this Jira release into 2.20.0, could you answer the following question to help me make a decision if I should adopt it: Is this a regression from a previous release? Is this a new feature or related to a new feature? What percentage of users would be impacted by this issue if it is not fixed? Would it be possible for the impacted users to skip this version? > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17065070#comment-17065070 ] Kenneth Knowles commented on BEAM-9557: --- Processing time is independent of event time, so the "end of window" should not be relevant. This is definitely a bug. Previously the output timestamp for processing time timers would always be the watermark time when fired. Now it should have that same behavior if unspecified, but otherwise the provided event time timestamp takes effect. > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > Fix For: 2.20.0 > > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17065036#comment-17065036 ] Kenneth Knowles commented on BEAM-9557: --- Is this caused by the default output timestamp of processing time timer? > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Affects Versions: 2.20.0 >Reporter: Steve Niemitz >Assignee: Reuven Lax >Priority: Critical > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)
[jira] [Commented] (BEAM-9557) Error setting processing time timers near end-of-window
[ https://issues.apache.org/jira/browse/BEAM-9557?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17064984#comment-17064984 ] Ahmet Altay commented on BEAM-9557: --- For reference commit from description (a005fd765a762183ca88df90f261f6d4a20cf3e0) was added in (https://github.com/apache/beam/pull/10627) [~reuvenlax] > Error setting processing time timers near end-of-window > --- > > Key: BEAM-9557 > URL: https://issues.apache.org/jira/browse/BEAM-9557 > Project: Beam > Issue Type: Bug > Components: runner-core >Affects Versions: 2.20.0 >Reporter: Steve Niemitz >Priority: Critical > > Previously, it was possible to set a processing time timer past the end of a > window, and it would simply not fire. > However, now, this results in an error: > {code:java} > java.lang.IllegalArgumentException: Attempted to set event time timer that > outputs for 2020-03-19T18:01:35.000Z but that is after the expiration of > window 2020-03-19T17:59:59.999Z > > org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument(Preconditions.java:440) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setAndVerifyOutputTimestamp(SimpleDoFnRunner.java:1011) > > org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$TimerInternalsTimer.setRelative(SimpleDoFnRunner.java:934) > .processElement(???.scala:187) > {code} > > I think the regression was introduced in commit > a005fd765a762183ca88df90f261f6d4a20cf3e0. Also notice that the error message > is wrong, it says that "event time timer" but the timer is in the processing > time domain. -- This message was sent by Atlassian Jira (v8.3.4#803005)