|
1 | 1 | import logging |
2 | 2 | from time import monotonic |
3 | | -from typing import Optional, TypedDict |
| 3 | +from typing import TYPE_CHECKING, Optional, TypedDict |
| 4 | + |
| 5 | +from confluent_kafka import TopicPartition |
4 | 6 |
|
5 | 7 | from quixstreams.internal_producer import InternalProducer |
| 8 | +from quixstreams.kafka.consumer import raise_for_msg_error |
6 | 9 | from quixstreams.models import Topic |
7 | 10 | from quixstreams.models.topics.manager import TopicManager |
8 | 11 | from quixstreams.utils.format import format_timestamp |
9 | 12 | from quixstreams.utils.json import dumps |
10 | 13 |
|
| 14 | +if TYPE_CHECKING: |
| 15 | + from quixstreams.kafka import BaseConsumer |
| 16 | + |
11 | 17 | logger = logging.getLogger(__name__) |
12 | 18 |
|
13 | 19 | __all__ = ("WatermarkManager", "WatermarkMessage") |
@@ -155,3 +161,175 @@ def _get_watermark(self) -> int: |
155 | 161 | if watermarks := self._watermarks.values(): |
156 | 162 | watermark = min(watermarks) |
157 | 163 | return watermark |
| 164 | + |
| 165 | + def bootstrap_watermarks(self, consumer: "BaseConsumer") -> None: |
| 166 | + """ |
| 167 | + Bootstrap watermarks by reading the watermarks topic progressively. |
| 168 | +
|
| 169 | + This method uses an exponential backoff strategy: |
| 170 | + 1. Try to read N messages from the end of the topic |
| 171 | + 2. If not all topic-partitions are found, seek further back exponentially |
| 172 | + 3. Continue until all TPs have watermarks or the beginning is reached |
| 173 | +
|
| 174 | + :param consumer: The Kafka consumer to use for reading watermarks |
| 175 | + """ |
| 176 | + watermarks_topic_name = self.watermarks_topic.name |
| 177 | + watermarks_partition = 0 # Watermarks topic always has 1 partition |
| 178 | + |
| 179 | + # Get the expected topic-partitions that need watermarks |
| 180 | + expected_tps = set(self._watermarks.keys()) |
| 181 | + if not expected_tps: |
| 182 | + logger.info("No topic-partitions to bootstrap watermarks for") |
| 183 | + return |
| 184 | + |
| 185 | + logger.info( |
| 186 | + f"Bootstrapping watermarks for {len(expected_tps)} topic-partitions " |
| 187 | + f"from topic '{watermarks_topic_name}'. Expected TPs: {expected_tps}" |
| 188 | + ) |
| 189 | + |
| 190 | + # Get the high watermark (end offset) of the watermarks topic |
| 191 | + tp = TopicPartition(watermarks_topic_name, watermarks_partition) |
| 192 | + logger.debug(f"Getting watermark offsets for {watermarks_topic_name}...") |
| 193 | + try: |
| 194 | + _, high_offset = consumer.get_watermark_offsets(tp, timeout=5.0) |
| 195 | + logger.debug(f"Watermarks topic high offset: {high_offset}") |
| 196 | + except Exception as e: |
| 197 | + # If we can't get watermark offsets, the topic might not be ready yet |
| 198 | + # Log a warning but allow the application to start with -1 watermarks |
| 199 | + logger.warning( |
| 200 | + f"Failed to get watermark offsets for topic {watermarks_topic_name}: {e}. " |
| 201 | + f"Watermarks will start at -1 and be updated as messages arrive." |
| 202 | + ) |
| 203 | + return |
| 204 | + |
| 205 | + if high_offset == 0: |
| 206 | + logger.info("Watermarks topic is empty, no bootstrapping needed") |
| 207 | + return |
| 208 | + |
| 209 | + # Progressive search parameters |
| 210 | + initial_lookback = 100 # Start by looking at last 100 messages |
| 211 | + lookback_step = min(initial_lookback, high_offset) |
| 212 | + found_tps: set[tuple[str, int]] = set() |
| 213 | + seek_offset = max(0, high_offset - lookback_step) |
| 214 | + |
| 215 | + iteration_count = 0 |
| 216 | + max_iterations = 20 # Safety limit to prevent infinite loops |
| 217 | + while found_tps != expected_tps: |
| 218 | + iteration_count += 1 |
| 219 | + if iteration_count > max_iterations: |
| 220 | + missing_tps = expected_tps - found_tps |
| 221 | + raise RuntimeError( |
| 222 | + f"Bootstrap failed: exceeded {max_iterations} iterations. " |
| 223 | + f"Found {len(found_tps)}/{len(expected_tps)} topic-partitions. " |
| 224 | + f"Missing: {missing_tps}. This suggests a bug in the bootstrap logic." |
| 225 | + ) |
| 226 | + logger.info( |
| 227 | + f"Bootstrap iteration {iteration_count}: seeking to offset {seek_offset} " |
| 228 | + f"(lookback_step={lookback_step}, found {len(found_tps)}/{len(expected_tps)} TPs)" |
| 229 | + ) |
| 230 | + |
| 231 | + # Seek to the calculated position |
| 232 | + tp_with_offset = TopicPartition( |
| 233 | + watermarks_topic_name, watermarks_partition, seek_offset |
| 234 | + ) |
| 235 | + try: |
| 236 | + consumer.seek(tp_with_offset) |
| 237 | + logger.debug(f"Seeked to offset {seek_offset}") |
| 238 | + except Exception as e: |
| 239 | + logger.error(f"Failed to seek to offset {seek_offset}: {e}") |
| 240 | + raise |
| 241 | + |
| 242 | + # Read messages from seek_offset towards previous seek_offset |
| 243 | + # or until all TPs are found |
| 244 | + messages_read = 0 |
| 245 | + max_messages_to_read = lookback_step |
| 246 | + |
| 247 | + # Timeout for this specific seek iteration (30 seconds) |
| 248 | + iteration_timeout = 30.0 |
| 249 | + iteration_start_time = monotonic() |
| 250 | + consecutive_poll_timeouts = 0 |
| 251 | + max_consecutive_poll_timeouts = 5 # Stop after 5 consecutive empty polls |
| 252 | + |
| 253 | + while messages_read < max_messages_to_read: |
| 254 | + # Check if this iteration has timed out |
| 255 | + if monotonic() - iteration_start_time > iteration_timeout: |
| 256 | + missing_tps = expected_tps - found_tps |
| 257 | + raise TimeoutError( |
| 258 | + f"Bootstrap failed: polling timeout after {iteration_timeout}s for seek offset {seek_offset}. " |
| 259 | + f"Found {len(found_tps)}/{len(expected_tps)} topic-partitions. " |
| 260 | + f"Missing: {missing_tps}. Cannot start application without complete watermark state." |
| 261 | + ) |
| 262 | + |
| 263 | + msg = consumer.poll(timeout=1.0) |
| 264 | + if msg is None: |
| 265 | + consecutive_poll_timeouts += 1 |
| 266 | + # If we've had many consecutive timeouts, assume we've read all available messages |
| 267 | + # in this range and move to the next iteration |
| 268 | + if consecutive_poll_timeouts >= max_consecutive_poll_timeouts: |
| 269 | + logger.info( |
| 270 | + f"No more messages available after {consecutive_poll_timeouts} empty polls at offset {seek_offset}, " |
| 271 | + f"moving to next iteration (read {messages_read}/{max_messages_to_read} messages)" |
| 272 | + ) |
| 273 | + break |
| 274 | + continue |
| 275 | + |
| 276 | + # Reset consecutive timeout counter when we get a message |
| 277 | + consecutive_poll_timeouts = 0 |
| 278 | + |
| 279 | + # Skip messages from other topics (shouldn't happen but be safe) |
| 280 | + if msg.topic() != watermarks_topic_name: |
| 281 | + continue |
| 282 | + |
| 283 | + messages_read += 1 |
| 284 | + |
| 285 | + # Deserialize and process the watermark message |
| 286 | + try: |
| 287 | + # Raise if message has an error |
| 288 | + msg = raise_for_msg_error(msg) |
| 289 | + watermark_msg = self.watermarks_topic.deserialize(msg).value |
| 290 | + tp_key = (watermark_msg["topic"], watermark_msg["partition"]) |
| 291 | + |
| 292 | + # Only track if it's an expected TP |
| 293 | + if tp_key in expected_tps: |
| 294 | + timestamp = watermark_msg["timestamp"] |
| 295 | + # Update the watermark (use max to handle out-of-order reads) |
| 296 | + current = self._watermarks.get(tp_key, -1) |
| 297 | + self._watermarks[tp_key] = max(current, timestamp) |
| 298 | + found_tps.add(tp_key) |
| 299 | + |
| 300 | + logger.debug( |
| 301 | + f"Bootstrapped watermark for {watermark_msg['topic']}[{watermark_msg['partition']}]: " |
| 302 | + f"{format_timestamp(timestamp)}" |
| 303 | + ) |
| 304 | + |
| 305 | + # Stop if we've found all TPs |
| 306 | + if found_tps == expected_tps: |
| 307 | + logger.info( |
| 308 | + f"Successfully bootstrapped all {len(expected_tps)} topic-partitions " |
| 309 | + f"after reading {messages_read} messages" |
| 310 | + ) |
| 311 | + return |
| 312 | + |
| 313 | + except Exception as e: |
| 314 | + logger.warning(f"Failed to deserialize watermark message: {e}") |
| 315 | + continue |
| 316 | + |
| 317 | + # If we've read everything and still missing TPs, expand lookback exponentially |
| 318 | + if found_tps != expected_tps: |
| 319 | + if seek_offset == 0: |
| 320 | + # We've read the entire topic from the beginning |
| 321 | + missing_tps = expected_tps - found_tps |
| 322 | + logger.warning( |
| 323 | + f"Reached beginning of watermarks topic but {len(missing_tps)} " |
| 324 | + f"topic-partitions still have no watermarks: {missing_tps}. " |
| 325 | + f"They will remain at -1 until new watermarks arrive." |
| 326 | + ) |
| 327 | + return |
| 328 | + |
| 329 | + # Double the step and seek further back from current position |
| 330 | + lookback_step = min(lookback_step * 2, seek_offset) |
| 331 | + seek_offset = max(0, seek_offset - lookback_step) |
| 332 | + |
| 333 | + logger.info( |
| 334 | + f"Finished bootstrapping watermarks: found {len(found_tps)}/{len(expected_tps)} topic-partitions" |
| 335 | + ) |
0 commit comments