4
4
import os
5
5
import signal
6
6
import warnings
7
- from typing import Optional , List , Callable , Union
7
+ from typing import Optional , List , Callable , Union , Literal , get_args
8
8
9
9
from confluent_kafka import TopicPartition
10
10
from typing_extensions import Self
44
44
__all__ = ("Application" ,)
45
45
46
46
logger = logging .getLogger (__name__ )
47
+ ProcessingGuarantee = Literal ["at-least-once" , "exactly-once" ]
47
48
MessageProcessedCallback = Callable [[str , int , int ], None ]
48
49
49
50
# Enforce idempotent producing for the internal RowProducer
@@ -78,7 +79,7 @@ class Application:
78
79
```python
79
80
from quixstreams import Application
80
81
81
- # Set up an `app = Application` and `sdf = StreamingDataFrame`;
82
+ # Set up an `app = Application` and `sdf = StreamingDataFrame`;
82
83
# add some operations to `sdf` and then run everything.
83
84
84
85
app = Application(broker_address='localhost:9092', consumer_group='group')
@@ -114,6 +115,7 @@ def __init__(
114
115
topic_manager : Optional [TopicManager ] = None ,
115
116
request_timeout : float = 30 ,
116
117
topic_create_timeout : float = 60 ,
118
+ processing_guarantee : ProcessingGuarantee = "at-least-once" ,
117
119
):
118
120
"""
119
121
:param broker_address: Connection settings for Kafka.
@@ -162,6 +164,7 @@ def __init__(
162
164
:param topic_manager: A `TopicManager` instance
163
165
:param request_timeout: timeout (seconds) for REST-based requests
164
166
:param topic_create_timeout: timeout (seconds) for topic create finalization
167
+ :param processing_guarantee: Use "exactly-once" or "at-least-once" processing.
165
168
166
169
<br><br>***Error Handlers***<br>
167
170
To handle errors, `Application` accepts callbacks triggered when
@@ -180,6 +183,13 @@ def __init__(
180
183
> NOTE: It is recommended to just use `quix_sdk_token` instead.
181
184
"""
182
185
configure_logging (loglevel = loglevel )
186
+
187
+ if processing_guarantee not in get_args (ProcessingGuarantee ):
188
+ raise ValueError (
189
+ f'Must provide a valid "processing_guarantee"; expected one of: '
190
+ f'{ get_args (ProcessingGuarantee )} , got "{ processing_guarantee } "'
191
+ )
192
+
183
193
producer_extra_config = producer_extra_config or {}
184
194
consumer_extra_config = consumer_extra_config or {}
185
195
@@ -248,6 +258,7 @@ def __init__(
248
258
self ._commit_interval = commit_interval
249
259
self ._producer_extra_config = producer_extra_config
250
260
self ._consumer_extra_config = consumer_extra_config
261
+ self ._processing_guarantee = processing_guarantee
251
262
self ._consumer = RowConsumer (
252
263
broker_address = broker_address ,
253
264
consumer_group = consumer_group ,
@@ -264,6 +275,7 @@ def __init__(
264
275
"max.poll.interval.ms" , _default_max_poll_interval_ms
265
276
)
266
277
/ 1000 , # convert to seconds
278
+ transactional = self ._uses_exactly_once ,
267
279
)
268
280
self ._consumer_poll_timeout = consumer_poll_timeout
269
281
self ._producer_poll_timeout = producer_poll_timeout
@@ -303,12 +315,17 @@ def __init__(
303
315
producer = self ._producer ,
304
316
consumer = self ._consumer ,
305
317
state_manager = self ._state_manager ,
318
+ exactly_once = self ._uses_exactly_once ,
306
319
)
307
320
308
321
@property
309
- def is_quix_app (self ):
322
+ def is_quix_app (self ) -> bool :
310
323
return self ._is_quix_app
311
324
325
+ @property
326
+ def _uses_exactly_once (self ) -> bool :
327
+ return self ._processing_guarantee == "exactly-once"
328
+
312
329
@classmethod
313
330
def Quix (
314
331
cls ,
@@ -331,6 +348,7 @@ def Quix(
331
348
topic_manager : Optional [QuixTopicManager ] = None ,
332
349
request_timeout : float = 30 ,
333
350
topic_create_timeout : float = 60 ,
351
+ processing_guarantee : Literal ["at-least-once" , "exactly-once" ] = "exactly-once" ,
334
352
) -> Self :
335
353
"""
336
354
>***NOTE:*** DEPRECATED: use Application with `quix_sdk_token` argument instead.
@@ -398,6 +416,7 @@ def Quix(
398
416
:param topic_manager: A `QuixTopicManager` instance
399
417
:param request_timeout: timeout (seconds) for REST-based requests
400
418
:param topic_create_timeout: timeout (seconds) for topic create finalization
419
+ :param processing_guarantee: Use "exactly-once" or "at-least-once" processing.
401
420
402
421
<br><br>***Error Handlers***<br>
403
422
To handle errors, `Application` accepts callbacks triggered when
@@ -445,6 +464,7 @@ def Quix(
445
464
request_timeout = request_timeout ,
446
465
topic_create_timeout = topic_create_timeout ,
447
466
quix_config_builder = quix_config_builder ,
467
+ processing_guarantee = processing_guarantee ,
448
468
)
449
469
return app
450
470
@@ -625,14 +645,18 @@ def get_consumer(self, auto_commit_enable: bool = True) -> Consumer:
625
645
Create and return a pre-configured Consumer instance.
626
646
The Consumer is initialized with params passed to Application.
627
647
628
- It's useful for consuming data from Kafka outside the standard Application processing flow.
629
- (e.g. to consume test data from a topic).
630
- Using it within the StreamingDataFrame functions is not recommended, as it creates a new Consumer instance
648
+ It's useful for consuming data from Kafka outside the standard
649
+ Application processing flow.
650
+ (e.g., to consume test data from a topic).
651
+ Using it within the StreamingDataFrame functions is not recommended, as it
652
+ creates a new Consumer instance
631
653
each time, which is not optimized for repeated use in a streaming pipeline.
632
654
633
- Note: By default this consumer does not autocommit consumed offsets to allow exactly-once processing.
655
+ Note: By default, this consumer does not autocommit the consumed offsets to allow
656
+ at-least-once processing.
634
657
To store the offset call store_offsets() after processing a message.
635
- If autocommit is necessary set `enable.auto.offset.store` to True in the consumer config when creating the app.
658
+ If autocommit is necessary set `enable.auto.offset.store` to True in
659
+ the consumer config when creating the app.
636
660
637
661
Example Snippet:
638
662
@@ -705,14 +729,16 @@ def run(
705
729
f'broker_address="{ self ._broker_address } " '
706
730
f'consumer_group="{ self ._consumer_group } " '
707
731
f'auto_offset_reset="{ self ._auto_offset_reset } " '
708
- f"commit_interval={ self ._commit_interval } s"
732
+ f"commit_interval={ self ._commit_interval } s "
733
+ f'processing_guarantee="{ self ._processing_guarantee } "'
709
734
)
710
735
if self .is_quix_app :
711
736
self ._quix_runtime_init ()
712
737
713
738
self ._setup_topics ()
714
739
715
740
exit_stack = contextlib .ExitStack ()
741
+ exit_stack .enter_context (self ._processing_context )
716
742
exit_stack .enter_context (self ._state_manager )
717
743
exit_stack .enter_context (self ._consumer )
718
744
exit_stack .push (
0 commit comments