69
69
# Enforce idempotent producing for the internal RowProducer
70
70
_default_producer_extra_config = {"enable.idempotence" : True }
71
71
72
+ # Default config for the internal consumer
73
+ _default_consumer_extra_config = {
74
+ "fetch.queue.backoff.ms" : 100 , # Make the consumer to fetch data more often
75
+ }
76
+
72
77
# Force assignment strategy to be "range" for co-partitioning in internal Consumers
73
78
consumer_extra_config_overrides = {"partition.assignment.strategy" : "range" }
74
79
@@ -151,6 +156,7 @@ def __init__(
151
156
request_timeout : float = 30 ,
152
157
topic_create_timeout : float = 60 ,
153
158
processing_guarantee : ProcessingGuarantee = "at-least-once" ,
159
+ max_partition_buffer_size : int = 10000 ,
154
160
):
155
161
"""
156
162
:param broker_address: Connection settings for Kafka.
@@ -210,6 +216,11 @@ def __init__(
210
216
:param request_timeout: timeout (seconds) for REST-based requests
211
217
:param topic_create_timeout: timeout (seconds) for topic create finalization
212
218
:param processing_guarantee: Use "exactly-once" or "at-least-once" processing.
219
+ :param max_partition_buffer_size: the maximum number of messages to buffer per topic partition to consider it full.
220
+ The buffering is used to consume messages in-order between multiple partitions with the same number.
221
+ It is a soft limit, and the actual number of buffered messages can be up to x2 higher.
222
+ Lower value decreases the memory use, but increases the latency.
223
+ Default - `10000`.
213
224
214
225
<br><br>***Error Handlers***<br>
215
226
To handle errors, `Application` accepts callbacks triggered when
@@ -305,7 +316,10 @@ def __init__(
305
316
** _default_producer_extra_config ,
306
317
** producer_extra_config ,
307
318
},
308
- consumer_extra_config = consumer_extra_config ,
319
+ consumer_extra_config = {
320
+ ** _default_consumer_extra_config ,
321
+ ** consumer_extra_config ,
322
+ },
309
323
processing_guarantee = processing_guarantee ,
310
324
consumer_poll_timeout = consumer_poll_timeout ,
311
325
producer_poll_timeout = producer_poll_timeout ,
@@ -315,6 +329,7 @@ def __init__(
315
329
state_dir = state_dir ,
316
330
rocksdb_options = rocksdb_options ,
317
331
use_changelog_topics = use_changelog_topics ,
332
+ max_partition_buffer_size = max_partition_buffer_size ,
318
333
)
319
334
320
335
self ._on_message_processed = on_message_processed
@@ -634,6 +649,7 @@ def _get_rowconsumer(
634
649
consumer_group = self ._config .consumer_group ,
635
650
auto_offset_reset = self ._config .auto_offset_reset ,
636
651
auto_commit_enable = False , # Disable auto commit and manage commits manually
652
+ max_partition_buffer_size = self ._config .max_partition_buffer_size ,
637
653
extra_config = extra_config ,
638
654
on_error = on_error ,
639
655
)
@@ -905,7 +921,10 @@ def _quix_runtime_init(self):
905
921
def _process_message (self , dataframe_composed ):
906
922
# Serve producer callbacks
907
923
self ._producer .poll (self ._config .producer_poll_timeout )
908
- rows = self ._consumer .poll_row (timeout = self ._config .consumer_poll_timeout )
924
+ rows = self ._consumer .poll_row (
925
+ timeout = self ._config .consumer_poll_timeout ,
926
+ buffered = self ._dataframe_registry .requires_time_alignment ,
927
+ )
909
928
910
929
if rows is None :
911
930
self ._run_tracker .set_current_message_tp (None )
@@ -1100,6 +1119,7 @@ class ApplicationConfig(BaseSettings):
1100
1119
state_dir : Path = Path ("state" )
1101
1120
rocksdb_options : Optional [RocksDBOptionsType ] = None
1102
1121
use_changelog_topics : bool = True
1122
+ max_partition_buffer_size : int = 10000
1103
1123
1104
1124
@classmethod
1105
1125
def settings_customise_sources (
0 commit comments