sentry_arroyo 2.39.1

A library for working with streaming data.
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
from __future__ import annotations

import functools
import logging
import time
from collections import defaultdict
from typing import (
    Any,
    Callable,
    Generic,
    Literal,
    Mapping,
    MutableMapping,
    Optional,
    Sequence,
    TypeVar,
    Union,
    cast,
)

from arroyo.backends.abstract import Consumer
from arroyo.commit import ONCE_PER_SECOND, CommitPolicy
from arroyo.dlq import BufferedMessages, DlqPolicy, DlqPolicyWrapper, InvalidMessage
from arroyo.errors import RecoverableError
from arroyo.processing.strategies.abstract import (
    MessageRejected,
    ProcessingStrategy,
    ProcessingStrategyFactory,
)
from arroyo.types import BrokerValue, Message, Partition, Topic, TStrategyPayload
from arroyo.utils.logging import handle_internal_error
from arroyo.utils.metrics import get_metrics
from arroyo.utils.stuck_detector import get_all_thread_stacks

logger = logging.getLogger(__name__)

METRICS_FREQUENCY_SEC = 1.0  # In seconds
BACKPRESSURE_THRESHOLD = 5.0  # In seconds
DEFAULT_JOIN_TIMEOUT = 25.0  # In seconds

F = TypeVar("F", bound=Callable[[Any], Any])


def _rdkafka_callback(metrics: MetricsBuffer) -> Callable[[F], F]:
    def decorator(f: F) -> F:
        @functools.wraps(f)
        def wrapper(*args: Any, **kwargs: Any) -> Any:
            start_time = time.time()
            try:
                return f(*args, **kwargs)
            except Exception as e:
                handle_internal_error(e)
                logger.exception(f"{f.__name__} crashed")
                raise
            finally:
                value = time.time() - start_time
                metrics.metrics.timing(
                    "arroyo.consumer.run.callback",
                    value,
                    tags={"callback_name": f.__name__},
                )
                metrics.incr_timing("arroyo.consumer.callback.time", value)

        return cast(F, wrapper)

    return decorator


class InvalidStateError(RuntimeError):
    pass


ConsumerTiming = Literal[
    "arroyo.consumer.poll.time",
    "arroyo.consumer.processing.time",
    "arroyo.consumer.backpressure.time",
    "arroyo.consumer.dlq.time",
    "arroyo.consumer.join.time",
    # This metric's timings overlap with DLQ/join time.
    "arroyo.consumer.callback.time",
    "arroyo.consumer.shutdown.time",
]

ConsumerCounter = Literal[
    "arroyo.consumer.run.count",
    "arroyo.consumer.invalid_message.count",
    "arroyo.consumer.pause",
    "arroyo.consumer.resume",
    "arroyo.consumer.dlq.dropped_messages",
    "arroyo.consumer.stuck",
]


class MetricsBuffer:
    def __init__(self) -> None:
        self.metrics = get_metrics()
        self.__timers: MutableMapping[ConsumerTiming, float] = defaultdict(float)
        self.__counters: MutableMapping[ConsumerCounter, int] = defaultdict(int)
        self.__reset()

    def incr_timing(self, metric: ConsumerTiming, duration: float) -> None:
        self.__timers[metric] += duration
        self.__throttled_record()

    def incr_counter(self, metric: ConsumerCounter, delta: int) -> None:
        self.__counters[metric] += delta
        self.__throttled_record()

    def flush(self) -> None:
        metric: Union[ConsumerTiming, ConsumerCounter]
        value: Union[float, int]

        for metric, value in self.__timers.items():
            self.metrics.timing(metric, value)
        for metric, value in self.__counters.items():
            self.metrics.increment(metric, value)
        self.__reset()

    def __reset(self) -> None:
        self.__timers.clear()
        self.__counters.clear()
        self.__last_record_time = time.time()

    def __throttled_record(self) -> None:
        if time.time() - self.__last_record_time > METRICS_FREQUENCY_SEC:
            self.flush()


