Skip to content

Commit 63272b9

Browse files
authored
"range" assignment updates (#712)
* Don't commit the checkpoint on assign * Enforce "range" assignor for internal consumers
1 parent 6837de7 commit 63272b9

File tree

3 files changed

+23
-10
lines changed

3 files changed

+23
-10
lines changed

quixstreams/app.py

Lines changed: 20 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -58,6 +58,9 @@
5858
# Enforce idempotent producing for the internal RowProducer
5959
_default_producer_extra_config = {"enable.idempotence": True}
6060

61+
# Force assignment strategy to be "range" for co-partitioning in internal Consumers
62+
consumer_extra_config_overrides = {"partition.assignment.strategy": "range"}
63+
6164
_default_max_poll_interval_ms = 300000
6265

6366

@@ -301,7 +304,10 @@ def __init__(
301304
self._on_message_processed = on_message_processed
302305
self._on_processing_error = on_processing_error or default_on_processing_error
303306

304-
self._consumer = self._get_rowconsumer(on_error=on_consumer_error)
307+
self._consumer = self._get_rowconsumer(
308+
on_error=on_consumer_error,
309+
extra_config_overrides=consumer_extra_config_overrides,
310+
)
305311
self._producer = self._get_rowproducer(on_error=on_producer_error)
306312
self._running = False
307313
self._failed = False
@@ -593,20 +599,27 @@ def get_producer(self) -> Producer:
593599
)
594600

595601
def _get_rowconsumer(
596-
self, on_error: Optional[ConsumerErrorCallback] = None
602+
self,
603+
on_error: Optional[ConsumerErrorCallback] = None,
604+
extra_config_overrides: Optional[dict] = None,
597605
) -> RowConsumer:
598606
"""
599607
Create a RowConsumer using the application config
600608
601609
Used to create the application consumer as well as the sources consumers
602610
"""
603-
611+
extra_config_overrides = extra_config_overrides or {}
612+
# Override the existing "extra_config" with new values
613+
extra_config = {
614+
**self._config.consumer_extra_config,
615+
**extra_config_overrides,
616+
}
604617
return RowConsumer(
605618
broker_address=self._config.broker_address,
606619
consumer_group=self._config.consumer_group,
607620
auto_offset_reset=self._config.auto_offset_reset,
608621
auto_commit_enable=False, # Disable auto commit and manage commits manually
609-
extra_config=self._config.consumer_extra_config,
622+
extra_config=extra_config,
610623
on_error=on_error,
611624
)
612625

@@ -692,7 +705,9 @@ def add_source(self, source: BaseSource, topic: Optional[Topic] = None) -> Topic
692705
source,
693706
topic,
694707
self._get_rowproducer(transactional=False),
695-
self._get_rowconsumer(),
708+
self._get_rowconsumer(
709+
extra_config_overrides=consumer_extra_config_overrides
710+
),
696711
self._get_topic_manager(),
697712
)
698713
return topic
@@ -897,10 +912,6 @@ def _on_assign(self, _, topic_partitions: List[TopicPartition]):
897912
# get the source data.
898913
self._source_manager.start_sources()
899914

900-
# First commit everything processed so far because assignment can take a while
901-
# and fail
902-
self._processing_context.commit_checkpoint(force=True)
903-
904915
# Assign partitions manually to pause the changelog topics
905916
self._consumer.assign(topic_partitions)
906917
# Pause changelog topic+partitions immediately after assignment

quixstreams/kafka/consumer.py

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -126,7 +126,6 @@ def __init__(
126126
self._consumer_config = {
127127
# previous Quix Streams defaults
128128
"enable.auto.offset.store": False,
129-
# Force assignment strategy to "range" for co-partitioning
130129
"partition.assignment.strategy": "range",
131130
**(extra_config or {}),
132131
**broker_address.as_librdkafka_dict(),

tests/test_quixstreams/test_app.py

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -839,6 +839,7 @@ def test_init_with_quix_sdk_token_arg(self):
839839
expected_consumer_extra_config = {
840840
**extra_config,
841841
**quix_extras,
842+
"partition.assignment.strategy": "range",
842843
}
843844

844845
def get_cfg_builder(quix_sdk_token):
@@ -903,6 +904,7 @@ def test_init_with_quix_sdk_token_env(self, monkeypatch):
903904
expected_consumer_extra_config = {
904905
**extra_config,
905906
**quix_extras,
907+
"partition.assignment.strategy": "range",
906908
}
907909

908910
def get_cfg_builder(quix_sdk_token):
@@ -965,6 +967,7 @@ def test_init_with_quix_config_builder(self):
965967
expected_consumer_extra_config = {
966968
**extra_config,
967969
**quix_extras,
970+
"partition.assignment.strategy": "range",
968971
}
969972

970973
def get_cfg_builder(quix_sdk_token):

0 commit comments

Comments
 (0)