Skip to content
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

Add exclude_internal_topics option and fixed pattern subscription #111

Merged
merged 2 commits into from
Feb 17, 2017
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
17 changes: 12 additions & 5 deletions aiokafka/client.py
Expand Up @@ -264,19 +264,26 @@ def add_topic(self, topic):
topic (str): topic to track
"""
if topic in self._topics:
return
res = asyncio.Future(loop=self._loop)
res.set_result(True)
else:
res = self.force_metadata_update()
self._topics.add(topic)
return res

def set_topics(self, topics):
"""Set specific topics to track for metadata.

Arguments:
topics (list of str): topics to track
"""
if set(topics).difference(self._topics):
self._topics = set(topics)
# update metadata in async manner
self.force_metadata_update()
if not topics or set(topics).difference(self._topics):
res = self.force_metadata_update()
else:
res = asyncio.Future(loop=self._loop)
res.set_result(True)
self._topics = set(topics)
return res

@asyncio.coroutine
def _get_conn(self, node_id):
Expand Down
15 changes: 13 additions & 2 deletions aiokafka/consumer.py
Expand Up @@ -124,6 +124,10 @@ class AIOKafkaConsumer(object):
socket connections. Directly passed into asyncio's
`create_connection`_. For more information see :ref:`ssl_auth`.
Default: None.
exclude_internal_topics (bool): Whether records from internal topics
(such as offsets) should be exposed to the consumer. If set to True
the only way to receive records from an internal topic is
subscribing to it. Requires 0.10+ Default: True

Note:
Many configuration parameters are taken from Java Client:
Expand Down Expand Up @@ -152,7 +156,8 @@ def __init__(self, *topics, loop,
max_poll_records=None,
ssl_context=None,
security_protocol='PLAINTEXT',
api_version='auto'):
api_version='auto',
exclude_internal_topics=True):
if api_version not in ('auto', '0.9', '0.10'):
raise ValueError("Unsupported Kafka API version")
self._client = AIOKafkaClient(
Expand All @@ -175,6 +180,7 @@ def __init__(self, *topics, loop,
self._fetch_min_bytes = fetch_min_bytes
self._fetch_max_wait_ms = fetch_max_wait_ms
self._max_partition_fetch_bytes = max_partition_fetch_bytes
self._exclude_internal_topics = exclude_internal_topics
if max_poll_records is not None and (
not isinstance(max_poll_records, int) or max_poll_records < 1):
raise ValueError("`max_poll_records` should be positive Integer")
Expand Down Expand Up @@ -219,7 +225,8 @@ def start(self):
retry_backoff_ms=self._retry_backoff_ms,
enable_auto_commit=self._enable_auto_commit,
auto_commit_interval_ms=self._auto_commit_interval_ms,
assignors=self._partition_assignment_strategy)
assignors=self._partition_assignment_strategy,
exclude_internal_topics=self._exclude_internal_topics)
self._coordinator.on_group_rebalanced(
self._on_change_subscription)

Expand Down Expand Up @@ -537,6 +544,10 @@ def subscribe(self, topics=(), pattern=None, listener=None):
self._subscription.subscribe(topics=topics,
pattern=pattern,
listener=listener)
# There's a bug in subscription, that pattern is not unset if we change
# from pattern to simple topic subscription
if not pattern:
self._subscription.subscribed_pattern = None

# regex will need all topic metadata
if pattern is not None:
Expand Down
10 changes: 6 additions & 4 deletions aiokafka/group_coordinator.py
Expand Up @@ -61,7 +61,8 @@ def __init__(self, client, subscription, *, loop,
session_timeout_ms=30000, heartbeat_interval_ms=3000,
retry_backoff_ms=100,
enable_auto_commit=True, auto_commit_interval_ms=5000,
assignors=(RoundRobinPartitionAssignor,)
assignors=(RoundRobinPartitionAssignor,),
exclude_internal_topics=True
):
"""Initialize the coordination manager.

