shunping opened a new issue, #35090: URL: https://github.com/apache/beam/issues/35090
### What happened? When we set an event-time timer in DoFn, prism (as well as direct runner) seems to trigger the timer before the water really passes the timer. This results in early firing of these timers. Below is the code to reproduce. The event comes in every second. We batch the events until the number reaches 5 and then return. On the first event, we set the timer to be event time of the first event + 10, and clear the timer at the end of the 5-element batch. In such a case, the timer is not supposed to be called. ``` import logging import apache_beam as beam from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.coders import coders from apache_beam.transforms.userstate import BagStateSpec from apache_beam.transforms.userstate import TimerSpec from apache_beam.transforms.userstate import on_timer from apache_beam.transforms.timeutil import TimeDomain from apache_beam.transforms.periodicsequence import PeriodicImpulse logging.basicConfig(level=logging.INFO) options = PipelineOptions([ "--streaming", "--environment_type=LOOPBACK", "--runner=PrismRunner", "--prism_beam_version_override=v2.65.0" #"--runner=PortableRunner", "--job_endpoint=localhost:8073", ]) class MyStatefulDoFn(beam.DoFn): ALL_ELEMENTS = BagStateSpec('buffer', coders.FloatCoder()) TIMER = TimerSpec('timer', TimeDomain.WATERMARK) def __init__(self, n): self._n = n def process(self, element_pair, t = beam.DoFn.TimestampParam, state=beam.DoFn.StateParam(ALL_ELEMENTS), timer = beam.DoFn.TimerParam(TIMER) ): state.add(element_pair[1]) all_elements = list(state.read()) # the start of new state cell if len(all_elements) == 1: print("set timer to", t+10) timer.set(t+10) # the state cell is full if len(all_elements) == self._n: print("batch process start") for e in all_elements: yield e print("batch process end") # clear the buffer state.clear() # don't need the timer now print("clear timer") timer.clear() @on_timer(TIMER) def timer_callback(self, t = beam.DoFn.TimestampParam, state = beam.DoFn.StateParam(ALL_ELEMENTS)): print("TIMER SHOULD NOT FIRE HERE!") print(f"timer callback start (timestamp={t})") all_elements = list(state.read()) print("timer callback elements:", all_elements) print("timer callback end") with beam.Pipeline(options=options) as p: _ = ( p | PeriodicImpulse(fire_interval=1) | beam.WithKeys(0) | beam.ParDo(MyStatefulDoFn(5)) | beam.Map(print)) ``` An output example from running the above code on prism from Beam 2.65.0: ``` set timer to Timestamp(1748576759.951000) batch process start 1748576749.951589 1748576750.951589 1748576751.951589 1748576752.951589 1748576753.951589 batch process end clear timer TIMER SHOULD NOT FIRE HERE! timer callback start (timestamp=Timestamp(1748576759.951000)) timer callback elements: [] timer callback end set timer to Timestamp(1748576764.951000) batch process start 1748576754.951589 1748576755.951589 1748576756.951589 1748576757.951589 1748576758.951589 batch process end clear timer TIMER SHOULD NOT FIRE HERE! timer callback start (timestamp=Timestamp(1748576764.951000)) timer callback elements: [] timer callback end set timer to Timestamp(1748576769.951000) batch process start 1748576759.951589 1748576760.951589 1748576761.951589 1748576762.951589 1748576763.951589 batch process end clear timer set timer to Timestamp(1748576774.951000) INFO:apache_beam.utils.subprocess_server:[2025-05-29T23:46:05.954524-04:00] INFO data.Recv timers for unknown bundle INFO:apache_beam.utils.subprocess_server: response: timers:{instruction_id:"inst16723" transform_id:"ref_AppliedPTransform_ParDo-MyStatefulDoFn-_12" timer_family_id:"ts-timer" is_last:true} INFO:apache_beam.utils.subprocess_server: ``` ### Issue Priority Priority: 2 (default / most bugs should be filed as P2) ### Issue Components - [ ] Component: Python SDK - [ ] Component: Java SDK - [ ] Component: Go SDK - [ ] Component: Typescript SDK - [ ] Component: IO connector - [ ] Component: Beam YAML - [ ] Component: Beam examples - [ ] Component: Beam playground - [ ] Component: Beam katas - [ ] Component: Website - [ ] Component: Infrastructure - [ ] Component: Spark Runner - [ ] Component: Flink Runner - [ ] Component: Samza Runner - [ ] Component: Twister2 Runner - [ ] Component: Hazelcast Jet Runner - [ ] Component: Google Cloud Dataflow Runner -- 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. To unsubscribe, e-mail: github-unsubscr...@beam.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org