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?
This seems strangely similar to BEAM-6757 actually.

What long-standing bug were you talking about here?  We've been running
these pipelines for years now and never run into this until now, although
maybe we had been, but there was no validation in the past to catch it
until now?

[1]
https://github.com/apache/beam/blob/master/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java#L1284


On Fri, Apr 1, 2022 at 5:11 PM Reuven Lax <[email protected]> wrote:

> There is a long-standing bug with processing timestamps.
>
> On Fri, Apr 1, 2022 at 2:01 PM Steve Niemitz <[email protected]> wrote:
>
>> 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 timestamp of the timer (2022-04-01T19:20:00.000Z) minus the allowed
>> skew (0 milliseconds) and no later than 294247-01-10T04:00:54.775Z. See the
>> DoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed
>> skew.
>> at
>> org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$OnTimerArgumentProvider.checkTimestamp(SimpleDoFnRunner.java:883)
>> at
>> org.apache.beam.runners.dataflow.worker.repackaged.org.apache.beam.runners.core.SimpleDoFnRunner$OnTimerArgumentProvider.outputWithTimestamp(SimpleDoFnRunner.java:863)
>> at
>> org.apache.beam.sdk.transforms.DoFnOutputReceivers$WindowedContextOutputReceiver.outputWithTimestamp(DoFnOutputReceivers.java:85)
>> <user code>
>>
>> This windowing is configured with 10 minute fixed windows and 10 minute
>> allowed lateness.  We're not specifically setting the output time on the
>> timer, so it seems like it's getting inferred from the element timestamp?
>> The code that emits elements from the timer uses window.maxTimestamp() to
>> set the output timestamp.  I'm not sure I understand how an element with a
>> timestamp in what should be the next window ended up in the previous one?
>> Given that this is the first stateful operation in the pipeline and we read
>> from pubsub using pubsub timestamps, so there should be no late data.
>>
>> I know there was a change recently to better validate the output
>> timestamp from timers [1], I'm having trouble understanding if there's a
>> bug in that, or if this is actually exposing a real bug in our pipeline.
>>
>> [1]
>> https://github.com/apache/beam/commit/15048929495ad66963b528d5bd71eb7b4a844c96
>>
>

Reply via email to