sentry-arroyo 2.31.1__py3-none-any.whl → 2.32.0__py3-none-any.whl
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.
- arroyo/backends/abstract.py +4 -1
- arroyo/backends/kafka/configuration.py +20 -16
- arroyo/backends/kafka/consumer.py +107 -28
- arroyo/backends/local/backend.py +4 -4
- arroyo/processing/processor.py +17 -4
- {sentry_arroyo-2.31.1.dist-info → sentry_arroyo-2.32.0.dist-info}/METADATA +1 -1
- {sentry_arroyo-2.31.1.dist-info → sentry_arroyo-2.32.0.dist-info}/RECORD +12 -12
- tests/backends/test_confluent_producer.py +3 -0
- tests/backends/test_kafka.py +64 -0
- {sentry_arroyo-2.31.1.dist-info → sentry_arroyo-2.32.0.dist-info}/WHEEL +0 -0
- {sentry_arroyo-2.31.1.dist-info → sentry_arroyo-2.32.0.dist-info}/licenses/LICENSE +0 -0
- {sentry_arroyo-2.31.1.dist-info → sentry_arroyo-2.32.0.dist-info}/top_level.txt +0 -0
arroyo/backends/abstract.py
CHANGED
|
@@ -153,10 +153,13 @@ class Consumer(Generic[TStrategyPayload], ABC):
|
|
|
153
153
|
raise NotImplementedError
|
|
154
154
|
|
|
155
155
|
@abstractmethod
|
|
156
|
-
def commit_offsets(self) -> Mapping[Partition, int]:
|
|
156
|
+
def commit_offsets(self) -> Optional[Mapping[Partition, int]]:
|
|
157
157
|
"""
|
|
158
158
|
Commit staged offsets. The return value of this method is a mapping
|
|
159
159
|
of streams with their committed offsets as values.
|
|
160
|
+
|
|
161
|
+
When auto-commit is enabled (in Kafka consumers), returns None since
|
|
162
|
+
the broker handles commits automatically.
|
|
160
163
|
"""
|
|
161
164
|
raise NotImplementedError
|
|
162
165
|
|
|
@@ -237,6 +237,7 @@ def build_kafka_consumer_configuration(
|
|
|
237
237
|
bootstrap_servers: Optional[Sequence[str]] = None,
|
|
238
238
|
override_params: Optional[Mapping[str, Any]] = None,
|
|
239
239
|
strict_offset_reset: Optional[bool] = None,
|
|
240
|
+
enable_auto_commit: bool = False,
|
|
240
241
|
) -> KafkaBrokerConfig:
|
|
241
242
|
|
|
242
243
|
if auto_offset_reset is None:
|
|
@@ -252,20 +253,23 @@ def build_kafka_consumer_configuration(
|
|
|
252
253
|
default_config, bootstrap_servers, override_params
|
|
253
254
|
)
|
|
254
255
|
|
|
255
|
-
|
|
256
|
-
|
|
257
|
-
|
|
258
|
-
|
|
259
|
-
|
|
260
|
-
|
|
261
|
-
|
|
262
|
-
|
|
263
|
-
|
|
264
|
-
|
|
265
|
-
|
|
266
|
-
|
|
267
|
-
|
|
268
|
-
|
|
269
|
-
|
|
270
|
-
|
|
256
|
+
# Default configuration with manual commit management
|
|
257
|
+
config_update = {
|
|
258
|
+
"enable.auto.commit": False,
|
|
259
|
+
"enable.auto.offset.store": False,
|
|
260
|
+
"group.id": group_id,
|
|
261
|
+
"auto.offset.reset": auto_offset_reset,
|
|
262
|
+
# this is an arroyo specific flag that only affects the consumer.
|
|
263
|
+
"arroyo.strict.offset.reset": strict_offset_reset,
|
|
264
|
+
# this is an arroyo specific flag to enable auto-commit mode
|
|
265
|
+
"arroyo.enable.auto.commit": enable_auto_commit,
|
|
266
|
+
# overridden to reduce memory usage when there's a large backlog
|
|
267
|
+
"queued.max.messages.kbytes": queued_max_messages_kbytes,
|
|
268
|
+
"queued.min.messages": queued_min_messages,
|
|
269
|
+
"enable.partition.eof": False,
|
|
270
|
+
"statistics.interval.ms": STATS_COLLECTION_FREQ_MS,
|
|
271
|
+
"stats_cb": stats_callback,
|
|
272
|
+
}
|
|
273
|
+
|
|
274
|
+
broker_config.update(config_update)
|
|
271
275
|
return broker_config
|
|
@@ -1,6 +1,8 @@
|
|
|
1
1
|
from __future__ import annotations
|
|
2
2
|
|
|
3
3
|
import logging
|
|
4
|
+
import time
|
|
5
|
+
from collections import defaultdict
|
|
4
6
|
from concurrent.futures import Future
|
|
5
7
|
from datetime import datetime
|
|
6
8
|
from enum import Enum
|
|
@@ -21,6 +23,7 @@ from typing import (
|
|
|
21
23
|
Tuple,
|
|
22
24
|
Type,
|
|
23
25
|
Union,
|
|
26
|
+
cast,
|
|
24
27
|
)
|
|
25
28
|
|
|
26
29
|
from confluent_kafka import (
|
|
@@ -183,6 +186,13 @@ class KafkaConsumer(Consumer[KafkaPayload]):
|
|
|
183
186
|
if self.__strict_offset_reset is None:
|
|
184
187
|
self.__strict_offset_reset = True
|
|
185
188
|
|
|
189
|
+
# Feature flag to enable rdkafka auto-commit with store_offsets
|
|
190
|
+
# When enabled, offsets are stored via store_offsets() and rdkafka
|
|
191
|
+
# automatically commits them periodically
|
|
192
|
+
self.__use_auto_commit = as_kafka_configuration_bool(
|
|
193
|
+
configuration.pop("arroyo.enable.auto.commit", False)
|
|
194
|
+
)
|
|
195
|
+
|
|
186
196
|
if auto_offset_reset in {"smallest", "earliest", "beginning"}:
|
|
187
197
|
self.__resolve_partition_starting_offset = (
|
|
188
198
|
self.__resolve_partition_offset_earliest
|
|
@@ -198,21 +208,32 @@ class KafkaConsumer(Consumer[KafkaPayload]):
|
|
|
198
208
|
else:
|
|
199
209
|
raise ValueError("invalid value for 'auto.offset.reset' configuration")
|
|
200
210
|
|
|
201
|
-
|
|
202
|
-
|
|
203
|
-
|
|
204
|
-
|
|
205
|
-
|
|
211
|
+
# When auto-commit is disabled (default), we require explicit configuration
|
|
212
|
+
# When auto-commit is enabled, we allow rdkafka to handle commits
|
|
213
|
+
if not self.__use_auto_commit:
|
|
214
|
+
if (
|
|
215
|
+
as_kafka_configuration_bool(
|
|
216
|
+
configuration.get("enable.auto.commit", "true")
|
|
217
|
+
)
|
|
218
|
+
is not False
|
|
219
|
+
):
|
|
220
|
+
raise ValueError("invalid value for 'enable.auto.commit' configuration")
|
|
206
221
|
|
|
207
|
-
|
|
208
|
-
|
|
209
|
-
|
|
210
|
-
|
|
211
|
-
|
|
212
|
-
|
|
213
|
-
|
|
214
|
-
|
|
215
|
-
|
|
222
|
+
if (
|
|
223
|
+
as_kafka_configuration_bool(
|
|
224
|
+
configuration.get("enable.auto.offset.store", "true")
|
|
225
|
+
)
|
|
226
|
+
is not False
|
|
227
|
+
):
|
|
228
|
+
raise ValueError(
|
|
229
|
+
"invalid value for 'enable.auto.offset.store' configuration"
|
|
230
|
+
)
|
|
231
|
+
else:
|
|
232
|
+
# In auto-commit mode, enable auto.commit and keep auto.offset.store disabled
|
|
233
|
+
# We'll use store_offsets() manually to control which offsets get committed
|
|
234
|
+
configuration["enable.auto.commit"] = True
|
|
235
|
+
configuration["enable.auto.offset.store"] = False
|
|
236
|
+
configuration["on_commit"] = self.__on_commit_callback
|
|
216
237
|
|
|
217
238
|
# NOTE: Offsets are explicitly managed as part of the assignment
|
|
218
239
|
# callback, so preemptively resetting offsets is not enabled when
|
|
@@ -232,6 +253,19 @@ class KafkaConsumer(Consumer[KafkaPayload]):
|
|
|
232
253
|
|
|
233
254
|
self.__state = KafkaConsumerState.CONSUMING
|
|
234
255
|
|
|
256
|
+
def __on_commit_callback(
|
|
257
|
+
self,
|
|
258
|
+
error: Optional[KafkaException],
|
|
259
|
+
partitions: Sequence[ConfluentTopicPartition],
|
|
260
|
+
) -> None:
|
|
261
|
+
if error:
|
|
262
|
+
partition_info = [f"{p.topic}:{p.partition}" for p in partitions]
|
|
263
|
+
logger.warning(
|
|
264
|
+
"Commit failed: %s. Partitions: %s",
|
|
265
|
+
error,
|
|
266
|
+
partition_info,
|
|
267
|
+
)
|
|
268
|
+
|
|
235
269
|
def __resolve_partition_offset_earliest(
|
|
236
270
|
self, partition: ConfluentTopicPartition
|
|
237
271
|
) -> ConfluentTopicPartition:
|
|
@@ -569,7 +603,21 @@ class KafkaConsumer(Consumer[KafkaPayload]):
|
|
|
569
603
|
# TODO: Maybe log a warning if these offsets exceed the current
|
|
570
604
|
# offsets, since that's probably a side effect of an incorrect usage
|
|
571
605
|
# pattern?
|
|
572
|
-
self.
|
|
606
|
+
if self.__use_auto_commit:
|
|
607
|
+
# When auto-commit is enabled, use store_offsets to stage offsets
|
|
608
|
+
# for rdkafka to auto-commit
|
|
609
|
+
if offsets:
|
|
610
|
+
self.__consumer.store_offsets(
|
|
611
|
+
offsets=[
|
|
612
|
+
ConfluentTopicPartition(
|
|
613
|
+
partition.topic.name, partition.index, offset
|
|
614
|
+
)
|
|
615
|
+
for partition, offset in offsets.items()
|
|
616
|
+
]
|
|
617
|
+
)
|
|
618
|
+
else:
|
|
619
|
+
# Default behavior: manually track staged offsets
|
|
620
|
+
self.__staged_offsets.update(offsets)
|
|
573
621
|
|
|
574
622
|
def __commit(self) -> Mapping[Partition, int]:
|
|
575
623
|
if self.__state in {KafkaConsumerState.CLOSED, KafkaConsumerState.ERROR}:
|
|
@@ -617,15 +665,24 @@ class KafkaConsumer(Consumer[KafkaPayload]):
|
|
|
617
665
|
|
|
618
666
|
return offsets
|
|
619
667
|
|
|
620
|
-
def commit_offsets(self) -> Mapping[Partition, int]:
|
|
668
|
+
def commit_offsets(self) -> Optional[Mapping[Partition, int]]:
|
|
621
669
|
"""
|
|
622
670
|
Commit staged offsets for all partitions that this consumer is
|
|
623
671
|
assigned to. The return value of this method is a mapping of
|
|
624
672
|
partitions with their committed offsets as values.
|
|
625
673
|
|
|
674
|
+
When auto-commit is enabled, returns None since rdkafka handles
|
|
675
|
+
commits automatically and we don't track which offsets were committed.
|
|
676
|
+
|
|
626
677
|
Raises an ``InvalidState`` if called on a closed consumer.
|
|
627
678
|
"""
|
|
628
|
-
|
|
679
|
+
if self.__use_auto_commit:
|
|
680
|
+
# When auto-commit is enabled, rdkafka commits automatically
|
|
681
|
+
# We don't track what was committed, so return None
|
|
682
|
+
# The offsets have already been staged via store_offsets()
|
|
683
|
+
return None
|
|
684
|
+
else:
|
|
685
|
+
return self.__commit_retry_policy.call(self.__commit)
|
|
629
686
|
|
|
630
687
|
def close(self, timeout: Optional[float] = None) -> None:
|
|
631
688
|
"""
|
|
@@ -748,6 +805,9 @@ class KafkaProducer(Producer[KafkaPayload]):
|
|
|
748
805
|
# Type alias for the delivery callback function
|
|
749
806
|
DeliveryCallback = Callable[[Optional[KafkaError], ConfluentMessage], None]
|
|
750
807
|
|
|
808
|
+
# Interval between metric flushes (in seconds)
|
|
809
|
+
METRICS_FREQUENCY_SEC = 1.0
|
|
810
|
+
|
|
751
811
|
|
|
752
812
|
class ConfluentProducer(ConfluentKafkaProducer): # type: ignore[misc]
|
|
753
813
|
"""
|
|
@@ -756,8 +816,10 @@ class ConfluentProducer(ConfluentKafkaProducer): # type: ignore[misc]
|
|
|
756
816
|
|
|
757
817
|
def __init__(self, configuration: Mapping[str, Any]) -> None:
|
|
758
818
|
super().__init__(configuration)
|
|
759
|
-
self.__metrics = get_metrics()
|
|
760
819
|
self.producer_name = configuration.get("client.id") or None
|
|
820
|
+
self.__metrics = get_metrics()
|
|
821
|
+
self.__produce_counters: MutableMapping[str, int] = defaultdict(int)
|
|
822
|
+
self.__reset_metrics()
|
|
761
823
|
|
|
762
824
|
def __metrics_delivery_callback(
|
|
763
825
|
self,
|
|
@@ -768,16 +830,8 @@ class ConfluentProducer(ConfluentKafkaProducer): # type: ignore[misc]
|
|
|
768
830
|
status = "error"
|
|
769
831
|
else:
|
|
770
832
|
status = "success"
|
|
771
|
-
|
|
772
|
-
|
|
773
|
-
if self.producer_name:
|
|
774
|
-
tags["producer_name"] = self.producer_name
|
|
775
|
-
|
|
776
|
-
self.__metrics.increment(
|
|
777
|
-
"arroyo.producer.produce_status",
|
|
778
|
-
1,
|
|
779
|
-
tags=tags,
|
|
780
|
-
)
|
|
833
|
+
self.__produce_counters[status] += 1
|
|
834
|
+
self.__throttled_record()
|
|
781
835
|
|
|
782
836
|
def __delivery_callback(
|
|
783
837
|
self,
|
|
@@ -797,3 +851,28 @@ class ConfluentProducer(ConfluentKafkaProducer): # type: ignore[misc]
|
|
|
797
851
|
user_callback = callback or on_delivery
|
|
798
852
|
wrapped_callback = self.__delivery_callback(user_callback)
|
|
799
853
|
super().produce(*args, on_delivery=wrapped_callback, **kwargs)
|
|
854
|
+
|
|
855
|
+
def __flush_metrics(self) -> None:
|
|
856
|
+
for status, count in self.__produce_counters.items():
|
|
857
|
+
tags = {"status": status}
|
|
858
|
+
if self.producer_name:
|
|
859
|
+
tags["producer_name"] = self.producer_name
|
|
860
|
+
self.__metrics.increment(
|
|
861
|
+
name="arroyo.producer.produce_status",
|
|
862
|
+
value=count,
|
|
863
|
+
tags=tags,
|
|
864
|
+
)
|
|
865
|
+
self.__reset_metrics()
|
|
866
|
+
|
|
867
|
+
def flush(self, timeout: float = -1) -> int:
|
|
868
|
+
# Kafka producer flush should flush metrics too
|
|
869
|
+
self.__flush_metrics()
|
|
870
|
+
return cast(int, super().flush(timeout))
|
|
871
|
+
|
|
872
|
+
def __reset_metrics(self) -> None:
|
|
873
|
+
self.__produce_counters.clear()
|
|
874
|
+
self.__last_record_time = time.time()
|
|
875
|
+
|
|
876
|
+
def __throttled_record(self) -> None:
|
|
877
|
+
if time.time() - self.__last_record_time > METRICS_FREQUENCY_SEC:
|
|
878
|
+
self.__flush_metrics()
|
arroyo/backends/local/backend.py
CHANGED
|
@@ -38,9 +38,9 @@ class LocalBroker(Generic[TStrategyPayload]):
|
|
|
38
38
|
self.__message_storage = message_storage
|
|
39
39
|
self.__clock = clock
|
|
40
40
|
|
|
41
|
-
self.__offsets: MutableMapping[
|
|
42
|
-
|
|
43
|
-
)
|
|
41
|
+
self.__offsets: MutableMapping[
|
|
42
|
+
str, MutableMapping[Partition, int]
|
|
43
|
+
] = defaultdict(dict)
|
|
44
44
|
|
|
45
45
|
# The active subscriptions are stored by consumer group as a mapping
|
|
46
46
|
# between the consumer and it's subscribed topics.
|
|
@@ -326,7 +326,7 @@ class LocalConsumer(Consumer[TStrategyPayload]):
|
|
|
326
326
|
# atomic
|
|
327
327
|
self.__staged_offsets.update(offsets)
|
|
328
328
|
|
|
329
|
-
def commit_offsets(self) -> Mapping[Partition, int]:
|
|
329
|
+
def commit_offsets(self) -> Optional[Mapping[Partition, int]]:
|
|
330
330
|
with self.__lock:
|
|
331
331
|
if self.__closed:
|
|
332
332
|
raise RuntimeError("consumer is closed")
|
arroyo/processing/processor.py
CHANGED
|
@@ -146,9 +146,9 @@ class StreamProcessor(Generic[TStrategyPayload]):
|
|
|
146
146
|
self.__processor_factory = processor_factory
|
|
147
147
|
self.__metrics_buffer = MetricsBuffer()
|
|
148
148
|
|
|
149
|
-
self.__processing_strategy: Optional[
|
|
150
|
-
|
|
151
|
-
|
|
149
|
+
self.__processing_strategy: Optional[ProcessingStrategy[TStrategyPayload]] = (
|
|
150
|
+
None
|
|
151
|
+
)
|
|
152
152
|
|
|
153
153
|
self.__message: Optional[BrokerValue[TStrategyPayload]] = None
|
|
154
154
|
|
|
@@ -465,8 +465,9 @@ class StreamProcessor(Generic[TStrategyPayload]):
|
|
|
465
465
|
|
|
466
466
|
elif self.__is_paused:
|
|
467
467
|
paused_partitions = set(self.__consumer.paused())
|
|
468
|
+
all_partitions = set(self.__consumer.tell())
|
|
468
469
|
unpaused_partitions = (
|
|
469
|
-
|
|
470
|
+
all_partitions - paused_partitions
|
|
470
471
|
)
|
|
471
472
|
if unpaused_partitions:
|
|
472
473
|
logger.warning(
|
|
@@ -484,6 +485,18 @@ class StreamProcessor(Generic[TStrategyPayload]):
|
|
|
484
485
|
# A paused consumer should still poll periodically to avoid it's partitions
|
|
485
486
|
# getting revoked by the broker after reaching the max.poll.interval.ms
|
|
486
487
|
# Polling a paused consumer should never yield a message.
|
|
488
|
+
logger.warning("consumer.tell() value right before poll() is: %s", self.__consumer.tell())
|
|
489
|
+
maybe_message = self.__consumer.poll(0.1)
|
|
490
|
+
if maybe_message is not None:
|
|
491
|
+
logger.warning("Received a message from partition: %s, \
|
|
492
|
+
consumer.tell() value right after poll() is: %s \
|
|
493
|
+
Some lines above consumer.tell() was called, all_partitons value was: %s \
|
|
494
|
+
Some lines above consumer.paused() was called, paused_partitions value is: %s",
|
|
495
|
+
maybe_message.partition,
|
|
496
|
+
self.__consumer.tell(),
|
|
497
|
+
all_partitions,
|
|
498
|
+
paused_partitions
|
|
499
|
+
)
|
|
487
500
|
assert self.__consumer.poll(0.1) is None
|
|
488
501
|
else:
|
|
489
502
|
time.sleep(0.01)
|
|
@@ -5,18 +5,18 @@ arroyo/errors.py,sha256=IbtoIbz_m5QrxNRBLOxiy-hOfJQTEwNPCyq6yqedJYk,1059
|
|
|
5
5
|
arroyo/py.typed,sha256=47DEQpj8HBSa-_TImW-5JCeuQeRkm5NMpJWZG3hSuFU,0
|
|
6
6
|
arroyo/types.py,sha256=sLY0x030np4UmbaW5C1KH1se7Z2pjQiPvAe5x2sXf7A,5684
|
|
7
7
|
arroyo/backends/__init__.py,sha256=47DEQpj8HBSa-_TImW-5JCeuQeRkm5NMpJWZG3hSuFU,0
|
|
8
|
-
arroyo/backends/abstract.py,sha256=
|
|
8
|
+
arroyo/backends/abstract.py,sha256=Wy9xhE1dtFiumG8Cz3JhksJ0rF74uJWZWq10UO1rxOI,9524
|
|
9
9
|
arroyo/backends/kafka/__init__.py,sha256=xgf-AqHbQkJsh73YokO2uoyyHfZf8XwUp6BULtM8stI,445
|
|
10
10
|
arroyo/backends/kafka/commit.py,sha256=LPsjvX5PPXR62DT6sa5GuSF78qk9F_L--Fz4kw7-m-s,3060
|
|
11
|
-
arroyo/backends/kafka/configuration.py,sha256=
|
|
12
|
-
arroyo/backends/kafka/consumer.py,sha256=
|
|
11
|
+
arroyo/backends/kafka/configuration.py,sha256=zB54w7qsyVeMVkH5MpV6F8ztXfEzIXrex6aKYX-GcqA,9141
|
|
12
|
+
arroyo/backends/kafka/consumer.py,sha256=zZ2ZoDaurLDBN9l9QR0fFWL16RJcf0D8Apaa3aff22k,33534
|
|
13
13
|
arroyo/backends/local/__init__.py,sha256=47DEQpj8HBSa-_TImW-5JCeuQeRkm5NMpJWZG3hSuFU,0
|
|
14
|
-
arroyo/backends/local/backend.py,sha256=
|
|
14
|
+
arroyo/backends/local/backend.py,sha256=hUXdCV6B5e7s4mjFC6HnIuUhjENU2tNZt5vuEOJmGZQ,13888
|
|
15
15
|
arroyo/backends/local/storages/__init__.py,sha256=AGYujdAAcn3osoj9jq84IzTywYbkIDv9wRg2rLhLXeg,104
|
|
16
16
|
arroyo/backends/local/storages/abstract.py,sha256=1qVQp6roxHkK6XT2aklZyZk1qq7RzcPN6Db_CA5--kg,2901
|
|
17
17
|
arroyo/backends/local/storages/memory.py,sha256=AoKDsVZzBXkOJyWArKWp3vfGfU9xLlKFXE9gsJiMIzQ,2613
|
|
18
18
|
arroyo/processing/__init__.py,sha256=vZVg0wJvJfoVzlzGvnL59bT6YNIRJNQ5t7oU045Qbk4,87
|
|
19
|
-
arroyo/processing/processor.py,sha256=
|
|
19
|
+
arroyo/processing/processor.py,sha256=BtNaIxBApuUAtSH-syGJnpeKADHUafut9Ve1KMe8JM0,22389
|
|
20
20
|
arroyo/processing/strategies/__init__.py,sha256=EU_JMb54eOxMxaC5mIFpI-sAF-X2ZScbE8czBZ7bQkY,1106
|
|
21
21
|
arroyo/processing/strategies/abstract.py,sha256=nu7juEz_aQmQIH35Z8u--FBuLjkK8_LQ1hIG2xpw9AA,4808
|
|
22
22
|
arroyo/processing/strategies/batching.py,sha256=s89xC6lQpBseEaApu1iNTipXGKeO95OMwinj2VBKn9s,4778
|
|
@@ -46,12 +46,12 @@ examples/transform_and_produce/__init__.py,sha256=47DEQpj8HBSa-_TImW-5JCeuQeRkm5
|
|
|
46
46
|
examples/transform_and_produce/batched.py,sha256=st2R6qTneAtV0JFbKP30Ti3sJDYj8Jkbmta9JckKdZU,2636
|
|
47
47
|
examples/transform_and_produce/script.py,sha256=8kSMIjQNqGYEVyE0PvrfJh-a_UYCrJSstTp_De7kyyg,2306
|
|
48
48
|
examples/transform_and_produce/simple.py,sha256=H7xqxItjl4tx34wVW5dy6mB9G39QucAtxkJSBzVmjgA,1637
|
|
49
|
-
sentry_arroyo-2.
|
|
49
|
+
sentry_arroyo-2.32.0.dist-info/licenses/LICENSE,sha256=0Ng3MFdEcnz0sVD1XvGBBzbavvNp_7OAM5yVObB46jU,10829
|
|
50
50
|
tests/backends/__init__.py,sha256=47DEQpj8HBSa-_TImW-5JCeuQeRkm5NMpJWZG3hSuFU,0
|
|
51
51
|
tests/backends/mixins.py,sha256=sfNyE0VTeiD3GHOnBYl-9urvPuURI2G1BWke0cz7Dvc,20445
|
|
52
52
|
tests/backends/test_commit.py,sha256=iTHfK1qsBxim0XwxgMvNNSMqDUMEHoYkYBDcgxGBFbs,831
|
|
53
|
-
tests/backends/test_confluent_producer.py,sha256=
|
|
54
|
-
tests/backends/test_kafka.py,sha256=
|
|
53
|
+
tests/backends/test_confluent_producer.py,sha256=KWqgvjDvqAdd0HxngdWKsUJaV7Hl1L5vAVQhBYlHeHU,3146
|
|
54
|
+
tests/backends/test_kafka.py,sha256=wBFCKEHoP6h0uG1bgDuzk84IZmrV_UVOFCrtbxztmJg,15506
|
|
55
55
|
tests/backends/test_kafka_producer.py,sha256=LpwkqnstcCDxemlKZ0FpzNKrP-1UuXXY15P7P-spjhE,3912
|
|
56
56
|
tests/backends/test_local.py,sha256=Mfd4DFuWVSVtl1GomQ6TIoWuJNcAliKqKU0BShPlEMY,3363
|
|
57
57
|
tests/processing/__init__.py,sha256=47DEQpj8HBSa-_TImW-5JCeuQeRkm5NMpJWZG3hSuFU,0
|
|
@@ -74,7 +74,7 @@ tests/utils/__init__.py,sha256=47DEQpj8HBSa-_TImW-5JCeuQeRkm5NMpJWZG3hSuFU,0
|
|
|
74
74
|
tests/utils/test_concurrent.py,sha256=Gwdzym2UZ1HO3rhOSGmzxImWcLFygY8P7MXHT3Q0xTE,455
|
|
75
75
|
tests/utils/test_metrics.py,sha256=bI0EtGgPokMQyEqX58i0-8zvLfxRP2nWaWr2wLMaJ_o,917
|
|
76
76
|
tests/utils/test_retries.py,sha256=AxJLkXWeL9AjHv_p1n0pe8CXXJp24ZQIuYBHfNcmiz4,3075
|
|
77
|
-
sentry_arroyo-2.
|
|
78
|
-
sentry_arroyo-2.
|
|
79
|
-
sentry_arroyo-2.
|
|
80
|
-
sentry_arroyo-2.
|
|
77
|
+
sentry_arroyo-2.32.0.dist-info/METADATA,sha256=mzNqLp5VxLgwXIgYJQUGelVVtP_Vy5RCCCrNiOEk2FU,2208
|
|
78
|
+
sentry_arroyo-2.32.0.dist-info/WHEEL,sha256=_zCd3N1l69ArxyTb8rzEoP9TpbYXkqRFSNOD5OuxnTs,91
|
|
79
|
+
sentry_arroyo-2.32.0.dist-info/top_level.txt,sha256=DVdMZKysL_iIxm5aY0sYgZtP5ZXMg9YBaBmGQHVmDXA,22
|
|
80
|
+
sentry_arroyo-2.32.0.dist-info/RECORD,,
|
|
@@ -29,6 +29,7 @@ class TestConfluentProducer:
|
|
|
29
29
|
)
|
|
30
30
|
mock_message = mock.Mock(spec=ConfluentMessage)
|
|
31
31
|
producer._ConfluentProducer__metrics_delivery_callback(None, mock_message)
|
|
32
|
+
producer.flush() # Flush buffered metrics
|
|
32
33
|
assert (
|
|
33
34
|
Increment(
|
|
34
35
|
"arroyo.producer.produce_status",
|
|
@@ -44,6 +45,7 @@ class TestConfluentProducer:
|
|
|
44
45
|
mock_error = mock.Mock(spec=KafkaError)
|
|
45
46
|
mock_message = mock.Mock(spec=ConfluentMessage)
|
|
46
47
|
producer._ConfluentProducer__metrics_delivery_callback(mock_error, mock_message)
|
|
48
|
+
producer.flush() # Flush buffered metrics
|
|
47
49
|
assert (
|
|
48
50
|
Increment("arroyo.producer.produce_status", 1, {"status": "error"})
|
|
49
51
|
in TestingMetricsBackend.calls
|
|
@@ -64,6 +66,7 @@ class TestConfluentProducer:
|
|
|
64
66
|
wrapped = producer._ConfluentProducer__delivery_callback(user_callback)
|
|
65
67
|
mock_message = mock.Mock(spec=ConfluentMessage)
|
|
66
68
|
wrapped(None, mock_message)
|
|
69
|
+
producer.flush() # Flush buffered metrics
|
|
67
70
|
assert (
|
|
68
71
|
Increment(
|
|
69
72
|
"arroyo.producer.produce_status",
|
tests/backends/test_kafka.py
CHANGED
|
@@ -275,6 +275,70 @@ class TestKafkaStreams(StreamsTestMixin[KafkaPayload]):
|
|
|
275
275
|
processor._run_once()
|
|
276
276
|
assert consumer.paused() == []
|
|
277
277
|
|
|
278
|
+
def test_auto_commit_mode(self) -> None:
|
|
279
|
+
"""Test that auto-commit mode uses store_offsets and commits on close"""
|
|
280
|
+
group_id = uuid.uuid1().hex
|
|
281
|
+
|
|
282
|
+
with self.get_topic() as topic:
|
|
283
|
+
# Produce some messages
|
|
284
|
+
with closing(self.get_producer()) as producer:
|
|
285
|
+
for i in range(5):
|
|
286
|
+
payload = KafkaPayload(None, f"msg_{i}".encode("utf8"), [])
|
|
287
|
+
producer.produce(topic, payload).result(5.0)
|
|
288
|
+
|
|
289
|
+
# Create consumer with auto-commit enabled
|
|
290
|
+
configuration = {
|
|
291
|
+
**self.configuration,
|
|
292
|
+
"auto.offset.reset": "earliest",
|
|
293
|
+
"arroyo.enable.auto.commit": True,
|
|
294
|
+
"group.id": group_id,
|
|
295
|
+
"session.timeout.ms": 10000,
|
|
296
|
+
}
|
|
297
|
+
|
|
298
|
+
# First consumer: consume messages and close
|
|
299
|
+
consumed_offsets = []
|
|
300
|
+
with closing(KafkaConsumer(configuration)) as consumer:
|
|
301
|
+
consumer.subscribe([topic])
|
|
302
|
+
|
|
303
|
+
# Consume all 5 messages and stage their offsets
|
|
304
|
+
for i in range(5):
|
|
305
|
+
value = consumer.poll(10.0)
|
|
306
|
+
assert value is not None
|
|
307
|
+
consumed_offsets.append(value.offset)
|
|
308
|
+
|
|
309
|
+
# Stage offsets (will use store_offsets internally in auto-commit mode)
|
|
310
|
+
consumer.stage_offsets(value.committable)
|
|
311
|
+
|
|
312
|
+
# commit_offsets should return None in auto-commit mode
|
|
313
|
+
result = consumer.commit_offsets()
|
|
314
|
+
assert result is None
|
|
315
|
+
|
|
316
|
+
# Close will commit any stored offsets
|
|
317
|
+
|
|
318
|
+
# Verify we consumed offsets 0-4
|
|
319
|
+
assert consumed_offsets == [0, 1, 2, 3, 4]
|
|
320
|
+
|
|
321
|
+
# Second consumer: verify offsets were committed on close
|
|
322
|
+
# This consumer uses manual commit to verify the committed offset
|
|
323
|
+
with closing(
|
|
324
|
+
self.get_consumer(
|
|
325
|
+
group=group_id,
|
|
326
|
+
auto_offset_reset="earliest",
|
|
327
|
+
enable_end_of_partition=True,
|
|
328
|
+
)
|
|
329
|
+
) as consumer:
|
|
330
|
+
consumer.subscribe([topic])
|
|
331
|
+
|
|
332
|
+
# Should start from offset 5, hitting EndOfPartition immediately
|
|
333
|
+
# If we got a message with offset < 5, auto-commit didn't work
|
|
334
|
+
try:
|
|
335
|
+
consumer.poll(10.0)
|
|
336
|
+
pytest.fail("Expected EndOfPartition, but poll succeeded")
|
|
337
|
+
except EndOfPartition as e:
|
|
338
|
+
# Verify we got EndOfPartition at offset 5
|
|
339
|
+
assert e.offset == 5
|
|
340
|
+
assert e.partition == Partition(topic, 0)
|
|
341
|
+
|
|
278
342
|
|
|
279
343
|
class TestKafkaStreamsIncrementalRebalancing(TestKafkaStreams):
|
|
280
344
|
# re-test the kafka consumer with cooperative-sticky rebalancing
|
|
File without changes
|
|
File without changes
|
|
File without changes
|