def test_coordinator_workflow(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() yield from self.wait_topic(client, 'topic1') yield from self.wait_topic(client, 'topic2') subscription = SubscriptionState('latest') subscription.subscribe(topics=('topic1', 'topic2')) coordinator = GroupCoordinator( client, subscription, loop=self.loop, session_timeout_ms=10000, heartbeat_interval_ms=500, retry_backoff_ms=100) self.assertEqual(coordinator.coordinator_id, None) self.assertEqual(coordinator.rejoin_needed, True) yield from coordinator.ensure_coordinator_known() self.assertNotEqual(coordinator.coordinator_id, None) yield from coordinator.ensure_active_group() self.assertNotEqual(coordinator.coordinator_id, None) self.assertEqual(coordinator.rejoin_needed, False) tp_list = subscription.assigned_partitions() self.assertEqual(tp_list, set([('topic1', 0), ('topic1', 1), ('topic2', 0), ('topic2', 1)])) # start second coordinator client2 = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client2.bootstrap() subscription2 = SubscriptionState('latest') subscription2.subscribe(topics=('topic1', 'topic2')) coordinator2 = GroupCoordinator( client2, subscription2, loop=self.loop, session_timeout_ms=10000, heartbeat_interval_ms=500, retry_backoff_ms=100) yield from coordinator2.ensure_active_group() yield from coordinator.ensure_active_group() tp_list = subscription.assigned_partitions() self.assertEqual(len(tp_list), 2) tp_list2 = subscription2.assigned_partitions() self.assertEqual(len(tp_list2), 2) tp_list |= tp_list2 self.assertEqual(tp_list, set([('topic1', 0), ('topic1', 1), ('topic2', 0), ('topic2', 1)])) yield from coordinator.close() yield from client.close() yield from asyncio.sleep(0.6, loop=self.loop) # wait heartbeat yield from coordinator2.ensure_active_group() tp_list = subscription2.assigned_partitions() self.assertEqual(tp_list, set([('topic1', 0), ('topic1', 1), ('topic2', 0), ('topic2', 1)])) yield from coordinator2.close() yield from client2.close()
def test_autocommit_enable_api_version(conn, api_version): coordinator = ConsumerCoordinator( KafkaClient(), SubscriptionState(), api_version=api_version) if api_version < (0, 8, 1): assert coordinator._auto_commit_task is None else: assert coordinator._auto_commit_task is not None
def test_maybe_auto_commit_offsets_sync(mocker, api_version, group_id, enable, error, has_auto_commit, commit_offsets, warn, exc): mock_warn = mocker.patch('kafka.coordinator.consumer.log.warning') mock_exc = mocker.patch('kafka.coordinator.consumer.log.exception') client = KafkaClient(api_version=api_version) coordinator = ConsumerCoordinator(client, SubscriptionState(), Metrics(), api_version=api_version, session_timeout_ms=30000, max_poll_interval_ms=30000, enable_auto_commit=enable, group_id=group_id) commit_sync = mocker.patch.object(coordinator, 'commit_offsets_sync', side_effect=error) if has_auto_commit: assert coordinator.next_auto_commit_deadline is not None else: assert coordinator.next_auto_commit_deadline is None assert coordinator._maybe_auto_commit_offsets_sync() is None if has_auto_commit: assert coordinator.next_auto_commit_deadline is not None assert commit_sync.call_count == (1 if commit_offsets else 0) assert mock_warn.call_count == (1 if warn else 0) assert mock_exc.call_count == (1 if exc else 0)
def test_init(conn): cli = KafkaClient() coordinator = ConsumerCoordinator(cli, SubscriptionState()) # metadata update on init assert cli.cluster._need_update is True assert WeakMethod(coordinator._handle_metadata_update) in cli.cluster._listeners
def _setup_error_after_data(self): subscriptions = SubscriptionState('latest') client = AIOKafkaClient( loop=self.loop, bootstrap_servers=[]) fetcher = Fetcher(client, subscriptions, loop=self.loop) tp1 = TopicPartition('some_topic', 0) tp2 = TopicPartition('some_topic', 1) state = TopicPartitionState() state.seek(0) subscriptions.assignment[tp1] = state state = TopicPartitionState() state.seek(0) subscriptions.assignment[tp2] = state subscriptions.needs_partition_assignment = False # Add some data messages = [ConsumerRecord( topic="some_topic", partition=1, offset=0, timestamp=0, timestamp_type=0, key=None, value=b"some", checksum=None, serialized_key_size=0, serialized_value_size=4)] fetcher._records[tp2] = FetchResult( tp2, subscriptions=subscriptions, loop=self.loop, records=iter(messages), backoff=0) # Add some error fetcher._records[tp1] = FetchError( loop=self.loop, error=OffsetOutOfRangeError({}), backoff=0) return fetcher, tp1, tp2, messages
def test_compacted_topic_consumption(self): # Compacted topics can have offsets skipped client = AIOKafkaClient( loop=self.loop, bootstrap_servers=[]) client.ready = mock.MagicMock() client.ready.side_effect = asyncio.coroutine(lambda a: True) client.force_metadata_update = mock.MagicMock() client.force_metadata_update.side_effect = asyncio.coroutine( lambda: False) client.send = mock.MagicMock() subscriptions = SubscriptionState('latest') fetcher = Fetcher(client, subscriptions, loop=self.loop) tp = TopicPartition('test', 0) req = FetchRequest( -1, # replica_id 100, 100, [(tp.topic, [(tp.partition, 155, 100000)])]) builder = LegacyRecordBatchBuilder( magic=1, compression_type=0, batch_size=99999999) builder.append(160, value=b"12345", key=b"1", timestamp=None) builder.append(162, value=b"23456", key=b"2", timestamp=None) builder.append(167, value=b"34567", key=b"3", timestamp=None) batch = bytes(builder.build()) resp = FetchResponse( [('test', [( 0, 0, 3000, # partition, error_code, highwater_offset batch # Batch raw bytes )])]) client.send.side_effect = asyncio.coroutine(lambda n, r: resp) state = TopicPartitionState() state.seek(155) state.drop_pending_message_set = False subscriptions.assignment[tp] = state subscriptions.needs_partition_assignment = False fetcher._in_flight.add(0) needs_wake_up = yield from fetcher._proc_fetch_request(0, req) self.assertEqual(needs_wake_up, True) buf = fetcher._records[tp] # Test successful getone first = buf.getone() self.assertEqual(state.position, 161) self.assertEqual( (first.value, first.key, first.offset), (b"12345", b"1", 160)) # Test successful getmany second, third = buf.getall() self.assertEqual(state.position, 168) self.assertEqual( (second.value, second.key, second.offset), (b"23456", b"2", 162)) self.assertEqual( (third.value, third.key, third.offset), (b"34567", b"3", 167))
def test_coordinator_ensure_active_group_on_expired_membership(self): # Do not fail ensure_active_group() if group membership has expired client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() yield from self.wait_topic(client, 'topic1') subscription = SubscriptionState('earliest') subscription.subscribe(topics=('topic1', )) coordinator = GroupCoordinator(client, subscription, loop=self.loop, group_id='test-offsets-group') yield from coordinator.ensure_active_group() # during OffsetCommit, UnknownMemberIdError is raised offsets = {TopicPartition('topic1', 0): OffsetAndMetadata(1, '')} with mock.patch('aiokafka.errors.for_code') as mocked: mocked.return_value = Errors.UnknownMemberIdError with self.assertRaises(Errors.UnknownMemberIdError): yield from coordinator.commit_offsets(offsets) self.assertEqual(subscription.needs_partition_assignment, True) # same exception is raised during ensure_active_group()'s call to # commit_offsets() via _on_join_prepare() but doesn't break this method with mock.patch.object(coordinator, "commit_offsets") as mocked: @asyncio.coroutine def mock_commit_offsets(*args, **kwargs): raise Errors.UnknownMemberIdError() mocked.side_effect = mock_commit_offsets yield from coordinator.ensure_active_group() yield from coordinator.close() yield from client.close()
def test_subscribe_pattern(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() test_listener = RebalanceListenerForTest() subscription = SubscriptionState('latest') subscription.subscribe(pattern='st-topic*', listener=test_listener) coordinator = GroupCoordinator(client, subscription, loop=self.loop, group_id='subs-pattern-group') yield from self.wait_topic(client, 'st-topic1') yield from self.wait_topic(client, 'st-topic2') yield from coordinator.ensure_active_group() self.assertNotEqual(coordinator.coordinator_id, None) self.assertEqual(coordinator.rejoin_needed, False) tp_list = subscription.assigned_partitions() assigned = set([('st-topic1', 0), ('st-topic1', 1), ('st-topic2', 0), ('st-topic2', 1)]) self.assertEqual(tp_list, assigned) self.assertEqual(test_listener.revoked, [set([])]) self.assertEqual(test_listener.assigned, [assigned]) yield from coordinator.close() yield from client.close()
def test_coordinator_subscription_append_on_rebalance(self): # same as above, but with adding topics instead of replacing them client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() yield from self.wait_topic(client, 'topic1') yield from self.wait_topic(client, 'topic2') subscription = SubscriptionState('earliest') subscription.subscribe(topics=('topic1',)) coordinator = GroupCoordinator( client, subscription, loop=self.loop, group_id='race-rebalance-subscribe-append', heartbeat_interval_ms=20000000) _perform_assignment = coordinator._perform_assignment with mock.patch.object(coordinator, '_perform_assignment') as mocked: def _new(*args, **kw): # Change the subscription to different topic before we finish # rebalance res = _perform_assignment(*args, **kw) subscription.subscribe(topics=('topic1', 'topic2', )) client.set_topics(('topic1', 'topic2', )) return res mocked.side_effect = _new yield from coordinator.ensure_active_group() self.assertEqual(subscription.needs_partition_assignment, False) topics = set([tp.topic for tp in subscription.assignment]) self.assertEqual(topics, {'topic1', 'topic2'}) yield from coordinator.close() yield from client.close()
def test_get_offsets(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() subscription = SubscriptionState('earliest') subscription.subscribe(topics=('topic1',)) coordinator = GroupCoordinator( client, subscription, loop=self.loop, group_id='getoffsets-group') yield from self.wait_topic(client, 'topic1') producer = AIOKafkaProducer( loop=self.loop, bootstrap_servers=self.hosts) yield from producer.start() yield from producer.send('topic1', b'first msg', partition=0) yield from producer.send('topic1', b'second msg', partition=1) yield from producer.send('topic1', b'third msg', partition=1) yield from producer.stop() yield from coordinator.ensure_active_group() offsets = {TopicPartition('topic1', 0): OffsetAndMetadata(1, ''), TopicPartition('topic1', 1): OffsetAndMetadata(2, '')} yield from coordinator.commit_offsets(offsets) self.assertEqual(subscription.all_consumed_offsets(), {}) subscription.seek(('topic1', 0), 0) subscription.seek(('topic1', 1), 0) yield from coordinator.refresh_committed_offsets() self.assertEqual(subscription.assignment[('topic1', 0)].committed, 1) self.assertEqual(subscription.assignment[('topic1', 1)].committed, 2) yield from coordinator.close() yield from client.close()
def test_maybe_auto_commit_offsets_sync(mocker, api_version, group_id, enable, error, has_auto_commit, commit_offsets, warn, exc): mock_warn = mocker.patch('kafka.coordinator.consumer.log.warning') mock_exc = mocker.patch('kafka.coordinator.consumer.log.exception') coordinator = ConsumerCoordinator(KafkaClient(), SubscriptionState(), Metrics(), 'consumer', api_version=api_version, enable_auto_commit=enable, group_id=group_id) commit_sync = mocker.patch.object(coordinator, 'commit_offsets_sync', side_effect=error) if has_auto_commit: assert coordinator._auto_commit_task is not None coordinator._auto_commit_task.enable() assert coordinator._auto_commit_task._enabled is True else: assert coordinator._auto_commit_task is None assert coordinator._maybe_auto_commit_offsets_sync() is None if has_auto_commit: assert coordinator._auto_commit_task is not None assert coordinator._auto_commit_task._enabled is False assert commit_sync.call_count == (1 if commit_offsets else 0) assert mock_warn.call_count == (1 if warn else 0) assert mock_exc.call_count == (1 if exc else 0)
def test_with_nocommit_support(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) subscription = SubscriptionState('latest') subscription.subscribe(topics=('topic1',)) coordinator = GroupCoordinator( client, subscription, loop=self.loop, enable_auto_commit=False) self.assertEqual(coordinator._auto_commit_task, None)
def test_failed_broker_conn(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) subscription = SubscriptionState('latest') subscription.subscribe(topics=('topic1', )) coordinator = GroupCoordinator(client, subscription, loop=self.loop) with self.assertRaises(NoBrokersAvailable): yield from coordinator.ensure_coordinator_known()
def test_offsets_failed_scenarios(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() yield from self.wait_topic(client, 'topic1') subscription = SubscriptionState('earliest') subscription.subscribe(topics=('topic1', )) coordinator = GroupCoordinator(client, subscription, loop=self.loop, group_id='test-offsets-group') yield from coordinator.ensure_active_group() offsets = {TopicPartition('topic1', 0): OffsetAndMetadata(1, '')} yield from coordinator.commit_offsets(offsets) with mock.patch('aiokafka.errors.for_code') as mocked: mocked.return_value = Errors.GroupAuthorizationFailedError with self.assertRaises(Errors.GroupAuthorizationFailedError): yield from coordinator.commit_offsets(offsets) mocked.return_value = Errors.TopicAuthorizationFailedError with self.assertRaises(Errors.TopicAuthorizationFailedError): yield from coordinator.commit_offsets(offsets) mocked.return_value = Errors.InvalidCommitOffsetSizeError with self.assertRaises(Errors.InvalidCommitOffsetSizeError): yield from coordinator.commit_offsets(offsets) mocked.return_value = Errors.GroupLoadInProgressError with self.assertRaises(Errors.GroupLoadInProgressError): yield from coordinator.commit_offsets(offsets) mocked.return_value = Errors.RebalanceInProgressError with self.assertRaises(Errors.RebalanceInProgressError): yield from coordinator.commit_offsets(offsets) self.assertEqual(subscription.needs_partition_assignment, True) subscription.needs_partition_assignment = False mocked.return_value = Errors.UnknownMemberIdError with self.assertRaises(Errors.UnknownMemberIdError): yield from coordinator.commit_offsets(offsets) self.assertEqual(subscription.needs_partition_assignment, True) mocked.return_value = KafkaError with self.assertRaises(KafkaError): yield from coordinator.commit_offsets(offsets) mocked.return_value = Errors.NotCoordinatorForGroupError with self.assertRaises(Errors.NotCoordinatorForGroupError): yield from coordinator.commit_offsets(offsets) self.assertEqual(coordinator.coordinator_id, None) with self.assertRaises(Errors.GroupCoordinatorNotAvailableError): yield from coordinator.commit_offsets(offsets) yield from coordinator.close() yield from client.close()
def __init__(self, *topics, loop, bootstrap_servers='localhost', client_id='aiokafka-' + __version__, group_id=None, key_deserializer=None, value_deserializer=None, fetch_max_wait_ms=500, fetch_min_bytes=1, max_partition_fetch_bytes=1 * 1024 * 1024, request_timeout_ms=40 * 1000, retry_backoff_ms=100, reconnect_backoff_ms=50, auto_offset_reset='latest', enable_auto_commit=True, auto_commit_interval_ms=5000, check_crcs=True, metadata_max_age_ms=5 * 60 * 1000, partition_assignment_strategy=(RoundRobinPartitionAssignor, ), heartbeat_interval_ms=3000, session_timeout_ms=30000, consumer_timeout_ms=100, api_version='auto'): if api_version not in ('auto', '0.9'): raise ValueError("Unsupported Kafka API version") self._client = AIOKafkaClient(loop=loop, bootstrap_servers=bootstrap_servers, client_id=client_id, metadata_max_age_ms=metadata_max_age_ms, request_timeout_ms=request_timeout_ms) self._api_version = api_version self._group_id = group_id self._heartbeat_interval_ms = heartbeat_interval_ms self._retry_backoff_ms = retry_backoff_ms self._enable_auto_commit = enable_auto_commit self._auto_commit_interval_ms = auto_commit_interval_ms self._partition_assignment_strategy = partition_assignment_strategy self._key_deserializer = key_deserializer self._value_deserializer = value_deserializer 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._consumer_timeout = consumer_timeout_ms / 1000 self._check_crcs = check_crcs self._subscription = SubscriptionState(auto_offset_reset) self._fetcher = None self._coordinator = None self._closed = False self._loop = loop self._topics = topics if topics: self._client.set_topics(topics) self._subscription.subscribe(topics=topics)
def __init__(self, *topics, **configs): self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: if key in configs: self.config[key] = configs.pop(key) # Only check for extra config keys in top-level class assert not configs, 'Unrecognized configs: %s' % configs deprecated = {'smallest': 'earliest', 'largest': 'latest'} if self.config['auto_offset_reset'] in deprecated: new_config = deprecated[self.config['auto_offset_reset']] log.warning('use auto_offset_reset=%s (%s is deprecated)', new_config, self.config['auto_offset_reset']) self.config['auto_offset_reset'] = new_config metrics_tags = {'client-id': self.config['client_id']} metric_config = MetricConfig(samples=self.config['metrics_num_samples'], time_window_ms=self.config['metrics_sample_window_ms'], tags=metrics_tags) reporters = [reporter() for reporter in self.config['metric_reporters']] self._metrics = Metrics(metric_config, reporters) # TODO _metrics likely needs to be passed to KafkaClient, etc. # api_version was previously a str. Accept old format for now if isinstance(self.config['api_version'], str): str_version = self.config['api_version'] if str_version == 'auto': self.config['api_version'] = None else: self.config['api_version'] = tuple(map(int, str_version.split('.'))) log.warning('use api_version=%s [tuple] -- "%s" as str is deprecated', str(self.config['api_version']), str_version) self._client = KafkaClient(metrics=self._metrics, **self.config) # Get auto-discovered version from client if necessary if self.config['api_version'] is None: self.config['api_version'] = self._client.config['api_version'] self._subscription = SubscriptionState(self.config['auto_offset_reset']) self._fetcher = Fetcher( self._client, self._subscription, self._metrics, **self.config) self._coordinator = ConsumerCoordinator( self._client, self._subscription, self._metrics, assignors=self.config['partition_assignment_strategy'], **self.config) self._closed = False self._iterator = None self._consumer_timeout = float('inf') if topics: self._subscription.subscribe(topics=topics) self._client.set_topics(topics)
def test_coordinator_metadata_update_during_rebalance(self): # Race condition where client.set_topics start MetadataUpdate, but it # fails to arrive before leader performed assignment # Just ensure topics are created client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() yield from self.wait_topic(client, 'topic1') yield from self.wait_topic(client, 'topic2') yield from client.close() client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() self.add_cleanup(client.close) subscription = SubscriptionState('earliest') client.set_topics(["topic1"]) subscription.subscribe(topics=('topic1', )) coordinator = GroupCoordinator( client, subscription, loop=self.loop, group_id='race-rebalance-metadata-update', heartbeat_interval_ms=20000000) self.add_cleanup(coordinator.close) yield from coordinator.ensure_active_group() # Check that topic's partitions are properly assigned self.assertEqual(subscription.needs_partition_assignment, False) self.assertEqual( set(subscription.assignment.keys()), {TopicPartition("topic1", 0), TopicPartition("topic1", 1)}) _metadata_update = client._metadata_update with mock.patch.object(client, '_metadata_update') as mocked: @asyncio.coroutine def _new(*args, **kw): # Just make metadata updates a bit more slow for test # robustness yield from asyncio.sleep(0.5, loop=self.loop) res = yield from _metadata_update(*args, **kw) return res mocked.side_effect = _new subscription.subscribe(topics=('topic2', )) client.set_topics(('topic2', )) yield from coordinator.ensure_active_group() self.assertEqual(subscription.needs_partition_assignment, False) self.assertEqual( set(subscription.assignment.keys()), {TopicPartition("topic2", 0), TopicPartition("topic2", 1)})
def test_autocommit_enable_api_version(client, api_version): coordinator = ConsumerCoordinator(client, SubscriptionState(), Metrics(), enable_auto_commit=True, group_id='foobar', api_version=api_version) if api_version < (0, 8, 1): assert coordinator._auto_commit_task is None assert coordinator.config['enable_auto_commit'] is False else: assert coordinator._auto_commit_task is not None assert coordinator.config['enable_auto_commit'] is True
def test_autocommit_enable_api_version(client, api_version): coordinator = ConsumerCoordinator(client, SubscriptionState(), Metrics(), enable_auto_commit=True, session_timeout_ms=30000, # session_timeout_ms and max_poll_interval_ms max_poll_interval_ms=30000, # should be the same to avoid KafkaConfigurationError group_id='foobar', api_version=api_version) if api_version < (0, 8, 1): assert coordinator.config['enable_auto_commit'] is False else: assert coordinator.config['enable_auto_commit'] is True
def test_failed_group_join(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) subscription = SubscriptionState('latest') subscription.subscribe(topics=('topic1', )) coordinator = GroupCoordinator(client, subscription, loop=self.loop, retry_backoff_ms=10) @asyncio.coroutine def do_rebalance(): rebalance = CoordinatorGroupRebalance( coordinator, coordinator.group_id, coordinator.coordinator_id, subscription.subscription, coordinator._assignors, coordinator._session_timeout_ms, coordinator._retry_backoff_ms, loop=self.loop) yield from rebalance.perform_group_join() yield from client.bootstrap() yield from self.wait_topic(client, 'topic1') mocked = mock.MagicMock() coordinator._client = mocked # no exception expected, just wait mocked.send.side_effect = Errors.GroupLoadInProgressError() yield from do_rebalance() self.assertEqual(coordinator.need_rejoin(), True) mocked.send.side_effect = Errors.InvalidGroupIdError() with self.assertRaises(Errors.InvalidGroupIdError): yield from do_rebalance() self.assertEqual(coordinator.need_rejoin(), True) # no exception expected, member_id should be reseted coordinator.member_id = 'some_invalid_member_id' mocked.send.side_effect = Errors.UnknownMemberIdError() yield from do_rebalance() self.assertEqual(coordinator.need_rejoin(), True) self.assertEqual(coordinator.member_id, JoinGroupRequest.UNKNOWN_MEMBER_ID) # no exception expected, coordinator_id should be reseted coordinator.coordinator_id = 'some_id' mocked.send.side_effect = Errors.GroupCoordinatorNotAvailableError() yield from do_rebalance() self.assertEqual(coordinator.need_rejoin(), True) self.assertEqual(coordinator.coordinator_id, None) yield from client.close()
def test_failed_sync_group(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) subscription = SubscriptionState('latest') subscription.subscribe(topics=('topic1', )) coordinator = GroupCoordinator(client, subscription, loop=self.loop, heartbeat_interval_ms=20000) @asyncio.coroutine def do_sync_group(): rebalance = CoordinatorGroupRebalance( coordinator, coordinator.group_id, coordinator.coordinator_id, subscription.subscription, coordinator._assignors, coordinator._session_timeout_ms, coordinator._retry_backoff_ms, loop=self.loop) yield from rebalance._on_join_follower() with self.assertRaises(GroupCoordinatorNotAvailableError): yield from do_sync_group() mocked = mock.MagicMock() coordinator._client = mocked coordinator.member_id = 'some_invalid_member_id' coordinator.coordinator_unknown = asyncio.coroutine(lambda: False) mocked.send.side_effect = Errors.UnknownMemberIdError() with self.assertRaises(Errors.UnknownMemberIdError): yield from do_sync_group() self.assertEqual(coordinator.member_id, JoinGroupRequest.UNKNOWN_MEMBER_ID) mocked.send.side_effect = Errors.NotCoordinatorForGroupError() coordinator.coordinator_id = 'some_id' with self.assertRaises(Errors.NotCoordinatorForGroupError): yield from do_sync_group() self.assertEqual(coordinator.coordinator_id, None) mocked.send.side_effect = KafkaError() with self.assertRaises(KafkaError): yield from do_sync_group() # client sends LeaveGroupRequest to group coordinator # if generation > 0 (means that client is a member of group) # expecting no exception in this case (error should be ignored in close # method) coordinator.generation = 33 yield from coordinator.close()
def __init__(self, *topics, **configs): self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: if key in configs: self.config[key] = configs.pop(key) # Only check for extra config keys in top-level class assert not configs, 'Unrecognized configs: %s' % configs deprecated = {'smallest': 'earliest', 'largest': 'latest' } if self.config['auto_offset_reset'] in deprecated: new_config = deprecated[self.config['auto_offset_reset']] log.warning('use auto_offset_reset=%s (%s is deprecated)', new_config, self.config['auto_offset_reset']) self.config['auto_offset_reset'] = new_config metrics_tags = {'client-id': self.config['client_id']} metric_config = MetricConfig(samples=self.config['metrics_num_samples'], time_window_ms=self.config['metrics_sample_window_ms'], tags=metrics_tags) reporters = [reporter() for reporter in self.config['metric_reporters']] reporters.append(DictReporter('kafka.consumer')) self._metrics = Metrics(metric_config, reporters) metric_group_prefix = 'consumer' # TODO _metrics likely needs to be passed to KafkaClient, etc. self._client = KafkaClient(**self.config) # Check Broker Version if not set explicitly if self.config['api_version'] == 'auto': self.config['api_version'] = self._client.check_version() assert self.config['api_version'] in ('0.9', '0.8.2', '0.8.1', '0.8.0'), 'Unrecognized api version' # Convert api_version config to tuple for easy comparisons self.config['api_version'] = tuple( map(int, self.config['api_version'].split('.'))) self._subscription = SubscriptionState(self.config['auto_offset_reset']) self._fetcher = Fetcher( self._client, self._subscription, self._metrics, metric_group_prefix, **self.config) self._coordinator = ConsumerCoordinator( self._client, self._subscription, self._metrics, metric_group_prefix, assignors=self.config['partition_assignment_strategy'], **self.config) self._closed = False self._iterator = None self._consumer_timeout = float('inf') if topics: self._subscription.subscribe(topics=topics) self._client.set_topics(topics)
def __init__(self, *topics, **configs): self.config = copy.copy(self.DEFAULT_CONFIG) for key in self.config: if key in configs: self.config[key] = configs.pop(key) # Only check for extra config keys in top-level class assert not configs, 'Unrecognized configs: %s' % configs deprecated = {'smallest': 'earliest', 'largest': 'latest'} if self.config['auto_offset_reset'] in deprecated: new_config = deprecated[self.config['auto_offset_reset']] log.warning('use auto_offset_reset=%s (%s is deprecated)', new_config, self.config['auto_offset_reset']) self.config['auto_offset_reset'] = new_config self._client = KafkaClient(**self.config) # Check Broker Version if not set explicitly if self.config['api_version'] == 'auto': self.config['api_version'] = self._client.check_version() assert self.config['api_version'] in ( '0.9', '0.8.2', '0.8.1', '0.8.0'), 'Unrecognized api version' # Convert api_version config to tuple for easy comparisons self.config['api_version'] = tuple( map(int, self.config['api_version'].split('.'))) self._subscription = SubscriptionState( self.config['auto_offset_reset']) self._fetcher = Fetcher(self._client, self._subscription, **self.config) self._coordinator = ConsumerCoordinator( self._client, self._subscription, assignors=self.config['partition_assignment_strategy'], **self.config) self._closed = False self._iterator = None self._consumer_timeout = float('inf') #self.metrics = None if topics: self._subscription.subscribe(topics=topics) self._client.set_topics(topics)
def test_update_fetch_positions(self): client = AIOKafkaClient( loop=self.loop, bootstrap_servers=[]) subscriptions = SubscriptionState('latest') fetcher = Fetcher(client, subscriptions, loop=self.loop) partition = TopicPartition('test', 0) # partition is not assigned, should be ignored yield from fetcher.update_fetch_positions([partition]) state = TopicPartitionState() state.seek(0) subscriptions.assignment[partition] = state # partition is fetchable, no need to update position yield from fetcher.update_fetch_positions([partition]) client.ready = mock.MagicMock() client.ready.side_effect = asyncio.coroutine(lambda a: True) client.force_metadata_update = mock.MagicMock() client.force_metadata_update.side_effect = asyncio.coroutine( lambda: False) client.send = mock.MagicMock() client.send.side_effect = asyncio.coroutine( lambda n, r: OffsetResponse[0]([('test', [(0, 0, [4])])])) state.await_reset(OffsetResetStrategy.LATEST) client.cluster.leader_for_partition = mock.MagicMock() client.cluster.leader_for_partition.side_effect = [None, -1, 0] yield from fetcher.update_fetch_positions([partition]) self.assertEqual(state.position, 4) client.cluster.leader_for_partition = mock.MagicMock() client.cluster.leader_for_partition.return_value = 1 client.send = mock.MagicMock() client.send.side_effect = asyncio.coroutine( lambda n, r: OffsetResponse[0]([('test', [(0, 3, [])])])) state.await_reset(OffsetResetStrategy.LATEST) with self.assertRaises(UnknownTopicOrPartitionError): yield from fetcher.update_fetch_positions([partition]) client.send.side_effect = asyncio.coroutine( lambda n, r: OffsetResponse[0]([('test', [(0, -1, [])])])) with self.assertRaises(UnknownError): yield from fetcher.update_fetch_positions([partition]) yield from fetcher.close()
def test_fetchoffsets_failed_scenarios(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() yield from self.wait_topic(client, 'topic1') subscription = SubscriptionState('earliest') subscription.subscribe(topics=('topic1', )) coordinator = GroupCoordinator(client, subscription, loop=self.loop, group_id='fetch-offsets-group') yield from coordinator.ensure_active_group() offsets = {TopicPartition('topic1', 0): OffsetAndMetadata(1, '')} with mock.patch('kafka.common.for_code') as mocked: mocked.side_effect = MockedKafkaErrCode( Errors.GroupLoadInProgressError, Errors.NoError) yield from coordinator.fetch_committed_offsets(offsets) mocked.side_effect = MockedKafkaErrCode( Errors.UnknownMemberIdError, Errors.NoError) with self.assertRaises(Errors.UnknownMemberIdError): yield from coordinator.fetch_committed_offsets(offsets) self.assertEqual(subscription.needs_partition_assignment, True) mocked.side_effect = None mocked.return_value = Errors.UnknownTopicOrPartitionError r = yield from coordinator.fetch_committed_offsets(offsets) self.assertEqual(r, {}) mocked.return_value = KafkaError with self.assertRaises(KafkaError): yield from coordinator.fetch_committed_offsets(offsets) mocked.side_effect = MockedKafkaErrCode( Errors.NotCoordinatorForGroupError, Errors.NoError, Errors.NoError, Errors.NoError) yield from coordinator.fetch_committed_offsets(offsets) yield from coordinator.close() yield from client.close()
def test_coordinator_metadata_change_by_broker(self): # Issue #108. We can have a misleading metadata change, that will # trigger additional rebalance client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() yield from self.wait_topic(client, 'topic1') yield from self.wait_topic(client, 'topic2') client.set_topics(['other_topic']) yield from client.force_metadata_update() subscription = SubscriptionState('earliest') coordinator = GroupCoordinator( client, subscription, loop=self.loop, group_id='race-rebalance-subscribe-append', heartbeat_interval_ms=2000000) subscription.subscribe(topics=('topic1', )) yield from client.set_topics(subscription.group_subscription()) yield from coordinator.ensure_active_group() _perform_assignment = coordinator._perform_assignment with mock.patch.object(coordinator, '_perform_assignment') as mocked: mocked.side_effect = _perform_assignment subscription.subscribe(topics=('topic2', )) yield from client.set_topics(subscription.group_subscription()) # Should only trigger 1 rebalance, but will trigger 2 with bug: # Metadata snapshot will change: # {'topic1': {0, 1}} -> {'topic1': {0, 1}, 'topic2': {0, 1}} # And then again: # {'topic1': {0, 1}, 'topic2': {0, 1}} -> {'topic2': {0, 1}} yield from coordinator.ensure_active_group() yield from client.force_metadata_update() yield from coordinator.ensure_active_group() self.assertEqual(mocked.call_count, 1) yield from coordinator.close() yield from client.close()
def subscription_state(): return SubscriptionState()
def coordinator(client): return ConsumerCoordinator(client, SubscriptionState(), Metrics())
def coordinator(conn): return ConsumerCoordinator(KafkaClient(), SubscriptionState(), Metrics(), 'consumer')
def test_proc_fetch_request(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=[]) subscriptions = SubscriptionState('latest') fetcher = Fetcher(client, subscriptions, loop=self.loop) tp = TopicPartition('test', 0) tp_info = (tp.topic, [(tp.partition, 155, 100000)]) req = FetchRequest( -1, # replica_id 100, 100, [tp_info]) client.ready = mock.MagicMock() client.ready.side_effect = asyncio.coroutine(lambda a: True) client.force_metadata_update = mock.MagicMock() client.force_metadata_update.side_effect = asyncio.coroutine( lambda: False) client.send = mock.MagicMock() msg = Message(b"test msg") msg._encode_self() client.send.side_effect = asyncio.coroutine(lambda n, r: FetchResponse( [('test', [(0, 0, 9, [(4, 10, msg)])])])) fetcher._in_flight.add(0) needs_wake_up = yield from fetcher._proc_fetch_request(0, req) self.assertEqual(needs_wake_up, False) state = TopicPartitionState() state.seek(0) subscriptions.assignment[tp] = state subscriptions.needs_partition_assignment = False fetcher._in_flight.add(0) needs_wake_up = yield from fetcher._proc_fetch_request(0, req) self.assertEqual(needs_wake_up, True) buf = fetcher._records[tp] self.assertEqual(buf.getone(), None) # invalid offset, msg is ignored state.seek(4) fetcher._in_flight.add(0) fetcher._records.clear() needs_wake_up = yield from fetcher._proc_fetch_request(0, req) self.assertEqual(needs_wake_up, True) buf = fetcher._records[tp] self.assertEqual(buf.getone().value, b"test msg") # error -> no partition found client.send.side_effect = asyncio.coroutine(lambda n, r: FetchResponse( [('test', [(0, 3, 9, [(4, 10, msg)])])])) fetcher._in_flight.add(0) fetcher._records.clear() needs_wake_up = yield from fetcher._proc_fetch_request(0, req) self.assertEqual(needs_wake_up, False) # error -> topic auth failed client.send.side_effect = asyncio.coroutine(lambda n, r: FetchResponse( [('test', [(0, 29, 9, [(4, 10, msg)])])])) fetcher._in_flight.add(0) fetcher._records.clear() needs_wake_up = yield from fetcher._proc_fetch_request(0, req) self.assertEqual(needs_wake_up, True) with self.assertRaises(TopicAuthorizationFailedError): yield from fetcher.next_record([]) # error -> unknown client.send.side_effect = asyncio.coroutine(lambda n, r: FetchResponse( [('test', [(0, -1, 9, [(4, 10, msg)])])])) fetcher._in_flight.add(0) fetcher._records.clear() needs_wake_up = yield from fetcher._proc_fetch_request(0, req) self.assertEqual(needs_wake_up, False) # error -> offset out of range client.send.side_effect = asyncio.coroutine(lambda n, r: FetchResponse( [('test', [(0, 1, 9, [(4, 10, msg)])])])) fetcher._in_flight.add(0) fetcher._records.clear() needs_wake_up = yield from fetcher._proc_fetch_request(0, req) self.assertEqual(needs_wake_up, False) self.assertEqual(state.is_fetchable(), False) state.seek(4) subscriptions._default_offset_reset_strategy = OffsetResetStrategy.NONE client.send.side_effect = asyncio.coroutine(lambda n, r: FetchResponse( [('test', [(0, 1, 9, [(4, 10, msg)])])])) fetcher._in_flight.add(0) fetcher._records.clear() needs_wake_up = yield from fetcher._proc_fetch_request(0, req) self.assertEqual(needs_wake_up, True) with self.assertRaises(OffsetOutOfRangeError): yield from fetcher.next_record([]) yield from fetcher.close()