class StreamProcessor(Generic[TStrategyPayload]):
    """
    A stream processor manages the relationship between a ``Consumer``
    instance and a ``ProcessingStrategy``, ensuring that processing
    strategies are instantiated on partition assignment and closed on
    partition revocation.
    """

    def __init__(
        self,
        consumer: Consumer[TStrategyPayload],
        topic: Topic,
        processor_factory: ProcessingStrategyFactory[TStrategyPayload],
        commit_policy: CommitPolicy = ONCE_PER_SECOND,
        dlq_policy: Optional[DlqPolicy[TStrategyPayload]] = None,
        join_timeout: Optional[float] = None,
        stuck_detector_timeout: Optional[int] = None,
    ) -> None:
        self.__consumer = consumer
        self.__processor_factory = processor_factory
        self.__metrics_buffer = MetricsBuffer()

        self.__processing_strategy: Optional[ProcessingStrategy[TStrategyPayload]] = (
            None
        )

        self.__message: Optional[BrokerValue[TStrategyPayload]] = None

        # The timestamp when backpressure state started
        self.__backpressure_timestamp: Optional[float] = None
        # Consumer is paused after it is in backpressure state for > BACKPRESSURE_THRESHOLD seconds
        self.__is_paused = False

        self.__commit_policy_state = commit_policy.get_state_machine()

        # Default join_timeout to DEFAULT_JOIN_TIMEOUT if not provided
        self.__join_timeout = (
            join_timeout if join_timeout is not None else DEFAULT_JOIN_TIMEOUT
        )

        self.__shutdown_requested = False
        self.__shutdown_done = False

        # Buffers messages for DLQ. Messages are added when they are submitted for processing and
        # removed once the commit callback is fired as they are guaranteed to be valid at that point.
        self.__buffered_messages: BufferedMessages[TStrategyPayload] = BufferedMessages(
            dlq_policy
        )

        self.__dlq_policy: Optional[DlqPolicyWrapper[TStrategyPayload]] = (
            DlqPolicyWrapper(dlq_policy) if dlq_policy is not None else None
        )

        self.__last_run = time.time()

        if stuck_detector_timeout:
            self.stuck_detector_run(stuck_detector_timeout)

        def _close_strategy() -> None:
            self._close_processing_strategy()

        def _create_strategy(partitions: Mapping[Partition, int]) -> None:
            start_create = time.time()

            self.__processing_strategy = (
                self.__processor_factory.create_with_partitions(
                    self.__commit, partitions
                )
            )

            self.__metrics_buffer.metrics.timing(
                "arroyo.consumer.run.create_strategy", time.time() - start_create
            )

            logger.debug(
                "Initialized processing strategy: %r", self.__processing_strategy
            )

        @_rdkafka_callback(metrics=self.__metrics_buffer)
        def on_partitions_assigned(partitions: Mapping[Partition, int]) -> None:
            logger.info("New partitions assigned: %r", partitions)
            logger.info("Member id: %r", self.__consumer.member_id)

            self.__metrics_buffer.metrics.increment(
                "arroyo.consumer.partitions_assigned.count", len(partitions)
            )

            current_partitions = dict(self.__consumer.tell())
            current_partitions.update(partitions)

            if self.__dlq_policy:
                self.__dlq_policy.reset_dlq_limits(current_partitions)

            if current_partitions:
                if self.__processing_strategy is not None:
                    # TODO: for cooperative-sticky rebalancing this can happen
                    # quite often. we should port the changes to
                    # ProcessingStrategyFactory that we made in Rust: Remove
                    # create_with_partitions, replace with create +
                    # update_partitions
                    logger.warning(
                        "Partition assignment while processing strategy active"
                    )
                    _close_strategy()
                _create_strategy(current_partitions)

        @_rdkafka_callback(metrics=self.__metrics_buffer)
        def on_partitions_revoked(partitions: Sequence[Partition]) -> None:
            logger.info("Partitions to revoke: %r", partitions)

            self.__metrics_buffer.metrics.increment(
                "arroyo.consumer.partitions_revoked.count", len(partitions)
            )

            if partitions:
                _close_strategy()

                # Recreate the strategy if the consumer still has other partitions
                # assigned and is not closed or errored
                try:
                    current_partitions = self.__consumer.tell()
                    if len(current_partitions.keys() - set(partitions)):
                        active_partitions = {
                            partition: offset
                            for partition, offset in current_partitions.items()
                            if partition not in partitions
                        }
                        logger.info(
                            "Recreating strategy since there are still active partitions: %r",
                            active_partitions,
                        )
                        _create_strategy(active_partitions)
                except RuntimeError:
                    pass

            for partition in partitions:
                self.__buffered_messages.remove(partition)

            # Partition revocation can happen anytime during the consumer lifecycle and happen
            # multiple times. What we want to know is that the consumer is not stuck somewhere.
            # The presence of this message as the last message of a consumer
            # indicates that the consumer was not stuck.
            logger.info("Partition revocation complete.")

        self.__consumer.subscribe(
            [topic], on_assign=on_partitions_assigned, on_revoke=on_partitions_revoked
        )

    def _close_processing_strategy(self) -> None:
        """Close the processing strategy and wait for it to exit."""
        start_close = time.time()

        if self.__processing_strategy is None:
            # Partitions are revoked when the consumer is shutting down, at
            # which point we already have closed the consumer.
            return

        logger.info("Closing %r...", self.__processing_strategy)
        logger.info("Member id: %r", self.__consumer.member_id)
        self.__processing_strategy.close()

        logger.info("Waiting for %r to exit...", self.__processing_strategy)

        while True:
            start_join = time.time()

            try:
                self.__processing_strategy.join(self.__join_timeout)
                self.__metrics_buffer.incr_timing(
                    "arroyo.consumer.join.time", time.time() - start_join
                )
                break
            except InvalidMessage as e:
                self.__metrics_buffer.incr_timing(
                    "arroyo.consumer.join.time", time.time() - start_join
                )
                self._handle_invalid_message(e)

        logger.info("%r exited successfully", self.__processing_strategy)
        self.__processing_strategy = None
        self.__message = None
        self.__is_paused = False
        self._clear_backpressure()

        value = time.time() - start_close
        self.__metrics_buffer.metrics.timing(
            "arroyo.consumer.run.close_strategy", value
        )
        self.__metrics_buffer.incr_timing("arroyo.consumer.shutdown.time", value)

    def __commit(self, offsets: Mapping[Partition, int], force: bool = False) -> None:
        """
        If force is passed, commit immediately and do not throttle. This should
        be used during consumer shutdown where we do not want to wait before committing.
        """
        for partition, offset in offsets.items():
            self.__buffered_messages.pop(partition, offset - 1)

        self.__consumer.stage_offsets(offsets)
        now = time.time()

        if force or self.__commit_policy_state.should_commit(
            now,
            offsets,
        ):
            if self.__dlq_policy:
                self.__dlq_policy.flush(offsets)

            self.__consumer.commit_offsets()
            logger.debug(
                "Waited %0.4f seconds for offsets to be committed to %r.",
                time.time() - now,
                self.__consumer,
            )
            self.__commit_policy_state.did_commit(now, offsets)

    def run(self) -> None:
        "The main run loop, see class docstring for more information."

        logger.debug("Starting")
        try:
            while not self.__shutdown_requested:
                self._run_once()

            self._shutdown()
        except Exception:
            logger.exception("Caught exception, shutting down...")

            if self.__processing_strategy is not None:
                logger.debug("Terminating %r...", self.__processing_strategy)
                self.__processing_strategy.terminate()
                self.__processing_strategy = None

            logger.info("Closing %r...", self.__consumer)
            self.__consumer.close()
            self.__processor_factory.shutdown()
            logger.info("Processor terminated")
            raise
        finally:
            self.__shutdown_done = True

    def stuck_detector_run(self, stuck_detector_timeout: int) -> None:
        import threading

        def f() -> None:
            while not self.__shutdown_done:
                time_since_last_run = time.time() - self.__last_run
                if time_since_last_run > stuck_detector_timeout:
                    stack_traces = get_all_thread_stacks()
                    logger.warning(
                        "main thread stuck for more than %s seconds, stacks: %s",
                        stuck_detector_timeout,
                        stack_traces,
                    )
                    # Use the metrics directly instead of going through the buffer
                    # to avoid race conditions with the main thread
                    self.__metrics_buffer.metrics.increment("arroyo.consumer.stuck", 1)
                    return

                time.sleep(1)

        t = threading.Thread(target=f)
        t.daemon = True
        t.start()

    def _clear_backpressure(self) -> None:
        if self.__backpressure_timestamp is not None:
            self.__metrics_buffer.incr_timing(
                "arroyo.consumer.backpressure.time",
                time.time() - self.__backpressure_timestamp,
            )
            self.__backpressure_timestamp = None

    def _handle_invalid_message(self, exc: InvalidMessage) -> None:
        # Do not "carry over" message if it is the invalid one. Every other
        # message should be re-submitted to the strategy.
        if (
            self.__message is not None
            and exc.partition == self.__message.partition
            and exc.offset == self.__message.offset
        ):
            self.__message = None

        if exc.log_exception:
            logger.exception(exc)
        self.__metrics_buffer.incr_counter("arroyo.consumer.invalid_message.count", 1)
        if self.__dlq_policy:
            start_dlq = time.time()
            invalid_message = self.__buffered_messages.pop(exc.partition, exc.offset)
            if invalid_message is None:
                logger.error(
                    "Invalid message not found in buffer %s %s",
                    exc.partition,
                    exc.offset,
                )
                return

            # XXX: This blocks if there are more than MAX_PENDING_FUTURES in the queue.
            try:
                self.__dlq_policy.produce(invalid_message, exc.reason)
            except Exception:
                logger.exception(
                    f"Failed to produce message (partition: {exc.partition} offset: {exc.offset}) to DLQ topic, dropping"
                )
                self.__metrics_buffer.incr_counter(
                    "arroyo.consumer.dlq.dropped_messages", 1
                )

            self.__metrics_buffer.incr_timing(
                "arroyo.consumer.dlq.time", time.time() - start_dlq
            )

    def _run_once(self) -> None:
        self.__metrics_buffer.incr_counter("arroyo.consumer.run.count", 1)
        self.__last_run = time.time()

        message_carried_over = self.__message is not None

        if not message_carried_over:
            # Poll for a new message from the consumer only if there is no carried
            # over message which we need to successfully submit first.
            try:
                start_poll = time.time()
                self.__message = self.__consumer.poll(timeout=1.0)
                if self.__message:
                    self.__buffered_messages.append(self.__message)
                self.__metrics_buffer.incr_timing(
                    "arroyo.consumer.poll.time", time.time() - start_poll
                )
            except RecoverableError:
                return

        if self.__processing_strategy is not None:
            start_poll = time.time()
            try:
                self.__processing_strategy.poll()
            except InvalidMessage as e:
                self._handle_invalid_message(e)
                return

            self.__metrics_buffer.incr_timing(
                "arroyo.consumer.processing.time", time.time() - start_poll
            )
            if self.__message is not None:
                try:
                    start_submit = time.time()
                    message = Message(self.__message)
                    self.__processing_strategy.submit(message)

                    self.__metrics_buffer.incr_timing(
                        "arroyo.consumer.processing.time",
                        time.time() - start_submit,
                    )
                except MessageRejected as e:
                    # If the processing strategy rejected our message, we need
                    # to pause the consumer and hold the message until it is
                    # accepted, at which point we can resume consuming.
                    # if not message_carried_over:
                    if self.__backpressure_timestamp is None:
                        self.__backpressure_timestamp = time.time()

                    elif not self.__is_paused and (
                        time.time() - self.__backpressure_timestamp
                        > BACKPRESSURE_THRESHOLD
                    ):
                        self.__metrics_buffer.incr_counter("arroyo.consumer.pause", 1)
                        logger.debug(
                            "Caught %r while submitting %r, pausing consumer...",
                            e,
                            self.__message,
                        )
                        self.__consumer.pause([*self.__consumer.tell().keys()])
                        self.__is_paused = True

                    elif self.__is_paused:
                        maybe_message = self.__consumer.poll(0.1)
                        if maybe_message is not None:
                            # The paused consumer, in the above poll, has
                            # gone through rebalancing. In this case we
                            # expect that partition revocation cleared the
                            # pause flag and the carried over message.
                            # this assumption will not hold for cooperative-sticky rebalancing.
                            assert (
                                not self.__is_paused
                            ), "consumer unpaused itself without rebalancing"
                            assert (
                                self.__message is None
                            ), "consumer unpaused itself without rebalancing"

                    else:
                        time.sleep(0.01)

                except InvalidMessage as e:
                    self._handle_invalid_message(e)

                    if self.__is_paused:
                        self.__metrics_buffer.incr_counter("arroyo.consumer.resume", 1)
                        self.__consumer.resume([*self.__consumer.tell().keys()])
                        self.__is_paused = False

                else:
                    # Resume if we are currently in a paused state
                    if self.__is_paused:
                        self.__metrics_buffer.incr_counter("arroyo.consumer.resume", 1)
                        self.__consumer.resume([*self.__consumer.tell().keys()])
                        self.__is_paused = False

                    # Clear backpressure timestamp if it is set
                    self._clear_backpressure()

                    self.__message = None
        else:
            if self.__message is not None:
                raise InvalidStateError(
                    "received message without active processing strategy"
                )

    def signal_shutdown(self) -> None:
        """
        Tells the stream processor to shutdown on the next run loop
        iteration.

        Typically called from a signal handler.
        """
        logger.info("Shutdown signalled")

        self.__shutdown_requested = True

    def _shutdown(self) -> None:
        # when we close() a consumer, rdkafka would would revoke our partition
        # and call revocation callbacks, but also immediately revoke our member
        # ID as well, causing join() of the CommitStrategy (that is running in
        # the partition revocation callback) to crash.
        self._close_processing_strategy()

        # close the consumer
        logger.info("Stopping consumer")
        self.__metrics_buffer.flush()
        self.__consumer.close()
        self.__processor_factory.shutdown()
        logger.info("Stopped")

        # if there was an active processing strategy, it should be shut down
        # and unset when the partitions are revoked during consumer close
        assert (
            self.__processing_strategy is None
        ), "processing strategy was not closed on shutdown"