-
Notifications
You must be signed in to change notification settings - Fork 1.4k
KIP-70: Auto-commit offsets on consumer.unsubscribe(), defer assignment changes to rejoin #2560
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
73d820a
KIP-70: Auto-commit offsets on consumer.unsubscribe()
dpkp 856c08d
Raise TypeError if subscription topics is not list or non-str sequence
dpkp dfca19c
test_assign
dpkp 6db022c
test_subscription_state
dpkp 21fbd6e
fixup fetcher test
dpkp 6ad4c51
test changing subscription retains assignment
dpkp File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,26 +1,52 @@ | ||
from __future__ import absolute_import | ||
|
||
import pytest | ||
|
||
from kafka import KafkaConsumer | ||
from kafka.errors import KafkaConfigurationError | ||
from kafka import KafkaConsumer, TopicPartition | ||
from kafka.errors import KafkaConfigurationError, IllegalStateError | ||
|
||
|
||
def test_session_timeout_larger_than_request_timeout_raises(): | ||
with pytest.raises(KafkaConfigurationError): | ||
KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0, 9), group_id='foo', session_timeout_ms=50000, request_timeout_ms=40000) | ||
|
||
|
||
def test_fetch_max_wait_larger_than_request_timeout_raises(): | ||
with pytest.raises(KafkaConfigurationError): | ||
KafkaConsumer(bootstrap_servers='localhost:9092', fetch_max_wait_ms=50000, request_timeout_ms=40000) | ||
|
||
|
||
def test_request_timeout_larger_than_connections_max_idle_ms_raises(): | ||
with pytest.raises(KafkaConfigurationError): | ||
KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0, 9), request_timeout_ms=50000, connections_max_idle_ms=40000) | ||
|
||
|
||
def test_subscription_copy(): | ||
consumer = KafkaConsumer('foo', api_version=(0, 10, 0)) | ||
sub = consumer.subscription() | ||
assert sub is not consumer.subscription() | ||
assert sub == set(['foo']) | ||
sub.add('fizz') | ||
assert consumer.subscription() == set(['foo']) | ||
|
||
class TestKafkaConsumer: | ||
def test_session_timeout_larger_than_request_timeout_raises(self): | ||
with pytest.raises(KafkaConfigurationError): | ||
KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0, 9), group_id='foo', session_timeout_ms=50000, request_timeout_ms=40000) | ||
|
||
def test_fetch_max_wait_larger_than_request_timeout_raises(self): | ||
with pytest.raises(KafkaConfigurationError): | ||
KafkaConsumer(bootstrap_servers='localhost:9092', fetch_max_wait_ms=50000, request_timeout_ms=40000) | ||
def test_assign(): | ||
# Consumer w/ subscription to topic 'foo' | ||
consumer = KafkaConsumer('foo', api_version=(0, 10, 0)) | ||
assert consumer.assignment() == set() | ||
# Cannot assign manually | ||
with pytest.raises(IllegalStateError): | ||
consumer.assign([TopicPartition('foo', 0)]) | ||
|
||
def test_request_timeout_larger_than_connections_max_idle_ms_raises(self): | ||
with pytest.raises(KafkaConfigurationError): | ||
KafkaConsumer(bootstrap_servers='localhost:9092', api_version=(0, 9), request_timeout_ms=50000, connections_max_idle_ms=40000) | ||
assert 'foo' in consumer._client._topics | ||
|
||
def test_subscription_copy(self): | ||
consumer = KafkaConsumer('foo', api_version=(0, 10, 0)) | ||
sub = consumer.subscription() | ||
assert sub is not consumer.subscription() | ||
assert sub == set(['foo']) | ||
sub.add('fizz') | ||
assert consumer.subscription() == set(['foo']) | ||
consumer = KafkaConsumer(api_version=(0, 10, 0)) | ||
assert consumer.assignment() == set() | ||
consumer.assign([TopicPartition('foo', 0)]) | ||
assert consumer.assignment() == set([TopicPartition('foo', 0)]) | ||
assert 'foo' in consumer._client._topics | ||
# Cannot subscribe | ||
with pytest.raises(IllegalStateError): | ||
consumer.subscribe(topics=['foo']) | ||
consumer.assign([]) | ||
assert consumer.assignment() == set() |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,57 @@ | ||
from __future__ import absolute_import | ||
|
||
import pytest | ||
|
||
from kafka import TopicPartition | ||
from kafka.consumer.subscription_state import SubscriptionState, TopicPartitionState | ||
from kafka.vendor import six | ||
|
||
|
||
def test_type_error(): | ||
s = SubscriptionState() | ||
with pytest.raises(TypeError): | ||
s.subscribe(topics='foo') | ||
|
||
s.subscribe(topics=['foo']) | ||
|
||
|
||
def test_change_subscription(): | ||
s = SubscriptionState() | ||
s.subscribe(topics=['foo']) | ||
assert s.subscription == set(['foo']) | ||
s.change_subscription(['bar']) | ||
assert s.subscription == set(['bar']) | ||
|
||
|
||
def test_group_subscribe(): | ||
s = SubscriptionState() | ||
s.subscribe(topics=['foo']) | ||
assert s.subscription == set(['foo']) | ||
s.group_subscribe(['bar']) | ||
assert s.subscription == set(['foo']) | ||
assert s._group_subscription == set(['foo', 'bar']) | ||
|
||
s.reset_group_subscription() | ||
assert s.subscription == set(['foo']) | ||
assert s._group_subscription == set(['foo']) | ||
|
||
|
||
def test_assign_from_subscribed(): | ||
s = SubscriptionState() | ||
s.subscribe(topics=['foo']) | ||
with pytest.raises(ValueError): | ||
s.assign_from_subscribed([TopicPartition('bar', 0)]) | ||
|
||
s.assign_from_subscribed([TopicPartition('foo', 0), TopicPartition('foo', 1)]) | ||
assert set(s.assignment.keys()) == set([TopicPartition('foo', 0), TopicPartition('foo', 1)]) | ||
assert all([isinstance(s, TopicPartitionState) for s in six.itervalues(s.assignment)]) | ||
assert all([not s.has_valid_position for s in six.itervalues(s.assignment)]) | ||
|
||
|
||
def test_change_subscription_after_assignment(): | ||
s = SubscriptionState() | ||
s.subscribe(topics=['foo']) | ||
s.assign_from_subscribed([TopicPartition('foo', 0), TopicPartition('foo', 1)]) | ||
# Changing subscription retains existing assignment until next rebalance | ||
s.change_subscription(['bar']) | ||
assert set(s.assignment.keys()) == set([TopicPartition('foo', 0), TopicPartition('foo', 1)]) |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This causes
DeprecationWarning: Using or importing the ABCs from 'collections' instead of from 'collections.abc' is deprecated since Python 3.3, and in 3.10 it will stop working
I think the imports should be reversed here:
Or possibly just replace the entire try/except with a single import since the other import is only valid on very old/outdated Pythons.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
thanks, yea we still support very old pythons (2.7). I did
try old, except new
but that does cause deprecation warnings for interim pythons. I can see how you might prefertry new, except old