Expand Down Expand Up @@ -97,6 +98,7 @@ def __init__(self, client, subscription, *, loop,
self._session_timeout_ms = session_timeout_ms
self._heartbeat_interval_ms = heartbeat_interval_ms
self._retry_backoff_ms = retry_backoff_ms
self._exclude_internal_topics = exclude_internal_topics
self.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID
self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID
self.group_id = group_id
Expand Down Expand Up @@ -185,12 +187,11 @@ def _send_req(self, node_id, request):
def _handle_metadata_update(self, cluster):
if self._subscription.subscribed_pattern:
topics = []
for topic in cluster.topics():
for topic in cluster.topics(self._exclude_internal_topics):
if self._subscription.subscribed_pattern.match(topic):
topics.append(topic)

self._subscription.change_subscription(topics)
self._client.set_topics(self._subscription.group_subscription())

# check if there are any changes to the metadata which should trigger
# a rebalance
Expand Down Expand Up @@ -254,7 +255,8 @@ def _perform_assignment(self, leader_id, assignment_strategy, members):
# the group is interested in, which ensures that all metadata changes
# will eventually be seen
self._subscription.group_subscribe(all_subscribed_topics)
self._client.set_topics(self._subscription.group_subscription())
if not self._subscription.subscribed_pattern:
self._client.set_topics(self._subscription.group_subscription())

log.debug("Performing assignment for group %s using strategy %s"
" with subscriptions %s", self.group_id, assignor.name,
Expand Down
64 changes: 59 additions & 5 deletions tests/test_consumer.py
Expand Up @@ -595,9 +595,19 @@ def test_consumer_subscribe_pattern_with_autocreate(self):
# Wait for consumer to refresh metadata with new topic
yield from asyncio.sleep(0.3, loop=self.loop)
self.assertFalse(consume_task.done())
self.assertEqual(consumer._client.cluster.topics(), {my_topic})
self.assertTrue(consumer._client.cluster.topics() >= {my_topic})
self.assertEqual(consumer.subscription(), {my_topic})

# Add another topic
my_topic2 = "some-autocreate-pattern-2"
yield from producer.client._wait_on_metadata(my_topic2)
# Wait for consumer to refresh metadata with new topic
yield from asyncio.sleep(0.3, loop=self.loop)
self.assertFalse(consume_task.done())
self.assertTrue(consumer._client.cluster.topics() >=
{my_topic, my_topic2})
self.assertEqual(consumer.subscription(), {my_topic, my_topic2})

# Now lets actualy produce some data and verify that it is consumed
yield from producer.send(my_topic, b'test msg')
data = yield from consume_task
Expand All @@ -611,6 +621,10 @@ def test_consumer_rebalance_on_new_topic(self):
# Test will create a consumer group and check if adding new topic
# will trigger a group rebalance and assign partitions
pattern = "^another-autocreate-pattern-.*$"
client = AIOKafkaClient(
loop=self.loop, bootstrap_servers=self.hosts,
client_id="test_autocreate")
yield from client.bootstrap()
listener1 = StubRebalanceListener(loop=self.loop)
listener2 = StubRebalanceListener(loop=self.loop)
consumer1 = AIOKafkaConsumer(
Expand All @@ -634,10 +648,6 @@ def test_consumer_rebalance_on_new_topic(self):

# Lets force autocreation of a topic
my_topic = "another-autocreate-pattern-1"
client = AIOKafkaClient(
loop=self.loop, bootstrap_servers=self.hosts,
client_id="test_autocreate")
yield from client.bootstrap()
yield from client._wait_on_metadata(my_topic)

# Wait for group to stabilize
Expand All @@ -651,8 +661,27 @@ def test_consumer_rebalance_on_new_topic(self):
consumer1.assignment() | consumer2.assignment(),
my_partitions)

# Lets add another topic
listener1.reset()
listener2.reset()
my_topic2 = "another-autocreate-pattern-2"
yield from client._wait_on_metadata(my_topic2)

# Wait for group to stabilize
assign1 = yield from listener1.wait_assign()
assign2 = yield from listener2.wait_assign()
# We expect 2 partitons for autocreated topics
my_partitions = set([
TopicPartition(my_topic, 0), TopicPartition(my_topic, 1),
TopicPartition(my_topic2, 0), TopicPartition(my_topic2, 1)])
self.assertEqual(assign1 | assign2, my_partitions)
self.assertEqual(
consumer1.assignment() | consumer2.assignment(),
my_partitions)

yield from consumer1.stop()
yield from consumer2.stop()
yield from client.close()

@run_until_complete
def test_consumer_stops_getone(self):
Expand Down Expand Up @@ -696,3 +725,28 @@ def test_consumer_stops_getmany(self):
with self.assertRaises(ConsumerStoppedError):
yield from self.loop.create_task(
consumer.getmany(timeout_ms=0))

@run_until_complete
def test_exclude_internal_topics(self):
# Create random topic
my_topic = "some_noninternal_topic"
client = AIOKafkaClient(
loop=self.loop, bootstrap_servers=self.hosts,
client_id="test_autocreate")
yield from client.bootstrap()
yield from client._wait_on_metadata(my_topic)
yield from client.close()

# Check if only it will be subscribed
pattern = "^.*$"
consumer = AIOKafkaConsumer(
loop=self.loop, bootstrap_servers=self.hosts,
metadata_max_age_ms=200, group_id="some_group_1",
auto_offset_reset="earliest",
exclude_internal_topics=False)
consumer.subscribe(pattern=pattern)
yield from consumer.start()
self.assertIn("__consumer_offsets", consumer.subscription())
yield from consumer._client.force_metadata_update()
self.assertIn("__consumer_offsets", consumer.subscription())
yield from consumer.stop()