|
| 1 | +import asyncio |
| 2 | +import time |
| 3 | +from collections import deque |
| 4 | +from dataclasses import dataclass |
| 5 | +from typing import Dict, List, Optional |
| 6 | + |
| 7 | +from ray._private.telemetry.open_telemetry_metric_recorder import ( |
| 8 | + OpenTelemetryMetricRecorder, |
| 9 | +) |
| 10 | +from ray.core.generated import ( |
| 11 | + events_base_event_pb2, |
| 12 | +) |
| 13 | +from ray.core.generated.events_base_event_pb2 import RayEvent |
| 14 | +from ray.dashboard.modules.aggregator.constants import ( |
| 15 | + AGGREGATOR_AGENT_METRIC_PREFIX, |
| 16 | + CONSUMER_TAG_KEY, |
| 17 | +) |
| 18 | + |
| 19 | + |
| 20 | +@dataclass |
| 21 | +class _ConsumerState: |
| 22 | + # Index of the next event to be consumed by this consumer |
| 23 | + cursor_index: int |
| 24 | + |
| 25 | + |
| 26 | +class MultiConsumerEventBuffer: |
| 27 | + """A buffer which allows adding one event at a time and consuming events in batches. |
| 28 | + Supports multiple consumers, each with their own cursor index. Tracks the number of events evicted for each consumer. |
| 29 | +
|
| 30 | + Buffer is not thread-safe but is asyncio-friendly. All operations must be called from within the same event loop. |
| 31 | +
|
| 32 | + Arguments: |
| 33 | + max_size: Maximum number of events to store in the buffer. |
| 34 | + max_batch_size: Maximum number of events to return in a batch when calling wait_for_batch. |
| 35 | + common_metric_tags: Tags to add to all metrics. |
| 36 | + """ |
| 37 | + |
| 38 | + def __init__( |
| 39 | + self, |
| 40 | + max_size: int, |
| 41 | + max_batch_size: int, |
| 42 | + common_metric_tags: Optional[Dict[str, str]] = None, |
| 43 | + ): |
| 44 | + self._buffer = deque(maxlen=max_size) |
| 45 | + self._max_size = max_size |
| 46 | + self._lock = asyncio.Lock() |
| 47 | + self._has_new_events_to_consume = asyncio.Condition(self._lock) |
| 48 | + self._consumers: Dict[str, _ConsumerState] = {} |
| 49 | + |
| 50 | + self._max_batch_size = max_batch_size |
| 51 | + |
| 52 | + self._common_metrics_tags = common_metric_tags or {} |
| 53 | + self._metric_recorder = OpenTelemetryMetricRecorder() |
| 54 | + self.evicted_events_metric_name = ( |
| 55 | + f"{AGGREGATOR_AGENT_METRIC_PREFIX}_queue_dropped_events" |
| 56 | + ) |
| 57 | + self._metric_recorder.register_counter_metric( |
| 58 | + self.evicted_events_metric_name, |
| 59 | + "Total number of events dropped because the publish/buffer queue was full.", |
| 60 | + ) |
| 61 | + |
| 62 | + async def add_event(self, event: events_base_event_pb2.RayEvent) -> None: |
| 63 | + """Add an event to the buffer. |
| 64 | +
|
| 65 | + If the buffer is full, the oldest event is dropped. |
| 66 | + """ |
| 67 | + async with self._lock: |
| 68 | + dropped_event = None |
| 69 | + if len(self._buffer) >= self._max_size: |
| 70 | + dropped_event = self._buffer.popleft() |
| 71 | + self._buffer.append(event) |
| 72 | + |
| 73 | + if dropped_event is not None: |
| 74 | + for consumer_name, consumer_state in self._consumers.items(): |
| 75 | + # Update consumer cursor index and evicted events metric if an event was dropped |
| 76 | + if consumer_state.cursor_index == 0: |
| 77 | + # The dropped event was the next event this consumer would have consumed, publish eviction metric |
| 78 | + self._metric_recorder.set_metric_value( |
| 79 | + self.evicted_events_metric_name, |
| 80 | + { |
| 81 | + **self._common_metrics_tags, |
| 82 | + CONSUMER_TAG_KEY: consumer_name, |
| 83 | + "event_type": RayEvent.EventType.Name( |
| 84 | + dropped_event.event_type |
| 85 | + ), |
| 86 | + }, |
| 87 | + 1, |
| 88 | + ) |
| 89 | + else: |
| 90 | + # The dropped event was already consumed by the consumer, so we need to adjust the cursor |
| 91 | + consumer_state.cursor_index -= 1 |
| 92 | + |
| 93 | + # Signal the consumers that there are new events to consume |
| 94 | + self._has_new_events_to_consume.notify_all() |
| 95 | + |
| 96 | + def _evict_old_events(self) -> None: |
| 97 | + """Clean the buffer by removing events from the buffer who have index lower than |
| 98 | + all the cursor indexes of all consumers and updating the cursor index of all |
| 99 | + consumers. |
| 100 | + """ |
| 101 | + if not self._consumers: |
| 102 | + return |
| 103 | + |
| 104 | + min_cursor_index = min( |
| 105 | + consumer_state.cursor_index for consumer_state in self._consumers.values() |
| 106 | + ) |
| 107 | + for _ in range(min_cursor_index): |
| 108 | + self._buffer.popleft() |
| 109 | + |
| 110 | + # update the cursor index of all consumers |
| 111 | + for consumer_state in self._consumers.values(): |
| 112 | + consumer_state.cursor_index -= min_cursor_index |
| 113 | + |
| 114 | + async def wait_for_batch( |
| 115 | + self, consumer_name: str, timeout_seconds: float = 1.0 |
| 116 | + ) -> List[events_base_event_pb2.RayEvent]: |
| 117 | + """Wait for batch respecting self.max_batch_size and timeout_seconds. |
| 118 | +
|
| 119 | + Returns a batch of up to self.max_batch_size items. Waits for up to |
| 120 | + timeout_seconds after receiving the first event that will be in |
| 121 | + the next batch. After the timeout, returns as many items as are ready. |
| 122 | +
|
| 123 | + Always returns a batch with at least one item - will block |
| 124 | + indefinitely until an item comes in. |
| 125 | +
|
| 126 | + Arguments: |
| 127 | + consumer_name: name of the consumer consuming the batch |
| 128 | + timeout_seconds: maximum time to wait for a batch |
| 129 | +
|
| 130 | + Returns: |
| 131 | + A list of up to max_batch_size events ready for consumption. |
| 132 | + The list always contains at least one event. |
| 133 | + """ |
| 134 | + max_batch = self._max_batch_size |
| 135 | + batch = [] |
| 136 | + async with self._has_new_events_to_consume: |
| 137 | + consumer_state = self._consumers.get(consumer_name) |
| 138 | + if consumer_state is None: |
| 139 | + raise KeyError(f"unknown consumer '{consumer_name}'") |
| 140 | + |
| 141 | + # Phase 1: read the first event, wait indefinitely until there is at least one event to consume |
| 142 | + while consumer_state.cursor_index >= len(self._buffer): |
| 143 | + await self._has_new_events_to_consume.wait() |
| 144 | + |
| 145 | + # Add the first event to the batch |
| 146 | + event = self._buffer[consumer_state.cursor_index] |
| 147 | + consumer_state.cursor_index += 1 |
| 148 | + batch.append(event) |
| 149 | + |
| 150 | + # Phase 2: add items to the batch up to timeout or until full |
| 151 | + deadline = time.monotonic() + max(0.0, float(timeout_seconds)) |
| 152 | + while len(batch) < max_batch: |
| 153 | + remaining = deadline - time.monotonic() |
| 154 | + if remaining <= 0: |
| 155 | + break |
| 156 | + |
| 157 | + # Drain whatever is available |
| 158 | + while len(batch) < max_batch and consumer_state.cursor_index < len( |
| 159 | + self._buffer |
| 160 | + ): |
| 161 | + batch.append(self._buffer[consumer_state.cursor_index]) |
| 162 | + consumer_state.cursor_index += 1 |
| 163 | + |
| 164 | + if len(batch) >= max_batch: |
| 165 | + break |
| 166 | + |
| 167 | + # There is still room in the batch, but no new events to consume; wait until notified or timeout |
| 168 | + try: |
| 169 | + await asyncio.wait_for( |
| 170 | + self._has_new_events_to_consume.wait(), remaining |
| 171 | + ) |
| 172 | + except asyncio.TimeoutError: |
| 173 | + # Timeout, return the current batch |
| 174 | + break |
| 175 | + |
| 176 | + self._evict_old_events() |
| 177 | + return batch |
| 178 | + |
| 179 | + async def register_consumer(self, consumer_name: str) -> None: |
| 180 | + """Register a new consumer with a name. |
| 181 | +
|
| 182 | + Arguments: |
| 183 | + consumer_name: A unique name for the consumer. |
| 184 | +
|
| 185 | + """ |
| 186 | + async with self._lock: |
| 187 | + if self._consumers.get(consumer_name) is not None: |
| 188 | + raise ValueError(f"consumer '{consumer_name}' already registered") |
| 189 | + |
| 190 | + self._consumers[consumer_name] = _ConsumerState(cursor_index=0) |
| 191 | + |
| 192 | + async def size(self) -> int: |
| 193 | + """Get total number of events in the buffer. Does not take consumer cursors into account.""" |
| 194 | + return len(self._buffer) |
0 commit comments