shunping opened a new issue, #35384: URL: https://github.com/apache/beam/issues/35384
### What happened? When we use sliding windows and groupby, prism seems to fire the windows earlier than expected (event watermark has not reached window end) Here is the code to reproduce. ```python import logging import apache_beam as beam from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.transforms.window import SlidingWindows from apache_beam.transforms.periodicsequence import PeriodicImpulse from apache_beam.utils.timestamp import Timestamp logging.basicConfig(level=logging.INFO) options = PipelineOptions([ "--streaming", "--environment_type=LOOPBACK", "--runner=PrismRunner", ]) WINDOW_SIZE = 5 SLIDE_INTERVAL = 3 now = Timestamp.now() with beam.Pipeline(options=options) as p: _ = ( p | PeriodicImpulse(start_timestamp=now, stop_timestamp=now + 10, fire_interval=1) | beam.WithKeys(0) | beam.WindowInto(SlidingWindows(WINDOW_SIZE, SLIDE_INTERVAL)) | beam.GroupByKey() | beam.LogElements(level=logging.WARNING, with_timestamp=True, with_window=True)) ``` The output is something like ``` WARNING:root:(0, [1750467252.346215, 1750467253.346215]), timestamp='2025-06-21T00:54:13.999000Z', window(start=2025-06-21T00:54:09Z, end=2025-06-21T00:54:14Z) WARNING:root:(0, [1750467252.346215, 1750467253.346215, 1750467254.346215]), timestamp='2025-06-21T00:54:16.999000Z', window(start=2025-06-21T00:54:12Z, end=2025-06-21T00:54:17Z) WARNING:root:(0, [1750467255.346215, 1750467256.346215]), timestamp='2025-06-21T00:54:16.999000Z', window(start=2025-06-21T00:54:12Z, end=2025-06-21T00:54:17Z) WARNING:root:(0, [1750467255.346215, 1750467256.346215, 1750467257.346215]), timestamp='2025-06-21T00:54:19.999000Z', window(start=2025-06-21T00:54:15Z, end=2025-06-21T00:54:20Z) WARNING:root:(0, [1750467258.346215, 1750467259.346215]), timestamp='2025-06-21T00:54:19.999000Z', window(start=2025-06-21T00:54:15Z, end=2025-06-21T00:54:20Z) WARNING:root:(0, [1750467258.346215, 1750467259.346215, 1750467260.346215, 1750467261.346215]), timestamp='2025-06-21T00:54:22.999000Z', window(start=2025-06-21T00:54:18Z, end=2025-06-21T00:54:23Z) WARNING:root:(0, [1750467261.346215]), timestamp='2025-06-21T00:54:25.999000Z', window(start=2025-06-21T00:54:21Z, end=2025-06-21T00:54:26Z) ``` Here window(start=2025-06-21T00:54:15Z, end=2025-06-21T00:54:20Z) fires twice: - [1750467255.346215, 1750467256.346215, 1750467257.346215] - [1750467258.346215, 1750467259.346215] However, the last timestamp of the first firing (i.e. 1750467257 or 2025-06-21T00:54:17) is earlier than the end of the window. ### 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