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_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 _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_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_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_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_send_request(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts) yield from client.bootstrap() node_id = client.get_random_node() resp = yield from client.send(node_id, MetadataRequest([])) self.assertTrue(isinstance(resp, MetadataResponse)) yield from client.close()
async def test_client_receive_zero_brokers(self): brokers = [(0, 'broker_1', 4567), (1, 'broker_2', 5678)] correct_meta = MetadataResponse(brokers, []) bad_response = MetadataResponse([], []) async def send(*args, **kwargs): return bad_response client = AIOKafkaClient(loop=self.loop, bootstrap_servers=['broker_1:4567'], api_version="0.10") conn = mock.Mock() client._conns = [mock.Mock()] client._get_conn = mock.Mock() client._get_conn.side_effect = asyncio.coroutine(lambda x: conn) conn.send = mock.Mock() conn.send.side_effect = send client.cluster.update_metadata(correct_meta) brokers_before = client.cluster.brokers() await client._metadata_update(client.cluster, []) # There broker list should not be purged self.assertNotEqual(client.cluster.brokers(), set([])) self.assertEqual(client.cluster.brokers(), brokers_before)
async def test_client_receive_zero_brokers_timeout_on_send(self): brokers = [(0, 'broker_1', 4567), (1, 'broker_2', 5678)] correct_meta = MetadataResponse(brokers, []) async def send(*args, **kwargs): raise asyncio.TimeoutError() client = AIOKafkaClient(bootstrap_servers=['broker_1:4567'], api_version="0.10") conn = mock.Mock() client._conns = [mock.Mock()] async def _get_conn(*args: Any, **kwargs: Any): return conn client._get_conn = mock.Mock() client._get_conn.side_effect = _get_conn conn.send = mock.Mock() conn.send.side_effect = send client.cluster.update_metadata(correct_meta) brokers_before = client.cluster.brokers() await client._metadata_update(client.cluster, []) # There broker list should not be purged self.assertNotEqual(client.cluster.brokers(), set([])) self.assertEqual(client.cluster.brokers(), brokers_before)
def test_has_metadata_for_topic(self, protocol): @asyncio.coroutine def recv(request_id): return b'response' mocked_conns = {('broker_1', 4567): mock.MagicMock()} mocked_conns[('broker_1', 4567)].recv.side_effect = recv client = AIOKafkaClient(['broker_1:4567'], loop=self.loop) client._conns = mocked_conns brokers = [ BrokerMetadata(0, 'broker_1', 4567), BrokerMetadata(1, 'broker_2', 5678) ] topics = [ TopicMetadata('topic_still_creating', NO_LEADER, []), TopicMetadata('topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []), TopicMetadata('topic_noleaders', NO_ERROR, [ PartitionMetadata('topic_noleaders', 0, -1, [], [], NO_LEADER), PartitionMetadata('topic_noleaders', 1, -1, [], [], NO_LEADER), ]), ] protocol.decode_metadata_response.return_value = MetadataResponse( brokers, topics) self.loop.run_until_complete(client.load_metadata_for_topics()) # Topics with no partitions return False self.assertFalse(client.has_metadata_for_topic('topic_still_creating')) self.assertFalse(client.has_metadata_for_topic('topic_doesnt_exist')) # Topic with partition metadata, but no leaders return True self.assertTrue(client.has_metadata_for_topic('topic_noleaders'))
def test_concurrent_send_on_different_connection_groups(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts, metadata_max_age_ms=10000) yield from client.bootstrap() self.add_cleanup(client.close) yield from self.wait_topic(client, self.topic) node_id = client.get_random_node() wait_request = FetchRequest_v0( -1, # replica_id 500, # max_wait_ms 1024 * 1024, # min_bytes [(self.topic, [(0, 0, 1024)])]) vanila_request = MetadataRequest([]) send_time = self.loop.time() long_task = self.loop.create_task(client.send(node_id, wait_request)) yield from asyncio.sleep(0.0001, loop=self.loop) self.assertFalse(long_task.done()) yield from client.send(node_id, vanila_request, group=ConnectionGroup.COORDINATION) resp_time = self.loop.time() self.assertFalse(long_task.done()) fetch_resp = yield from long_task # Check error code like resp->topics[0]->partitions[0]->error_code self.assertEqual(fetch_resp.topics[0][1][0][1], 0) # Check that vanila request actually executed after wait request self.assertLess(resp_time - send_time, 0.5)
async def test_send_request(self): client = AIOKafkaClient(bootstrap_servers=self.hosts) await client.bootstrap() node_id = client.get_random_node() resp = await client.send(node_id, MetadataRequest([])) self.assertTrue(isinstance(resp, MetadataResponse)) await client.close()
def _setup_error_after_data(self): subscriptions = SubscriptionState(loop=self.loop) client = AIOKafkaClient( loop=self.loop, bootstrap_servers=[]) fetcher = Fetcher(client, subscriptions, loop=self.loop) tp1 = TopicPartition('some_topic', 0) tp2 = TopicPartition('some_topic', 1) subscriptions.subscribe(set(["some_topic"])) subscriptions.assign_from_subscribed({tp1, tp2}) assignment = subscriptions.subscription.assignment subscriptions.seek(tp1, 0) subscriptions.seek(tp2, 0) # 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, assignment=assignment, loop=self.loop, message_iterator=iter(messages), backoff=0, fetch_offset=0) # Add some error fetcher._records[tp1] = FetchError( loop=self.loop, error=OffsetOutOfRangeError({}), backoff=0) return fetcher, tp1, tp2, messages
def test_send_broker_unaware_request_fail(self): 'Tests that call fails when all hosts are unavailable' mocked_conns = { ('kafka01', 9092): mock.MagicMock(), ('kafka02', 9092): mock.MagicMock() } # inject KafkaConnection side effects fut1 = asyncio.Future(loop=self.loop) fut1.set_exception(RuntimeError("kafka01 went away (unittest)")) mocked_conns[('kafka01', 9092)].send.return_value = fut1 fut2 = asyncio.Future(loop=self.loop) fut2.set_exception(RuntimeError("kafka02 went away (unittest)")) mocked_conns[('kafka02', 9092)].send.return_value = fut2 client = AIOKafkaClient(['kafka01:9092', 'kafka02:9092'], loop=self.loop) client._conns = mocked_conns @asyncio.coroutine def go(): with self.assertRaises(KafkaUnavailableError): yield from client._send_broker_unaware_request( payloads=['fake request'], encoder_fn=mock.MagicMock( return_value=b'fake encoded message'), decoder_fn=lambda x: x) for key, conn in mocked_conns.items(): conn.send.assert_called_with(b'fake encoded message') self.loop.run_until_complete(go())
async def test_check_version(self): kafka_version = tuple(int(x) for x in self.kafka_version.split(".")) client = AIOKafkaClient(bootstrap_servers=self.hosts) await client.bootstrap() ver = await client.check_version() expected_version = kafka_version[:2] # No significant protocol changed, no way to differencieate if expected_version == (2, 2): expected_version = (2, 1) elif expected_version == (2, 4): expected_version = (2, 3) self.assertEqual(expected_version, ver[:2]) await self.wait_topic(client, 'some_test_topic') ver2 = await client.check_version() self.assertEqual(ver, ver2) ver2 = await client.check_version(client.get_random_node()) self.assertEqual(ver, ver2) with mock.patch.object(AIOKafkaConnection, 'send') as mocked: mocked.side_effect = KafkaError('mocked exception') with self.assertRaises(UnrecognizedBrokerVersion): await client.check_version(client.get_random_node()) async def _get_conn(*args: Any, **kw: Any): return None client._get_conn = _get_conn with self.assertRaises(KafkaConnectionError): await client.check_version() await client.close()
def test_send_broker_unaware_request(self): 'Tests that call works when at least one of the host is available' mocked_conns = { ('kafka01', 9092): mock.MagicMock(), ('kafka02', 9092): mock.MagicMock(), ('kafka03', 9092): mock.MagicMock() } # inject KafkaConnection side effects fut = asyncio.Future(loop=self.loop) fut.set_exception(RuntimeError("kafka01 went away (unittest)")) mocked_conns[('kafka01', 9092)].send.return_value = fut fut2 = asyncio.Future(loop=self.loop) fut2.set_result(b'valid response') mocked_conns[('kafka02', 9092)].send.return_value = fut2 fut3 = asyncio.Future(loop=self.loop) fut3.set_exception(RuntimeError("kafka03 went away (unittest)")) mocked_conns[('kafka03', 9092)].send.return_value = fut3 client = AIOKafkaClient('kafka01:9092,kafka02:9092', loop=self.loop) client._conns = mocked_conns resp = self.loop.run_until_complete( client._send_broker_unaware_request(payloads=[b'fake request'], encoder_fn=mock.MagicMock(), decoder_fn=lambda x: x)) self.assertEqual(b'valid response', resp)
async def test_no_concurrent_send_on_connection(self): client = AIOKafkaClient(bootstrap_servers=self.hosts, metadata_max_age_ms=10000) await client.bootstrap() self.add_cleanup(client.close) await self.wait_topic(client, self.topic) node_id = client.get_random_node() wait_request = FetchRequest_v0( -1, # replica_id 500, # max_wait_ms 1024 * 1024, # min_bytes [(self.topic, [(0, 0, 1024)])]) vanila_request = MetadataRequest([]) loop = get_running_loop() send_time = loop.time() long_task = create_task(client.send(node_id, wait_request)) await asyncio.sleep(0.0001) self.assertFalse(long_task.done()) await client.send(node_id, vanila_request) resp_time = loop.time() fetch_resp = await long_task # Check error code like resp->topics[0]->partitions[0]->error_code self.assertEqual(fetch_resp.topics[0][1][0][1], 0) # Check that vanila request actually executed after wait request self.assertGreaterEqual(resp_time - send_time, 0.5)
def test_send_produce_request_raises_when_topic_unknown(self, protocol): @asyncio.coroutine def recv(request_id): return b'response' mocked_conns = {('broker_1', 4567): mock.MagicMock()} mocked_conns[('broker_1', 4567)].recv.side_effect = recv client = AIOKafkaClient(['broker_1:4567'], loop=self.loop) client._conns = mocked_conns brokers = [ BrokerMetadata(0, 'broker_1', 4567), BrokerMetadata(1, 'broker_2', 5678) ] topics = [ TopicMetadata('topic_doesnt_exist', UNKNOWN_TOPIC_OR_PARTITION, []), ] protocol.decode_metadata_response.return_value = MetadataResponse( brokers, topics) self.loop.run_until_complete(client.load_metadata_for_topics()) requests = [ ProduceRequest( "topic_doesnt_exist", 0, [create_message("a"), create_message("b")]) ] with self.assertRaises(UnknownTopicOrPartitionError): self.loop.run_until_complete(client.send_produce_request(requests))
def test_get_leader_for_unassigned_partitions(self, protocol): @asyncio.coroutine def recv(request_id): return b'response' mocked_conns = {('broker_1', 4567): mock.MagicMock()} mocked_conns[('broker_1', 4567)].recv.side_effect = recv client = AIOKafkaClient(['broker_1:4567'], loop=self.loop) client._conns = mocked_conns brokers = [ BrokerMetadata(0, 'broker_1', 4567), BrokerMetadata(1, 'broker_2', 5678) ] topics = [ TopicMetadata('topic_no_partitions', NO_LEADER, []), TopicMetadata('topic_unknown', UNKNOWN_TOPIC_OR_PARTITION, []), ] protocol.decode_metadata_response.return_value = MetadataResponse( brokers, topics) self.loop.run_until_complete(client.load_metadata_for_topics()) self.assertDictEqual({}, client._topics_to_brokers) with self.assertRaises(LeaderNotAvailableError): self.loop.run_until_complete( client._get_leader_for_partition('topic_no_partitions', 0)) with self.assertRaises(UnknownTopicOrPartitionError): self.loop.run_until_complete( client._get_leader_for_partition('topic_unknown', 0))
def test_force_metadata_update_multiple_times(self): client = AIOKafkaClient(loop=self.loop, bootstrap_servers=self.hosts, metadata_max_age_ms=10000) yield from client.bootstrap() self.add_cleanup(client.close) orig = client._metadata_update with mock.patch.object(client, '_metadata_update') as mocked: @asyncio.coroutine def new(*args, **kw): yield from asyncio.sleep(0.01, loop=self.loop) return (yield from orig(*args, **kw)) mocked.side_effect = new client.force_metadata_update() yield from asyncio.sleep(0.001, loop=self.loop) self.assertEqual(len(client._metadata_update.mock_calls), 1) client.force_metadata_update() yield from asyncio.sleep(0.001, loop=self.loop) self.assertEqual(len(client._metadata_update.mock_calls), 1) client.force_metadata_update() yield from asyncio.sleep(0.05, loop=self.loop) self.assertEqual(len(client._metadata_update.mock_calls), 1)
async 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(loop=self.loop) 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 )])]) subscriptions.assign_from_user({tp}) assignment = subscriptions.subscription.assignment tp_state = assignment.state_value(tp) client.send.side_effect = asyncio.coroutine(lambda n, r: resp) tp_state.seek(155) fetcher._in_flight.add(0) needs_wake_up = await fetcher._proc_fetch_request( assignment, 0, req) self.assertEqual(needs_wake_up, True) buf = fetcher._records[tp] # Test successful getone, the closest in batch offset=160 first = buf.getone() self.assertEqual(tp_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(tp_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_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_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)
async def test_init_with_csv(self): client = AIOKafkaClient( bootstrap_servers='127.0.0.1:9092,127.0.0.2:9092,127.0.0.3:9092') self.assertEqual( sorted([('127.0.0.1', 9092, socket.AF_INET), ('127.0.0.2', 9092, socket.AF_INET), ('127.0.0.3', 9092, socket.AF_INET)]), sorted(client.hosts))
def test_close(self): client = AIOKafkaClient(['broker_1:4567'], loop=self.loop) m1 = mock.Mock() m2 = mock.Mock() client._conns = {('host1', 4567): m1, ('host2', 5678): m2} client.close() self.assertEqual({}, client._conns) m1.close.assert_raises_with() m2.close.assert_raises_with()
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 test_init_with_csv(self): client = AIOKafkaClient( loop=self.loop, bootstrap_servers='kafka01:9092,kafka02:9092,kafka03:9092') self.assertEqual(sorted({'kafka01': 9092, 'kafka02': 9092, 'kafka03': 9092}.items()), sorted(client.hosts))
async def test_bootstrap(self): client = AIOKafkaClient(bootstrap_servers='0.42.42.42:444') with self.assertRaises(KafkaConnectionError): await client.bootstrap() client = AIOKafkaClient(bootstrap_servers=self.hosts) await client.bootstrap() await self.wait_topic(client, 'test_topic') metadata = await client.fetch_all_metadata() self.assertTrue('test_topic' in metadata.topics()) client.set_topics(['t2', 't3']) client.set_topics(['t2', 't3']) # should be ignored client.add_topic('t2') # should be ignored # bootstrap again -- no error expected await client.bootstrap() await client.close()
def __init__(self, *, loop, bootstrap_servers='localhost', client_id=None, metadata_max_age_ms=300000, request_timeout_ms=40000, api_version='auto', acks=1, key_serializer=None, value_serializer=None, compression_type=None, max_batch_size=16384, partitioner=DefaultPartitioner(), max_request_size=1048576, linger_ms=0, send_backoff_ms=100, retry_backoff_ms=100): if acks not in (0, 1, -1, 'all'): raise ValueError("Invalid ACKS parameter") if compression_type not in ('gzip', 'snappy', 'lz4', None): raise ValueError("Invalid compression type!") if api_version not in ('auto', '0.10', '0.9', '0.8.2', '0.8.1', '0.8.0'): raise ValueError("Unsupported Kafka version") self._PRODUCER_CLIENT_ID_SEQUENCE += 1 if client_id is None: client_id = 'aiokafka-producer-%s' % \ self._PRODUCER_CLIENT_ID_SEQUENCE if acks == 'all': acks = -1 self._acks = acks self._key_serializer = key_serializer self._value_serializer = value_serializer self._compression_type = compression_type self._partitioner = partitioner self._max_request_size = max_request_size self._request_timeout_ms = request_timeout_ms 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, api_version=api_version) self._metadata = self.client.cluster self._message_accumulator = MessageAccumulator( self._metadata, max_batch_size, self._compression_type, self._request_timeout_ms / 1000, loop) self._sender_task = None self._in_flight = set() self._closed = False self._loop = loop self._retry_backoff = retry_backoff_ms / 1000 self._linger_time = linger_ms / 1000