Timer bug in 2.37 around output timestamps?

2022-04-01 Thread Steve Niemitz
We have a job that uses processing time timers, and just upgraded from 2.33 to 2.37. Sporadically we've started seeing jobs fail with this error: java.lang.IllegalArgumentException: Cannot output with timestamp 2022-04-01T19:19:59.999Z. Output timestamps must be no earlier than the output timesta

Re: Timer bug in 2.37 around output timestamps?

2022-04-01 Thread Steve Niemitz
I'm unclear how the timer would have even ever been set to output at that timestamp though. The output timestamp falls into the next window, and if unset (like in this case) the output timestamp is derived from the element timestamp [1]. This means we somehow had an element in the wrong window? T

Re: Timer bug in 2.37 around output timestamps?

2022-04-01 Thread Steve Niemitz
> but this only impacts non-portable runners. heh like Dataflow? :( I'm not sure what the solution is here then? I managed to hit this bug within 2 hours of running my first pipeline on 2.37. I can't just live with pipelines breaking randomly, and it seems like everything worked fine before tha

Re: Timer bug in 2.37 around output timestamps?

2022-04-01 Thread Evan Galpin
I believe that this thread is entirely related to an another thread[1] where there is discussion that the correct fix for this issue could be to enforce that watermark updates would only happen at bundle boundaries. There’s another related thread[2] citing the same error with ElasticsearchIO. This

Re: Timer bug in 2.37 around output timestamps?

2022-04-01 Thread Steve Niemitz
While I do agree the symptoms are similar, I don't believe these are related. The ESIO bug was centered around both "bundle" operations (StartBundle/EndBundle) and watermark updates during bundle processing. In my case I'm not using anything related to bundle operations (no Start/EndBundle), and

Re: Timer bug in 2.37 around output timestamps?

2022-04-02 Thread Steve Niemitz
I've dug into this some more and have a couple observations/questions: - I added some logging to my DoFn in both @ProcessElement and @OnTimer, I can confirm that I never have late data coming into ProcessElement (element.timestamp() is never after the end of the window) - The OnTimer method does en

Re: Timer bug in 2.37 around output timestamps?

2022-04-02 Thread Steve Niemitz
I can confirm that changing input watermark to output watermark in SimpleDoFnRunner.onTimer [1] seems to fix this. The question is, would that have any other unintended consequences? It seems safe to me. It still seems like the best option would be to use the actual output timestamp here though.

Re: Timer bug in 2.37 around output timestamps?

2022-04-04 Thread Jan Lukavský
On 4/2/22 16:41, Steve Niemitz wrote: I've dug into this some more and have a couple observations/questions: - I added some logging to my DoFn in both @ProcessElement and @OnTimer, I can confirm that I never have late data coming into ProcessElement (element.timestamp() is never after the end o

Re: Timer bug in 2.37 around output timestamps?

2022-04-04 Thread Steve Niemitz
Oh I had forgotten about that thread, good point, it is very related to this. I agree we should fix this, to force the conversation, I've opened a PR to do so: https://github.com/apache/beam/pull/17262 As a good example of this behavior simply in the beam SDK, GroupIntoBatches exhibits this proble

Re: Timer bug in 2.37 around output timestamps?

2022-04-04 Thread Steve Niemitz
Agreed, let's resolve the conversation in that PR and we should be good to land it. On Mon, Apr 4, 2022 at 5:15 PM Reuven Lax wrote: > FYI - the bug you found is the one I was referring to. I actually had > fixed this as part of a larger PR (that fixes bugs in DirectRunner), but to > get this in