sentry-arroyo 2.32.4__tar.gz → 2.32.5__tar.gz
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
- {sentry_arroyo-2.32.4/sentry_arroyo.egg-info → sentry_arroyo-2.32.5}/PKG-INFO +1 -1
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/processor.py +57 -35
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5/sentry_arroyo.egg-info}/PKG-INFO +1 -1
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/setup.py +1 -1
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/backends/test_kafka.py +88 -1
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/test_processor.py +72 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/LICENSE +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/MANIFEST.in +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/README.md +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/abstract.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/kafka/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/kafka/commit.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/kafka/configuration.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/kafka/consumer.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/local/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/local/backend.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/local/storages/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/local/storages/abstract.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/backends/local/storages/memory.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/commit.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/dlq.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/errors.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/abstract.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/batching.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/buffer.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/commit.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/filter.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/guard.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/healthcheck.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/noop.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/produce.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/reduce.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/run_task.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/run_task_in_threads.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/run_task_with_multiprocessing.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/unfold.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/py.typed +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/types.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/utils/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/utils/clock.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/utils/codecs.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/utils/concurrent.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/utils/logging.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/utils/metricDefs.json +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/utils/metric_defs.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/utils/metrics.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/utils/profiler.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/utils/retries.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/examples/transform_and_produce/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/examples/transform_and_produce/batched.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/examples/transform_and_produce/script.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/examples/transform_and_produce/simple.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/requirements.txt +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/sentry_arroyo.egg-info/SOURCES.txt +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/sentry_arroyo.egg-info/dependency_links.txt +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/sentry_arroyo.egg-info/not-zip-safe +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/sentry_arroyo.egg-info/requires.txt +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/sentry_arroyo.egg-info/top_level.txt +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/setup.cfg +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/backends/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/backends/mixins.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/backends/test_commit.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/backends/test_confluent_producer.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/backends/test_kafka_commit_callback.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/backends/test_kafka_producer.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/backends/test_local.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_all.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_batching.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_buffer.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_commit.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_filter.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_guard.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_noop.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_produce.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_reduce.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_run_task.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_run_task_in_threads.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_run_task_with_multiprocessing.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/processing/strategies/test_unfold.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/test_commit.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/test_dlq.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/test_kip848_e2e.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/test_types.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/utils/__init__.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/utils/test_concurrent.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/utils/test_metrics.py +0 -0
- {sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/tests/utils/test_retries.py +0 -0
|
@@ -141,14 +141,15 @@ class StreamProcessor(Generic[TStrategyPayload]):
|
|
|
141
141
|
dlq_policy: Optional[DlqPolicy[TStrategyPayload]] = None,
|
|
142
142
|
join_timeout: Optional[float] = None,
|
|
143
143
|
shutdown_strategy_before_consumer: bool = False,
|
|
144
|
+
handle_poll_while_paused: Optional[bool] = False,
|
|
144
145
|
) -> None:
|
|
145
146
|
self.__consumer = consumer
|
|
146
147
|
self.__processor_factory = processor_factory
|
|
147
148
|
self.__metrics_buffer = MetricsBuffer()
|
|
148
149
|
|
|
149
|
-
self.__processing_strategy: Optional[
|
|
150
|
-
|
|
151
|
-
|
|
150
|
+
self.__processing_strategy: Optional[
|
|
151
|
+
ProcessingStrategy[TStrategyPayload]
|
|
152
|
+
] = None
|
|
152
153
|
|
|
153
154
|
self.__message: Optional[BrokerValue[TStrategyPayload]] = None
|
|
154
155
|
|
|
@@ -167,6 +168,8 @@ class StreamProcessor(Generic[TStrategyPayload]):
|
|
|
167
168
|
self.__shutdown_requested = False
|
|
168
169
|
self.__shutdown_strategy_before_consumer = shutdown_strategy_before_consumer
|
|
169
170
|
|
|
171
|
+
self.__handle_poll_while_paused = handle_poll_while_paused
|
|
172
|
+
|
|
170
173
|
# Buffers messages for DLQ. Messages are added when they are submitted for processing and
|
|
171
174
|
# removed once the commit callback is fired as they are guaranteed to be valid at that point.
|
|
172
175
|
self.__buffered_messages: BufferedMessages[TStrategyPayload] = BufferedMessages(
|
|
@@ -464,40 +467,59 @@ class StreamProcessor(Generic[TStrategyPayload]):
|
|
|
464
467
|
self.__is_paused = True
|
|
465
468
|
|
|
466
469
|
elif self.__is_paused:
|
|
467
|
-
|
|
468
|
-
all_partitions = set(self.__consumer.tell())
|
|
469
|
-
unpaused_partitions = (
|
|
470
|
-
all_partitions - paused_partitions
|
|
471
|
-
)
|
|
472
|
-
if unpaused_partitions:
|
|
473
|
-
logger.warning(
|
|
474
|
-
"Processor in paused state while consumer is partially unpaused: %s, paused: %s",
|
|
475
|
-
unpaused_partitions,
|
|
476
|
-
paused_partitions,
|
|
477
|
-
)
|
|
478
|
-
self.__is_paused = False
|
|
479
|
-
# unpause paused partitions... just in case a subset is paused
|
|
480
|
-
self.__metrics_buffer.incr_counter(
|
|
481
|
-
"arroyo.consumer.resume", 1
|
|
482
|
-
)
|
|
483
|
-
self.__consumer.resume([*paused_partitions])
|
|
484
|
-
else:
|
|
485
|
-
# A paused consumer should still poll periodically to avoid it's partitions
|
|
486
|
-
# getting revoked by the broker after reaching the max.poll.interval.ms
|
|
487
|
-
# Polling a paused consumer should never yield a message.
|
|
488
|
-
logger.warning("consumer.tell() value right before poll() is: %s", self.__consumer.tell())
|
|
470
|
+
if self.__handle_poll_while_paused:
|
|
489
471
|
maybe_message = self.__consumer.poll(0.1)
|
|
490
472
|
if maybe_message is not None:
|
|
491
|
-
|
|
492
|
-
|
|
493
|
-
|
|
494
|
-
|
|
495
|
-
|
|
496
|
-
|
|
497
|
-
|
|
498
|
-
|
|
499
|
-
|
|
500
|
-
|
|
473
|
+
# The paused consumer, in the above poll, has
|
|
474
|
+
# gone through rebalancing. In this case we
|
|
475
|
+
# expect that partition revocation cleared the
|
|
476
|
+
# pause flag and the carried over message.
|
|
477
|
+
# this assumption will not hold for cooperative-sticky rebalancing.
|
|
478
|
+
assert (
|
|
479
|
+
not self.__is_paused
|
|
480
|
+
), "consumer unpaused itself without rebalancing"
|
|
481
|
+
assert (
|
|
482
|
+
self.__message is None
|
|
483
|
+
), "consumer unpaused itself without rebalancing"
|
|
484
|
+
# this path might raise AssertionErrors
|
|
485
|
+
else:
|
|
486
|
+
paused_partitions = set(self.__consumer.paused())
|
|
487
|
+
all_partitions = set(self.__consumer.tell())
|
|
488
|
+
unpaused_partitions = all_partitions - paused_partitions
|
|
489
|
+
if unpaused_partitions:
|
|
490
|
+
logger.warning(
|
|
491
|
+
"Processor in paused state while consumer is partially unpaused: %s, paused: %s",
|
|
492
|
+
unpaused_partitions,
|
|
493
|
+
paused_partitions,
|
|
494
|
+
)
|
|
495
|
+
self.__is_paused = False
|
|
496
|
+
# unpause paused partitions... just in case a subset is paused
|
|
497
|
+
self.__metrics_buffer.incr_counter(
|
|
498
|
+
"arroyo.consumer.resume", 1
|
|
499
|
+
)
|
|
500
|
+
self.__consumer.resume([*paused_partitions])
|
|
501
|
+
else:
|
|
502
|
+
# A paused consumer should still poll periodically to avoid it's partitions
|
|
503
|
+
# getting revoked by the broker after reaching the max.poll.interval.ms
|
|
504
|
+
# Polling a paused consumer should never yield a message.
|
|
505
|
+
logger.warning(
|
|
506
|
+
"consumer.tell() value right before poll() is: %s",
|
|
507
|
+
self.__consumer.tell(),
|
|
508
|
+
)
|
|
509
|
+
maybe_message = self.__consumer.poll(0.1)
|
|
510
|
+
if maybe_message is not None:
|
|
511
|
+
logger.warning(
|
|
512
|
+
"Received a message from partition: %s, \
|
|
513
|
+
consumer.tell() value right after poll() is: %s \
|
|
514
|
+
Some lines above consumer.tell() was called, all_partitons value was: %s \
|
|
515
|
+
Some lines above consumer.paused() was called, paused_partitions value is: %s",
|
|
516
|
+
maybe_message.partition,
|
|
517
|
+
self.__consumer.tell(),
|
|
518
|
+
all_partitions,
|
|
519
|
+
paused_partitions,
|
|
520
|
+
)
|
|
521
|
+
assert maybe_message is None
|
|
522
|
+
|
|
501
523
|
else:
|
|
502
524
|
time.sleep(0.01)
|
|
503
525
|
|
|
@@ -80,7 +80,7 @@ class TestKafkaStreams(StreamsTestMixin[KafkaPayload]):
|
|
|
80
80
|
@property
|
|
81
81
|
def configuration(self) -> KafkaBrokerConfig:
|
|
82
82
|
config = {
|
|
83
|
-
"bootstrap.servers": os.environ.get("DEFAULT_BROKERS", "
|
|
83
|
+
"bootstrap.servers": os.environ.get("DEFAULT_BROKERS", "127.0.0.1:9092"),
|
|
84
84
|
}
|
|
85
85
|
|
|
86
86
|
return build_kafka_configuration(config)
|
|
@@ -231,6 +231,93 @@ class TestKafkaStreams(StreamsTestMixin[KafkaPayload]):
|
|
|
231
231
|
with pytest.raises(RuntimeError):
|
|
232
232
|
processor.run()
|
|
233
233
|
|
|
234
|
+
@mock.patch("arroyo.processing.processor.BACKPRESSURE_THRESHOLD", 0)
|
|
235
|
+
def test_assign_partition_during_pause(self) -> None:
|
|
236
|
+
if self.cooperative_sticky or self.kip_848:
|
|
237
|
+
pytest.skip("test does not work with cooperative-sticky rebalancing")
|
|
238
|
+
|
|
239
|
+
payloads = self.get_payloads()
|
|
240
|
+
|
|
241
|
+
strategy = mock.Mock()
|
|
242
|
+
strategy.submit.side_effect = MessageRejected()
|
|
243
|
+
factory = mock.Mock()
|
|
244
|
+
factory.create_with_partitions.return_value = strategy
|
|
245
|
+
|
|
246
|
+
partition_count = 2
|
|
247
|
+
|
|
248
|
+
with self.get_topic(partition_count) as topic, closing(
|
|
249
|
+
self.get_producer()
|
|
250
|
+
) as producer, closing(
|
|
251
|
+
self.get_consumer(
|
|
252
|
+
"test_assign_partition_during_pause", enable_end_of_partition=True
|
|
253
|
+
)
|
|
254
|
+
) as consumer_a, closing(
|
|
255
|
+
self.get_consumer(
|
|
256
|
+
"test_assign_partition_during_pause", enable_end_of_partition=True
|
|
257
|
+
)
|
|
258
|
+
) as consumer_b:
|
|
259
|
+
for i in range(partition_count):
|
|
260
|
+
producer.produce(Partition(topic, i), next(payloads)).result(
|
|
261
|
+
timeout=5.0
|
|
262
|
+
)
|
|
263
|
+
|
|
264
|
+
processor_a = StreamProcessor(
|
|
265
|
+
consumer_a, topic, factory, IMMEDIATE, handle_poll_while_paused=True
|
|
266
|
+
)
|
|
267
|
+
|
|
268
|
+
def wait_until_consumer_pauses(processor: StreamProcessor[Any]) -> None:
|
|
269
|
+
for _ in range(20):
|
|
270
|
+
try:
|
|
271
|
+
processor._run_once()
|
|
272
|
+
except EndOfPartition:
|
|
273
|
+
pass
|
|
274
|
+
|
|
275
|
+
if processor._StreamProcessor__is_paused: # type:ignore
|
|
276
|
+
return
|
|
277
|
+
raise RuntimeError("processor was not paused")
|
|
278
|
+
|
|
279
|
+
# calling _run_once will pause both consumers because of the MessageRejected strategy above
|
|
280
|
+
wait_until_consumer_pauses(processor_a)
|
|
281
|
+
|
|
282
|
+
# consumer A has all the partitions
|
|
283
|
+
assert len(consumer_a.tell()) == 2
|
|
284
|
+
assert len(consumer_b.tell()) == 0
|
|
285
|
+
|
|
286
|
+
# consumer A has all partitions paused (both from consumer and from
|
|
287
|
+
# StreamProcessor POV)
|
|
288
|
+
assert consumer_a.paused()
|
|
289
|
+
assert processor_a._StreamProcessor__is_paused is True # type:ignore
|
|
290
|
+
|
|
291
|
+
# subscribe with another consumer, now we should have rebalancing in the next few polls
|
|
292
|
+
processor_b = StreamProcessor(
|
|
293
|
+
consumer_b, topic, factory, IMMEDIATE, handle_poll_while_paused=True
|
|
294
|
+
)
|
|
295
|
+
|
|
296
|
+
for _ in range(10):
|
|
297
|
+
try:
|
|
298
|
+
processor_a._run_once()
|
|
299
|
+
except EndOfPartition:
|
|
300
|
+
pass
|
|
301
|
+
try:
|
|
302
|
+
processor_b._run_once()
|
|
303
|
+
except EndOfPartition:
|
|
304
|
+
pass
|
|
305
|
+
|
|
306
|
+
# balanced
|
|
307
|
+
assert len(consumer_a.tell()) == 1
|
|
308
|
+
assert len(consumer_b.tell()) == 1
|
|
309
|
+
|
|
310
|
+
# close B, but A has not polled yet, so it only has one partition still
|
|
311
|
+
consumer_b.close()
|
|
312
|
+
assert len(consumer_a.tell()) == 1
|
|
313
|
+
|
|
314
|
+
for _ in range(20):
|
|
315
|
+
try:
|
|
316
|
+
processor_a._run_once()
|
|
317
|
+
except EndOfPartition:
|
|
318
|
+
pass
|
|
319
|
+
assert len(consumer_a.tell()) == 2
|
|
320
|
+
|
|
234
321
|
def test_consumer_polls_when_paused(self) -> None:
|
|
235
322
|
strategy = mock.Mock()
|
|
236
323
|
factory = mock.Mock()
|
|
@@ -690,3 +690,75 @@ def test_processor_pause_with_invalid_message() -> None:
|
|
|
690
690
|
|
|
691
691
|
processor._run_once()
|
|
692
692
|
assert strategy.submit.call_args_list[-1] == mock.call(new_message)
|
|
693
|
+
|
|
694
|
+
def test_processor_poll_while_paused() -> None:
|
|
695
|
+
|
|
696
|
+
topic = Topic("topic")
|
|
697
|
+
|
|
698
|
+
consumer = mock.Mock()
|
|
699
|
+
strategy = mock.Mock()
|
|
700
|
+
factory = mock.Mock()
|
|
701
|
+
factory.create_with_partitions.return_value = strategy
|
|
702
|
+
|
|
703
|
+
processor: StreamProcessor[int] = StreamProcessor(
|
|
704
|
+
consumer, topic, factory, IMMEDIATE, handle_poll_while_paused=True
|
|
705
|
+
)
|
|
706
|
+
|
|
707
|
+
# Subscribe to topic
|
|
708
|
+
subscribe_args, subscribe_kwargs = consumer.subscribe.call_args
|
|
709
|
+
assert subscribe_args[0] == [topic]
|
|
710
|
+
|
|
711
|
+
# Partition assignment
|
|
712
|
+
partition = Partition(topic, 0)
|
|
713
|
+
new_partition = Partition(topic, 1)
|
|
714
|
+
consumer.tell.return_value = {}
|
|
715
|
+
assignment_callback = subscribe_kwargs["on_assign"]
|
|
716
|
+
offsets = {partition: 0}
|
|
717
|
+
assignment_callback(offsets)
|
|
718
|
+
|
|
719
|
+
# Message that we will get from polling
|
|
720
|
+
message = Message(BrokerValue(0, partition, 0, datetime.now()))
|
|
721
|
+
|
|
722
|
+
# Message will be rejected
|
|
723
|
+
consumer.poll.return_value = message.value
|
|
724
|
+
strategy.submit.side_effect = MessageRejected()
|
|
725
|
+
with assert_changes(lambda: int(consumer.pause.call_count), 0, 1):
|
|
726
|
+
processor._run_once()
|
|
727
|
+
assert strategy.submit.call_args_list[-1] == mock.call(message)
|
|
728
|
+
|
|
729
|
+
with mock.patch("time.time", return_value=time.time() + 5):
|
|
730
|
+
processor._run_once() # Should pause now
|
|
731
|
+
|
|
732
|
+
# Consumer is in paused state
|
|
733
|
+
# The same rejected message should be carried over
|
|
734
|
+
|
|
735
|
+
# All partitions are paused
|
|
736
|
+
consumer.paused.return_value = set(p for p in offsets)
|
|
737
|
+
# Simulate a continuous backpressure state where messages are being rejected
|
|
738
|
+
strategy.submit.side_effect = MessageRejected()
|
|
739
|
+
|
|
740
|
+
# Simulate Kafka returning nothing since the consumer is paused
|
|
741
|
+
consumer.poll.return_value = None
|
|
742
|
+
|
|
743
|
+
# The next poll returns nothing, but we are still carrying over the rejected message
|
|
744
|
+
processor._run_once()
|
|
745
|
+
assert consumer.poll.return_value is None
|
|
746
|
+
|
|
747
|
+
# At this point, let's say the message carried over is invalid (e.g. it could be stale)
|
|
748
|
+
strategy.submit.side_effect = InvalidMessage(partition, 0, needs_commit=False)
|
|
749
|
+
|
|
750
|
+
# Handles the invalid message and unpauses the consumer
|
|
751
|
+
with assert_changes(lambda: int(consumer.resume.call_count), 0, 1):
|
|
752
|
+
processor._run_once()
|
|
753
|
+
|
|
754
|
+
# Poll for the next message from Kafka, but this time the partition has changed
|
|
755
|
+
new_message = Message(BrokerValue(0, new_partition, 1, datetime.now()))
|
|
756
|
+
consumer.poll.return_value = new_message.value
|
|
757
|
+
processor._run_once()
|
|
758
|
+
assert processor._StreamProcessor__is_paused is False # type:ignore
|
|
759
|
+
|
|
760
|
+
strategy.submit.return_value = None
|
|
761
|
+
strategy.submit.side_effect = None
|
|
762
|
+
|
|
763
|
+
processor._run_once()
|
|
764
|
+
assert strategy.submit.call_args_list[-1] == mock.call(new_message)
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
{sentry_arroyo-2.32.4 → sentry_arroyo-2.32.5}/arroyo/processing/strategies/run_task_in_threads.py
RENAMED
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|
|
File without changes
|