gemini-code-assist[bot] commented on code in PR #36621: URL: https://github.com/apache/beam/pull/36621#discussion_r2462485776
########## sdks/python/apache_beam/examples/cookbook/ordered_window_elements/batch.py: ########## @@ -0,0 +1,523 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import logging +from enum import Enum +from typing import Any +from typing import Callable +from typing import Optional + +import apache_beam as beam + +from apache_beam.coders import BooleanCoder +from apache_beam.coders import PickleCoder +from apache_beam.pvalue import AsDict +from apache_beam.transforms.combiners import ToListCombineFn +from apache_beam.transforms.ptransform import PTransform +from apache_beam.transforms.timeutil import TimeDomain +from apache_beam.transforms.userstate import OrderedListStateSpec +from apache_beam.transforms.userstate import ReadModifyWriteStateSpec +from apache_beam.transforms.userstate import TimerSpec +from apache_beam.transforms.userstate import on_timer +from apache_beam.transforms.window import GlobalWindow +from apache_beam.utils.timestamp import MIN_TIMESTAMP +from apache_beam.utils.timestamp import DurationTypes # pylint: disable=unused-import +from apache_beam.utils.timestamp import Timestamp +from apache_beam.utils.timestamp import TimestampTypes # pylint: disable=unused-import + + +class FanOutToWindows(beam.DoFn): + """ + Assigns each element to all the windows that contain it. + + This DoFn is used to expand a single element into multiple elements, each + associated with a specific window. + + Args: + duration: The duration of each window in seconds. + slide_interval: The interval at which windows slide in seconds. + offset: The offset for window alignment in seconds. +""" + def __init__(self, duration, slide_interval, offset): + self.duration = duration + self.slide_interval = slide_interval + self.offset = offset + + def process(self, element): + """ + Processes an element and assigns it to relevant windows. + + Args: + element: A tuple (timestamp, value) where timestamp is a Timestamp object + and value is the actual element data. + + Yields: + A tuple ((window_start, window_end), element) for each window the + element belongs to. + """ + timestamp = element[0] + timestamp_secs = timestamp.micros / 1e6 + + # Align the timestamp with the windowing scheme. + aligned_timestamp = timestamp_secs - self.offset + + # Calculate the start of the last window that could contain this timestamp. + last_window_start_aligned = ((aligned_timestamp // self.slide_interval) * + self.slide_interval) + last_window_start = last_window_start_aligned + self.offset + + # To find out the start of the first possible window that covers this + # timestamp, we start with the last window and assume we slide backward n + # times: + # first_possible_start = last_window_start - n * slide_interval + # first_possible_end = last_window_start - n * slide_interval + duration + # The conditions hold: + # first_possible_end > timestamp. + # first_possible_end - slide_interval <= timestamp + # Therefore, + # n < (last_window_start + duration - timestamp) / slide_interval + # n >= (last_window_start + duration - timestamp) / slide_interval - 1 + # The worst case is that the element is at the beginning of the slide: + # i.e. timestamp = last_window_start + # And n is an integer satisfies + # duration / slide_interval - 1 <= n < duration / slide_interval + # Case 1: if duration is divisible by slide_interval, + # then n = duration / slide_interval - 1 + # Case 2: if duration is not divisible by slide_interval, + # then n = duration // slide_interval + # A unified solution is n = (duration - 1) // slide_interval + n = (self.duration - 1) // self.slide_interval + first_possible_start = last_window_start - n * self.slide_interval + + # We iterate from the first possible window start up to the last one. + current_start = first_possible_start + while current_start <= last_window_start: + # An element is in a window [start, start + duration) if: + # start <= timestamp < start + duration + if current_start <= timestamp_secs < current_start + self.duration: + yield (current_start, current_start + self.duration), element + current_start += self.slide_interval + + +class FanOutToSlideBoundaries(beam.DoFn): + """ + Assigns each element to a window representing its slide. + + This DoFn is used to group elements by the start of the slide they belong to. + This is a preliminary step for generating context information for window gaps. + + Args: + slide_interval: The interval at which windows slide in seconds. + offset: The offset for window alignment in seconds. + """ + def __init__(self, slide_interval, offset): + self.slide_interval = slide_interval + self.offset = offset + + def process(self, element): + """ + Processes an element and assigns it to its corresponding slide boundary. + + Args: + element: A tuple (timestamp, value) where timestamp is a Timestamp object + and value is the actual element data. + + Yields: + A tuple (slide_start, element) where slide_start is the beginning + timestamp of the slide the element belongs to. + """ + timestamp = element[0] + timestamp_secs = timestamp.micros / 1e6 + + # Align the timestamp with the windowing scheme. + aligned_timestamp = timestamp_secs - self.offset + + # Calculate the start of the slide containing this timestamp. + slide_start_aligned = ((aligned_timestamp // self.slide_interval) * + self.slide_interval) + slide_start = slide_start_aligned + self.offset + + # slide_end = slide_start + self.slide_interval + yield slide_start, element + + +class GenerateContextDoFn(beam.DoFn): + """ + Generates context information for filling gaps in windows. + + This DoFn uses Beam's state and timer features to collect elements within + slides and emit a "context" value for each slide. This context value is + typically the element with the maximum timestamp within that slide, which + can then be used to forward-fill empty windows or gaps at the start of + windows. + + Args: + duration: The duration of each window in seconds. + slide_interval: The interval at which windows slide in seconds. + offset: The offset for window alignment in seconds. + default: The default value to use when no context is available. + """ + ORDERED_BUFFER_STATE = OrderedListStateSpec('ordered_buffer', PickleCoder()) + WINDOW_TIMER = TimerSpec('window_timer', TimeDomain.WATERMARK) + TIMER_STATE = ReadModifyWriteStateSpec('timer_state', BooleanCoder()) + + def __init__(self, duration, slide_interval, offset, default): + self.duration = duration + self.slide_interval = slide_interval + self.offset = offset + self.default = default + + def process( + self, + element=beam.DoFn.ElementParam, + timestamp=beam.DoFn.TimestampParam, + window_timer=beam.DoFn.TimerParam(WINDOW_TIMER), + timer_state=beam.DoFn.StateParam(TIMER_STATE), + ordered_buffer=beam.DoFn.StateParam(ORDERED_BUFFER_STATE), + ): + """ + Buffers elements and sets a timer to process them when the window closes. + + Args: + element: The input element, expected to be (key, (slide_start, value)). + timestamp: The timestamp of the element. + window_timer: The timer for the current window. + timer_state: State to track if the timer has been started. + ordered_buffer: Ordered list state to buffer elements. + """ + _, (slide_start, value) = element + + ordered_buffer.add((Timestamp.of(slide_start), value)) + + timer_started = timer_state.read() + if not timer_started: + window_timer.set(GlobalWindow().end) + timer_state.write(True) + return [] + + @on_timer(WINDOW_TIMER) + def on_timer( + self, + ordered_buffer=beam.DoFn.StateParam(ORDERED_BUFFER_STATE), + ): + """ + Emits context results when the window timer fires. + + This method processes the buffered elements, identifies the maximum + timestamp element for each slide, and yields context values to fill + potential gaps in subsequent windows. + + Args: + ordered_buffer: Ordered list state containing buffered elements. + + Yields: + A tuple (timestamp, element) representing the context for a slide. + """ + # Emit the context result once we collect all elements + prev_max_timestamp_element = None + prev_max_timestamp = MIN_TIMESTAMP + prev_slide_start = None + for slide_start, max_timestamp_event in ordered_buffer.read(): + event_ts = max_timestamp_event[0] + if prev_slide_start != slide_start: + # a new slide starts + if prev_max_timestamp_element is not None: + # Use the last available max timestamp element for slide between + # the last seen slide and the current slide (which includes + # empty slides in the middle). + start = prev_slide_start + while start < slide_start: + yield (start + self.slide_interval, prev_max_timestamp_element) + start += self.slide_interval + else: + yield (slide_start, (MIN_TIMESTAMP, self.default)) + + prev_slide_start = slide_start + + if prev_max_timestamp < event_ts < slide_start + self.slide_interval: + prev_max_timestamp = event_ts + prev_max_timestamp_element = max_timestamp_event Review Comment:  The logic in this `on_timer` method seems overly complex, particularly the part that appears to re-calculate the maximum timestamp element (lines 251-253). Since the upstream pipeline uses `beam.CombinePerKey(max_timestamp_element)`, we can assume that this `DoFn` receives only one element (the one with the maximum timestamp) per slide. Simplifying this logic would improve readability and maintainability. Here is a suggested simplification that is clearer and achieves the same result: ```python @on_timer(WINDOW_TIMER) def on_timer( self, ordered_buffer=beam.DoFn.StateParam(ORDERED_BUFFER_STATE), ): """ Emits context results when the window timer fires. This method processes the buffered elements, which are the max-timestamp elements for each slide, and yields context values to fill potential gaps in subsequent windows. Args: ordered_buffer: Ordered list state containing buffered elements. Yields: A tuple (timestamp, element) representing the context for a slide. """ # The upstream CombinePerKey ensures one max_timestamp_event per slide_start. # This method iterates through them and generates context for empty slides. last_element = (MIN_TIMESTAMP, self.default) last_slide_start = None for slide_start, max_timestamp_event in ordered_buffer.read(): if last_slide_start is not None: # Fill gaps between the last slide and this one. current_slide = last_slide_start + self.slide_interval while current_slide < slide_start: yield (current_slide, last_element) current_slide += self.slide_interval # Yield context for the current slide, which is the element from the # previous slide. yield (slide_start, last_element) last_slide_start = slide_start last_element = max_timestamp_event ``` ########## sdks/python/apache_beam/examples/cookbook/ordered_window_elements/batch_test.py: ########## @@ -0,0 +1,325 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import logging +import random +import unittest + +from parameterized import param +from parameterized import parameterized + +import apache_beam as beam +from apache_beam.examples.cookbook.ordered_window_elements.batch import OrderedWindowElements # pylint: disable=line-too-long +from apache_beam.examples.cookbook.ordered_window_elements.batch import WindowGapStrategy # pylint: disable=line-too-long +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to +from apache_beam.utils.timestamp import Timestamp + +logging.basicConfig(level=logging.INFO) +#logging.basicConfig(level=logging.WARNING) + +options = PipelineOptions([ + "--environment_type=LOOPBACK", + "--runner=PrismRunner", #"--runner=FnApiRunner", + "--prism_log_kind=dev", + # "--runner=PortableRunner", + # "--job_endpoint=localhost:8073", +]) + +ENABLE_LOGGING = False +WINDOW_SIZE = 3 + + +def _maybe_log_elements(pcoll, prefix="result="): + if ENABLE_LOGGING: + return pcoll | beam.LogElements( + prefix=prefix, + level=logging.WARNING, + with_timestamp=True, + with_window=True, + use_epoch_time=True) + else: + return pcoll + + +def _create_input_batch(elements: list[int], shuffle_data=True): + if shuffle_data: + random.shuffle(elements) + return beam.Create([(Timestamp.of(e), e) for e in elements]) + + +def _create_input_batch_without_timestamp( + elements: list[int], shuffle_data=True): + if shuffle_data: + random.shuffle(elements) + return beam.Create(elements) + + +def _convert_timestamp_to_int(): + return beam.MapTuple( + lambda window, elements: + ((int(window[0].micros // 1e6), int(window[1].micros // 1e6)), + [(int(t.micros // 1e6), v) for t, v in elements])) + + +class OrderedWindowElementsTest(unittest.TestCase): + def setUp(self) -> None: + self.options = PipelineOptions([ + "--streaming", Review Comment:  The pipeline options in `setUp` include `"--streaming"`. Since this test is for a batch pipeline, this flag is unnecessary and potentially misleading. It should be removed to accurately reflect the test environment. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
