From 782a4c7851fa65c2817d556cc755c0882e55acce Mon Sep 17 00:00:00 2001 From: Hoang Phan Date: Tue, 3 Jun 2025 06:49:58 -0400 Subject: [PATCH 01/19] Add support for session window --- docs/windowing.md | 236 +++++- quixstreams/dataframe/dataframe.py | 98 +++ quixstreams/dataframe/windows/__init__.py | 2 + quixstreams/dataframe/windows/definitions.py | 76 +- quixstreams/dataframe/windows/time_based.py | 342 ++++++++ .../test_windows/test_session.py | 760 ++++++++++++++++++ 6 files changed, 1505 insertions(+), 9 deletions(-) create mode 100644 tests/test_quixstreams/test_dataframe/test_windows/test_session.py diff --git a/docs/windowing.md b/docs/windowing.md index ea5a9a72f..6ce57c5d2 100644 --- a/docs/windowing.md +++ b/docs/windowing.md @@ -9,7 +9,9 @@ With windows, you can calculate such aggregations as: - Total of website visitors for every hour - The average speed of a vehicle over the last 10 minutes - Maximum temperature of a sensor observed over 30 second ranges -- Give an user a reward after 10 succesful actions +- Give an user a reward after 10 succesful actions +- Track user activity sessions on a website +- Detect fraud patterns in financial transactions ## Types of Time in Streaming @@ -500,6 +502,220 @@ sdf = ( ``` +## Session Windows + +Session windows group events that occur within a specified timeout period. Unlike fixed-time windows (tumbling, hopping, sliding), session windows have dynamic durations based on the actual timing of events, making them ideal for user activity tracking, fraud detection, and other event-driven scenarios. + +A session starts with the first event and extends each time a new event arrives within the timeout period. The session closes after the timeout period with no new events. + +Key characteristics of session windows: + +- **Dynamic boundaries**: Each session can have different start and end times based on actual events +- **Activity-based**: Sessions extend automatically when events arrive within the timeout period +- **Event-driven closure**: Sessions close when no events arrive within the timeout period +- **Grace period support**: Late events can still extend sessions if they arrive within the grace period + +### How Session Windows Work + +``` +Time: 0 5 10 15 20 25 30 35 40 45 50 +Events: A B C D E + +Timeout: 10 seconds +Grace: 2 seconds + +Session 1: [0, 20] - Events A, B (B extends the session from A) +Session 2: [25, 35] - Events C, D (D extends the session from C) +Session 3: [45, 55] - Event E (session will close at 55 if no more events) +``` + +In this example: +- Event A starts Session 1 at time 0, session would timeout at time 10 +- Event B arrives at time 10, extending Session 1 to timeout at time 20 +- Event C arrives at time 25, starting Session 2 (too late for Session 1) +- Event D arrives at time 30, extending Session 2 to timeout at time 40 +- Event E arrives at time 45, starting Session 3 + +### Basic Session Window Example + +Imagine you want to track user activity sessions on a website, where a session continues as long as user actions occur within 30 minutes of each other: + +Input: +```json +{"user_action": "page_view", "page": "/home", "timestamp": 1000} +{"user_action": "click", "element": "button", "timestamp": 800000} +{"user_action": "page_view", "page": "/products", "timestamp": 1200000} +{"user_action": "purchase", "amount": 50, "timestamp": 2000000} +``` + +Here's how to track user sessions using session windows: + +```python +from datetime import timedelta +from quixstreams import Application +from quixstreams.dataframe.windows import Count, Collect + +app = Application(...) +sdf = app.dataframe(...) + +sdf = ( + # Define a session window with 30-minute timeout and 5-minute grace period + .session_window( + timeout_ms=timedelta(minutes=30), + grace_ms=timedelta(minutes=5) + ) + + # Count the number of actions in each session and collect all actions + .agg( + action_count=Count(), + actions=Collect("user_action") + ) + + # Emit results when sessions are complete + .final() +) + +# Expected output (when session expires): +# { +# "start": 1000, +# "end": 2000000 + 1800000, # last event + timeout +# "action_count": 4, +# "actions": ["page_view", "click", "page_view", "purchase"] +# } +``` + +### Session Window with Current Mode + +For real-time monitoring, you can use `.current()` mode to get updates as the session progresses: + +```python +from datetime import timedelta +from quixstreams import Application +from quixstreams.dataframe.windows import Sum, Count + +app = Application(...) +sdf = app.dataframe(...) + +sdf = ( + # Define a session window with 10-second timeout + .session_window(timeout_ms=timedelta(seconds=10)) + + # Track total purchase amount and count in each session + .agg( + total_amount=Sum("amount"), + purchase_count=Count() + ) + + # Emit updates for each message (real-time session tracking) + .current() +) + +# Output for each incoming event: +# Event 1: {"start": 1000, "end": 11000, "total_amount": 25, "purchase_count": 1} +# Event 2: {"start": 1000, "end": 15000, "total_amount": 75, "purchase_count": 2} # session extended +# Event 3: {"start": 1000, "end": 18000, "total_amount": 125, "purchase_count": 3} # session extended again +``` + +### Handling Late Events in Sessions + +Session windows support grace periods to handle out-of-order events: + +```python +from datetime import timedelta +from quixstreams import Application +from quixstreams.dataframe.windows import Count + +def on_late_session_event( + value, key, timestamp_ms, late_by_ms, start, end, name, topic, partition, offset +): + """Handle late events that couldn't extend any session""" + print(f"Late event for key {key}: {late_by_ms}ms late") + print(f"Event would have belonged to session [{start}, {end}]") + return False # Suppress default logging + +app = Application(...) +sdf = app.dataframe(...) + +sdf = ( + # Session window with 5-minute timeout and 1-minute grace period + .session_window( + timeout_ms=timedelta(minutes=5), + grace_ms=timedelta(minutes=1), + on_late=on_late_session_event + ) + .agg(event_count=Count()) + .final() +) +``` + +### Session Window Use Cases + +**1. User Activity Tracking** +```python +# Track user sessions on a website or app +.session_window(timeout_ms=timedelta(minutes=30)) +.agg( + page_views=Count(), + unique_pages=Count("page_url", unique=True), + session_duration=Max("timestamp") - Min("timestamp") +) +``` + +**2. Fraud Detection** +```python +# Detect suspicious transaction patterns +.session_window(timeout_ms=timedelta(minutes=10)) +.agg( + transaction_count=Count(), + total_amount=Sum("amount"), + locations=Collect("location") +) +``` + +**3. IoT Device Monitoring** +```python +# Monitor device activity sessions +.session_window(timeout_ms=timedelta(hours=1)) +.agg( + readings_count=Count(), + avg_temperature=Mean("temperature"), + max_pressure=Max("pressure") +) +``` + +**4. Gaming Analytics** +```python +# Track gaming sessions +.session_window(timeout_ms=timedelta(minutes=20)) +.agg( + actions_performed=Count(), + points_earned=Sum("points"), + levels_completed=Count("level_completed") +) +``` + +### Session Window Parameters + +- **`timeout_ms`**: The session timeout period. If no new events arrive within this period, the session will be closed. Can be specified as either an `int` (milliseconds) or a `timedelta` object. + +- **`grace_ms`**: The grace period for data arrival. Allows late-arriving data to be included in the session, even if it arrives after the session has theoretically timed out. Can be specified as either an `int` (milliseconds) or a `timedelta` object. + +- **`name`**: Optional unique identifier for the window. If not provided, it will be automatically generated based on the window's properties. + +- **`on_late`**: Optional callback to react to late records that cannot extend any existing session. Use this to customize logging or route late events to a dead-letter queue. + +### Session Window Behavior + +**Session Creation**: A new session starts when an event arrives and no existing session can accommodate it (i.e., all existing sessions have timed out). + +**Session Extension**: An existing session is extended when an event arrives within `timeout + grace_period` of the session's last activity. + +**Session Closure**: A session closes when the current time exceeds `session_end_time + grace_period`, where `session_end_time = last_event_time + timeout`. + +**Key Grouping**: Like all windows in Quix Streams, sessions are grouped by message key. Each key maintains its own independent sessions. + +**Event Time**: Sessions use event time (from Kafka message timestamps) rather than processing time. + ## Lateness and Out-of-Order Processing When working with event time, some events may be processed later than they're supposed to. Such events are called **"out-of-order"** because they violate the expected order of time in the data stream. @@ -540,7 +756,7 @@ The appropriate value for a grace period varies depending on the use case. ### Reacting on late events !!! info New in v3.8.0 -To react on late records coming into time windows, you can pass the `on_late` callbacks to `.tumbling_window()`, `.hopping_window()` and `.sliding_window()` methods. +To react on late records coming into time windows, you can pass the `on_late` callbacks to `.tumbling_window()`, `.hopping_window()`, `.sliding_window()`, and `.session_window()` methods. You can use this callback to customize the logging of such messages or to send them to some dead-letter queue, for example. @@ -667,6 +883,8 @@ In this strategy, messages advance time and close only windows with the **same** If some message keys appear irregularly in the stream, the latest windows can remain unprocessed until the message with the same key is received. +Session windows also support both closing strategies. With **key** strategy, sessions for each key close independently. With **partition** strategy, any message can advance time and close sessions for all keys in the partition. + ```python from datetime import timedelta from quixstreams import Application @@ -780,7 +998,7 @@ described in [the "Updating Kafka Headers" section](./processing.md#updating-kaf Here are some general concepts about how windowed aggregations are implemented in Quix Streams: -- Only time-based windows are supported. +- Time-based windows (tumbling, hopping, sliding, session) and count-based windows are supported. - Every window is grouped by the current Kafka message key. - Messages with `None` key will be ignored. - The minimal window unit is a **millisecond**. More fine-grained values (e.g. microseconds) will be rounded towards the closest millisecond number. @@ -794,10 +1012,12 @@ window specification. The state store name is auto-generated by default using the following window attributes: -- Window type: `"tumbling"` or `"hopping"` -- Window parameters: `duration_ms` and `step_ms` +- Window type: `"tumbling"`, `"hopping"`, `"sliding"`, or `"session"` +- Window parameters: `duration_ms` and `step_ms` for time-based windows, `timeout_ms` for session windows -E.g. a store name for a hopping window of 30 seconds with a 5 second step will be `hopping_window_30000_5000`. +Examples: +- A hopping window of 30 seconds with a 5 second step: `hopping_window_30000_5000` +- A session window with 30 second timeout: `session_window_30000` ### Updating Window Definitions @@ -807,8 +1027,8 @@ When you change the definition of the window (e.g. its size), the data in the st Quix Streams handles some of the situations, like: -- Updating window type (e.g. from tumbling to hopping) -- Updating window period or step +- Updating window type (e.g. from tumbling to hopping, from hopping to session) +- Updating window period, step, or timeout - Adding/Removing/Updating an aggregation function (except `Reduce()`) Updating the window type and parameters will change the name of the underlying state store, and the new window definition will use a different one. diff --git a/quixstreams/dataframe/dataframe.py b/quixstreams/dataframe/dataframe.py index 53e90c767..3f7684a3d 100644 --- a/quixstreams/dataframe/dataframe.py +++ b/quixstreams/dataframe/dataframe.py @@ -67,6 +67,7 @@ from .windows import ( HoppingCountWindowDefinition, HoppingTimeWindowDefinition, + SessionWindowDefinition, SlidingCountWindowDefinition, SlidingTimeWindowDefinition, TumblingCountWindowDefinition, @@ -1526,6 +1527,103 @@ def sliding_count_window( name=name, ) + def session_window( + self, + timeout_ms: Union[int, timedelta], + grace_ms: Union[int, timedelta] = 0, + name: Optional[str] = None, + on_late: Optional[WindowOnLateCallback] = None, + ) -> SessionWindowDefinition: + """ + Create a session window transformation on this StreamingDataFrame. + + Session windows group events that occur within a specified timeout period. + A session starts with the first event and extends each time a new event arrives + within the timeout period. The session closes after the timeout period with no + new events. + + Unlike fixed-time windows, session windows have dynamic durations based on the + actual events and their timing, making them ideal for user activity tracking, + fraud detection, and other event-driven scenarios. + + They allow performing stateful aggregations like `sum`, `reduce`, etc. + on top of the data and emit results downstream. + + Notes: + + - The timestamp of the aggregation result is set to the session start timestamp. + - Every session is grouped by the current Kafka message key. + - Messages with `None` key will be ignored. + - Sessions always use the current event time. + + Example Snippet: + + ```python + from quixstreams import Application + import quixstreams.dataframe.windows.aggregations as agg + + app = Application() + sdf = app.dataframe(...) + + sdf = ( + # Define a session window with 30-second timeout and 10-second grace period + sdf.session_window( + timeout_ms=timedelta(seconds=30), + grace_ms=timedelta(seconds=10) + ) + + # Specify the aggregation function + .agg(value=agg.Sum()) + + # Specify how the results should be emitted downstream. + # "current()" will emit results as they come for each updated session, + # possibly producing multiple messages per key-session pair + # "final()" will emit sessions only when they are closed and cannot + # receive any updates anymore. + .final() + ) + ``` + + :param timeout_ms: The session timeout period. + If no new events arrive within this period, the session will be closed. + Can be specified as either an `int` representing milliseconds + or a `timedelta` object. + >***NOTE:*** `timedelta` objects will be rounded to the closest millisecond + value. + + :param grace_ms: The grace period for data arrival. + It allows late-arriving data to be included in the session, + even if it arrives after the session has theoretically timed out. + Can be specified as either an `int` representing milliseconds + or a `timedelta` object. + >***NOTE:*** `timedelta` objects will be rounded to the closest millisecond + value. + + :param name: The unique identifier for the window. If not provided, it will be + automatically generated based on the window's properties. + + :param on_late: an optional callback to react on late records in sessions and + to configure the logging of such events. + If the callback returns `True`, the message about a late record will be logged + (default behavior). + Otherwise, no message will be logged. + + :return: `SessionWindowDefinition` instance representing the session window + configuration. + This object can be further configured with aggregation functions + like `sum`, `count`, etc. applied to the StreamingDataFrame. + """ + timeout_ms = ensure_milliseconds(timeout_ms) + grace_ms = ensure_milliseconds(grace_ms) + + return SessionWindowDefinition( + timeout_ms=timeout_ms, + grace_ms=grace_ms, + dataframe=self, + name=name, + on_late=on_late, + ) + def fill(self, *columns: str, **mapping: Any) -> "StreamingDataFrame": """ Fill missing values in the message value with a constant value. diff --git a/quixstreams/dataframe/windows/__init__.py b/quixstreams/dataframe/windows/__init__.py index 14e7f98e3..ce6655a90 100644 --- a/quixstreams/dataframe/windows/__init__.py +++ b/quixstreams/dataframe/windows/__init__.py @@ -16,6 +16,7 @@ from .definitions import ( HoppingCountWindowDefinition, HoppingTimeWindowDefinition, + SessionWindowDefinition, SlidingCountWindowDefinition, SlidingTimeWindowDefinition, TumblingCountWindowDefinition, @@ -38,6 +39,7 @@ "Collector", "HoppingCountWindowDefinition", "HoppingTimeWindowDefinition", + "SessionWindowDefinition", "SlidingCountWindowDefinition", "SlidingTimeWindowDefinition", "TumblingCountWindowDefinition", diff --git a/quixstreams/dataframe/windows/definitions.py b/quixstreams/dataframe/windows/definitions.py index 90d4d815b..35f5db111 100644 --- a/quixstreams/dataframe/windows/definitions.py +++ b/quixstreams/dataframe/windows/definitions.py @@ -28,6 +28,9 @@ SlidingWindowSingleAggregation, ) from .time_based import ( + SessionWindow, + SessionWindowMultiAggregation, + SessionWindowSingleAggregation, TimeWindow, TimeWindowMultiAggregation, TimeWindowSingleAggregation, @@ -43,6 +46,7 @@ "HoppingTimeWindowDefinition", "SlidingTimeWindowDefinition", "TumblingTimeWindowDefinition", + "SessionWindowDefinition", ] WindowT = TypeVar("WindowT", bound=Window) @@ -528,4 +532,74 @@ def _get_name(self, func_name: Optional[str]) -> str: ) if func_name: return f"{prefix}_{func_name}" - return prefix + else: + return prefix + + +class SessionWindowDefinition(WindowDefinition): + """ + Definition for session windows that group events by activity sessions. + + Session windows group events that occur within a specified timeout period. + A session starts with the first event and extends each time a new event arrives + within the timeout period. The session closes after the timeout period with no + new events. + """ + + def __init__( + self, + timeout_ms: int, + grace_ms: int, + dataframe: "StreamingDataFrame", + name: Optional[str] = None, + on_late: Optional[WindowOnLateCallback] = None, + ): + if not isinstance(timeout_ms, int): + raise TypeError("Session timeout must be an integer") + if timeout_ms < 1: + raise ValueError("Session timeout cannot be smaller than 1ms") + if grace_ms < 0: + raise ValueError("Session grace cannot be smaller than 0ms") + + super().__init__(name, dataframe, on_late) + + self._timeout_ms = timeout_ms + self._grace_ms = grace_ms + + @property + def timeout_ms(self) -> int: + return self._timeout_ms + + @property + def grace_ms(self) -> int: + return self._grace_ms + + def _get_name(self, func_name: Optional[str]) -> str: + prefix = f"{self._name}_session_window" if self._name else "session_window" + if func_name: + return f"{prefix}_{self._timeout_ms}_{func_name}" + else: + return f"{prefix}_{self._timeout_ms}" + + def _create_window( + self, + func_name: Optional[str], + aggregators: Optional[dict[str, BaseAggregator]] = None, + collectors: Optional[dict[str, BaseCollector]] = None, + ) -> SessionWindow: + if func_name: + window_type: Union[ + type[SessionWindowSingleAggregation], type[SessionWindowMultiAggregation] + ] = SessionWindowSingleAggregation + else: + window_type = SessionWindowMultiAggregation + + return window_type( + timeout_ms=self._timeout_ms, + grace_ms=self._grace_ms, + name=self._get_name(func_name=func_name), + dataframe=self._dataframe, + aggregators=aggregators or {}, + collectors=collectors or {}, + on_late=self._on_late, + ) diff --git a/quixstreams/dataframe/windows/time_based.py b/quixstreams/dataframe/windows/time_based.py index c403cfdfa..5647274fd 100644 --- a/quixstreams/dataframe/windows/time_based.py +++ b/quixstreams/dataframe/windows/time_based.py @@ -271,9 +271,351 @@ def _on_expired_window( ) +class SessionWindow(Window): + """ + Session window groups events that occur within a specified timeout period. + + A session starts with the first event and extends each time a new event arrives + within the timeout period. The session closes after the timeout period with no + new events. + + Each session window can have different start and end times based on the actual + events, making sessions dynamic rather than fixed-time intervals. + """ + + def __init__( + self, + timeout_ms: int, + grace_ms: int, + name: str, + dataframe: "StreamingDataFrame", + on_late: Optional[WindowOnLateCallback] = None, + ): + super().__init__( + name=name, + dataframe=dataframe, + ) + + self._timeout_ms = timeout_ms + self._grace_ms = grace_ms + self._on_late = on_late + self._closing_strategy = ClosingStrategy.KEY + + def final( + self, closing_strategy: ClosingStrategyValues = "key" + ) -> "StreamingDataFrame": + """ + Apply the session window aggregation and return results only when the sessions + are closed. + + The format of returned sessions: + ```python + { + "start": , + "end": , + "value: , + } + ``` + + The individual session is closed when the event time + (the maximum observed timestamp across the partition) passes + the last event timestamp + timeout + grace period. + The closed sessions cannot receive updates anymore and are considered final. + + :param closing_strategy: the strategy to use when closing sessions. + Possible values: + - `"key"` - messages advance time and close sessions with the same key. + If some message keys appear irregularly in the stream, the latest sessions can remain unprocessed until a message with the same key is received. + - `"partition"` - messages advance time and close sessions for the whole partition to which this message key belongs. + If timestamps between keys are not ordered, it may increase the number of discarded late messages. + Default - `"key"`. + """ + self._closing_strategy = ClosingStrategy.new(closing_strategy) + return super().final() + + def current( + self, closing_strategy: ClosingStrategyValues = "key" + ) -> "StreamingDataFrame": + """ + Apply the session window transformation to the StreamingDataFrame to return results + for each updated session. + + The format of returned sessions: + ```python + { + "start": , + "end": , + "value: , + } + ``` + + This method processes streaming data and returns results as they come, + regardless of whether the session is closed or not. + + :param closing_strategy: the strategy to use when closing sessions. + Possible values: + - `"key"` - messages advance time and close sessions with the same key. + If some message keys appear irregularly in the stream, the latest sessions can remain unprocessed until a message with the same key is received. + - `"partition"` - messages advance time and close sessions for the whole partition to which this message key belongs. + If timestamps between keys are not ordered, it may increase the number of discarded late messages. + Default - `"key"`. + """ + self._closing_strategy = ClosingStrategy.new(closing_strategy) + return super().current() + + def process_window( + self, + value: Any, + key: Any, + timestamp_ms: int, + transaction: WindowedPartitionTransaction, + ) -> tuple[Iterable[WindowKeyResult], Iterable[WindowKeyResult]]: + state = transaction.as_state(prefix=key) + timeout_ms = self._timeout_ms + grace_ms = self._grace_ms + + collect = self.collect + aggregate = self.aggregate + + # Determine the latest timestamp for expiration logic + if self._closing_strategy == ClosingStrategy.PARTITION: + latest_expired_timestamp = transaction.get_latest_expired(prefix=b"") + latest_timestamp = max(timestamp_ms, latest_expired_timestamp) + else: + state_ts = state.get_latest_timestamp() or 0 + latest_timestamp = max(timestamp_ms, state_ts) + + # Calculate session expiry threshold + session_expiry_threshold = latest_timestamp - grace_ms + + # Check if the event is too late + if timestamp_ms < session_expiry_threshold: + late_by_ms = session_expiry_threshold - timestamp_ms + self._on_expired_session( + value=value, + key=key, + start=timestamp_ms, + end=timestamp_ms + timeout_ms, + timestamp_ms=timestamp_ms, + late_by_ms=late_by_ms, + ) + return [], [] + + # Look for an existing session that can be extended + session_start = None + session_end = None + can_extend_session = False + existing_aggregated = None + old_window_to_delete = None + + # Search for active sessions that can accommodate the new event + search_start = max(0, timestamp_ms - timeout_ms * 2) + windows = state.get_windows(search_start, timestamp_ms + timeout_ms + 1, backwards=True) + + for (window_start, window_end), aggregated_value, _ in windows: + # Calculate the time gap between the new event and the session's last activity + session_last_activity = window_end - timeout_ms + time_gap = timestamp_ms - session_last_activity + + # Check if this session can be extended + if time_gap <= timeout_ms + grace_ms and timestamp_ms >= window_start: + session_start = window_start + session_end = timestamp_ms + timeout_ms + can_extend_session = True + existing_aggregated = aggregated_value + old_window_to_delete = (window_start, window_end) + break + + # If no extendable session found, start a new one + if not can_extend_session: + session_start = timestamp_ms + session_end = timestamp_ms + timeout_ms + + # Process the event for this session + updated_windows: list[WindowKeyResult] = [] + + # Delete the old window if extending an existing session + if can_extend_session and old_window_to_delete: + old_start, old_end = old_window_to_delete + transaction.delete_window(old_start, old_end, prefix=key) + + # Add to collection if needed + if collect: + state.add_to_collection( + value=self._collect_value(value), + id=timestamp_ms, + ) + + # Update the session window aggregation + aggregated = None + if aggregate: + current_value = existing_aggregated if can_extend_session else None + if current_value is None: + current_value = self._initialize_value() + + aggregated = self._aggregate_value(current_value, value, timestamp_ms) + + # Output intermediate results for aggregations + if aggregate: + updated_windows.append( + ( + key, + self._results(aggregated, [], session_start, session_end), + ) + ) + + state.update_window(session_start, session_end, value=aggregated, timestamp_ms=timestamp_ms) + + # Expire old sessions + if self._closing_strategy == ClosingStrategy.PARTITION: + expired_windows = self.expire_sessions_by_partition( + transaction, session_expiry_threshold, collect + ) + else: + expired_windows = self.expire_sessions_by_key( + key, state, session_expiry_threshold, collect + ) + + return updated_windows, expired_windows + + def expire_sessions_by_partition( + self, + transaction: WindowedPartitionTransaction, + expiry_threshold: int, + collect: bool, + ) -> Iterable[WindowKeyResult]: + start = time.monotonic() + count = 0 + + # Import the parsing function to extract message keys from window keys + from quixstreams.state.rocksdb.windowed.serialization import parse_window_key + + expired_results = [] + + # Collect all keys and extract unique prefixes to avoid iteration conflicts + all_keys = list(transaction.keys()) + seen_prefixes = set() + + for key_bytes in all_keys: + try: + prefix, start_ms, end_ms = parse_window_key(key_bytes) + if prefix not in seen_prefixes: + seen_prefixes.add(prefix) + except (ValueError, IndexError): + # Skip invalid window key formats + continue + + # Expire sessions for each unique prefix + for prefix in seen_prefixes: + state = transaction.as_state(prefix=prefix) + prefix_expired = list(self.expire_sessions_by_key( + prefix, state, expiry_threshold, collect + )) + expired_results.extend(prefix_expired) + count += len(prefix_expired) + + if count: + logger.debug( + "Expired %s session windows in %ss", count, round(time.monotonic() - start, 2) + ) + + return expired_results + + def expire_sessions_by_key( + self, + key: Any, + state: WindowedState, + expiry_threshold: int, + collect: bool, + ) -> Iterable[WindowKeyResult]: + start = time.monotonic() + count = 0 + + # Get all windows and check which ones have expired + all_windows = list(state.get_windows(0, expiry_threshold + self._timeout_ms, backwards=False)) + + windows_to_delete = [] + for (window_start, window_end), aggregated, _ in all_windows: + # Session expires when the session end time has passed the expiry threshold + if window_end <= expiry_threshold: + collected = [] + if collect: + collected = state.get_from_collection(window_start, window_end) + + windows_to_delete.append((window_start, window_end)) + count += 1 + yield (key, self._results(aggregated, collected, window_start, window_end)) + + # Clean up expired windows + for window_start, window_end in windows_to_delete: + state._transaction.delete_window(window_start, window_end, prefix=state._prefix) + if collect: + state.delete_from_collection(window_end, start=window_start) + + if count: + logger.debug( + "Expired %s session windows in %ss", count, round(time.monotonic() - start, 2) + ) + + def _on_expired_session( + self, + value: Any, + key: Any, + start: int, + end: int, + timestamp_ms: int, + late_by_ms: int, + ) -> None: + try: + ctx = message_context() + topic = ctx.topic + partition = ctx.partition + offset = ctx.offset + except: + # In test environments, message context might not be available + topic = "unknown" + partition = -1 + offset = -1 + + to_log = True + + # Trigger the "on_late" callback if provided + if self._on_late: + to_log = self._on_late( + value, + key, + timestamp_ms, + late_by_ms, + start, + end, + self._name, + topic, + partition, + offset, + ) + if to_log: + logger.warning( + "Skipping session processing for the closed session " + f"timestamp_ms={timestamp_ms} " + f"session={(start, end)} " + f"late_by_ms={late_by_ms} " + f"store_name={self._name} " + f"partition={topic}[{partition}] " + f"offset={offset}" + ) + + class TimeWindowSingleAggregation(SingleAggregationWindowMixin, TimeWindow): pass class TimeWindowMultiAggregation(MultiAggregationWindowMixin, TimeWindow): pass + + +class SessionWindowSingleAggregation(SingleAggregationWindowMixin, SessionWindow): + pass + + +class SessionWindowMultiAggregation(MultiAggregationWindowMixin, SessionWindow): + pass diff --git a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py new file mode 100644 index 000000000..80f7156af --- /dev/null +++ b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py @@ -0,0 +1,760 @@ +import pytest + +import quixstreams.dataframe.windows.aggregations as agg +from quixstreams.dataframe import DataFrameRegistry +from quixstreams.dataframe.windows.definitions import SessionWindowDefinition +from quixstreams.dataframe.windows.time_based import ClosingStrategy + + +@pytest.fixture() +def session_window_definition_factory(state_manager, dataframe_factory): + def factory(timeout_ms: int, grace_ms: int = 0) -> SessionWindowDefinition: + sdf = dataframe_factory( + state_manager=state_manager, registry=DataFrameRegistry() + ) + window_def = SessionWindowDefinition( + timeout_ms=timeout_ms, grace_ms=grace_ms, dataframe=sdf + ) + return window_def + + return factory + + +def process(window, value, key, transaction, timestamp_ms): + updated, expired = window.process_window( + value=value, key=key, transaction=transaction, timestamp_ms=timestamp_ms + ) + return list(updated), list(expired) + + +class TestSessionWindow: + @pytest.mark.parametrize( + "timeout, grace, provided_name, func_name, expected_name", + [ + (30000, 5000, "custom_window", "sum", "custom_window_session_window_30000_sum"), + (30000, 5000, None, "sum", "session_window_30000_sum"), + (15000, 5000, None, "count", "session_window_15000_count"), + ], + ) + def test_session_window_definition_get_name( + self, + timeout, + grace, + provided_name, + func_name, + expected_name, + dataframe_factory, + ): + swd = SessionWindowDefinition( + timeout_ms=timeout, + grace_ms=grace, + dataframe=dataframe_factory(), + name=provided_name, + ) + name = swd._get_name(func_name) + assert name == expected_name + + def test_multiaggregation( + self, + session_window_definition_factory, + state_manager, + ): + window = session_window_definition_factory(timeout_ms=10000, grace_ms=1000).agg( + count=agg.Count(), + sum=agg.Sum(), + mean=agg.Mean(), + max=agg.Max(), + min=agg.Min(), + collect=agg.Collect(), + ) + window.final(closing_strategy="key") + assert window.name == "session_window_10000" + + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + key = b"key" + with store.start_partition_transaction(0) as tx: + # First event starts a session + updated, expired = process( + window, value=1, key=key, transaction=tx, timestamp_ms=1000 + ) + assert not expired + assert updated == [ + ( + key, + { + "start": 1000, + "end": 11000, # 1000 + 10000 timeout + "count": 1, + "sum": 1, + "mean": 1.0, + "max": 1, + "min": 1, + "collect": [], + }, + ) + ] + + # Second event within timeout extends the session + updated, expired = process( + window, value=4, key=key, transaction=tx, timestamp_ms=5000 + ) + assert not expired + assert updated == [ + ( + key, + { + "start": 1000, + "end": 15000, # 5000 + 10000 timeout + "count": 2, + "sum": 5, + "mean": 2.5, + "max": 4, + "min": 1, + "collect": [], + }, + ) + ] + + # Third event outside timeout starts new session, expires previous + updated, expired = process( + window, value=2, key=key, transaction=tx, timestamp_ms=26000 + ) + assert expired == [ + ( + key, + { + "start": 1000, + "end": 15000, + "count": 2, + "sum": 5, + "mean": 2.5, + "max": 4, + "min": 1, + "collect": [1, 4], + }, + ) + ] + assert updated == [ + ( + key, + { + "start": 26000, + "end": 36000, # 26000 + 10000 timeout + "count": 1, + "sum": 2, + "mean": 2.0, + "max": 2, + "min": 2, + "collect": [], + }, + ) + ] + + @pytest.mark.parametrize("expiration", ("key", "partition")) + def test_sessionwindow_count( + self, expiration, session_window_definition_factory, state_manager + ): + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window = window_def.count() + assert window.name == "session_window_10000_count" + + window.final(closing_strategy=expiration) + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + # Start session + process(window, value=0, key=key, transaction=tx, timestamp_ms=1000) + # Add to session + updated, expired = process( + window, value=0, key=key, transaction=tx, timestamp_ms=5000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == 2 + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 15000 # 5000 + 10000 + assert not expired + + @pytest.mark.parametrize("expiration", ("key", "partition")) + def test_sessionwindow_sum( + self, expiration, session_window_definition_factory, state_manager + ): + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window = window_def.sum() + assert window.name == "session_window_10000_sum" + + window.final(closing_strategy=expiration) + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + process(window, value=2, key=key, transaction=tx, timestamp_ms=1000) + updated, expired = process( + window, value=3, key=key, transaction=tx, timestamp_ms=5000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == 5 + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 15000 + assert not expired + + @pytest.mark.parametrize("expiration", ("key", "partition")) + def test_sessionwindow_mean( + self, expiration, session_window_definition_factory, state_manager + ): + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window = window_def.mean() + assert window.name == "session_window_10000_mean" + + window.final(closing_strategy=expiration) + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + process(window, value=2, key=key, transaction=tx, timestamp_ms=1000) + updated, expired = process( + window, value=4, key=key, transaction=tx, timestamp_ms=5000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == 3.0 + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 15000 + assert not expired + + @pytest.mark.parametrize("expiration", ("key", "partition")) + def test_sessionwindow_reduce( + self, expiration, session_window_definition_factory, state_manager + ): + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window = window_def.reduce( + reducer=lambda agg, current: agg + [current], + initializer=lambda value: [value], + ) + assert window.name == "session_window_10000_reduce" + + window.final(closing_strategy=expiration) + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + process(window, value=2, key=key, transaction=tx, timestamp_ms=1000) + updated, expired = process( + window, value=3, key=key, transaction=tx, timestamp_ms=5000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == [2, 3] + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 15000 + assert not expired + + @pytest.mark.parametrize("expiration", ("key", "partition")) + def test_sessionwindow_max( + self, expiration, session_window_definition_factory, state_manager + ): + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window = window_def.max() + assert window.name == "session_window_10000_max" + + window.final(closing_strategy=expiration) + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + process(window, value=2, key=key, transaction=tx, timestamp_ms=1000) + updated, expired = process( + window, value=5, key=key, transaction=tx, timestamp_ms=5000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == 5 + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 15000 + assert not expired + + @pytest.mark.parametrize("expiration", ("key", "partition")) + def test_sessionwindow_min( + self, expiration, session_window_definition_factory, state_manager + ): + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window = window_def.min() + assert window.name == "session_window_10000_min" + + window.final(closing_strategy=expiration) + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + process(window, value=5, key=key, transaction=tx, timestamp_ms=1000) + updated, expired = process( + window, value=2, key=key, transaction=tx, timestamp_ms=5000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == 2 + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 15000 + assert not expired + + @pytest.mark.parametrize("expiration", ("key", "partition")) + def test_sessionwindow_collect( + self, expiration, session_window_definition_factory, state_manager + ): + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window = window_def.collect() + assert window.name == "session_window_10000_collect" + + window.final(closing_strategy=expiration) + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + process(window, value=1, key=key, transaction=tx, timestamp_ms=1000) + process(window, value=2, key=key, transaction=tx, timestamp_ms=5000) + process(window, value=3, key=key, transaction=tx, timestamp_ms=8000) + # Event outside timeout triggers session closure + updated, expired = process( + window, value=4, key=key, transaction=tx, timestamp_ms=25000 + ) + assert not updated + assert expired == [(key, {"start": 1000, "end": 18000, "value": [1, 2, 3]})] + + @pytest.mark.parametrize( + "timeout, grace, name", + [ + (-10000, 1000, "test"), # timeout < 0 + (10000, -1000, "test"), # grace < 0 + (0, 1000, "test"), # timeout == 0 + ], + ) + def test_session_window_def_init_invalid( + self, timeout, grace, name, dataframe_factory + ): + with pytest.raises(ValueError): + SessionWindowDefinition( + timeout_ms=timeout, + grace_ms=grace, + name=name, + dataframe=dataframe_factory(), + ) + + def test_session_window_def_init_invalid_type(self, dataframe_factory): + with pytest.raises(TypeError): + SessionWindowDefinition( + timeout_ms="invalid", # should be int + grace_ms=1000, + name="test", + dataframe=dataframe_factory(), + ) + + @pytest.mark.parametrize("expiration", ("key", "partition")) + def test_session_window_process_timeout_behavior( + self, + expiration, + session_window_definition_factory, + state_manager, + ): + """Test that sessions properly timeout and new sessions start correctly""" + window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=0) + window = window_def.sum() + window.final(closing_strategy=expiration) + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + + # Start session 1 + updated, expired = process( + window, value=1, key=key, transaction=tx, timestamp_ms=1000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == 1 + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 6000 # 1000 + 5000 + assert not expired + + # Add to session 1 (within timeout) + updated, expired = process( + window, value=2, key=key, transaction=tx, timestamp_ms=4000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == 3 + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 9000 # 4000 + 5000 + assert not expired + + # Start session 2 (outside timeout) - should expire session 1 + updated, expired = process( + window, value=5, key=key, transaction=tx, timestamp_ms=15000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == 5 + assert updated[0][1]["start"] == 15000 + assert updated[0][1]["end"] == 20000 # 15000 + 5000 + + assert len(expired) == 1 + assert expired[0][1]["value"] == 3 + assert expired[0][1]["start"] == 1000 + assert expired[0][1]["end"] == 9000 + + def test_session_window_grace_period( + self, session_window_definition_factory, state_manager + ): + """Test that grace period allows late events""" + window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=2000) + window = window_def.sum() + window.final(closing_strategy="key") + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + + # Start session + updated, expired = process( + window, value=1, key=key, transaction=tx, timestamp_ms=1000 + ) + assert len(updated) == 1 + assert updated[0][1]["start"] == 1000 + assert not expired + + # Event that would normally expire the session, but within grace + updated, expired = process( + window, value=2, key=key, transaction=tx, timestamp_ms=8000 + ) + # Session should still be active due to grace period + assert len(updated) == 1 + assert updated[0][1]["value"] == 3 + assert not expired + + # Event outside grace period - should expire session + updated, expired = process( + window, value=3, key=key, transaction=tx, timestamp_ms=16000 + ) + assert len(expired) == 1 + assert expired[0][1]["value"] == 3 + assert expired[0][1]["start"] == 1000 + + def test_session_window_multiple_keys( + self, session_window_definition_factory, state_manager + ): + """Test that different keys maintain separate sessions""" + window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=0) + window = window_def.sum() + window.final(closing_strategy="key") + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key1 = b"key1" + key2 = b"key2" + + # Start session for key1 + updated, expired = process( + window, value=1, key=key1, transaction=tx, timestamp_ms=1000 + ) + assert len(updated) == 1 + assert updated[0][0] == key1 + assert updated[0][1]["value"] == 1 + assert not expired + + # Start session for key2 + updated, expired = process( + window, value=10, key=key2, transaction=tx, timestamp_ms=2000 + ) + assert len(updated) == 1 + assert updated[0][0] == key2 + assert updated[0][1]["value"] == 10 + assert not expired + + # Add to key1 session + updated, expired = process( + window, value=2, key=key1, transaction=tx, timestamp_ms=3000 + ) + assert len(updated) == 1 + assert updated[0][0] == key1 + assert updated[0][1]["value"] == 3 + assert not expired + + # Add to key2 session + updated, expired = process( + window, value=20, key=key2, transaction=tx, timestamp_ms=4000 + ) + assert len(updated) == 1 + assert updated[0][0] == key2 + assert updated[0][1]["value"] == 30 + assert not expired + + def test_session_partition_expiration( + self, session_window_definition_factory, state_manager + ): + """Test partition-level session expiration""" + window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=1000) + window = window_def.sum() + window.final(closing_strategy="partition") + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key1 = b"key1" + key2 = b"key2" + + # Start sessions for both keys + process(window, value=1, key=key1, transaction=tx, timestamp_ms=1000) + process(window, value=10, key=key2, transaction=tx, timestamp_ms=2000) + + # Add to both sessions + process(window, value=2, key=key1, transaction=tx, timestamp_ms=3000) + process(window, value=20, key=key2, transaction=tx, timestamp_ms=4000) + + # Event that advances partition time beyond grace period + # Should expire sessions for both keys + updated, expired = process( + window, value=3, key=key1, transaction=tx, timestamp_ms=15000 + ) + + # Should get new session for key1 + assert len(updated) == 1 + assert updated[0][0] == key1 + assert updated[0][1]["value"] == 3 + assert updated[0][1]["start"] == 15000 + + # Should expire sessions for both keys + expired_keys = {exp[0] for exp in expired} + assert key1 in expired_keys + assert key2 in expired_keys + + def test_session_window_late_events( + self, session_window_definition_factory, state_manager + ): + """Test handling of late events that arrive after session closure""" + window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=1000) + window = window_def.sum() + window.final(closing_strategy="key") + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + + # Start and finish a session + process(window, value=1, key=key, transaction=tx, timestamp_ms=1000) + process(window, value=2, key=key, transaction=tx, timestamp_ms=3000) + + # Start new session that will cause first to expire + updated, expired = process( + window, value=5, key=key, transaction=tx, timestamp_ms=15000 + ) + assert len(expired) == 1 + assert expired[0][1]["value"] == 3 + + # Now send a late event that would belong to the first session + # Should be ignored due to being too late + updated, expired = process( + window, value=10, key=key, transaction=tx, timestamp_ms=2500 + ) + # Should not affect any sessions since it's too late + assert not updated + assert not expired + + def test_session_window_current_mode( + self, session_window_definition_factory, state_manager + ): + """Test session window with current() mode""" + window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=0) + window = window_def.sum() + window.current(closing_strategy="key") + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + + # Start session - should get update immediately + updated, expired = process( + window, value=1, key=key, transaction=tx, timestamp_ms=1000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == 1 + assert not expired + + # Add to session - should get update immediately + updated, expired = process( + window, value=2, key=key, transaction=tx, timestamp_ms=3000 + ) + assert len(updated) == 1 + assert updated[0][1]["value"] == 3 + assert not expired + + def test_session_window_overlapping_sessions( + self, session_window_definition_factory, state_manager + ): + """Test that sessions don't overlap for the same key""" + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=0) + window = window_def.sum() + window.final(closing_strategy="key") + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + + # Start session 1 + updated, expired = process( + window, value=1, key=key, transaction=tx, timestamp_ms=1000 + ) + session1_end = updated[0][1]["end"] + + # Event within timeout - extends session 1 + updated, expired = process( + window, value=2, key=key, transaction=tx, timestamp_ms=5000 + ) + new_end = updated[0][1]["end"] + assert new_end > session1_end # Session extended + assert updated[0][1]["value"] == 3 # Accumulated value + + # Event far in future - starts session 2, expires session 1 + updated, expired = process( + window, value=10, key=key, transaction=tx, timestamp_ms=30000 + ) + assert len(expired) == 1 + assert expired[0][1]["value"] == 3 # Final value of session 1 + assert len(updated) == 1 + assert updated[0][1]["value"] == 10 # New session 2 starts fresh + assert updated[0][1]["start"] == 30000 + + def test_session_window_merge_sessions( + self, session_window_definition_factory, state_manager + ): + """Test that an event can merge two previously separate sessions""" + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window = window_def.sum() + window.final(closing_strategy="key") + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + + # Create first session + updated, expired = process( + window, value=1, key=key, transaction=tx, timestamp_ms=1000 + ) + assert len(updated) == 1 + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 11000 # 1000 + 10000 + assert updated[0][1]["value"] == 1 + assert not expired + + # Create second session that doesn't expire the first one yet + # (13000 is still within timeout + grace of first session: 11000 + 1000 = 12000) + # Actually, let's make it further: 20000ms to ensure two separate sessions + updated, expired = process( + window, value=10, key=key, transaction=tx, timestamp_ms=20000 + ) + # First session should now be expired + assert len(expired) == 1 + assert expired[0][1]["start"] == 1000 + assert expired[0][1]["end"] == 11000 + assert expired[0][1]["value"] == 1 + + assert len(updated) == 1 + assert updated[0][1]["start"] == 20000 + assert updated[0][1]["end"] == 30000 # 20000 + 10000 + assert updated[0][1]["value"] == 10 + + # Add another event to the second session + updated, expired = process( + window, value=5, key=key, transaction=tx, timestamp_ms=25000 + ) + assert len(updated) == 1 + assert updated[0][1]["start"] == 20000 + assert updated[0][1]["end"] == 35000 # 25000 + 10000 + assert updated[0][1]["value"] == 15 # 10 + 5 + assert not expired + + # Now test the limitation: we'll create a third session that could theoretically + # merge with the second session if there was a bridging event + # But since sessions don't auto-merge, they'll remain separate + updated, expired = process( + window, value=100, key=key, transaction=tx, timestamp_ms=50000 + ) + # Second session should be expired + assert len(expired) == 1 + assert expired[0][1]["start"] == 20000 + assert expired[0][1]["end"] == 35000 + assert expired[0][1]["value"] == 15 + + # Third session starts + assert len(updated) == 1 + assert updated[0][1]["start"] == 50000 + assert updated[0][1]["end"] == 60000 # 50000 + 10000 + assert updated[0][1]["value"] == 100 + + def test_session_window_bridging_event_scenario( + self, session_window_definition_factory, state_manager + ): + """ + Test scenario where an event arrives that could theoretically bridge two sessions. + + This test documents the current behavior where sessions don't auto-merge, + even when a bridging event could logically connect them. + + Scenario: + 1. Session A: [1000, 11000] with value 5 + 2. Session B: [15000, 25000] with value 10 + 3. Bridging event at 12000ms that: + - Can extend Session A to [1000, 22000] + - Now overlaps with Session B [15000, 25000] + - Ideally should merge into single session [1000, 25000] with value 15+bridge_value + + Current behavior: Session A gets extended, Session B remains separate + Ideal behavior: Sessions A and B get merged when bridging event arrives + """ + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=2000) + window = window_def.sum() + window.final(closing_strategy="key") + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + key = b"key" + + # Create Session A + updated, expired = process( + window, value=5, key=key, transaction=tx, timestamp_ms=1000 + ) + assert len(updated) == 1 + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 11000 # 1000 + 10000 + assert updated[0][1]["value"] == 5 + assert not expired + + # Create Session B - close enough that it doesn't expire Session A + # Session A expires when time > 11000 + 2000 = 13000 + # So event at 12000 should keep Session A alive + updated, expired = process( + window, value=10, key=key, transaction=tx, timestamp_ms=12000 + ) + # This should extend Session A since 12000 is within timeout+grace of Session A + # Session A last activity was at 1000, so it expires at 1000+10000+2000=13000 + # Event at 12000 is before 13000, so it should extend Session A + assert len(updated) == 1 + assert updated[0][1]["start"] == 1000 # Session A extended + assert updated[0][1]["end"] == 22000 # 12000 + 10000 + assert updated[0][1]["value"] == 15 # 5 + 10 + assert not expired + + # Now create what would be Session B if Session A hadn't been extended + updated, expired = process( + window, value=20, key=key, transaction=tx, timestamp_ms=15000 + ) + # This should extend the already extended Session A further + assert len(updated) == 1 + assert updated[0][1]["start"] == 1000 # Still Session A + assert updated[0][1]["end"] == 25000 # 15000 + 10000 + assert updated[0][1]["value"] == 35 # 5 + 10 + 20 + assert not expired + + # Final event to expire the session + updated, expired = process( + window, value=1, key=key, transaction=tx, timestamp_ms=40000 + ) + assert len(expired) == 1 + assert expired[0][1]["start"] == 1000 + assert expired[0][1]["end"] == 25000 + assert expired[0][1]["value"] == 35 # All events combined + + assert len(updated) == 1 + assert updated[0][1]["start"] == 40000 + assert updated[0][1]["value"] == 1 \ No newline at end of file From d93de6d6fef5ff5a5520f695baf851beec27ed01 Mon Sep 17 00:00:00 2001 From: Hoang Phan Date: Tue, 3 Jun 2025 12:09:30 -0400 Subject: [PATCH 02/19] Add fix for serializer --- quixstreams/dataframe/windows/time_based.py | 81 ++++++---- quixstreams/state/types.py | 12 ++ .../test_windows/test_session.py | 147 ++++++++++++++---- 3 files changed, 181 insertions(+), 59 deletions(-) diff --git a/quixstreams/dataframe/windows/time_based.py b/quixstreams/dataframe/windows/time_based.py index 5647274fd..250b0bab6 100644 --- a/quixstreams/dataframe/windows/time_based.py +++ b/quixstreams/dataframe/windows/time_based.py @@ -274,11 +274,11 @@ def _on_expired_window( class SessionWindow(Window): """ Session window groups events that occur within a specified timeout period. - + A session starts with the first event and extends each time a new event arrives within the timeout period. The session closes after the timeout period with no new events. - + Each session window can have different start and end times based on the actual events, making sessions dynamic rather than fixed-time intervals. """ @@ -400,23 +400,25 @@ def process_window( late_by_ms=late_by_ms, ) return [], [] - + # Look for an existing session that can be extended session_start = None session_end = None can_extend_session = False existing_aggregated = None old_window_to_delete = None - + # Search for active sessions that can accommodate the new event search_start = max(0, timestamp_ms - timeout_ms * 2) - windows = state.get_windows(search_start, timestamp_ms + timeout_ms + 1, backwards=True) - + windows = state.get_windows( + search_start, timestamp_ms + timeout_ms + 1, backwards=True + ) + for (window_start, window_end), aggregated_value, _ in windows: # Calculate the time gap between the new event and the session's last activity session_last_activity = window_end - timeout_ms time_gap = timestamp_ms - session_last_activity - + # Check if this session can be extended if time_gap <= timeout_ms + grace_ms and timestamp_ms >= window_start: session_start = window_start @@ -433,12 +435,12 @@ def process_window( # Process the event for this session updated_windows: list[WindowKeyResult] = [] - + # Delete the old window if extending an existing session if can_extend_session and old_window_to_delete: old_start, old_end = old_window_to_delete - transaction.delete_window(old_start, old_end, prefix=key) - + transaction.delete_window(old_start, old_end, prefix=state._prefix) # type: ignore # noqa: SLF001 + # Add to collection if needed if collect: state.add_to_collection( @@ -454,7 +456,11 @@ def process_window( current_value = self._initialize_value() aggregated = self._aggregate_value(current_value, value, timestamp_ms) - + + # By this point, session_start and session_end are guaranteed to be set + assert session_start is not None # noqa: S101 + assert session_end is not None # noqa: S101 + # Output intermediate results for aggregations if aggregate: updated_windows.append( @@ -463,8 +469,10 @@ def process_window( self._results(aggregated, [], session_start, session_end), ) ) - - state.update_window(session_start, session_end, value=aggregated, timestamp_ms=timestamp_ms) + + state.update_window( + session_start, session_end, value=aggregated, timestamp_ms=timestamp_ms + ) # Expire old sessions if self._closing_strategy == ClosingStrategy.PARTITION: @@ -489,13 +497,13 @@ def expire_sessions_by_partition( # Import the parsing function to extract message keys from window keys from quixstreams.state.rocksdb.windowed.serialization import parse_window_key - + expired_results = [] - + # Collect all keys and extract unique prefixes to avoid iteration conflicts all_keys = list(transaction.keys()) seen_prefixes = set() - + for key_bytes in all_keys: try: prefix, start_ms, end_ms = parse_window_key(key_bytes) @@ -504,21 +512,23 @@ def expire_sessions_by_partition( except (ValueError, IndexError): # Skip invalid window key formats continue - + # Expire sessions for each unique prefix for prefix in seen_prefixes: state = transaction.as_state(prefix=prefix) - prefix_expired = list(self.expire_sessions_by_key( - prefix, state, expiry_threshold, collect - )) + prefix_expired = list( + self.expire_sessions_by_key(prefix, state, expiry_threshold, collect) + ) expired_results.extend(prefix_expired) count += len(prefix_expired) if count: logger.debug( - "Expired %s session windows in %ss", count, round(time.monotonic() - start, 2) + "Expired %s session windows in %ss", + count, + round(time.monotonic() - start, 2), ) - + return expired_results def expire_sessions_by_key( @@ -532,8 +542,10 @@ def expire_sessions_by_key( count = 0 # Get all windows and check which ones have expired - all_windows = list(state.get_windows(0, expiry_threshold + self._timeout_ms, backwards=False)) - + all_windows = list( + state.get_windows(0, expiry_threshold + self._timeout_ms, backwards=False) + ) + windows_to_delete = [] for (window_start, window_end), aggregated, _ in all_windows: # Session expires when the session end time has passed the expiry threshold @@ -541,20 +553,29 @@ def expire_sessions_by_key( collected = [] if collect: collected = state.get_from_collection(window_start, window_end) - + windows_to_delete.append((window_start, window_end)) count += 1 - yield (key, self._results(aggregated, collected, window_start, window_end)) + yield ( + key, + self._results(aggregated, collected, window_start, window_end), + ) # Clean up expired windows for window_start, window_end in windows_to_delete: - state._transaction.delete_window(window_start, window_end, prefix=state._prefix) + state._transaction.delete_window( # type: ignore # noqa: SLF001 + window_start, + window_end, + prefix=state._prefix, # type: ignore # noqa: SLF001 + ) if collect: state.delete_from_collection(window_end, start=window_start) if count: logger.debug( - "Expired %s session windows in %ss", count, round(time.monotonic() - start, 2) + "Expired %s session windows in %ss", + count, + round(time.monotonic() - start, 2), ) def _on_expired_session( @@ -576,9 +597,9 @@ def _on_expired_session( topic = "unknown" partition = -1 offset = -1 - + to_log = True - + # Trigger the "on_late" callback if provided if self._on_late: to_log = self._on_late( diff --git a/quixstreams/state/types.py b/quixstreams/state/types.py index c80c9e2ad..beb8b0e0b 100644 --- a/quixstreams/state/types.py +++ b/quixstreams/state/types.py @@ -391,6 +391,18 @@ def expire_all_windows( """ ... + def delete_window(self, start_ms: int, end_ms: int, prefix: bytes) -> None: + """ + Delete a specific window from RocksDB. + + This method removes a single window entry with the specified start and end timestamps. + + :param start_ms: The start timestamp of the window to delete + :param end_ms: The end timestamp of the window to delete + :param prefix: The key prefix for the window + """ + ... + def delete_windows( self, max_start_time: int, delete_values: bool, prefix: bytes ) -> None: diff --git a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py index 80f7156af..a342ac69b 100644 --- a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py +++ b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py @@ -3,7 +3,6 @@ import quixstreams.dataframe.windows.aggregations as agg from quixstreams.dataframe import DataFrameRegistry from quixstreams.dataframe.windows.definitions import SessionWindowDefinition -from quixstreams.dataframe.windows.time_based import ClosingStrategy @pytest.fixture() @@ -31,7 +30,13 @@ class TestSessionWindow: @pytest.mark.parametrize( "timeout, grace, provided_name, func_name, expected_name", [ - (30000, 5000, "custom_window", "sum", "custom_window_session_window_30000_sum"), + ( + 30000, + 5000, + "custom_window", + "sum", + "custom_window_session_window_30000_sum", + ), (30000, 5000, None, "sum", "session_window_30000_sum"), (15000, 5000, None, "count", "session_window_15000_count"), ], @@ -360,7 +365,7 @@ def test_session_window_process_timeout_behavior( store.assign_partition(0) with store.start_partition_transaction(0) as tx: key = b"key" - + # Start session 1 updated, expired = process( window, value=1, key=key, transaction=tx, timestamp_ms=1000 @@ -406,7 +411,7 @@ def test_session_window_grace_period( store.assign_partition(0) with store.start_partition_transaction(0) as tx: key = b"key" - + # Start session updated, expired = process( window, value=1, key=key, transaction=tx, timestamp_ms=1000 @@ -444,7 +449,7 @@ def test_session_window_multiple_keys( with store.start_partition_transaction(0) as tx: key1 = b"key1" key2 = b"key2" - + # Start session for key1 updated, expired = process( window, value=1, key=key1, transaction=tx, timestamp_ms=1000 @@ -530,11 +535,11 @@ def test_session_window_late_events( store.assign_partition(0) with store.start_partition_transaction(0) as tx: key = b"key" - + # Start and finish a session process(window, value=1, key=key, transaction=tx, timestamp_ms=1000) process(window, value=2, key=key, transaction=tx, timestamp_ms=3000) - + # Start new session that will cause first to expire updated, expired = process( window, value=5, key=key, transaction=tx, timestamp_ms=15000 @@ -562,7 +567,7 @@ def test_session_window_current_mode( store.assign_partition(0) with store.start_partition_transaction(0) as tx: key = b"key" - + # Start session - should get update immediately updated, expired = process( window, value=1, key=key, transaction=tx, timestamp_ms=1000 @@ -590,13 +595,13 @@ def test_session_window_overlapping_sessions( store.assign_partition(0) with store.start_partition_transaction(0) as tx: key = b"key" - + # Start session 1 updated, expired = process( window, value=1, key=key, transaction=tx, timestamp_ms=1000 ) session1_end = updated[0][1]["end"] - + # Event within timeout - extends session 1 updated, expired = process( window, value=2, key=key, transaction=tx, timestamp_ms=5000 @@ -604,7 +609,7 @@ def test_session_window_overlapping_sessions( new_end = updated[0][1]["end"] assert new_end > session1_end # Session extended assert updated[0][1]["value"] == 3 # Accumulated value - + # Event far in future - starts session 2, expires session 1 updated, expired = process( window, value=10, key=key, transaction=tx, timestamp_ms=30000 @@ -626,7 +631,7 @@ def test_session_window_merge_sessions( store.assign_partition(0) with store.start_partition_transaction(0) as tx: key = b"key" - + # Create first session updated, expired = process( window, value=1, key=key, transaction=tx, timestamp_ms=1000 @@ -636,7 +641,7 @@ def test_session_window_merge_sessions( assert updated[0][1]["end"] == 11000 # 1000 + 10000 assert updated[0][1]["value"] == 1 assert not expired - + # Create second session that doesn't expire the first one yet # (13000 is still within timeout + grace of first session: 11000 + 1000 = 12000) # Actually, let's make it further: 20000ms to ensure two separate sessions @@ -648,12 +653,12 @@ def test_session_window_merge_sessions( assert expired[0][1]["start"] == 1000 assert expired[0][1]["end"] == 11000 assert expired[0][1]["value"] == 1 - + assert len(updated) == 1 assert updated[0][1]["start"] == 20000 assert updated[0][1]["end"] == 30000 # 20000 + 10000 assert updated[0][1]["value"] == 10 - + # Add another event to the second session updated, expired = process( window, value=5, key=key, transaction=tx, timestamp_ms=25000 @@ -663,7 +668,7 @@ def test_session_window_merge_sessions( assert updated[0][1]["end"] == 35000 # 25000 + 10000 assert updated[0][1]["value"] == 15 # 10 + 5 assert not expired - + # Now test the limitation: we'll create a third session that could theoretically # merge with the second session if there was a bridging event # But since sessions don't auto-merge, they'll remain separate @@ -675,7 +680,7 @@ def test_session_window_merge_sessions( assert expired[0][1]["start"] == 20000 assert expired[0][1]["end"] == 35000 assert expired[0][1]["value"] == 15 - + # Third session starts assert len(updated) == 1 assert updated[0][1]["start"] == 50000 @@ -687,18 +692,18 @@ def test_session_window_bridging_event_scenario( ): """ Test scenario where an event arrives that could theoretically bridge two sessions. - + This test documents the current behavior where sessions don't auto-merge, even when a bridging event could logically connect them. - + Scenario: 1. Session A: [1000, 11000] with value 5 - 2. Session B: [15000, 25000] with value 10 + 2. Session B: [15000, 25000] with value 10 3. Bridging event at 12000ms that: - Can extend Session A to [1000, 22000] - Now overlaps with Session B [15000, 25000] - Ideally should merge into single session [1000, 25000] with value 15+bridge_value - + Current behavior: Session A gets extended, Session B remains separate Ideal behavior: Sessions A and B get merged when bridging event arrives """ @@ -709,7 +714,7 @@ def test_session_window_bridging_event_scenario( store.assign_partition(0) with store.start_partition_transaction(0) as tx: key = b"key" - + # Create Session A updated, expired = process( window, value=5, key=key, transaction=tx, timestamp_ms=1000 @@ -719,7 +724,7 @@ def test_session_window_bridging_event_scenario( assert updated[0][1]["end"] == 11000 # 1000 + 10000 assert updated[0][1]["value"] == 5 assert not expired - + # Create Session B - close enough that it doesn't expire Session A # Session A expires when time > 11000 + 2000 = 13000 # So event at 12000 should keep Session A alive @@ -732,9 +737,9 @@ def test_session_window_bridging_event_scenario( assert len(updated) == 1 assert updated[0][1]["start"] == 1000 # Session A extended assert updated[0][1]["end"] == 22000 # 12000 + 10000 - assert updated[0][1]["value"] == 15 # 5 + 10 + assert updated[0][1]["value"] == 15 # 5 + 10 assert not expired - + # Now create what would be Session B if Session A hadn't been extended updated, expired = process( window, value=20, key=key, transaction=tx, timestamp_ms=15000 @@ -743,9 +748,9 @@ def test_session_window_bridging_event_scenario( assert len(updated) == 1 assert updated[0][1]["start"] == 1000 # Still Session A assert updated[0][1]["end"] == 25000 # 15000 + 10000 - assert updated[0][1]["value"] == 35 # 5 + 10 + 20 + assert updated[0][1]["value"] == 35 # 5 + 10 + 20 assert not expired - + # Final event to expire the session updated, expired = process( window, value=1, key=key, transaction=tx, timestamp_ms=40000 @@ -754,7 +759,91 @@ def test_session_window_bridging_event_scenario( assert expired[0][1]["start"] == 1000 assert expired[0][1]["end"] == 25000 assert expired[0][1]["value"] == 35 # All events combined - + assert len(updated) == 1 assert updated[0][1]["start"] == 40000 - assert updated[0][1]["value"] == 1 \ No newline at end of file + assert updated[0][1]["value"] == 1 + + def test_session_window_string_key_extension( + self, session_window_definition_factory, state_manager + ): + """ + Test session window extension with string keys. + + This test specifically verifies that session extension works correctly + when using string keys (which need to be serialized to bytes internally). + + This test would have caught the original TypeError bug where + `transaction.delete_window()` was called with a string key instead of + the properly serialized bytes prefix. + """ + window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window = window_def.sum() + window.final(closing_strategy="key") + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + with store.start_partition_transaction(0) as tx: + # Use a string key instead of bytes to trigger the serialization path + key = "user_123" + + # Start a session + updated, expired = process( + window, value=100, key=key, transaction=tx, timestamp_ms=1000 + ) + assert len(updated) == 1 + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 11000 # 1000 + 10000 + assert updated[0][1]["value"] == 100 + assert not expired + + # Extend the session - this should trigger the delete_window call + # that would have failed with the original bug + updated, expired = process( + window, value=200, key=key, transaction=tx, timestamp_ms=5000 + ) + assert len(updated) == 1 + assert updated[0][1]["start"] == 1000 # Session extended, same start + assert updated[0][1]["end"] == 15000 # 5000 + 10000 (new end time) + assert updated[0][1]["value"] == 300 # 100 + 200 + assert not expired + + # Extend the session again to make sure it still works + updated, expired = process( + window, value=50, key=key, transaction=tx, timestamp_ms=8000 + ) + assert len(updated) == 1 + assert updated[0][1]["start"] == 1000 # Session extended again + assert updated[0][1]["end"] == 18000 # 8000 + 10000 + assert updated[0][1]["value"] == 350 # 100 + 200 + 50 + assert not expired + + # Test with a different string key to make sure multiple keys work + key2 = "user_456" + updated, expired = process( + window, value=75, key=key2, transaction=tx, timestamp_ms=9000 + ) + assert len(updated) == 1 + assert updated[0][0] == key2 # Different key + assert updated[0][1]["start"] == 9000 + assert updated[0][1]["end"] == 19000 # 9000 + 10000 + assert updated[0][1]["value"] == 75 + assert not expired + + # Expire the first session by advancing time far enough + updated, expired = process( + window, value=25, key=key, transaction=tx, timestamp_ms=30000 + ) + + # Should have expired the first session + assert len(expired) == 1 + assert expired[0][0] == key + assert expired[0][1]["start"] == 1000 + assert expired[0][1]["end"] == 18000 + assert expired[0][1]["value"] == 350 + + # Should have started a new session for the first key + assert len(updated) == 1 + assert updated[0][0] == key + assert updated[0][1]["start"] == 30000 + assert updated[0][1]["end"] == 40000 + assert updated[0][1]["value"] == 25 From 7db4ffef99b8053d871618809d9fc88a05bf3005 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Fri, 1 Aug 2025 15:17:25 +0200 Subject: [PATCH 03/19] Fix linters --- quixstreams/dataframe/dataframe.py | 4 ++-- quixstreams/dataframe/windows/definitions.py | 7 ++++--- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/quixstreams/dataframe/dataframe.py b/quixstreams/dataframe/dataframe.py index 3f7684a3d..52ecd9d57 100644 --- a/quixstreams/dataframe/dataframe.py +++ b/quixstreams/dataframe/dataframe.py @@ -1536,12 +1536,12 @@ def session_window( ) -> SessionWindowDefinition: """ Create a session window transformation on this StreamingDataFrame. - + Session windows group events that occur within a specified timeout period. A session starts with the first event and extends each time a new event arrives within the timeout period. The session closes after the timeout period with no new events. - + Unlike fixed-time windows, session windows have dynamic durations based on the actual events and their timing, making them ideal for user activity tracking, fraud detection, and other event-driven scenarios. diff --git a/quixstreams/dataframe/windows/definitions.py b/quixstreams/dataframe/windows/definitions.py index 35f5db111..174f920f4 100644 --- a/quixstreams/dataframe/windows/definitions.py +++ b/quixstreams/dataframe/windows/definitions.py @@ -539,13 +539,13 @@ def _get_name(self, func_name: Optional[str]) -> str: class SessionWindowDefinition(WindowDefinition): """ Definition for session windows that group events by activity sessions. - + Session windows group events that occur within a specified timeout period. A session starts with the first event and extends each time a new event arrives within the timeout period. The session closes after the timeout period with no new events. """ - + def __init__( self, timeout_ms: int, @@ -589,7 +589,8 @@ def _create_window( ) -> SessionWindow: if func_name: window_type: Union[ - type[SessionWindowSingleAggregation], type[SessionWindowMultiAggregation] + type[SessionWindowSingleAggregation], + type[SessionWindowMultiAggregation], ] = SessionWindowSingleAggregation else: window_type = SessionWindowMultiAggregation From 587c77134456b75b72e723b5f9e3dc41e26ab666 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Mon, 4 Aug 2025 13:47:01 +0200 Subject: [PATCH 04/19] Correct docs --- docs/windowing.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/windowing.md b/docs/windowing.md index 6ce57c5d2..acd792250 100644 --- a/docs/windowing.md +++ b/docs/windowing.md @@ -9,7 +9,7 @@ With windows, you can calculate such aggregations as: - Total of website visitors for every hour - The average speed of a vehicle over the last 10 minutes - Maximum temperature of a sensor observed over 30 second ranges -- Give an user a reward after 10 succesful actions +- Give a user a reward after 10 successful actions - Track user activity sessions on a website - Detect fraud patterns in financial transactions @@ -205,14 +205,14 @@ Count-based Tumbling Windows slice incoming events into batch of a fixed size. For example, a tumbling window configured with a count of 4 will batch and aggregate message 1 to 4, then 5 to 8, 9 to 12 and so on. ``` -Count Tumbing Windows +Count Tumbling Windows [0, 3] : .... [4, 7] : .... [8, 11] : .... [12, 15] : .... ``` -In a tumbing window each message is only assigned to a **single** interval. +In a tumbling window each message is only assigned to a **single** interval. **Example** @@ -358,7 +358,7 @@ Count Hopping Windows [6, 11] : ...... ``` -In hopping windows each messages can be assigned to multiple windows because the windows overlap. +In hopping windows each message can be assigned to multiple windows because the windows overlap. ## Time-based Sliding Windows Sliding windows are overlapping time-based windows that advance with each incoming message, rather than at fixed time intervals like hopping windows. They have a fixed 1 ms resolution and perform better and are less resource-intensive than hopping windows with a 1 ms step. Sliding windows do not produce redundant windows; every interval has a distinct aggregation. @@ -440,7 +440,7 @@ sdf = ( Sliding windows are overlapping windows that advance with each incoming message. They are equal to count-based hopping windows with a step of 1. -For example a sliding window of 4 messagew will generate the followiwng windows: +For example a sliding window of 4 messages will generate the following windows: ``` Count Sliding Windows @@ -504,7 +504,7 @@ sdf = ( ## Session Windows -Session windows group events that occur within a specified timeout period. Unlike fixed-time windows (tumbling, hopping, sliding), session windows have dynamic durations based on the actual timing of events, making them ideal for user activity tracking, fraud detection, and other event-driven scenarios. +Session windows group events that occur within a specified timeout period. Unlike fixed-time windows (tumbling, hopping, sliding), session windows have dynamic durations based on the actual timing of events. This makes them ideal for user activity tracking, fraud detection, and other event-driven scenarios. A session starts with the first event and extends each time a new event arrives within the timeout period. The session closes after the timeout period with no new events. @@ -525,7 +525,7 @@ Timeout: 10 seconds Grace: 2 seconds Session 1: [0, 20] - Events A, B (B extends the session from A) -Session 2: [25, 35] - Events C, D (D extends the session from C) +Session 2: [25, 40] - Events C, D (D extends the session from C) Session 3: [45, 55] - Event E (session will close at 55 if no more events) ``` @@ -914,7 +914,7 @@ sdf = sdf.tumbling_window(timedelta(seconds=10)).agg(value=Sum()).final(closing_ An alternative is to use the **partition** closing strategy. In this strategy, messages advance time and close windows for the whole partition to which this key belongs. -If messages aren't ordered accross keys some message can be skipped if the windows are already closed. +If messages aren't ordered across keys some message can be skipped if the windows are already closed. ```python from datetime import timedelta From 3d777e752c3006011ee80c718d79e195bfbf0848 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Mon, 4 Aug 2025 14:58:16 +0200 Subject: [PATCH 05/19] Rename timeout_ms to inactivity_gap_ms --- quixstreams/dataframe/dataframe.py | 10 +++++----- quixstreams/dataframe/windows/definitions.py | 18 +++++++----------- 2 files changed, 12 insertions(+), 16 deletions(-) diff --git a/quixstreams/dataframe/dataframe.py b/quixstreams/dataframe/dataframe.py index 52ecd9d57..b2a83ebab 100644 --- a/quixstreams/dataframe/dataframe.py +++ b/quixstreams/dataframe/dataframe.py @@ -1529,7 +1529,7 @@ def sliding_count_window( def session_window( self, - timeout_ms: Union[int, timedelta], + inactivity_gap_ms: Union[int, timedelta], grace_ms: Union[int, timedelta] = 0, name: Optional[str] = None, on_late: Optional[WindowOnLateCallback] = None, @@ -1568,7 +1568,7 @@ def session_window( sdf = ( # Define a session window with 30-second timeout and 10-second grace period sdf.session_window( - timeout_ms=timedelta(seconds=30), + inactivity_gap_ms=timedelta(seconds=30), grace_ms=timedelta(seconds=10) ) @@ -1584,7 +1584,7 @@ def session_window( ) ``` - :param timeout_ms: The session timeout period. + :param inactivity_gap_ms: The session timeout period. If no new events arrive within this period, the session will be closed. Can be specified as either an `int` representing milliseconds or a `timedelta` object. @@ -1613,11 +1613,11 @@ def session_window( This object can be further configured with aggregation functions like `sum`, `count`, etc. applied to the StreamingDataFrame. """ - timeout_ms = ensure_milliseconds(timeout_ms) + inactivity_gap_ms = ensure_milliseconds(inactivity_gap_ms) grace_ms = ensure_milliseconds(grace_ms) return SessionWindowDefinition( - timeout_ms=timeout_ms, + inactivity_gap_ms=inactivity_gap_ms, grace_ms=grace_ms, dataframe=self, name=name, diff --git a/quixstreams/dataframe/windows/definitions.py b/quixstreams/dataframe/windows/definitions.py index 174f920f4..58edb3a90 100644 --- a/quixstreams/dataframe/windows/definitions.py +++ b/quixstreams/dataframe/windows/definitions.py @@ -548,28 +548,24 @@ class SessionWindowDefinition(WindowDefinition): def __init__( self, - timeout_ms: int, + inactivity_gap_ms: int, grace_ms: int, dataframe: "StreamingDataFrame", name: Optional[str] = None, on_late: Optional[WindowOnLateCallback] = None, ): - if not isinstance(timeout_ms, int): + if not isinstance(inactivity_gap_ms, int): raise TypeError("Session timeout must be an integer") - if timeout_ms < 1: + if inactivity_gap_ms < 1: raise ValueError("Session timeout cannot be smaller than 1ms") if grace_ms < 0: raise ValueError("Session grace cannot be smaller than 0ms") super().__init__(name, dataframe, on_late) - self._timeout_ms = timeout_ms + self._inactivity_gap_ms = inactivity_gap_ms self._grace_ms = grace_ms - @property - def timeout_ms(self) -> int: - return self._timeout_ms - @property def grace_ms(self) -> int: return self._grace_ms @@ -577,9 +573,9 @@ def grace_ms(self) -> int: def _get_name(self, func_name: Optional[str]) -> str: prefix = f"{self._name}_session_window" if self._name else "session_window" if func_name: - return f"{prefix}_{self._timeout_ms}_{func_name}" + return f"{prefix}_{self._inactivity_gap_ms}_{func_name}" else: - return f"{prefix}_{self._timeout_ms}" + return f"{prefix}_{self._inactivity_gap_ms}" def _create_window( self, @@ -596,7 +592,7 @@ def _create_window( window_type = SessionWindowMultiAggregation return window_type( - timeout_ms=self._timeout_ms, + timeout_ms=self._inactivity_gap_ms, grace_ms=self._grace_ms, name=self._get_name(func_name=func_name), dataframe=self._dataframe, From 3bed3678372858646920b0927b824ddadd6f41bb Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Mon, 4 Aug 2025 14:58:52 +0200 Subject: [PATCH 06/19] Remove asserts --- quixstreams/dataframe/windows/time_based.py | 6 ------ 1 file changed, 6 deletions(-) diff --git a/quixstreams/dataframe/windows/time_based.py b/quixstreams/dataframe/windows/time_based.py index 250b0bab6..bc99535af 100644 --- a/quixstreams/dataframe/windows/time_based.py +++ b/quixstreams/dataframe/windows/time_based.py @@ -402,8 +402,6 @@ def process_window( return [], [] # Look for an existing session that can be extended - session_start = None - session_end = None can_extend_session = False existing_aggregated = None old_window_to_delete = None @@ -457,10 +455,6 @@ def process_window( aggregated = self._aggregate_value(current_value, value, timestamp_ms) - # By this point, session_start and session_end are guaranteed to be set - assert session_start is not None # noqa: S101 - assert session_end is not None # noqa: S101 - # Output intermediate results for aggregations if aggregate: updated_windows.append( From bec010bdfa830edb387f94b25e769c4858910427 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Mon, 4 Aug 2025 15:08:56 +0200 Subject: [PATCH 07/19] Tiny refactor --- quixstreams/dataframe/windows/time_based.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/quixstreams/dataframe/windows/time_based.py b/quixstreams/dataframe/windows/time_based.py index bc99535af..e9162f24a 100644 --- a/quixstreams/dataframe/windows/time_based.py +++ b/quixstreams/dataframe/windows/time_based.py @@ -449,9 +449,9 @@ def process_window( # Update the session window aggregation aggregated = None if aggregate: - current_value = existing_aggregated if can_extend_session else None - if current_value is None: - current_value = self._initialize_value() + current_value = ( + existing_aggregated if can_extend_session else self._initialize_value() + ) aggregated = self._aggregate_value(current_value, value, timestamp_ms) From 68687ab92a7d92900ff3b2a993ec3a6920ed16e6 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Mon, 4 Aug 2025 15:41:55 +0200 Subject: [PATCH 08/19] Rename timeout_ms to inactivity_gap_ms in tests --- .../test_windows/test_session.py | 82 +++++++++++++------ 1 file changed, 59 insertions(+), 23 deletions(-) diff --git a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py index a342ac69b..7c7b7cccd 100644 --- a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py +++ b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py @@ -7,12 +7,12 @@ @pytest.fixture() def session_window_definition_factory(state_manager, dataframe_factory): - def factory(timeout_ms: int, grace_ms: int = 0) -> SessionWindowDefinition: + def factory(inactivity_gap_ms: int, grace_ms: int = 0) -> SessionWindowDefinition: sdf = dataframe_factory( state_manager=state_manager, registry=DataFrameRegistry() ) window_def = SessionWindowDefinition( - timeout_ms=timeout_ms, grace_ms=grace_ms, dataframe=sdf + inactivity_gap_ms=inactivity_gap_ms, grace_ms=grace_ms, dataframe=sdf ) return window_def @@ -51,7 +51,7 @@ def test_session_window_definition_get_name( dataframe_factory, ): swd = SessionWindowDefinition( - timeout_ms=timeout, + inactivity_gap_ms=timeout, grace_ms=grace, dataframe=dataframe_factory(), name=provided_name, @@ -64,7 +64,9 @@ def test_multiaggregation( session_window_definition_factory, state_manager, ): - window = session_window_definition_factory(timeout_ms=10000, grace_ms=1000).agg( + window = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=1000 + ).agg( count=agg.Count(), sum=agg.Sum(), mean=agg.Mean(), @@ -160,7 +162,9 @@ def test_multiaggregation( def test_sessionwindow_count( self, expiration, session_window_definition_factory, state_manager ): - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=1000 + ) window = window_def.count() assert window.name == "session_window_10000_count" @@ -185,7 +189,9 @@ def test_sessionwindow_count( def test_sessionwindow_sum( self, expiration, session_window_definition_factory, state_manager ): - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=1000 + ) window = window_def.sum() assert window.name == "session_window_10000_sum" @@ -208,7 +214,9 @@ def test_sessionwindow_sum( def test_sessionwindow_mean( self, expiration, session_window_definition_factory, state_manager ): - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=1000 + ) window = window_def.mean() assert window.name == "session_window_10000_mean" @@ -231,7 +239,9 @@ def test_sessionwindow_mean( def test_sessionwindow_reduce( self, expiration, session_window_definition_factory, state_manager ): - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=1000 + ) window = window_def.reduce( reducer=lambda agg, current: agg + [current], initializer=lambda value: [value], @@ -257,7 +267,9 @@ def test_sessionwindow_reduce( def test_sessionwindow_max( self, expiration, session_window_definition_factory, state_manager ): - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=1000 + ) window = window_def.max() assert window.name == "session_window_10000_max" @@ -280,7 +292,9 @@ def test_sessionwindow_max( def test_sessionwindow_min( self, expiration, session_window_definition_factory, state_manager ): - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=1000 + ) window = window_def.min() assert window.name == "session_window_10000_min" @@ -303,7 +317,9 @@ def test_sessionwindow_min( def test_sessionwindow_collect( self, expiration, session_window_definition_factory, state_manager ): - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=1000 + ) window = window_def.collect() assert window.name == "session_window_10000_collect" @@ -335,7 +351,7 @@ def test_session_window_def_init_invalid( ): with pytest.raises(ValueError): SessionWindowDefinition( - timeout_ms=timeout, + inactivity_gap_ms=timeout, grace_ms=grace, name=name, dataframe=dataframe_factory(), @@ -344,7 +360,7 @@ def test_session_window_def_init_invalid( def test_session_window_def_init_invalid_type(self, dataframe_factory): with pytest.raises(TypeError): SessionWindowDefinition( - timeout_ms="invalid", # should be int + inactivity_gap_ms="invalid", # should be int grace_ms=1000, name="test", dataframe=dataframe_factory(), @@ -358,7 +374,9 @@ def test_session_window_process_timeout_behavior( state_manager, ): """Test that sessions properly timeout and new sessions start correctly""" - window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=0) + window_def = session_window_definition_factory( + inactivity_gap_ms=5000, grace_ms=0 + ) window = window_def.sum() window.final(closing_strategy=expiration) store = state_manager.get_store(stream_id="test", store_name=window.name) @@ -404,7 +422,9 @@ def test_session_window_grace_period( self, session_window_definition_factory, state_manager ): """Test that grace period allows late events""" - window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=2000) + window_def = session_window_definition_factory( + inactivity_gap_ms=5000, grace_ms=2000 + ) window = window_def.sum() window.final(closing_strategy="key") store = state_manager.get_store(stream_id="test", store_name=window.name) @@ -441,7 +461,9 @@ def test_session_window_multiple_keys( self, session_window_definition_factory, state_manager ): """Test that different keys maintain separate sessions""" - window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=0) + window_def = session_window_definition_factory( + inactivity_gap_ms=5000, grace_ms=0 + ) window = window_def.sum() window.final(closing_strategy="key") store = state_manager.get_store(stream_id="test", store_name=window.name) @@ -490,7 +512,9 @@ def test_session_partition_expiration( self, session_window_definition_factory, state_manager ): """Test partition-level session expiration""" - window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=5000, grace_ms=1000 + ) window = window_def.sum() window.final(closing_strategy="partition") store = state_manager.get_store(stream_id="test", store_name=window.name) @@ -528,7 +552,9 @@ def test_session_window_late_events( self, session_window_definition_factory, state_manager ): """Test handling of late events that arrive after session closure""" - window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=5000, grace_ms=1000 + ) window = window_def.sum() window.final(closing_strategy="key") store = state_manager.get_store(stream_id="test", store_name=window.name) @@ -560,7 +586,9 @@ def test_session_window_current_mode( self, session_window_definition_factory, state_manager ): """Test session window with current() mode""" - window_def = session_window_definition_factory(timeout_ms=5000, grace_ms=0) + window_def = session_window_definition_factory( + inactivity_gap_ms=5000, grace_ms=0 + ) window = window_def.sum() window.current(closing_strategy="key") store = state_manager.get_store(stream_id="test", store_name=window.name) @@ -588,7 +616,9 @@ def test_session_window_overlapping_sessions( self, session_window_definition_factory, state_manager ): """Test that sessions don't overlap for the same key""" - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=0) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=0 + ) window = window_def.sum() window.final(closing_strategy="key") store = state_manager.get_store(stream_id="test", store_name=window.name) @@ -624,7 +654,9 @@ def test_session_window_merge_sessions( self, session_window_definition_factory, state_manager ): """Test that an event can merge two previously separate sessions""" - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=1000 + ) window = window_def.sum() window.final(closing_strategy="key") store = state_manager.get_store(stream_id="test", store_name=window.name) @@ -707,7 +739,9 @@ def test_session_window_bridging_event_scenario( Current behavior: Session A gets extended, Session B remains separate Ideal behavior: Sessions A and B get merged when bridging event arrives """ - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=2000) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=2000 + ) window = window_def.sum() window.final(closing_strategy="key") store = state_manager.get_store(stream_id="test", store_name=window.name) @@ -777,7 +811,9 @@ def test_session_window_string_key_extension( `transaction.delete_window()` was called with a string key instead of the properly serialized bytes prefix. """ - window_def = session_window_definition_factory(timeout_ms=10000, grace_ms=1000) + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=1000 + ) window = window_def.sum() window.final(closing_strategy="key") store = state_manager.get_store(stream_id="test", store_name=window.name) From 048020a4169b632ee4e2d8da55a01c811ce51470 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Mon, 4 Aug 2025 16:44:29 +0200 Subject: [PATCH 09/19] Use mock_message_context --- quixstreams/dataframe/windows/time_based.py | 22 +++++-------------- .../test_dataframe/fixtures.py | 8 ++++++- .../test_windows/test_session.py | 2 +- .../test_windows/test_sliding.py | 7 ------ 4 files changed, 14 insertions(+), 25 deletions(-) diff --git a/quixstreams/dataframe/windows/time_based.py b/quixstreams/dataframe/windows/time_based.py index e9162f24a..5258ca7d0 100644 --- a/quixstreams/dataframe/windows/time_based.py +++ b/quixstreams/dataframe/windows/time_based.py @@ -581,17 +581,7 @@ def _on_expired_session( timestamp_ms: int, late_by_ms: int, ) -> None: - try: - ctx = message_context() - topic = ctx.topic - partition = ctx.partition - offset = ctx.offset - except: - # In test environments, message context might not be available - topic = "unknown" - partition = -1 - offset = -1 - + ctx = message_context() to_log = True # Trigger the "on_late" callback if provided @@ -604,9 +594,9 @@ def _on_expired_session( start, end, self._name, - topic, - partition, - offset, + ctx.topic, + ctx.partition, + ctx.offset, ) if to_log: logger.warning( @@ -615,8 +605,8 @@ def _on_expired_session( f"session={(start, end)} " f"late_by_ms={late_by_ms} " f"store_name={self._name} " - f"partition={topic}[{partition}] " - f"offset={offset}" + f"partition={ctx.topic}[{ctx.partition}] " + f"offset={ctx.offset}" ) diff --git a/tests/test_quixstreams/test_dataframe/fixtures.py b/tests/test_quixstreams/test_dataframe/fixtures.py index 1955c17a3..aabc1529a 100644 --- a/tests/test_quixstreams/test_dataframe/fixtures.py +++ b/tests/test_quixstreams/test_dataframe/fixtures.py @@ -1,5 +1,5 @@ from typing import Optional -from unittest.mock import MagicMock +from unittest.mock import MagicMock, patch import pytest @@ -59,3 +59,9 @@ def factory( return sdf return factory + + +@pytest.fixture +def mock_message_context(): + with patch("quixstreams.dataframe.windows.time_based.message_context"): + yield diff --git a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py index 7c7b7cccd..ea3b91a05 100644 --- a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py +++ b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py @@ -549,7 +549,7 @@ def test_session_partition_expiration( assert key2 in expired_keys def test_session_window_late_events( - self, session_window_definition_factory, state_manager + self, session_window_definition_factory, state_manager, mock_message_context ): """Test handling of late events that arrive after session closure""" window_def = session_window_definition_factory( diff --git a/tests/test_quixstreams/test_dataframe/test_windows/test_sliding.py b/tests/test_quixstreams/test_dataframe/test_windows/test_sliding.py index fc5ab8eba..2d12ecd8d 100644 --- a/tests/test_quixstreams/test_dataframe/test_windows/test_sliding.py +++ b/tests/test_quixstreams/test_dataframe/test_windows/test_sliding.py @@ -2,7 +2,6 @@ from dataclasses import dataclass, field from itertools import chain from typing import Any -from unittest import mock import pytest @@ -741,12 +740,6 @@ def expected_windows_in_state(self) -> set[tuple[int, int]]: ] -@pytest.fixture -def mock_message_context(): - with mock.patch("quixstreams.dataframe.windows.time_based.message_context"): - yield - - @pytest.fixture def sliding_window_definition_factory( state_manager, dataframe_factory, topic_manager_topic_factory From e61b9993fa557b3f2aa38e297832d818e4eb253a Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Tue, 5 Aug 2025 11:39:42 +0200 Subject: [PATCH 10/19] Remove redundant if --- quixstreams/dataframe/windows/time_based.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/quixstreams/dataframe/windows/time_based.py b/quixstreams/dataframe/windows/time_based.py index 5258ca7d0..bed9e116d 100644 --- a/quixstreams/dataframe/windows/time_based.py +++ b/quixstreams/dataframe/windows/time_based.py @@ -454,9 +454,6 @@ def process_window( ) aggregated = self._aggregate_value(current_value, value, timestamp_ms) - - # Output intermediate results for aggregations - if aggregate: updated_windows.append( ( key, From ea35cc0fda574a6c0f4e08927c96db3064f7570a Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Tue, 5 Aug 2025 12:41:25 +0200 Subject: [PATCH 11/19] Move SessionWindow to a dedicated module --- quixstreams/dataframe/windows/definitions.py | 8 +- quixstreams/dataframe/windows/session.py | 364 ++++++++++++++++++ quixstreams/dataframe/windows/time_based.py | 344 ----------------- .../test_dataframe/fixtures.py | 5 +- 4 files changed, 373 insertions(+), 348 deletions(-) create mode 100644 quixstreams/dataframe/windows/session.py diff --git a/quixstreams/dataframe/windows/definitions.py b/quixstreams/dataframe/windows/definitions.py index 58edb3a90..987a4ed5c 100644 --- a/quixstreams/dataframe/windows/definitions.py +++ b/quixstreams/dataframe/windows/definitions.py @@ -22,15 +22,17 @@ CountWindowMultiAggregation, CountWindowSingleAggregation, ) +from .session import ( + SessionWindow, + SessionWindowMultiAggregation, + SessionWindowSingleAggregation, +) from .sliding import ( SlidingWindow, SlidingWindowMultiAggregation, SlidingWindowSingleAggregation, ) from .time_based import ( - SessionWindow, - SessionWindowMultiAggregation, - SessionWindowSingleAggregation, TimeWindow, TimeWindowMultiAggregation, TimeWindowSingleAggregation, diff --git a/quixstreams/dataframe/windows/session.py b/quixstreams/dataframe/windows/session.py new file mode 100644 index 000000000..8af54d839 --- /dev/null +++ b/quixstreams/dataframe/windows/session.py @@ -0,0 +1,364 @@ +import logging +import time +from typing import TYPE_CHECKING, Any, Iterable, Optional + +from quixstreams.context import message_context +from quixstreams.state import WindowedPartitionTransaction, WindowedState + +from .base import ( + MultiAggregationWindowMixin, + SingleAggregationWindowMixin, + Window, + WindowKeyResult, + WindowOnLateCallback, +) +from .time_based import ClosingStrategy, ClosingStrategyValues + +if TYPE_CHECKING: + from quixstreams.dataframe.dataframe import StreamingDataFrame + +logger = logging.getLogger(__name__) + + +class SessionWindow(Window): + """ + Session window groups events that occur within a specified timeout period. + + A session starts with the first event and extends each time a new event arrives + within the timeout period. The session closes after the timeout period with no + new events. + + Each session window can have different start and end times based on the actual + events, making sessions dynamic rather than fixed-time intervals. + """ + + def __init__( + self, + timeout_ms: int, + grace_ms: int, + name: str, + dataframe: "StreamingDataFrame", + on_late: Optional[WindowOnLateCallback] = None, + ): + super().__init__( + name=name, + dataframe=dataframe, + ) + + self._timeout_ms = timeout_ms + self._grace_ms = grace_ms + self._on_late = on_late + self._closing_strategy = ClosingStrategy.KEY + + def final( + self, closing_strategy: ClosingStrategyValues = "key" + ) -> "StreamingDataFrame": + """ + Apply the session window aggregation and return results only when the sessions + are closed. + + The format of returned sessions: + ```python + { + "start": , + "end": , + "value: , + } + ``` + + The individual session is closed when the event time + (the maximum observed timestamp across the partition) passes + the last event timestamp + timeout + grace period. + The closed sessions cannot receive updates anymore and are considered final. + + :param closing_strategy: the strategy to use when closing sessions. + Possible values: + - `"key"` - messages advance time and close sessions with the same key. + If some message keys appear irregularly in the stream, the latest sessions can remain unprocessed until a message with the same key is received. + - `"partition"` - messages advance time and close sessions for the whole partition to which this message key belongs. + If timestamps between keys are not ordered, it may increase the number of discarded late messages. + Default - `"key"`. + """ + self._closing_strategy = ClosingStrategy.new(closing_strategy) + return super().final() + + def current( + self, closing_strategy: ClosingStrategyValues = "key" + ) -> "StreamingDataFrame": + """ + Apply the session window transformation to the StreamingDataFrame to return results + for each updated session. + + The format of returned sessions: + ```python + { + "start": , + "end": , + "value: , + } + ``` + + This method processes streaming data and returns results as they come, + regardless of whether the session is closed or not. + + :param closing_strategy: the strategy to use when closing sessions. + Possible values: + - `"key"` - messages advance time and close sessions with the same key. + If some message keys appear irregularly in the stream, the latest sessions can remain unprocessed until a message with the same key is received. + - `"partition"` - messages advance time and close sessions for the whole partition to which this message key belongs. + If timestamps between keys are not ordered, it may increase the number of discarded late messages. + Default - `"key"`. + """ + self._closing_strategy = ClosingStrategy.new(closing_strategy) + return super().current() + + def process_window( + self, + value: Any, + key: Any, + timestamp_ms: int, + transaction: WindowedPartitionTransaction, + ) -> tuple[Iterable[WindowKeyResult], Iterable[WindowKeyResult]]: + state = transaction.as_state(prefix=key) + timeout_ms = self._timeout_ms + grace_ms = self._grace_ms + + collect = self.collect + aggregate = self.aggregate + + # Determine the latest timestamp for expiration logic + if self._closing_strategy == ClosingStrategy.PARTITION: + latest_expired_timestamp = transaction.get_latest_expired(prefix=b"") + latest_timestamp = max(timestamp_ms, latest_expired_timestamp) + else: + state_ts = state.get_latest_timestamp() or 0 + latest_timestamp = max(timestamp_ms, state_ts) + + # Calculate session expiry threshold + session_expiry_threshold = latest_timestamp - grace_ms + + # Check if the event is too late + if timestamp_ms < session_expiry_threshold: + late_by_ms = session_expiry_threshold - timestamp_ms + self._on_expired_session( + value=value, + key=key, + start=timestamp_ms, + end=timestamp_ms + timeout_ms, + timestamp_ms=timestamp_ms, + late_by_ms=late_by_ms, + ) + return [], [] + + # Look for an existing session that can be extended + can_extend_session = False + existing_aggregated = None + old_window_to_delete = None + + # Search for active sessions that can accommodate the new event + search_start = max(0, timestamp_ms - timeout_ms * 2) + windows = state.get_windows( + search_start, timestamp_ms + timeout_ms + 1, backwards=True + ) + + for (window_start, window_end), aggregated_value, _ in windows: + # Calculate the time gap between the new event and the session's last activity + session_last_activity = window_end - timeout_ms + time_gap = timestamp_ms - session_last_activity + + # Check if this session can be extended + if time_gap <= timeout_ms + grace_ms and timestamp_ms >= window_start: + session_start = window_start + session_end = timestamp_ms + timeout_ms + can_extend_session = True + existing_aggregated = aggregated_value + old_window_to_delete = (window_start, window_end) + break + + # If no extendable session found, start a new one + if not can_extend_session: + session_start = timestamp_ms + session_end = timestamp_ms + timeout_ms + + # Process the event for this session + updated_windows: list[WindowKeyResult] = [] + + # Delete the old window if extending an existing session + if can_extend_session and old_window_to_delete: + old_start, old_end = old_window_to_delete + transaction.delete_window(old_start, old_end, prefix=state._prefix) # type: ignore # noqa: SLF001 + + # Add to collection if needed + if collect: + state.add_to_collection( + value=self._collect_value(value), + id=timestamp_ms, + ) + + # Update the session window aggregation + aggregated = None + if aggregate: + current_value = ( + existing_aggregated if can_extend_session else self._initialize_value() + ) + + aggregated = self._aggregate_value(current_value, value, timestamp_ms) + updated_windows.append( + ( + key, + self._results(aggregated, [], session_start, session_end), + ) + ) + + state.update_window( + session_start, session_end, value=aggregated, timestamp_ms=timestamp_ms + ) + + # Expire old sessions + if self._closing_strategy == ClosingStrategy.PARTITION: + expired_windows = self.expire_sessions_by_partition( + transaction, session_expiry_threshold, collect + ) + else: + expired_windows = self.expire_sessions_by_key( + key, state, session_expiry_threshold, collect + ) + + return updated_windows, expired_windows + + def expire_sessions_by_partition( + self, + transaction: WindowedPartitionTransaction, + expiry_threshold: int, + collect: bool, + ) -> Iterable[WindowKeyResult]: + start = time.monotonic() + count = 0 + + # Import the parsing function to extract message keys from window keys + from quixstreams.state.rocksdb.windowed.serialization import parse_window_key + + expired_results = [] + + # Collect all keys and extract unique prefixes to avoid iteration conflicts + all_keys = list(transaction.keys()) + seen_prefixes = set() + + for key_bytes in all_keys: + try: + prefix, start_ms, end_ms = parse_window_key(key_bytes) + if prefix not in seen_prefixes: + seen_prefixes.add(prefix) + except (ValueError, IndexError): + # Skip invalid window key formats + continue + + # Expire sessions for each unique prefix + for prefix in seen_prefixes: + state = transaction.as_state(prefix=prefix) + prefix_expired = list( + self.expire_sessions_by_key(prefix, state, expiry_threshold, collect) + ) + expired_results.extend(prefix_expired) + count += len(prefix_expired) + + if count: + logger.debug( + "Expired %s session windows in %ss", + count, + round(time.monotonic() - start, 2), + ) + + return expired_results + + def expire_sessions_by_key( + self, + key: Any, + state: WindowedState, + expiry_threshold: int, + collect: bool, + ) -> Iterable[WindowKeyResult]: + start = time.monotonic() + count = 0 + + # Get all windows and check which ones have expired + all_windows = list( + state.get_windows(0, expiry_threshold + self._timeout_ms, backwards=False) + ) + + windows_to_delete = [] + for (window_start, window_end), aggregated, _ in all_windows: + # Session expires when the session end time has passed the expiry threshold + if window_end <= expiry_threshold: + collected = [] + if collect: + collected = state.get_from_collection(window_start, window_end) + + windows_to_delete.append((window_start, window_end)) + count += 1 + yield ( + key, + self._results(aggregated, collected, window_start, window_end), + ) + + # Clean up expired windows + for window_start, window_end in windows_to_delete: + state._transaction.delete_window( # type: ignore # noqa: SLF001 + window_start, + window_end, + prefix=state._prefix, # type: ignore # noqa: SLF001 + ) + if collect: + state.delete_from_collection(window_end, start=window_start) + + if count: + logger.debug( + "Expired %s session windows in %ss", + count, + round(time.monotonic() - start, 2), + ) + + def _on_expired_session( + self, + value: Any, + key: Any, + start: int, + end: int, + timestamp_ms: int, + late_by_ms: int, + ) -> None: + ctx = message_context() + to_log = True + + # Trigger the "on_late" callback if provided + if self._on_late: + to_log = self._on_late( + value, + key, + timestamp_ms, + late_by_ms, + start, + end, + self._name, + ctx.topic, + ctx.partition, + ctx.offset, + ) + if to_log: + logger.warning( + "Skipping session processing for the closed session " + f"timestamp_ms={timestamp_ms} " + f"session={(start, end)} " + f"late_by_ms={late_by_ms} " + f"store_name={self._name} " + f"partition={ctx.topic}[{ctx.partition}] " + f"offset={ctx.offset}" + ) + + +class SessionWindowSingleAggregation(SingleAggregationWindowMixin, SessionWindow): + pass + + +class SessionWindowMultiAggregation(MultiAggregationWindowMixin, SessionWindow): + pass diff --git a/quixstreams/dataframe/windows/time_based.py b/quixstreams/dataframe/windows/time_based.py index bed9e116d..c403cfdfa 100644 --- a/quixstreams/dataframe/windows/time_based.py +++ b/quixstreams/dataframe/windows/time_based.py @@ -271,353 +271,9 @@ def _on_expired_window( ) -class SessionWindow(Window): - """ - Session window groups events that occur within a specified timeout period. - - A session starts with the first event and extends each time a new event arrives - within the timeout period. The session closes after the timeout period with no - new events. - - Each session window can have different start and end times based on the actual - events, making sessions dynamic rather than fixed-time intervals. - """ - - def __init__( - self, - timeout_ms: int, - grace_ms: int, - name: str, - dataframe: "StreamingDataFrame", - on_late: Optional[WindowOnLateCallback] = None, - ): - super().__init__( - name=name, - dataframe=dataframe, - ) - - self._timeout_ms = timeout_ms - self._grace_ms = grace_ms - self._on_late = on_late - self._closing_strategy = ClosingStrategy.KEY - - def final( - self, closing_strategy: ClosingStrategyValues = "key" - ) -> "StreamingDataFrame": - """ - Apply the session window aggregation and return results only when the sessions - are closed. - - The format of returned sessions: - ```python - { - "start": , - "end": , - "value: , - } - ``` - - The individual session is closed when the event time - (the maximum observed timestamp across the partition) passes - the last event timestamp + timeout + grace period. - The closed sessions cannot receive updates anymore and are considered final. - - :param closing_strategy: the strategy to use when closing sessions. - Possible values: - - `"key"` - messages advance time and close sessions with the same key. - If some message keys appear irregularly in the stream, the latest sessions can remain unprocessed until a message with the same key is received. - - `"partition"` - messages advance time and close sessions for the whole partition to which this message key belongs. - If timestamps between keys are not ordered, it may increase the number of discarded late messages. - Default - `"key"`. - """ - self._closing_strategy = ClosingStrategy.new(closing_strategy) - return super().final() - - def current( - self, closing_strategy: ClosingStrategyValues = "key" - ) -> "StreamingDataFrame": - """ - Apply the session window transformation to the StreamingDataFrame to return results - for each updated session. - - The format of returned sessions: - ```python - { - "start": , - "end": , - "value: , - } - ``` - - This method processes streaming data and returns results as they come, - regardless of whether the session is closed or not. - - :param closing_strategy: the strategy to use when closing sessions. - Possible values: - - `"key"` - messages advance time and close sessions with the same key. - If some message keys appear irregularly in the stream, the latest sessions can remain unprocessed until a message with the same key is received. - - `"partition"` - messages advance time and close sessions for the whole partition to which this message key belongs. - If timestamps between keys are not ordered, it may increase the number of discarded late messages. - Default - `"key"`. - """ - self._closing_strategy = ClosingStrategy.new(closing_strategy) - return super().current() - - def process_window( - self, - value: Any, - key: Any, - timestamp_ms: int, - transaction: WindowedPartitionTransaction, - ) -> tuple[Iterable[WindowKeyResult], Iterable[WindowKeyResult]]: - state = transaction.as_state(prefix=key) - timeout_ms = self._timeout_ms - grace_ms = self._grace_ms - - collect = self.collect - aggregate = self.aggregate - - # Determine the latest timestamp for expiration logic - if self._closing_strategy == ClosingStrategy.PARTITION: - latest_expired_timestamp = transaction.get_latest_expired(prefix=b"") - latest_timestamp = max(timestamp_ms, latest_expired_timestamp) - else: - state_ts = state.get_latest_timestamp() or 0 - latest_timestamp = max(timestamp_ms, state_ts) - - # Calculate session expiry threshold - session_expiry_threshold = latest_timestamp - grace_ms - - # Check if the event is too late - if timestamp_ms < session_expiry_threshold: - late_by_ms = session_expiry_threshold - timestamp_ms - self._on_expired_session( - value=value, - key=key, - start=timestamp_ms, - end=timestamp_ms + timeout_ms, - timestamp_ms=timestamp_ms, - late_by_ms=late_by_ms, - ) - return [], [] - - # Look for an existing session that can be extended - can_extend_session = False - existing_aggregated = None - old_window_to_delete = None - - # Search for active sessions that can accommodate the new event - search_start = max(0, timestamp_ms - timeout_ms * 2) - windows = state.get_windows( - search_start, timestamp_ms + timeout_ms + 1, backwards=True - ) - - for (window_start, window_end), aggregated_value, _ in windows: - # Calculate the time gap between the new event and the session's last activity - session_last_activity = window_end - timeout_ms - time_gap = timestamp_ms - session_last_activity - - # Check if this session can be extended - if time_gap <= timeout_ms + grace_ms and timestamp_ms >= window_start: - session_start = window_start - session_end = timestamp_ms + timeout_ms - can_extend_session = True - existing_aggregated = aggregated_value - old_window_to_delete = (window_start, window_end) - break - - # If no extendable session found, start a new one - if not can_extend_session: - session_start = timestamp_ms - session_end = timestamp_ms + timeout_ms - - # Process the event for this session - updated_windows: list[WindowKeyResult] = [] - - # Delete the old window if extending an existing session - if can_extend_session and old_window_to_delete: - old_start, old_end = old_window_to_delete - transaction.delete_window(old_start, old_end, prefix=state._prefix) # type: ignore # noqa: SLF001 - - # Add to collection if needed - if collect: - state.add_to_collection( - value=self._collect_value(value), - id=timestamp_ms, - ) - - # Update the session window aggregation - aggregated = None - if aggregate: - current_value = ( - existing_aggregated if can_extend_session else self._initialize_value() - ) - - aggregated = self._aggregate_value(current_value, value, timestamp_ms) - updated_windows.append( - ( - key, - self._results(aggregated, [], session_start, session_end), - ) - ) - - state.update_window( - session_start, session_end, value=aggregated, timestamp_ms=timestamp_ms - ) - - # Expire old sessions - if self._closing_strategy == ClosingStrategy.PARTITION: - expired_windows = self.expire_sessions_by_partition( - transaction, session_expiry_threshold, collect - ) - else: - expired_windows = self.expire_sessions_by_key( - key, state, session_expiry_threshold, collect - ) - - return updated_windows, expired_windows - - def expire_sessions_by_partition( - self, - transaction: WindowedPartitionTransaction, - expiry_threshold: int, - collect: bool, - ) -> Iterable[WindowKeyResult]: - start = time.monotonic() - count = 0 - - # Import the parsing function to extract message keys from window keys - from quixstreams.state.rocksdb.windowed.serialization import parse_window_key - - expired_results = [] - - # Collect all keys and extract unique prefixes to avoid iteration conflicts - all_keys = list(transaction.keys()) - seen_prefixes = set() - - for key_bytes in all_keys: - try: - prefix, start_ms, end_ms = parse_window_key(key_bytes) - if prefix not in seen_prefixes: - seen_prefixes.add(prefix) - except (ValueError, IndexError): - # Skip invalid window key formats - continue - - # Expire sessions for each unique prefix - for prefix in seen_prefixes: - state = transaction.as_state(prefix=prefix) - prefix_expired = list( - self.expire_sessions_by_key(prefix, state, expiry_threshold, collect) - ) - expired_results.extend(prefix_expired) - count += len(prefix_expired) - - if count: - logger.debug( - "Expired %s session windows in %ss", - count, - round(time.monotonic() - start, 2), - ) - - return expired_results - - def expire_sessions_by_key( - self, - key: Any, - state: WindowedState, - expiry_threshold: int, - collect: bool, - ) -> Iterable[WindowKeyResult]: - start = time.monotonic() - count = 0 - - # Get all windows and check which ones have expired - all_windows = list( - state.get_windows(0, expiry_threshold + self._timeout_ms, backwards=False) - ) - - windows_to_delete = [] - for (window_start, window_end), aggregated, _ in all_windows: - # Session expires when the session end time has passed the expiry threshold - if window_end <= expiry_threshold: - collected = [] - if collect: - collected = state.get_from_collection(window_start, window_end) - - windows_to_delete.append((window_start, window_end)) - count += 1 - yield ( - key, - self._results(aggregated, collected, window_start, window_end), - ) - - # Clean up expired windows - for window_start, window_end in windows_to_delete: - state._transaction.delete_window( # type: ignore # noqa: SLF001 - window_start, - window_end, - prefix=state._prefix, # type: ignore # noqa: SLF001 - ) - if collect: - state.delete_from_collection(window_end, start=window_start) - - if count: - logger.debug( - "Expired %s session windows in %ss", - count, - round(time.monotonic() - start, 2), - ) - - def _on_expired_session( - self, - value: Any, - key: Any, - start: int, - end: int, - timestamp_ms: int, - late_by_ms: int, - ) -> None: - ctx = message_context() - to_log = True - - # Trigger the "on_late" callback if provided - if self._on_late: - to_log = self._on_late( - value, - key, - timestamp_ms, - late_by_ms, - start, - end, - self._name, - ctx.topic, - ctx.partition, - ctx.offset, - ) - if to_log: - logger.warning( - "Skipping session processing for the closed session " - f"timestamp_ms={timestamp_ms} " - f"session={(start, end)} " - f"late_by_ms={late_by_ms} " - f"store_name={self._name} " - f"partition={ctx.topic}[{ctx.partition}] " - f"offset={ctx.offset}" - ) - - class TimeWindowSingleAggregation(SingleAggregationWindowMixin, TimeWindow): pass class TimeWindowMultiAggregation(MultiAggregationWindowMixin, TimeWindow): pass - - -class SessionWindowSingleAggregation(SingleAggregationWindowMixin, SessionWindow): - pass - - -class SessionWindowMultiAggregation(MultiAggregationWindowMixin, SessionWindow): - pass diff --git a/tests/test_quixstreams/test_dataframe/fixtures.py b/tests/test_quixstreams/test_dataframe/fixtures.py index aabc1529a..92167d49b 100644 --- a/tests/test_quixstreams/test_dataframe/fixtures.py +++ b/tests/test_quixstreams/test_dataframe/fixtures.py @@ -63,5 +63,8 @@ def factory( @pytest.fixture def mock_message_context(): - with patch("quixstreams.dataframe.windows.time_based.message_context"): + with ( + patch("quixstreams.dataframe.windows.time_based.message_context"), + patch("quixstreams.dataframe.windows.session.message_context"), + ): yield From 68027278702f27156514ea67eed209a8b3d45e62 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Tue, 5 Aug 2025 13:22:10 +0200 Subject: [PATCH 12/19] Introduce FixedTimeWindow intermediary class --- quixstreams/dataframe/windows/definitions.py | 18 +-- quixstreams/dataframe/windows/session.py | 124 ++---------------- quixstreams/dataframe/windows/sliding.py | 4 +- quixstreams/dataframe/windows/time_based.py | 108 ++++++++------- .../test_dataframe/fixtures.py | 5 +- 5 files changed, 79 insertions(+), 180 deletions(-) diff --git a/quixstreams/dataframe/windows/definitions.py b/quixstreams/dataframe/windows/definitions.py index 987a4ed5c..4cebb5e05 100644 --- a/quixstreams/dataframe/windows/definitions.py +++ b/quixstreams/dataframe/windows/definitions.py @@ -33,9 +33,9 @@ SlidingWindowSingleAggregation, ) from .time_based import ( + FixedTimeWindowMultiAggregation, + FixedTimeWindowSingleAggregation, TimeWindow, - TimeWindowMultiAggregation, - TimeWindowSingleAggregation, ) if TYPE_CHECKING: @@ -312,10 +312,11 @@ def _create_window( ) -> TimeWindow: if func_name: window_type: Union[ - type[TimeWindowSingleAggregation], type[TimeWindowMultiAggregation] - ] = TimeWindowSingleAggregation + type[FixedTimeWindowSingleAggregation], + type[FixedTimeWindowMultiAggregation], + ] = FixedTimeWindowSingleAggregation else: - window_type = TimeWindowMultiAggregation + window_type = FixedTimeWindowMultiAggregation return window_type( duration_ms=self._duration_ms, @@ -361,10 +362,11 @@ def _create_window( ) -> TimeWindow: if func_name: window_type: Union[ - type[TimeWindowSingleAggregation], type[TimeWindowMultiAggregation] - ] = TimeWindowSingleAggregation + type[FixedTimeWindowSingleAggregation], + type[FixedTimeWindowMultiAggregation], + ] = FixedTimeWindowSingleAggregation else: - window_type = TimeWindowMultiAggregation + window_type = FixedTimeWindowMultiAggregation return window_type( duration_ms=self._duration_ms, diff --git a/quixstreams/dataframe/windows/session.py b/quixstreams/dataframe/windows/session.py index 8af54d839..40ee17ea9 100644 --- a/quixstreams/dataframe/windows/session.py +++ b/quixstreams/dataframe/windows/session.py @@ -2,17 +2,15 @@ import time from typing import TYPE_CHECKING, Any, Iterable, Optional -from quixstreams.context import message_context from quixstreams.state import WindowedPartitionTransaction, WindowedState from .base import ( MultiAggregationWindowMixin, SingleAggregationWindowMixin, - Window, WindowKeyResult, WindowOnLateCallback, ) -from .time_based import ClosingStrategy, ClosingStrategyValues +from .time_based import ClosingStrategy, TimeWindow if TYPE_CHECKING: from quixstreams.dataframe.dataframe import StreamingDataFrame @@ -20,7 +18,7 @@ logger = logging.getLogger(__name__) -class SessionWindow(Window): +class SessionWindow(TimeWindow): """ Session window groups events that occur within a specified timeout period. @@ -40,77 +38,10 @@ def __init__( dataframe: "StreamingDataFrame", on_late: Optional[WindowOnLateCallback] = None, ): - super().__init__( - name=name, - dataframe=dataframe, - ) + super().__init__(name=name, dataframe=dataframe, on_late=on_late) self._timeout_ms = timeout_ms self._grace_ms = grace_ms - self._on_late = on_late - self._closing_strategy = ClosingStrategy.KEY - - def final( - self, closing_strategy: ClosingStrategyValues = "key" - ) -> "StreamingDataFrame": - """ - Apply the session window aggregation and return results only when the sessions - are closed. - - The format of returned sessions: - ```python - { - "start": , - "end": , - "value: , - } - ``` - - The individual session is closed when the event time - (the maximum observed timestamp across the partition) passes - the last event timestamp + timeout + grace period. - The closed sessions cannot receive updates anymore and are considered final. - - :param closing_strategy: the strategy to use when closing sessions. - Possible values: - - `"key"` - messages advance time and close sessions with the same key. - If some message keys appear irregularly in the stream, the latest sessions can remain unprocessed until a message with the same key is received. - - `"partition"` - messages advance time and close sessions for the whole partition to which this message key belongs. - If timestamps between keys are not ordered, it may increase the number of discarded late messages. - Default - `"key"`. - """ - self._closing_strategy = ClosingStrategy.new(closing_strategy) - return super().final() - - def current( - self, closing_strategy: ClosingStrategyValues = "key" - ) -> "StreamingDataFrame": - """ - Apply the session window transformation to the StreamingDataFrame to return results - for each updated session. - - The format of returned sessions: - ```python - { - "start": , - "end": , - "value: , - } - ``` - - This method processes streaming data and returns results as they come, - regardless of whether the session is closed or not. - - :param closing_strategy: the strategy to use when closing sessions. - Possible values: - - `"key"` - messages advance time and close sessions with the same key. - If some message keys appear irregularly in the stream, the latest sessions can remain unprocessed until a message with the same key is received. - - `"partition"` - messages advance time and close sessions for the whole partition to which this message key belongs. - If timestamps between keys are not ordered, it may increase the number of discarded late messages. - Default - `"key"`. - """ - self._closing_strategy = ClosingStrategy.new(closing_strategy) - return super().current() def process_window( self, @@ -140,7 +71,7 @@ def process_window( # Check if the event is too late if timestamp_ms < session_expiry_threshold: late_by_ms = session_expiry_threshold - timestamp_ms - self._on_expired_session( + self._on_expired_window( value=value, key=key, start=timestamp_ms, @@ -216,17 +147,17 @@ def process_window( # Expire old sessions if self._closing_strategy == ClosingStrategy.PARTITION: - expired_windows = self.expire_sessions_by_partition( + expired_windows = self.expire_by_partition( transaction, session_expiry_threshold, collect ) else: - expired_windows = self.expire_sessions_by_key( + expired_windows = self.expire_by_key( key, state, session_expiry_threshold, collect ) return updated_windows, expired_windows - def expire_sessions_by_partition( + def expire_by_partition( self, transaction: WindowedPartitionTransaction, expiry_threshold: int, @@ -257,7 +188,7 @@ def expire_sessions_by_partition( for prefix in seen_prefixes: state = transaction.as_state(prefix=prefix) prefix_expired = list( - self.expire_sessions_by_key(prefix, state, expiry_threshold, collect) + self.expire_by_key(prefix, state, expiry_threshold, collect) ) expired_results.extend(prefix_expired) count += len(prefix_expired) @@ -271,7 +202,7 @@ def expire_sessions_by_partition( return expired_results - def expire_sessions_by_key( + def expire_by_key( self, key: Any, state: WindowedState, @@ -318,43 +249,6 @@ def expire_sessions_by_key( round(time.monotonic() - start, 2), ) - def _on_expired_session( - self, - value: Any, - key: Any, - start: int, - end: int, - timestamp_ms: int, - late_by_ms: int, - ) -> None: - ctx = message_context() - to_log = True - - # Trigger the "on_late" callback if provided - if self._on_late: - to_log = self._on_late( - value, - key, - timestamp_ms, - late_by_ms, - start, - end, - self._name, - ctx.topic, - ctx.partition, - ctx.offset, - ) - if to_log: - logger.warning( - "Skipping session processing for the closed session " - f"timestamp_ms={timestamp_ms} " - f"session={(start, end)} " - f"late_by_ms={late_by_ms} " - f"store_name={self._name} " - f"partition={ctx.topic}[{ctx.partition}] " - f"offset={ctx.offset}" - ) - class SessionWindowSingleAggregation(SingleAggregationWindowMixin, SessionWindow): pass diff --git a/quixstreams/dataframe/windows/sliding.py b/quixstreams/dataframe/windows/sliding.py index d3dfdbb39..ba93d0d19 100644 --- a/quixstreams/dataframe/windows/sliding.py +++ b/quixstreams/dataframe/windows/sliding.py @@ -7,13 +7,13 @@ SingleAggregationWindowMixin, WindowKeyResult, ) -from .time_based import ClosingStrategyValues, TimeWindow +from .time_based import ClosingStrategyValues, FixedTimeWindow if TYPE_CHECKING: from quixstreams.dataframe.dataframe import StreamingDataFrame -class SlidingWindow(TimeWindow): +class SlidingWindow(FixedTimeWindow): def final( self, closing_strategy: ClosingStrategyValues = "key" ) -> "StreamingDataFrame": diff --git a/quixstreams/dataframe/windows/time_based.py b/quixstreams/dataframe/windows/time_based.py index c403cfdfa..40ed710a2 100644 --- a/quixstreams/dataframe/windows/time_based.py +++ b/quixstreams/dataframe/windows/time_based.py @@ -40,23 +40,12 @@ def new(cls, value: str) -> "ClosingStrategy": class TimeWindow(Window): def __init__( self, - duration_ms: int, - grace_ms: int, name: str, dataframe: "StreamingDataFrame", - step_ms: Optional[int] = None, on_late: Optional[WindowOnLateCallback] = None, ): - super().__init__( - name=name, - dataframe=dataframe, - ) - - self._duration_ms = duration_ms - self._grace_ms = grace_ms - self._step_ms = step_ms + super().__init__(name=name, dataframe=dataframe) self._on_late = on_late - self._closing_strategy = ClosingStrategy.KEY def final( @@ -122,6 +111,60 @@ def current( self._closing_strategy = ClosingStrategy.new(closing_strategy) return super().current() + def _on_expired_window( + self, + value: Any, + key: Any, + start: int, + end: int, + timestamp_ms: int, + late_by_ms: int, + ) -> None: + ctx = message_context() + to_log = True + # Trigger the "on_late" callback if provided. + # Log the lateness warning if the callback returns True + if self._on_late: + to_log = self._on_late( + value, + key, + timestamp_ms, + late_by_ms, + start, + end, + self._name, + ctx.topic, + ctx.partition, + ctx.offset, + ) + if to_log: + logger.warning( + "Skipping window processing for the closed window " + f"timestamp_ms={timestamp_ms} " + f"window={(start, end)} " + f"late_by_ms={late_by_ms} " + f"store_name={self._name} " + f"partition={ctx.topic}[{ctx.partition}] " + f"offset={ctx.offset}" + ) + + +class FixedTimeWindow(TimeWindow): + def __init__( + self, + duration_ms: int, + grace_ms: int, + name: str, + dataframe: "StreamingDataFrame", + step_ms: Optional[int] = None, + on_late: Optional[WindowOnLateCallback] = None, + ): + super().__init__(name=name, dataframe=dataframe, on_late=on_late) + + self._duration_ms = duration_ms + self._grace_ms = grace_ms + self._step_ms = step_ms + def process_window( self, value: Any, @@ -233,47 +276,10 @@ def expire_by_key( ): yield (key, self._results(aggregated, collected, window_start, window_end)) - def _on_expired_window( - self, - value: Any, - key: Any, - start: int, - end: int, - timestamp_ms: int, - late_by_ms: int, - ) -> None: - ctx = message_context() - to_log = True - # Trigger the "on_late" callback if provided. - # Log the lateness warning if the callback returns True - if self._on_late: - to_log = self._on_late( - value, - key, - timestamp_ms, - late_by_ms, - start, - end, - self._name, - ctx.topic, - ctx.partition, - ctx.offset, - ) - if to_log: - logger.warning( - "Skipping window processing for the closed window " - f"timestamp_ms={timestamp_ms} " - f"window={(start, end)} " - f"late_by_ms={late_by_ms} " - f"store_name={self._name} " - f"partition={ctx.topic}[{ctx.partition}] " - f"offset={ctx.offset}" - ) - -class TimeWindowSingleAggregation(SingleAggregationWindowMixin, TimeWindow): +class FixedTimeWindowSingleAggregation(SingleAggregationWindowMixin, FixedTimeWindow): pass -class TimeWindowMultiAggregation(MultiAggregationWindowMixin, TimeWindow): +class FixedTimeWindowMultiAggregation(MultiAggregationWindowMixin, FixedTimeWindow): pass diff --git a/tests/test_quixstreams/test_dataframe/fixtures.py b/tests/test_quixstreams/test_dataframe/fixtures.py index 92167d49b..aabc1529a 100644 --- a/tests/test_quixstreams/test_dataframe/fixtures.py +++ b/tests/test_quixstreams/test_dataframe/fixtures.py @@ -63,8 +63,5 @@ def factory( @pytest.fixture def mock_message_context(): - with ( - patch("quixstreams.dataframe.windows.time_based.message_context"), - patch("quixstreams.dataframe.windows.session.message_context"), - ): + with patch("quixstreams.dataframe.windows.time_based.message_context"): yield From 6bee68a559c823a56dc86292f49f696271008c93 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Wed, 6 Aug 2025 10:37:34 +0200 Subject: [PATCH 13/19] End time equals to last event time --- quixstreams/dataframe/windows/session.py | 19 +++--- .../test_windows/test_session.py | 62 +++++++++---------- 2 files changed, 42 insertions(+), 39 deletions(-) diff --git a/quixstreams/dataframe/windows/session.py b/quixstreams/dataframe/windows/session.py index 40ee17ea9..af71226c0 100644 --- a/quixstreams/dataframe/windows/session.py +++ b/quixstreams/dataframe/windows/session.py @@ -75,7 +75,7 @@ def process_window( value=value, key=key, start=timestamp_ms, - end=timestamp_ms + timeout_ms, + end=timestamp_ms, # End time is the timestamp of the last event timestamp_ms=timestamp_ms, late_by_ms=late_by_ms, ) @@ -94,13 +94,14 @@ def process_window( for (window_start, window_end), aggregated_value, _ in windows: # Calculate the time gap between the new event and the session's last activity - session_last_activity = window_end - timeout_ms + # window_end is stored as last_event_timestamp + 1, so subtract 1 to get actual last event time + session_last_activity = window_end - 1 time_gap = timestamp_ms - session_last_activity # Check if this session can be extended if time_gap <= timeout_ms + grace_ms and timestamp_ms >= window_start: session_start = window_start - session_end = timestamp_ms + timeout_ms + session_end = timestamp_ms + 1 can_extend_session = True existing_aggregated = aggregated_value old_window_to_delete = (window_start, window_end) @@ -109,7 +110,7 @@ def process_window( # If no extendable session found, start a new one if not can_extend_session: session_start = timestamp_ms - session_end = timestamp_ms + timeout_ms + session_end = timestamp_ms + 1 # Process the event for this session updated_windows: list[WindowKeyResult] = [] @@ -137,7 +138,7 @@ def process_window( updated_windows.append( ( key, - self._results(aggregated, [], session_start, session_end), + self._results(aggregated, [], session_start, session_end - 1), ) ) @@ -219,8 +220,10 @@ def expire_by_key( windows_to_delete = [] for (window_start, window_end), aggregated, _ in all_windows: - # Session expires when the session end time has passed the expiry threshold - if window_end <= expiry_threshold: + # Session expires when the session end time + timeout has passed the expiry threshold + # window_end is stored as last_event_timestamp + 1, so we subtract 1 and add timeout_ms + last_event_timestamp = window_end - 1 + if last_event_timestamp + self._timeout_ms <= expiry_threshold: collected = [] if collect: collected = state.get_from_collection(window_start, window_end) @@ -229,7 +232,7 @@ def expire_by_key( count += 1 yield ( key, - self._results(aggregated, collected, window_start, window_end), + self._results(aggregated, collected, window_start, window_end - 1), ) # Clean up expired windows diff --git a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py index ea3b91a05..df1a83f44 100644 --- a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py +++ b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py @@ -91,7 +91,7 @@ def test_multiaggregation( key, { "start": 1000, - "end": 11000, # 1000 + 10000 timeout + "end": 1000, # timestamp of last event "count": 1, "sum": 1, "mean": 1.0, @@ -112,7 +112,7 @@ def test_multiaggregation( key, { "start": 1000, - "end": 15000, # 5000 + 10000 timeout + "end": 5000, # timestamp of last event "count": 2, "sum": 5, "mean": 2.5, @@ -132,7 +132,7 @@ def test_multiaggregation( key, { "start": 1000, - "end": 15000, + "end": 5000, # timestamp of last event "count": 2, "sum": 5, "mean": 2.5, @@ -147,7 +147,7 @@ def test_multiaggregation( key, { "start": 26000, - "end": 36000, # 26000 + 10000 timeout + "end": 26000, # timestamp of last event "count": 1, "sum": 2, "mean": 2.0, @@ -182,7 +182,7 @@ def test_sessionwindow_count( assert len(updated) == 1 assert updated[0][1]["value"] == 2 assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 15000 # 5000 + 10000 + assert updated[0][1]["end"] == 5000 # timestamp of last event assert not expired @pytest.mark.parametrize("expiration", ("key", "partition")) @@ -207,7 +207,7 @@ def test_sessionwindow_sum( assert len(updated) == 1 assert updated[0][1]["value"] == 5 assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 15000 + assert updated[0][1]["end"] == 5000 # timestamp of last event assert not expired @pytest.mark.parametrize("expiration", ("key", "partition")) @@ -232,7 +232,7 @@ def test_sessionwindow_mean( assert len(updated) == 1 assert updated[0][1]["value"] == 3.0 assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 15000 + assert updated[0][1]["end"] == 5000 # timestamp of last event assert not expired @pytest.mark.parametrize("expiration", ("key", "partition")) @@ -260,7 +260,7 @@ def test_sessionwindow_reduce( assert len(updated) == 1 assert updated[0][1]["value"] == [2, 3] assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 15000 + assert updated[0][1]["end"] == 5000 # timestamp of last event assert not expired @pytest.mark.parametrize("expiration", ("key", "partition")) @@ -285,7 +285,7 @@ def test_sessionwindow_max( assert len(updated) == 1 assert updated[0][1]["value"] == 5 assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 15000 + assert updated[0][1]["end"] == 5000 # timestamp of last event assert not expired @pytest.mark.parametrize("expiration", ("key", "partition")) @@ -310,7 +310,7 @@ def test_sessionwindow_min( assert len(updated) == 1 assert updated[0][1]["value"] == 2 assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 15000 + assert updated[0][1]["end"] == 5000 # timestamp of last event assert not expired @pytest.mark.parametrize("expiration", ("key", "partition")) @@ -336,7 +336,7 @@ def test_sessionwindow_collect( window, value=4, key=key, transaction=tx, timestamp_ms=25000 ) assert not updated - assert expired == [(key, {"start": 1000, "end": 18000, "value": [1, 2, 3]})] + assert expired == [(key, {"start": 1000, "end": 8000, "value": [1, 2, 3]})] @pytest.mark.parametrize( "timeout, grace, name", @@ -391,7 +391,7 @@ def test_session_window_process_timeout_behavior( assert len(updated) == 1 assert updated[0][1]["value"] == 1 assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 6000 # 1000 + 5000 + assert updated[0][1]["end"] == 1000 # timestamp of last event assert not expired # Add to session 1 (within timeout) @@ -401,7 +401,7 @@ def test_session_window_process_timeout_behavior( assert len(updated) == 1 assert updated[0][1]["value"] == 3 assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 9000 # 4000 + 5000 + assert updated[0][1]["end"] == 4000 # timestamp of last event assert not expired # Start session 2 (outside timeout) - should expire session 1 @@ -411,12 +411,12 @@ def test_session_window_process_timeout_behavior( assert len(updated) == 1 assert updated[0][1]["value"] == 5 assert updated[0][1]["start"] == 15000 - assert updated[0][1]["end"] == 20000 # 15000 + 5000 + assert updated[0][1]["end"] == 15000 # timestamp of last event assert len(expired) == 1 assert expired[0][1]["value"] == 3 assert expired[0][1]["start"] == 1000 - assert expired[0][1]["end"] == 9000 + assert expired[0][1]["end"] == 4000 # timestamp of last event def test_session_window_grace_period( self, session_window_definition_factory, state_manager @@ -670,7 +670,7 @@ def test_session_window_merge_sessions( ) assert len(updated) == 1 assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 11000 # 1000 + 10000 + assert updated[0][1]["end"] == 1000 # timestamp of last event assert updated[0][1]["value"] == 1 assert not expired @@ -683,12 +683,12 @@ def test_session_window_merge_sessions( # First session should now be expired assert len(expired) == 1 assert expired[0][1]["start"] == 1000 - assert expired[0][1]["end"] == 11000 + assert expired[0][1]["end"] == 1000 # timestamp of last event assert expired[0][1]["value"] == 1 assert len(updated) == 1 assert updated[0][1]["start"] == 20000 - assert updated[0][1]["end"] == 30000 # 20000 + 10000 + assert updated[0][1]["end"] == 20000 # timestamp of last event assert updated[0][1]["value"] == 10 # Add another event to the second session @@ -697,7 +697,7 @@ def test_session_window_merge_sessions( ) assert len(updated) == 1 assert updated[0][1]["start"] == 20000 - assert updated[0][1]["end"] == 35000 # 25000 + 10000 + assert updated[0][1]["end"] == 25000 # timestamp of last event assert updated[0][1]["value"] == 15 # 10 + 5 assert not expired @@ -710,13 +710,13 @@ def test_session_window_merge_sessions( # Second session should be expired assert len(expired) == 1 assert expired[0][1]["start"] == 20000 - assert expired[0][1]["end"] == 35000 + assert expired[0][1]["end"] == 25000 # timestamp of last event assert expired[0][1]["value"] == 15 # Third session starts assert len(updated) == 1 assert updated[0][1]["start"] == 50000 - assert updated[0][1]["end"] == 60000 # 50000 + 10000 + assert updated[0][1]["end"] == 50000 # timestamp of last event assert updated[0][1]["value"] == 100 def test_session_window_bridging_event_scenario( @@ -755,7 +755,7 @@ def test_session_window_bridging_event_scenario( ) assert len(updated) == 1 assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 11000 # 1000 + 10000 + assert updated[0][1]["end"] == 1000 # timestamp of last event assert updated[0][1]["value"] == 5 assert not expired @@ -770,7 +770,7 @@ def test_session_window_bridging_event_scenario( # Event at 12000 is before 13000, so it should extend Session A assert len(updated) == 1 assert updated[0][1]["start"] == 1000 # Session A extended - assert updated[0][1]["end"] == 22000 # 12000 + 10000 + assert updated[0][1]["end"] == 12000 # timestamp of last event assert updated[0][1]["value"] == 15 # 5 + 10 assert not expired @@ -781,7 +781,7 @@ def test_session_window_bridging_event_scenario( # This should extend the already extended Session A further assert len(updated) == 1 assert updated[0][1]["start"] == 1000 # Still Session A - assert updated[0][1]["end"] == 25000 # 15000 + 10000 + assert updated[0][1]["end"] == 15000 # timestamp of last event assert updated[0][1]["value"] == 35 # 5 + 10 + 20 assert not expired @@ -791,7 +791,7 @@ def test_session_window_bridging_event_scenario( ) assert len(expired) == 1 assert expired[0][1]["start"] == 1000 - assert expired[0][1]["end"] == 25000 + assert expired[0][1]["end"] == 15000 # timestamp of last event assert expired[0][1]["value"] == 35 # All events combined assert len(updated) == 1 @@ -828,7 +828,7 @@ def test_session_window_string_key_extension( ) assert len(updated) == 1 assert updated[0][1]["start"] == 1000 - assert updated[0][1]["end"] == 11000 # 1000 + 10000 + assert updated[0][1]["end"] == 1000 # timestamp of last event assert updated[0][1]["value"] == 100 assert not expired @@ -839,7 +839,7 @@ def test_session_window_string_key_extension( ) assert len(updated) == 1 assert updated[0][1]["start"] == 1000 # Session extended, same start - assert updated[0][1]["end"] == 15000 # 5000 + 10000 (new end time) + assert updated[0][1]["end"] == 5000 # timestamp of last event assert updated[0][1]["value"] == 300 # 100 + 200 assert not expired @@ -849,7 +849,7 @@ def test_session_window_string_key_extension( ) assert len(updated) == 1 assert updated[0][1]["start"] == 1000 # Session extended again - assert updated[0][1]["end"] == 18000 # 8000 + 10000 + assert updated[0][1]["end"] == 8000 # timestamp of last event assert updated[0][1]["value"] == 350 # 100 + 200 + 50 assert not expired @@ -861,7 +861,7 @@ def test_session_window_string_key_extension( assert len(updated) == 1 assert updated[0][0] == key2 # Different key assert updated[0][1]["start"] == 9000 - assert updated[0][1]["end"] == 19000 # 9000 + 10000 + assert updated[0][1]["end"] == 9000 # timestamp of last event assert updated[0][1]["value"] == 75 assert not expired @@ -874,12 +874,12 @@ def test_session_window_string_key_extension( assert len(expired) == 1 assert expired[0][0] == key assert expired[0][1]["start"] == 1000 - assert expired[0][1]["end"] == 18000 + assert expired[0][1]["end"] == 8000 # timestamp of last event assert expired[0][1]["value"] == 350 # Should have started a new session for the first key assert len(updated) == 1 assert updated[0][0] == key assert updated[0][1]["start"] == 30000 - assert updated[0][1]["end"] == 40000 + assert updated[0][1]["end"] == 30000 # timestamp of last event assert updated[0][1]["value"] == 25 From e3717155fbeb689e5e0b90ef668c5458a94af0f8 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Wed, 6 Aug 2025 13:53:04 +0200 Subject: [PATCH 14/19] Lift the equality check from validation to simplify session windows --- quixstreams/dataframe/windows/session.py | 20 +++++++++---------- .../state/rocksdb/windowed/transaction.py | 4 ++-- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/quixstreams/dataframe/windows/session.py b/quixstreams/dataframe/windows/session.py index af71226c0..a1a01c83c 100644 --- a/quixstreams/dataframe/windows/session.py +++ b/quixstreams/dataframe/windows/session.py @@ -94,14 +94,14 @@ def process_window( for (window_start, window_end), aggregated_value, _ in windows: # Calculate the time gap between the new event and the session's last activity - # window_end is stored as last_event_timestamp + 1, so subtract 1 to get actual last event time - session_last_activity = window_end - 1 + # window_end now directly represents the timestamp of the last event + session_last_activity = window_end time_gap = timestamp_ms - session_last_activity # Check if this session can be extended if time_gap <= timeout_ms + grace_ms and timestamp_ms >= window_start: session_start = window_start - session_end = timestamp_ms + 1 + session_end = timestamp_ms can_extend_session = True existing_aggregated = aggregated_value old_window_to_delete = (window_start, window_end) @@ -110,7 +110,7 @@ def process_window( # If no extendable session found, start a new one if not can_extend_session: session_start = timestamp_ms - session_end = timestamp_ms + 1 + session_end = timestamp_ms # End time is the timestamp of the last event # Process the event for this session updated_windows: list[WindowKeyResult] = [] @@ -138,7 +138,7 @@ def process_window( updated_windows.append( ( key, - self._results(aggregated, [], session_start, session_end - 1), + self._results(aggregated, [], session_start, session_end), ) ) @@ -221,18 +221,18 @@ def expire_by_key( windows_to_delete = [] for (window_start, window_end), aggregated, _ in all_windows: # Session expires when the session end time + timeout has passed the expiry threshold - # window_end is stored as last_event_timestamp + 1, so we subtract 1 and add timeout_ms - last_event_timestamp = window_end - 1 - if last_event_timestamp + self._timeout_ms <= expiry_threshold: + # window_end directly represents the timestamp of the last event + if window_end + self._timeout_ms <= expiry_threshold: collected = [] if collect: - collected = state.get_from_collection(window_start, window_end) + # window_end is now the timestamp of the last event, so we need +1 to include it + collected = state.get_from_collection(window_start, window_end + 1) windows_to_delete.append((window_start, window_end)) count += 1 yield ( key, - self._results(aggregated, collected, window_start, window_end - 1), + self._results(aggregated, collected, window_start, window_end), ) # Clean up expired windows diff --git a/quixstreams/state/rocksdb/windowed/transaction.py b/quixstreams/state/rocksdb/windowed/transaction.py index 3779b3e29..2ecf87186 100644 --- a/quixstreams/state/rocksdb/windowed/transaction.py +++ b/quixstreams/state/rocksdb/windowed/transaction.py @@ -485,9 +485,9 @@ def _set_timestamp(self, cache: Cache, prefix: bytes, timestamp_ms: int): ) def _validate_duration(self, start_ms: int, end_ms: int): - if end_ms <= start_ms: + if end_ms < start_ms: raise ValueError( - f"Invalid window duration: window end {end_ms} is smaller or equal " + f"Invalid window duration: window end {end_ms} is smaller " f"than window start {start_ms}" ) From f2c71fc2c2b8f01d10b641810d77dac2890173b8 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Wed, 6 Aug 2025 14:26:52 +0200 Subject: [PATCH 15/19] Fix end time resolution for out-of-order events --- quixstreams/dataframe/windows/session.py | 3 +- .../test_windows/test_session.py | 51 +++++++++++++++++++ 2 files changed, 53 insertions(+), 1 deletion(-) diff --git a/quixstreams/dataframe/windows/session.py b/quixstreams/dataframe/windows/session.py index a1a01c83c..4609815b9 100644 --- a/quixstreams/dataframe/windows/session.py +++ b/quixstreams/dataframe/windows/session.py @@ -101,7 +101,8 @@ def process_window( # Check if this session can be extended if time_gap <= timeout_ms + grace_ms and timestamp_ms >= window_start: session_start = window_start - session_end = timestamp_ms + # Only update end time if the new event is newer than the current end time + session_end = max(window_end, timestamp_ms) can_extend_session = True existing_aggregated = aggregated_value old_window_to_delete = (window_start, window_end) diff --git a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py index df1a83f44..0d0075bfa 100644 --- a/tests/test_quixstreams/test_dataframe/test_windows/test_session.py +++ b/tests/test_quixstreams/test_dataframe/test_windows/test_session.py @@ -883,3 +883,54 @@ def test_session_window_string_key_extension( assert updated[0][1]["start"] == 30000 assert updated[0][1]["end"] == 30000 # timestamp of last event assert updated[0][1]["value"] == 25 + + def test_out_of_order_events_end_time( + self, session_window_definition_factory, state_manager + ): + """Test that out-of-order events correctly maintain the latest timestamp as end time""" + window_def = session_window_definition_factory( + inactivity_gap_ms=10000, grace_ms=5000 + ) + window = window_def.sum() + window.final(closing_strategy="key") + + store = state_manager.get_store(stream_id="test", store_name=window.name) + store.assign_partition(0) + + with store.start_partition_transaction(0) as tx: + key = b"key" + + # 1. Start session with event at timestamp 1000 + updated, expired = process( + window, value=1, key=key, transaction=tx, timestamp_ms=1000 + ) + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 1000 # End should be 1000 + assert updated[0][1]["value"] == 1 + + # 2. Add event at timestamp 8000 (in order) + updated, expired = process( + window, value=2, key=key, transaction=tx, timestamp_ms=8000 + ) + assert updated[0][1]["start"] == 1000 + assert updated[0][1]["end"] == 8000 # End should be 8000 (latest event) + assert updated[0][1]["value"] == 3 + + # 3. Add OUT-OF-ORDER event at timestamp 3000 (before 8000) + # This should be accepted (within grace period) but should NOT change the end time + updated, expired = process( + window, value=10, key=key, transaction=tx, timestamp_ms=3000 + ) + assert updated[0][1]["start"] == 1000 + # KEY TEST: End time should remain 8000, not become 3000! + assert updated[0][1]["end"] == 8000 + assert updated[0][1]["value"] == 13 + + # 4. Add event NEWER than current end (timestamp 9000) + updated, expired = process( + window, value=4, key=key, transaction=tx, timestamp_ms=9000 + ) + assert updated[0][1]["start"] == 1000 + # NOW the end time should update to 9000 + assert updated[0][1]["end"] == 9000 + assert updated[0][1]["value"] == 17 From 08dd3d5e2490443b09e2d497c9dcbf84f2a01285 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Wed, 6 Aug 2025 14:53:21 +0200 Subject: [PATCH 16/19] Correct docs --- docs/windowing.md | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/docs/windowing.md b/docs/windowing.md index acd792250..a29fb3094 100644 --- a/docs/windowing.md +++ b/docs/windowing.md @@ -578,7 +578,7 @@ sdf = ( # Expected output (when session expires): # { # "start": 1000, -# "end": 2000000 + 1800000, # last event + timeout +# "end": 2000000, # timestamp of last event # "action_count": 4, # "actions": ["page_view", "click", "page_view", "purchase"] # } @@ -588,6 +588,13 @@ sdf = ( For real-time monitoring, you can use `.current()` mode to get updates as the session progresses: +Input: +```json +{"amount": 25, "timestamp": 1000} +{"amount": 50, "timestamp": 5000} +{"amount": 50, "timestamp": 8000} +``` + ```python from datetime import timedelta from quixstreams import Application @@ -611,9 +618,9 @@ sdf = ( ) # Output for each incoming event: -# Event 1: {"start": 1000, "end": 11000, "total_amount": 25, "purchase_count": 1} -# Event 2: {"start": 1000, "end": 15000, "total_amount": 75, "purchase_count": 2} # session extended -# Event 3: {"start": 1000, "end": 18000, "total_amount": 125, "purchase_count": 3} # session extended again +# Event 1: {"start": 1000, "end": 1000, "total_amount": 25, "purchase_count": 1} +# Event 2: {"start": 1000, "end": 5000, "total_amount": 75, "purchase_count": 2} +# Event 3: {"start": 1000, "end": 8000, "total_amount": 125, "purchase_count": 3} ``` ### Handling Late Events in Sessions @@ -710,7 +717,9 @@ sdf = ( **Session Extension**: An existing session is extended when an event arrives within `timeout + grace_period` of the session's last activity. -**Session Closure**: A session closes when the current time exceeds `session_end_time + grace_period`, where `session_end_time = last_event_time + timeout`. +**Session Closure**: A session closes when the current time exceeds `last_event_time + timeout + grace_period`. The session end time in the output represents the timestamp of the last event in the session. + +**Out-of-Order Events**: When out-of-order events arrive within the grace period, they extend the session but do not change the end time if they are older than the current latest event. The end time always represents the timestamp of the chronologically latest event in the session. **Key Grouping**: Like all windows in Quix Streams, sessions are grouped by message key. Each key maintains its own independent sessions. From 610d38ccc7ac5d6363351ac2da826abbcc4a927a Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Wed, 6 Aug 2025 15:22:15 +0200 Subject: [PATCH 17/19] Introduce delete_window method to eliminate private member usage --- quixstreams/dataframe/windows/session.py | 8 ++------ quixstreams/state/rocksdb/windowed/state.py | 15 ++++++++++++++ quixstreams/state/types.py | 11 ++++++++++ .../test_rocksdb/test_windowed/test_state.py | 20 +++++++++++++++++++ 4 files changed, 48 insertions(+), 6 deletions(-) diff --git a/quixstreams/dataframe/windows/session.py b/quixstreams/dataframe/windows/session.py index 4609815b9..d35251869 100644 --- a/quixstreams/dataframe/windows/session.py +++ b/quixstreams/dataframe/windows/session.py @@ -119,7 +119,7 @@ def process_window( # Delete the old window if extending an existing session if can_extend_session and old_window_to_delete: old_start, old_end = old_window_to_delete - transaction.delete_window(old_start, old_end, prefix=state._prefix) # type: ignore # noqa: SLF001 + state.delete_window(old_start, old_end) # Add to collection if needed if collect: @@ -238,11 +238,7 @@ def expire_by_key( # Clean up expired windows for window_start, window_end in windows_to_delete: - state._transaction.delete_window( # type: ignore # noqa: SLF001 - window_start, - window_end, - prefix=state._prefix, # type: ignore # noqa: SLF001 - ) + state.delete_window(window_start, window_end) if collect: state.delete_from_collection(window_end, start=window_start) diff --git a/quixstreams/state/rocksdb/windowed/state.py b/quixstreams/state/rocksdb/windowed/state.py index 3e3021b20..d4c775da7 100644 --- a/quixstreams/state/rocksdb/windowed/state.py +++ b/quixstreams/state/rocksdb/windowed/state.py @@ -182,3 +182,18 @@ def delete_windows(self, max_start_time: int, delete_values: bool) -> None: delete_values=delete_values, prefix=self._prefix, ) + + def delete_window(self, start_ms: int, end_ms: int) -> None: + """ + Delete a specific window from the state store. + + This method removes a single window entry with the specified start and end timestamps. + + :param start_ms: The start timestamp of the window to delete + :param end_ms: The end timestamp of the window to delete + """ + return self._transaction.delete_window( + start_ms=start_ms, + end_ms=end_ms, + prefix=self._prefix, + ) diff --git a/quixstreams/state/types.py b/quixstreams/state/types.py index beb8b0e0b..c5f8c975f 100644 --- a/quixstreams/state/types.py +++ b/quixstreams/state/types.py @@ -187,6 +187,17 @@ def delete_windows(self, max_start_time: int, delete_values: bool) -> None: """ ... + def delete_window(self, start_ms: int, end_ms: int) -> None: + """ + Delete a specific window from the state store. + + This method removes a single window entry with the specified start and end timestamps. + + :param start_ms: The start timestamp of the window to delete + :param end_ms: The end timestamp of the window to delete + """ + ... + def get_windows( self, start_from_ms: int, start_to_ms: int, backwards: bool = False ) -> list[WindowDetail[V]]: diff --git a/tests/test_quixstreams/test_state/test_rocksdb/test_windowed/test_state.py b/tests/test_quixstreams/test_state/test_rocksdb/test_windowed/test_state.py index 5217b5961..7161b6d73 100644 --- a/tests/test_quixstreams/test_state/test_rocksdb/test_windowed/test_state.py +++ b/tests/test_quixstreams/test_state/test_rocksdb/test_windowed/test_state.py @@ -388,6 +388,26 @@ def test_delete_windows_with_values(transaction_state, get_value): assert get_value(timestamp_ms=2, counter=1) == "b" +def test_delete_window(transaction_state): + with transaction_state() as state: + state.update_window(start_ms=1, end_ms=2, value=1, timestamp_ms=1) + state.update_window(start_ms=2, end_ms=3, value=2, timestamp_ms=2) + state.update_window(start_ms=3, end_ms=4, value=3, timestamp_ms=3) + + with transaction_state() as state: + assert state.get_window(start_ms=1, end_ms=2) + assert state.get_window(start_ms=2, end_ms=3) + assert state.get_window(start_ms=3, end_ms=4) + + # Delete a specific window + state.delete_window(start_ms=2, end_ms=3) + + # Only the specified window should be deleted + assert state.get_window(start_ms=1, end_ms=2) + assert not state.get_window(start_ms=2, end_ms=3) + assert state.get_window(start_ms=3, end_ms=4) + + @pytest.mark.parametrize("value", [1, "string", None, ["list"], {"dict": "dict"}]) def test_add_to_collection(transaction_state, get_value, value): with transaction_state() as state: From 6c52b8a5ee03211c4752afef44be6c1e73188cfb Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Wed, 6 Aug 2025 15:58:35 +0200 Subject: [PATCH 18/19] Tweak the session windows code --- quixstreams/dataframe/windows/session.py | 65 +++++++++--------------- 1 file changed, 24 insertions(+), 41 deletions(-) diff --git a/quixstreams/dataframe/windows/session.py b/quixstreams/dataframe/windows/session.py index d35251869..40324eb27 100644 --- a/quixstreams/dataframe/windows/session.py +++ b/quixstreams/dataframe/windows/session.py @@ -66,73 +66,56 @@ def process_window( latest_timestamp = max(timestamp_ms, state_ts) # Calculate session expiry threshold - session_expiry_threshold = latest_timestamp - grace_ms + expiry_threshold = latest_timestamp - grace_ms # Check if the event is too late - if timestamp_ms < session_expiry_threshold: - late_by_ms = session_expiry_threshold - timestamp_ms + if timestamp_ms < expiry_threshold: self._on_expired_window( value=value, key=key, start=timestamp_ms, - end=timestamp_ms, # End time is the timestamp of the last event + end=timestamp_ms, timestamp_ms=timestamp_ms, - late_by_ms=late_by_ms, + late_by_ms=expiry_threshold - timestamp_ms, ) return [], [] - # Look for an existing session that can be extended - can_extend_session = False - existing_aggregated = None - old_window_to_delete = None - # Search for active sessions that can accommodate the new event - search_start = max(0, timestamp_ms - timeout_ms * 2) - windows = state.get_windows( - search_start, timestamp_ms + timeout_ms + 1, backwards=True - ) - - for (window_start, window_end), aggregated_value, _ in windows: + for (window_start, window_end), aggregated_value, _ in state.get_windows( + start_from_ms=max(0, timestamp_ms - timeout_ms * 2), + start_to_ms=timestamp_ms + timeout_ms + 1, + backwards=True, + ): # Calculate the time gap between the new event and the session's last activity # window_end now directly represents the timestamp of the last event - session_last_activity = window_end - time_gap = timestamp_ms - session_last_activity + time_gap = timestamp_ms - window_end # Check if this session can be extended if time_gap <= timeout_ms + grace_ms and timestamp_ms >= window_start: + extend_session = True session_start = window_start - # Only update end time if the new event is newer than the current end time + # Only update end time if the new event is greater than the current end time session_end = max(window_end, timestamp_ms) - can_extend_session = True existing_aggregated = aggregated_value - old_window_to_delete = (window_start, window_end) + # Delete the old window if extending an existing session + state.delete_window(window_start, window_end) break - - # If no extendable session found, start a new one - if not can_extend_session: - session_start = timestamp_ms - session_end = timestamp_ms # End time is the timestamp of the last event + else: + # If no extendable session found, start a new one + extend_session = False + session_start = session_end = timestamp_ms # Process the event for this session updated_windows: list[WindowKeyResult] = [] - # Delete the old window if extending an existing session - if can_extend_session and old_window_to_delete: - old_start, old_end = old_window_to_delete - state.delete_window(old_start, old_end) - # Add to collection if needed if collect: - state.add_to_collection( - value=self._collect_value(value), - id=timestamp_ms, - ) + state.add_to_collection(value=self._collect_value(value), id=timestamp_ms) # Update the session window aggregation - aggregated = None if aggregate: current_value = ( - existing_aggregated if can_extend_session else self._initialize_value() + existing_aggregated if extend_session else self._initialize_value() ) aggregated = self._aggregate_value(current_value, value, timestamp_ms) @@ -142,6 +125,8 @@ def process_window( self._results(aggregated, [], session_start, session_end), ) ) + else: + aggregated = None state.update_window( session_start, session_end, value=aggregated, timestamp_ms=timestamp_ms @@ -150,12 +135,10 @@ def process_window( # Expire old sessions if self._closing_strategy == ClosingStrategy.PARTITION: expired_windows = self.expire_by_partition( - transaction, session_expiry_threshold, collect + transaction, expiry_threshold, collect ) else: - expired_windows = self.expire_by_key( - key, state, session_expiry_threshold, collect - ) + expired_windows = self.expire_by_key(key, state, expiry_threshold, collect) return updated_windows, expired_windows From e30c97ed3769de678610b1535fbd235e151cf842 Mon Sep 17 00:00:00 2001 From: Remy Gwaramadze Date: Wed, 6 Aug 2025 16:22:41 +0200 Subject: [PATCH 19/19] Correct duration validation tests --- .../test_windowed/test_transaction.py | 43 ++++--------------- 1 file changed, 8 insertions(+), 35 deletions(-) diff --git a/tests/test_quixstreams/test_state/test_rocksdb/test_windowed/test_transaction.py b/tests/test_quixstreams/test_state/test_rocksdb/test_windowed/test_transaction.py index 6808b0fef..efc3239d7 100644 --- a/tests/test_quixstreams/test_state/test_rocksdb/test_windowed/test_transaction.py +++ b/tests/test_quixstreams/test_state/test_rocksdb/test_windowed/test_transaction.py @@ -220,62 +220,35 @@ def test_expire_windows_with_grace_empty(self, windowed_rocksdb_store_factory): assert not expired - @pytest.mark.parametrize( - "start_ms, end_ms", - [ - (0, 0), - (1, 0), - ], - ) - def test_get_window_invalid_duration( - self, start_ms, end_ms, windowed_rocksdb_store_factory - ): + def test_get_window_invalid_duration(self, windowed_rocksdb_store_factory): store = windowed_rocksdb_store_factory() store.assign_partition(0) prefix = b"__key__" with store.start_partition_transaction(0) as tx: with pytest.raises(ValueError, match="Invalid window duration"): - tx.get_window(start_ms=start_ms, end_ms=end_ms, prefix=prefix) - - @pytest.mark.parametrize( - "start_ms, end_ms", - [ - (0, 0), - (1, 0), - ], - ) - def test_update_window_invalid_duration( - self, start_ms, end_ms, windowed_rocksdb_store_factory - ): + tx.get_window(start_ms=1, end_ms=0, prefix=prefix) + + def test_update_window_invalid_duration(self, windowed_rocksdb_store_factory): store = windowed_rocksdb_store_factory() store.assign_partition(0) prefix = b"__key__" with store.start_partition_transaction(0) as tx: with pytest.raises(ValueError, match="Invalid window duration"): tx.update_window( - start_ms=start_ms, - end_ms=end_ms, + start_ms=1, + end_ms=0, value=1, timestamp_ms=1, prefix=prefix, ) - @pytest.mark.parametrize( - "start_ms, end_ms", - [ - (0, 0), - (1, 0), - ], - ) - def test_delete_window_invalid_duration( - self, start_ms, end_ms, windowed_rocksdb_store_factory - ): + def test_delete_window_invalid_duration(self, windowed_rocksdb_store_factory): store = windowed_rocksdb_store_factory() store.assign_partition(0) prefix = b"__key__" with store.start_partition_transaction(0) as tx: with pytest.raises(ValueError, match="Invalid window duration"): - tx.delete_window(start_ms=start_ms, end_ms=end_ms, prefix=prefix) + tx.delete_window(start_ms=1, end_ms=0, prefix=prefix) def test_expire_windows_no_expired(self, windowed_rocksdb_store_factory): store = windowed_rocksdb_store_factory()