def _maybe_do_transactional_request(self): txn_manager = self._txn_manager # If we have any new partitions, still not added to the transaction # we need to do that before committing tps = txn_manager.partitions_to_add() if tps: return ensure_future(self._do_add_partitions_to_txn(tps), loop=self._loop) # We need to add group to transaction before we can commit the offset group_id = txn_manager.consumer_group_to_add() if group_id is not None: return ensure_future(self._do_add_offsets_to_txn(group_id), loop=self._loop) # Now commit the added group's offset commit_data = txn_manager.offsets_to_commit() if commit_data is not None: offsets, group_id, fut = commit_data return ensure_future(self._do_txn_offset_commit( offsets, group_id, fut), loop=self._loop) commit_result = txn_manager.needs_transaction_commit() if commit_result is not None: return ensure_future(self._do_txn_commit(commit_result), loop=self._loop)
async def test_producer_transactional_send_batch_outside_txn(self): producer = AIOKafkaProducer( loop=self.loop, bootstrap_servers=self.hosts, transactional_id="sobaka_producer", client_id="p1") await producer.start() self.add_cleanup(producer.stop) batch = producer.create_batch() batch.append(timestamp=None, key=None, value=b"2") batch.close() # Can not send if not yet in transaction with self.assertRaises(IllegalOperation): await producer.send_batch(batch, self.topic, partition=0) await producer.begin_transaction() await producer.send(self.topic, value=b"1", partition=0) commit_task = ensure_future(producer.commit_transaction()) await asyncio.sleep(0.001, loop=self.loop) self.assertFalse(commit_task.done()) # Already not in transaction with self.assertRaises(IllegalOperation): await producer.send_batch(batch, self.topic, partition=0) await commit_task # Transaction needs to be restarted with self.assertRaises(IllegalOperation): await producer.send_batch(batch, self.topic, partition=0)
def connect(self): loop = self._loop self._closed_fut = create_future(loop=loop) if self._secutity_protocol == "PLAINTEXT": ssl = None else: assert self._secutity_protocol == "SSL" assert self._ssl_context is not None ssl = self._ssl_context # Create streams same as `open_connection`, but using custom protocol reader = asyncio.StreamReader(limit=READER_LIMIT, loop=loop) protocol = AIOKafkaProtocol(self._closed_fut, reader, loop=loop) transport, _ = yield from asyncio.wait_for( loop.create_connection( lambda: protocol, self.host, self.port, ssl=ssl), loop=loop, timeout=self._request_timeout) writer = asyncio.StreamWriter(transport, protocol, reader, loop) self._reader, self._writer, self._protocol = reader, writer, protocol # Start reader task. self._read_task = ensure_future(self._read(), loop=loop) self._read_task.add_done_callback(self._on_read_task_error) # Start idle checker if self._max_idle_ms is not None: self._idle_handle = self._loop.call_soon(self._idle_check) return reader, writer
async def test_producer_transactional_send_batch_outside_txn(self): producer = AIOKafkaProducer(loop=self.loop, bootstrap_servers=self.hosts, transactional_id="sobaka_producer", client_id="p1") await producer.start() self.add_cleanup(producer.stop) batch = producer.create_batch() batch.append(timestamp=None, key=None, value=b"2") batch.close() # Can not send if not yet in transaction with self.assertRaises(IllegalOperation): await producer.send_batch(batch, self.topic, partition=0) await producer.begin_transaction() await producer.send(self.topic, value=b"1", partition=0) commit_task = ensure_future(producer.commit_transaction()) await asyncio.sleep(0.001, loop=self.loop) self.assertFalse(commit_task.done()) # Already not in transaction with self.assertRaises(IllegalOperation): await producer.send_batch(batch, self.topic, partition=0) await commit_task # Transaction needs to be restarted with self.assertRaises(IllegalOperation): await producer.send_batch(batch, self.topic, partition=0)
def start_pending_task(coro, node_id, self=self): task = ensure_future(coro, loop=self._loop) self._pending_tasks.add(task) self._in_flight.add(node_id) def on_done(fut, self=self): self._in_flight.discard(node_id) task.add_done_callback(on_done)
def start_pending_task(coro, node_id, self=self): task = ensure_future(coro, loop=self._loop) self._pending_tasks.add(task) self._in_flight.add(node_id) def on_done(fut, self=self): self._in_flight.discard(node_id) task.add_done_callback(on_done)
def bootstrap(self): """Try to to bootstrap initial cluster metadata""" # using request v0 for bootstap if not sure v1 is available if self.api_version == "auto" or self.api_version < (0, 10): metadata_request = MetadataRequest[0]([]) else: metadata_request = MetadataRequest[1]([]) for host, port, _ in self.hosts: log.debug("Attempting to bootstrap via node at %s:%s", host, port) try: bootstrap_conn = yield from create_conn( host, port, loop=self._loop, client_id=self._client_id, request_timeout_ms=self._request_timeout_ms, ssl_context=self._ssl_context, security_protocol=self._security_protocol, max_idle_ms=self._connections_max_idle_ms, sasl_mechanism=self._sasl_mechanism, sasl_plain_username=self._sasl_plain_username, sasl_plain_password=self._sasl_plain_password) except (OSError, asyncio.TimeoutError) as err: log.error('Unable connect to "%s:%s": %s', host, port, err) continue try: metadata = yield from bootstrap_conn.send(metadata_request) except KafkaError as err: log.warning('Unable to request metadata from "%s:%s": %s', host, port, err) bootstrap_conn.close() continue self.cluster.update_metadata(metadata) # A cluster with no topics can return no broker metadata... # In that case, we should keep the bootstrap connection till # we get a normal cluster layout. if not len(self.cluster.brokers()): bootstrap_id = ('bootstrap', ConnectionGroup.DEFAULT) self._conns[bootstrap_id] = bootstrap_conn else: bootstrap_conn.close() log.debug('Received cluster metadata: %s', self.cluster) break else: raise ConnectionError( 'Unable to bootstrap from {}'.format(self.hosts)) # detect api version if need if self._api_version == 'auto': self._api_version = yield from self.check_version() if self._sync_task is None: # starting metadata synchronizer task self._sync_task = ensure_future( self._md_synchronizer(), loop=self._loop)
async def test_producer_transactional_cancel_txn_methods(self): producer = AIOKafkaProducer(loop=self.loop, bootstrap_servers=self.hosts, transactional_id="sobaka_producer", client_id="p1") txn_manager = producer._txn_manager self.assertEqual(txn_manager.state, TransactionState.UNINITIALIZED) await producer.start() self.add_cleanup(producer.stop) self.assertEqual(txn_manager.state, TransactionState.READY) async def cancel(task): # Coroutines will not be started until we yield at least 1ce await asyncio.sleep(0) task.cancel() try: await task except asyncio.CancelledError: pass # test cancel begin_transaction. task = ensure_future(producer.begin_transaction()) await cancel(task) self.assertEqual(txn_manager.state, TransactionState.READY) # test cancel commit_transaction. Commit should not be cancelled. await producer.begin_transaction() self.assertEqual(txn_manager.state, TransactionState.IN_TRANSACTION) task = ensure_future(producer.commit_transaction()) await cancel(task) self.assertEqual(txn_manager.state, TransactionState.COMMITTING_TRANSACTION) await asyncio.sleep(0.1) self.assertEqual(txn_manager.state, TransactionState.READY) # test cancel abort_transaction. Abort should also not be cancelled. await producer.begin_transaction() self.assertEqual(txn_manager.state, TransactionState.IN_TRANSACTION) task = ensure_future(producer.abort_transaction()) await cancel(task) self.assertEqual(txn_manager.state, TransactionState.ABORTING_TRANSACTION) await asyncio.sleep(0.1) self.assertEqual(txn_manager.state, TransactionState.READY)
def test_load_metadata(self): brokers = [(0, 'broker_1', 4567), (1, 'broker_2', 5678)] topics = [(NO_ERROR, 'topic_1', [(NO_ERROR, 0, 1, [1, 2], [1, 2])]), (NO_ERROR, 'topic_2', [ (NO_LEADER, 0, -1, [], []), (NO_LEADER, 1, 1, [], []), ]), (NO_LEADER, 'topic_no_partitions', []), (UNKNOWN_TOPIC_OR_PARTITION, 'topic_unknown', []), (NO_ERROR, 'topic_3', [(NO_ERROR, 0, 0, [0, 1], [0, 1]), (NO_ERROR, 1, 1, [1, 0], [1, 0]), (NO_ERROR, 2, 0, [0, 1], [0, 1])]), (NO_ERROR, 'topic_4', [ (NO_ERROR, 0, 0, [0, 1], [0, 1]), (REPLICA_NOT_AVAILABLE, 1, 1, [1, 0], [1, 0]), ])] @asyncio.coroutine def send(request_id): return MetadataResponse(brokers, topics) mocked_conns = {(0, 0): mock.MagicMock()} mocked_conns[(0, 0)].send.side_effect = send client = AIOKafkaClient(loop=self.loop, bootstrap_servers=['broker_1:4567']) task = ensure_future(client._md_synchronizer(), loop=self.loop) client._conns = mocked_conns client.cluster.update_metadata(MetadataResponse(brokers[:1], [])) self.loop.run_until_complete(client.force_metadata_update()) task.cancel() md = client.cluster c_brokers = md.brokers() self.assertEqual(len(c_brokers), 2) expected_brokers = [(0, 'broker_1', 4567, None), (1, 'broker_2', 5678, None)] self.assertEqual(sorted(expected_brokers), sorted(list(c_brokers))) c_topics = md.topics() self.assertEqual(len(c_topics), 4) self.assertEqual(md.partitions_for_topic('topic_1'), set([0])) self.assertEqual(md.partitions_for_topic('topic_2'), set([0, 1])) self.assertEqual(md.partitions_for_topic('topic_3'), set([0, 1, 2])) self.assertEqual(md.partitions_for_topic('topic_4'), set([0, 1])) self.assertEqual(md.available_partitions_for_topic('topic_2'), set([1])) mocked_conns[(0, 0)].connected.return_value = False is_ready = self.loop.run_until_complete(client.ready(0)) self.assertEqual(is_ready, False) is_ready = self.loop.run_until_complete(client.ready(1)) self.assertEqual(is_ready, False) self.assertEqual(mocked_conns, {}) with self.assertRaises(NodeNotReadyError): self.loop.run_until_complete(client.send(0, None))
async def test_producer_transactional_cancel_txn_methods(self): producer = AIOKafkaProducer( loop=self.loop, bootstrap_servers=self.hosts, transactional_id="sobaka_producer", client_id="p1") txn_manager = producer._txn_manager self.assertEqual(txn_manager.state, TransactionState.UNINITIALIZED) await producer.start() self.add_cleanup(producer.stop) self.assertEqual(txn_manager.state, TransactionState.READY) async def cancel(task): # Coroutines will not be started until we yield at least 1ce await asyncio.sleep(0) task.cancel() try: await task except asyncio.CancelledError: pass # test cancel begin_transaction. task = ensure_future(producer.begin_transaction()) await cancel(task) self.assertEqual(txn_manager.state, TransactionState.READY) # test cancel commit_transaction. Commit should not be cancelled. await producer.begin_transaction() self.assertEqual(txn_manager.state, TransactionState.IN_TRANSACTION) task = ensure_future(producer.commit_transaction()) await cancel(task) self.assertEqual( txn_manager.state, TransactionState.COMMITTING_TRANSACTION) await asyncio.sleep(0.1) self.assertEqual(txn_manager.state, TransactionState.READY) # test cancel abort_transaction. Abort should also not be cancelled. await producer.begin_transaction() self.assertEqual(txn_manager.state, TransactionState.IN_TRANSACTION) task = ensure_future(producer.abort_transaction()) await cancel(task) self.assertEqual( txn_manager.state, TransactionState.ABORTING_TRANSACTION) await asyncio.sleep(0.1) self.assertEqual(txn_manager.state, TransactionState.READY)
def start(self): """Connect to Kafka cluster and check server version""" log.debug("Starting the Kafka producer") # trace yield from self.client.bootstrap() if self._compression_type == 'lz4': assert self.client.api_version >= (0, 8, 2), \ 'LZ4 Requires >= Kafka 0.8.2 Brokers' self._sender_task = ensure_future( self._sender_routine(), loop=self._loop) self._message_accumulator.set_api_version(self.client.api_version) log.debug("Kafka producer started")
async def _wait_for_data_or_error(self, coro): if self._group_id is None: return (await coro) else: coordination_error_fut = self._coordinator.error_future data_task = ensure_future(coro, loop=self._loop) await asyncio.wait([data_task, coordination_error_fut], return_when=asyncio.FIRST_COMPLETED) # Check for errors in coordination and raise if any if coordination_error_fut.done(): coordination_error_fut.result() # Raises set exception if any return (await data_task)
def start(self): """Connect to Kafka cluster and check server version""" log.debug("Starting the Kafka producer") # trace yield from self.client.bootstrap() if self._compression_type == 'lz4': assert self.client.api_version >= (0, 8, 2), \ 'LZ4 Requires >= Kafka 0.8.2 Brokers' if self._txn_manager is not None and self.client.api_version < (0, 11): raise UnsupportedVersionError( "Indempotent producer available only for Broker vesion 0.11" " and above") self._sender_task = ensure_future(self._sender_routine(), loop=self._loop) self._message_accumulator.set_api_version(self.client.api_version) self._producer_magic = 0 if self.client.api_version < (0, 10) else 1 log.debug("Kafka producer started")
def _on_change_subscription(self): """ This is `group rebalanced` signal handler used to update fetch positions of assigned partitions """ if self._closed: # pragma: no cover return # fetch positions if we have partitions we're subscribed # to that we don't know the offset for if not self._subscription.has_all_fetch_positions(): task = ensure_future(self._update_fetch_positions( self._subscription.missing_fetch_positions()), loop=self._loop) self._pending_position_fetches.add(task) def on_done(fut, tasks=self._pending_position_fetches): tasks.discard(fut) try: fut.result() except Exception as err: # pragma: no cover log.error("Failed to update fetch positions: %r", err) task.add_done_callback(on_done)
def _sender_routine(self): """ Background task, that sends pending batches to leader nodes for batch's partition. This incapsulates same logic as Java's `Sender` background thread. Because we use asyncio this is more event based loop, rather than counting timeout till next possible even like in Java. """ tasks = set() txn_task = None # Track a single task for transaction interactions try: while True: # If indempotence or transactions are turned on we need to # have a valid PID to send any request below log.debug('+maybe wait for pid') yield from self._maybe_wait_for_pid() log.debug('-maybe wait for pid') waiters = set() # As transaction coordination is done via a single, separate # socket we do not need to pump it to several nodes, as we do # with produce requests. # We will only have 1 task at a time and will try to spawn # another once that is done. txn_manager = self._txn_manager muted_partitions = self._muted_partitions if txn_manager is not None and \ txn_manager.transactional_id is not None: if txn_task is None or txn_task.done(): txn_task = self._maybe_do_transactional_request() if txn_task is not None: log.debug('has txn task') tasks.add(txn_task) else: log.debug('creating new txn task') # Waiters will not be awaited on exit, tasks will waiters.add(txn_manager.make_task_waiter()) # We can't have a race condition between # AddPartitionsToTxnRequest and a ProduceRequest, so we # mute the partition until added. muted_partitions = (muted_partitions | txn_manager.partitions_to_add()) batches, unknown_leaders_exist = \ self._message_accumulator.drain_by_nodes( ignore_nodes=self._in_flight, muted_partitions=muted_partitions) # create produce task for every batch for node_id, batches in batches.items(): task = ensure_future(self._send_produce_req( node_id, batches), loop=self._loop) self._in_flight.add(node_id) for tp in batches: self._muted_partitions.add(tp) tasks.add(task) if unknown_leaders_exist: # we have at least one unknown partition's leader, # try to update cluster metadata and wait backoff time fut = self.client.force_metadata_update() waiters |= tasks.union([fut]) else: fut = self._message_accumulator.data_waiter() waiters |= tasks.union([fut]) # wait when: # * At least one of produce task is finished # * Data for new partition arrived # * Metadata update if partition leader unknown log.debug('+SENDER WAIT FOR %r' % (waiters, )) if waiters: done, _ = yield from asyncio.wait( waiters, return_when=asyncio.FIRST_COMPLETED, loop=self._loop) log.debug('-SENDER WAIT FOR') else: yield from asyncio.sleep(0.5) # done tasks should never produce errors, if they are it's a # bug for task in done: task.result() tasks -= done except asyncio.CancelledError: # done tasks should never produce errors, if they are it's a bug for task in tasks: yield from task except (ProducerFenced, OutOfOrderSequenceNumber, TransactionalIdAuthorizationFailed): raise except Exception as exc: # pragma: no cover log.error("Unexpected error in sender routine", exc_info=True) if self._on_irrecoverable_error: res = self._on_irrecoverable_error(exc) if asyncio.iscoroutine(res): # callback can be async def yield from res raise KafkaError("Unexpected error during batch delivery")
def start(self): # If producer is indempotent we need to assure we have PID found yield from self._maybe_wait_for_pid() self._sender_task = ensure_future(self._sender_routine(), loop=self._loop) self._sender_task.add_done_callback(self._fail_all)
def test_fetcher__update_fetch_positions(self): client = AIOKafkaClient( loop=self.loop, bootstrap_servers=[]) subscriptions = SubscriptionState(loop=self.loop) fetcher = Fetcher(client, subscriptions, loop=self.loop) self.add_cleanup(fetcher.close) # Disable backgroud task fetcher._fetch_task.cancel() try: yield from fetcher._fetch_task except asyncio.CancelledError: pass fetcher._fetch_task = ensure_future( asyncio.sleep(1000000, loop=self.loop), loop=self.loop) partition = TopicPartition('test', 0) offsets = {partition: OffsetAndTimestamp(12, -1)} @asyncio.coroutine def _proc_offset_request(node_id, topic_data): return offsets fetcher._proc_offset_request = mock.Mock() fetcher._proc_offset_request.side_effect = _proc_offset_request def reset_assignment(): subscriptions.assign_from_user({partition}) assignment = subscriptions.subscription.assignment tp_state = assignment.state_value(partition) return assignment, tp_state assignment, tp_state = reset_assignment() self.assertIsNone(tp_state._position) # CASE: reset from committed # In basic case we will need to wait for committed update_task = ensure_future( fetcher._update_fetch_positions(assignment, 0, [partition]), loop=self.loop ) yield from asyncio.sleep(0.1, loop=self.loop) self.assertFalse(update_task.done()) # Will continue only after committed is resolved tp_state.update_committed(OffsetAndMetadata(4, "")) needs_wakeup = yield from update_task self.assertFalse(needs_wakeup) self.assertEqual(tp_state._position, 4) self.assertEqual(fetcher._proc_offset_request.call_count, 0) # CASE: will not query committed if position already present yield from fetcher._update_fetch_positions(assignment, 0, [partition]) self.assertEqual(tp_state._position, 4) self.assertEqual(fetcher._proc_offset_request.call_count, 0) # CASE: awaiting_reset for the partition tp_state.await_reset(OffsetResetStrategy.LATEST) self.assertIsNone(tp_state._position) yield from fetcher._update_fetch_positions(assignment, 0, [partition]) self.assertEqual(tp_state._position, 12) self.assertEqual(fetcher._proc_offset_request.call_count, 1) # CASE: seeked while waiting for committed to be resolved assignment, tp_state = reset_assignment() update_task = ensure_future( fetcher._update_fetch_positions(assignment, 0, [partition]), loop=self.loop ) yield from asyncio.sleep(0.1, loop=self.loop) self.assertFalse(update_task.done()) tp_state.seek(8) tp_state.update_committed(OffsetAndMetadata(4, "")) yield from update_task self.assertEqual(tp_state._position, 8) self.assertEqual(fetcher._proc_offset_request.call_count, 1) # CASE: awaiting_reset during waiting for committed assignment, tp_state = reset_assignment() update_task = ensure_future( fetcher._update_fetch_positions(assignment, 0, [partition]), loop=self.loop ) yield from asyncio.sleep(0.1, loop=self.loop) self.assertFalse(update_task.done()) tp_state.await_reset(OffsetResetStrategy.LATEST) tp_state.update_committed(OffsetAndMetadata(4, "")) yield from update_task self.assertEqual(tp_state._position, 12) self.assertEqual(fetcher._proc_offset_request.call_count, 2) # CASE: reset using default strategy if committed offset undefined assignment, tp_state = reset_assignment() self.loop.call_later( 0.01, tp_state.update_committed, OffsetAndMetadata(-1, "")) yield from fetcher._update_fetch_positions(assignment, 0, [partition]) self.assertEqual(tp_state._position, 12) self.assertEqual(fetcher._records, {}) # CASE: set error if _default_reset_strategy = OffsetResetStrategy.NONE assignment, tp_state = reset_assignment() self.loop.call_later( 0.01, tp_state.update_committed, OffsetAndMetadata(-1, "")) fetcher._default_reset_strategy = OffsetResetStrategy.NONE needs_wakeup = yield from fetcher._update_fetch_positions( assignment, 0, [partition]) self.assertTrue(needs_wakeup) self.assertIsNone(tp_state._position) self.assertIsInstance(fetcher._records[partition], FetchError) fetcher._records.clear() # CASE: if _proc_offset_request errored, we will retry on another spin fetcher._proc_offset_request.side_effect = UnknownError() assignment, tp_state = reset_assignment() tp_state.await_reset(OffsetResetStrategy.LATEST) yield from fetcher._update_fetch_positions(assignment, 0, [partition]) self.assertIsNone(tp_state._position) self.assertTrue(tp_state.awaiting_reset) # CASE: reset 2 partitions separately, 1 will raise, 1 will get # committed fetcher._proc_offset_request.side_effect = _proc_offset_request partition2 = TopicPartition('test', 1) subscriptions.assign_from_user({partition, partition2}) assignment = subscriptions.subscription.assignment tp_state = assignment.state_value(partition) tp_state2 = assignment.state_value(partition2) tp_state.await_reset(OffsetResetStrategy.LATEST) self.loop.call_later( 0.01, tp_state2.update_committed, OffsetAndMetadata(5, "")) yield from fetcher._update_fetch_positions( assignment, 0, [partition, partition2]) self.assertEqual(tp_state.position, 12) self.assertEqual(tp_state2.position, 5)
def __init__(self, client, subscriptions, *, loop, key_deserializer=None, value_deserializer=None, fetch_min_bytes=1, fetch_max_bytes=52428800, fetch_max_wait_ms=500, max_partition_fetch_bytes=1048576, check_crcs=True, fetcher_timeout=0.2, prefetch_backoff=0.1, retry_backoff_ms=100, auto_offset_reset='latest', isolation_level="read_uncommitted"): self._client = client self._loop = loop self._key_deserializer = key_deserializer self._value_deserializer = value_deserializer self._fetch_min_bytes = fetch_min_bytes self._fetch_max_bytes = fetch_max_bytes self._fetch_max_wait_ms = fetch_max_wait_ms self._max_partition_fetch_bytes = max_partition_fetch_bytes self._check_crcs = check_crcs self._fetcher_timeout = fetcher_timeout self._prefetch_backoff = prefetch_backoff self._retry_backoff = retry_backoff_ms / 1000 self._subscriptions = subscriptions self._default_reset_strategy = OffsetResetStrategy.from_str( auto_offset_reset) if isolation_level == "read_uncommitted": self._isolation_level = READ_UNCOMMITTED elif isolation_level == "read_committed": self._isolation_level = READ_COMMITTED else: raise ValueError( "Incorrect isolation level {}".format(isolation_level)) self._records = collections.OrderedDict() self._in_flight = set() self._pending_tasks = set() self._wait_consume_future = None self._fetch_waiters = set() # SubscriptionState will pass Coordination critical errors to those # waiters directly self._subscriptions.register_fetch_waiters(self._fetch_waiters) if client.api_version >= (0, 11): req_version = 4 elif client.api_version >= (0, 10, 1): req_version = 3 elif client.api_version >= (0, 10): req_version = 2 else: req_version = 1 self._fetch_request_class = FetchRequest[req_version] self._fetch_task = ensure_future(self._fetch_requests_routine(), loop=loop) self._closed = False
def _create_reader_task(self): self_ref = weakref.ref(self) read_task = ensure_future(self._read(self_ref), loop=self._loop) read_task.add_done_callback( functools.partial(self._on_read_task_error, self_ref)) return read_task
def _fetch_requests_routine(self): """ Background task, that always prefetches next result page. The algorithm: * Group partitions per node, which is the leader for it. * If all partitions for this node need prefetch - do it right away * If any partition has some data (in `self._records`) wait up till `prefetch_backoff` so application can consume data from it. * If data in `self._records` is not consumed up to `prefetch_backoff` just request data for other partitions from this node. We request data in such manner cause Kafka blocks the connection if we perform a FetchRequest and we don't have enough data. This means we must perform a FetchRequest to as many partitions as we can in a node. Original Java Kafka client processes data differently, as it only prefetches data if all messages were given to application (i.e. if `self._records` are empty). We don't use this method, cause we allow to process partitions separately (by passing `partitions` list to `getall()` call of the consumer), which can end up in a long wait if some partitions (or topics) are processed slower, than others. """ try: while True: # Reset consuming signal future. self._wait_consume_future = create_future(loop=self._loop) # Create and send fetch requests requests, timeout = self._create_fetch_requests() for node_id, request in requests: node_ready = yield from self._client.ready(node_id) if not node_ready: # We will request it on next routine continue log.debug("Sending FetchRequest to node %s", node_id) task = ensure_future(self._proc_fetch_request( node_id, request), loop=self._loop) self._fetch_tasks.add(task) self._in_flight.add(node_id) done_set, _ = yield from asyncio.wait( chain(self._fetch_tasks, [self._wait_consume_future]), loop=self._loop, timeout=timeout, return_when=asyncio.FIRST_COMPLETED) # Process fetch tasks results if any done_fetches = self._fetch_tasks.intersection(done_set) if done_fetches: has_new_data = any(fut.result() for fut in done_fetches) if has_new_data: # we added some messages to self._records, # wake up getters self._notify(self._wait_empty_future) self._fetch_tasks -= done_fetches except asyncio.CancelledError: pass except Exception: # pragma: no cover log.error("Unexpected error in fetcher routine", exc_info=True)
def __init__(self, client, subscriptions, *, loop, key_deserializer=None, value_deserializer=None, fetch_min_bytes=1, fetch_max_wait_ms=500, max_partition_fetch_bytes=1048576, check_crcs=True, fetcher_timeout=0.2, prefetch_backoff=0.1, retry_backoff_ms=100): """Initialize a Kafka Message Fetcher. Parameters: client (AIOKafkaClient): kafka client subscription (SubscriptionState): instance of SubscriptionState located in kafka.consumer.subscription_state key_deserializer (callable): Any callable that takes a raw message key and returns a deserialized key. value_deserializer (callable, optional): Any callable that takes a raw message value and returns a deserialized value. fetch_min_bytes (int): Minimum amount of data the server should return for a fetch request, otherwise wait up to fetch_max_wait_ms for more data to accumulate. Default: 1. fetch_max_wait_ms (int): The maximum amount of time in milliseconds the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch_min_bytes. Default: 500. max_partition_fetch_bytes (int): The maximum amount of data per-partition the server will return. The maximum total memory used for a request = #partitions * max_partition_fetch_bytes. This size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large message on a certain partition. Default: 1048576. check_crcs (bool): Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance. Default: True fetcher_timeout (float): Maximum polling interval in the background fetching routine. Default: 0.2 prefetch_backoff (float): number of seconds to wait until consumption of partition is paused. Paused partitions will not request new data from Kafka server (will not be included in next poll request). """ self._client = client self._loop = loop 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._check_crcs = check_crcs self._fetcher_timeout = fetcher_timeout self._prefetch_backoff = prefetch_backoff self._retry_backoff = retry_backoff_ms / 1000 self._subscriptions = subscriptions self._records = collections.OrderedDict() self._in_flight = set() self._fetch_tasks = set() self._wait_consume_future = None self._wait_empty_future = None req_version = 2 if client.api_version >= (0, 10) else 1 self._fetch_request_class = FetchRequest[req_version] self._fetch_task = ensure_future(self._fetch_requests_routine(), loop=loop)
def _create_reader_task(self): self_ref = weakref.ref(self) read_task = ensure_future(self._read(self_ref), loop=self._loop) read_task.add_done_callback( functools.partial(self._on_read_task_error, self_ref)) return read_task
def __init__(self, client, subscription, *, loop, group_id='aiokafka-default-group', session_timeout_ms=30000, heartbeat_interval_ms=3000, retry_backoff_ms=100, enable_auto_commit=True, auto_commit_interval_ms=5000, assignors=(RoundRobinPartitionAssignor, ), exclude_internal_topics=True, assignment_changed_cb=None): """Initialize the coordination manager. Parameters: client (AIOKafkaClient): kafka client subscription (SubscriptionState): instance of SubscriptionState located in kafka.consumer.subscription_state group_id (str): name of the consumer group to join for dynamic partition assignment (if enabled), and to use for fetching and committing offsets. Default: 'aiokafka-default-group' enable_auto_commit (bool): If true the consumer's offset will be periodically committed in the background. Default: True. auto_commit_interval_ms (int): milliseconds between automatic offset commits, if enable_auto_commit is True. Default: 5000. assignors (list): List of objects to use to distribute partition ownership amongst consumer instances when group management is used. Default: [RoundRobinPartitionAssignor] heartbeat_interval_ms (int): The expected time in milliseconds between heartbeats to the consumer coordinator when using Kafka's group management feature. Heartbeats are used to ensure that the consumer's session stays active and to facilitate rebalancing when new consumers join or leave the group. The value must be set lower than session_timeout_ms, but typically should be set no higher than 1/3 of that value. It can be adjusted even lower to control the expected time for normal rebalances. Default: 3000 session_timeout_ms (int): The timeout used to detect failures when using Kafka's group managementment facilities. Default: 30000 retry_backoff_ms (int): Milliseconds to backoff when retrying on errors. Default: 100. """ super().__init__(client, subscription, loop=loop, exclude_internal_topics=exclude_internal_topics, assignment_changed_cb=assignment_changed_cb) self._loop = loop self._session_timeout_ms = session_timeout_ms self._heartbeat_interval_ms = heartbeat_interval_ms self._retry_backoff_ms = retry_backoff_ms self._assignors = assignors self._enable_auto_commit = enable_auto_commit self._auto_commit_interval_ms = auto_commit_interval_ms self.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID self.member_id = JoinGroupRequest.UNKNOWN_MEMBER_ID self.group_id = group_id self.coordinator_id = None self.rejoin_needed = True self.pending_rejoin_fut = None # `ensure_active_group` can be called from several places # (from consumer and from heartbeat task), so we need lock self._rejoin_lock = asyncio.Lock(loop=loop) self._auto_commit_task = None # _closing future used as a signal to heartbeat task for finish ASAP self._closing = create_future(loop=loop) self.heartbeat_task = ensure_future(self._heartbeat_task_routine(), loop=loop) if self._enable_auto_commit: interval = self._auto_commit_interval_ms / 1000 self._auto_commit_task = ensure_future( self.auto_commit_routine(interval), loop=loop)
def __init__( self, client, subscriptions, *, loop, key_deserializer=None, value_deserializer=None, fetch_min_bytes=1, fetch_max_bytes=52428800, fetch_max_wait_ms=500, max_partition_fetch_bytes=1048576, check_crcs=True, fetcher_timeout=0.2, prefetch_backoff=0.1, retry_backoff_ms=100, auto_offset_reset='latest', isolation_level="read_uncommitted"): self._client = client self._loop = loop self._key_deserializer = key_deserializer self._value_deserializer = value_deserializer self._fetch_min_bytes = fetch_min_bytes self._fetch_max_bytes = fetch_max_bytes self._fetch_max_wait_ms = fetch_max_wait_ms self._max_partition_fetch_bytes = max_partition_fetch_bytes self._check_crcs = check_crcs self._fetcher_timeout = fetcher_timeout self._prefetch_backoff = prefetch_backoff self._retry_backoff = retry_backoff_ms / 1000 self._subscriptions = subscriptions self._default_reset_strategy = OffsetResetStrategy.from_str( auto_offset_reset) if isolation_level == "read_uncommitted": self._isolation_level = READ_UNCOMMITTED elif isolation_level == "read_committed": self._isolation_level = READ_COMMITTED else: raise ValueError( "Incorrect isolation level {}".format(isolation_level)) self._records = collections.OrderedDict() self._in_flight = set() self._pending_tasks = set() self._wait_consume_future = None self._fetch_waiters = set() # SubscriptionState will pass Coordination critical errors to those # waiters directly self._subscriptions.register_fetch_waiters(self._fetch_waiters) if client.api_version >= (0, 11): req_version = 4 elif client.api_version >= (0, 10, 1): req_version = 3 elif client.api_version >= (0, 10): req_version = 2 else: req_version = 1 self._fetch_request_class = FetchRequest[req_version] self._fetch_task = ensure_future( self._fetch_requests_routine(), loop=loop) self._closed = False
def bootstrap(self): """Try to to bootstrap initial cluster metadata""" # using request v0 for bootstap if not sure v1 is available if self._api_version == "auto" or self._api_version < (0, 10): metadata_request = MetadataRequest[0]([]) else: metadata_request = MetadataRequest[1]([]) version_hint = None if self._api_version != "auto": version_hint = self._api_version for host, port, _ in self.hosts: log.debug("Attempting to bootstrap via node at %s:%s", host, port) try: bootstrap_conn = yield from create_conn( host, port, loop=self._loop, client_id=self._client_id, request_timeout_ms=self._request_timeout_ms, ssl_context=self._ssl_context, security_protocol=self._security_protocol, max_idle_ms=self._connections_max_idle_ms, sasl_mechanism=self._sasl_mechanism, sasl_plain_username=self._sasl_plain_username, sasl_plain_password=self._sasl_plain_password, sasl_kerberos_service_name=self._sasl_kerberos_service_name, # noqa: ignore=E501 sasl_kerberos_domain_name=self._sasl_kerberos_domain_name, version_hint=version_hint) except (OSError, asyncio.TimeoutError) as err: log.error('Unable connect to "%s:%s": %s', host, port, err) continue try: metadata = yield from bootstrap_conn.send(metadata_request) except KafkaError as err: log.warning('Unable to request metadata from "%s:%s": %s', host, port, err) bootstrap_conn.close() continue self.cluster.update_metadata(metadata) # A cluster with no topics can return no broker metadata... # In that case, we should keep the bootstrap connection till # we get a normal cluster layout. if not len(self.cluster.brokers()): bootstrap_id = ('bootstrap', ConnectionGroup.DEFAULT) self._conns[bootstrap_id] = bootstrap_conn else: bootstrap_conn.close() log.debug('Received cluster metadata: %s', self.cluster) break else: raise ConnectionError( 'Unable to bootstrap from {}'.format(self.hosts)) # detect api version if need if self._api_version == 'auto': self._api_version = yield from self.check_version() if self._sync_task is None: # starting metadata synchronizer task self._sync_task = ensure_future( self._md_synchronizer(), loop=self._loop)
def __init__(self, client, subscriptions, *, loop, key_deserializer=None, value_deserializer=None, fetch_min_bytes=1, fetch_max_bytes=52428800, fetch_max_wait_ms=500, max_partition_fetch_bytes=1048576, check_crcs=True, fetcher_timeout=0.2, prefetch_backoff=0.1, retry_backoff_ms=100, auto_offset_reset='latest'): """Initialize a Kafka Message Fetcher. Parameters: client (AIOKafkaClient): kafka client subscription (SubscriptionState): instance of SubscriptionState located in kafka.consumer.subscription_state key_deserializer (callable): Any callable that takes a raw message key and returns a deserialized key. value_deserializer (callable, optional): Any callable that takes a raw message value and returns a deserialized value. fetch_min_bytes (int): Minimum amount of data the server should return for a fetch request, otherwise wait up to fetch_max_wait_ms for more data to accumulate. Default: 1. fetch_max_bytes (int): The maximum amount of data the server should return for a fetch request. This is not an absolute maximum, if the first message in the first non-empty partition of the fetch is larger than this value, the message will still be returned to ensure that the consumer can make progress. NOTE: consumer performs fetches to multiple brokers in parallel so memory usage will depend on the number of brokers containing partitions for the topic. Supported Kafka version >= 0.10.1.0. Default: 52428800 (50 Mb). fetch_max_wait_ms (int): The maximum amount of time in milliseconds the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch_min_bytes. Default: 500. max_partition_fetch_bytes (int): The maximum amount of data per-partition the server will return. The maximum total memory used for a request = #partitions * max_partition_fetch_bytes. This size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large message on a certain partition. Default: 1048576. check_crcs (bool): Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance. Default: True fetcher_timeout (float): Maximum polling interval in the background fetching routine. Default: 0.2 prefetch_backoff (float): number of seconds to wait until consumption of partition is paused. Paused partitions will not request new data from Kafka server (will not be included in next poll request). auto_offset_reset (str): A policy for resetting offsets on OffsetOutOfRange errors: 'earliest' will move to the oldest available message, 'latest' will move to the most recent. Any ofther value will raise the exception. Default: 'latest'. """ self._client = client self._loop = loop self._key_deserializer = key_deserializer self._value_deserializer = value_deserializer self._fetch_min_bytes = fetch_min_bytes self._fetch_max_bytes = fetch_max_bytes self._fetch_max_wait_ms = fetch_max_wait_ms self._max_partition_fetch_bytes = max_partition_fetch_bytes self._check_crcs = check_crcs self._fetcher_timeout = fetcher_timeout self._prefetch_backoff = prefetch_backoff self._retry_backoff = retry_backoff_ms / 1000 self._subscriptions = subscriptions self._default_reset_strategy = OffsetResetStrategy.from_str( auto_offset_reset) self._isolation_level = READ_UNCOMMITTED self._records = collections.OrderedDict() self._in_flight = set() self._pending_tasks = set() self._wait_consume_future = None self._fetch_waiters = set() if client.api_version >= (0, 11): req_version = 4 elif client.api_version >= (0, 10, 1): req_version = 3 elif client.api_version >= (0, 10): req_version = 2 else: req_version = 1 self._fetch_request_class = FetchRequest[req_version] self._fetch_task = ensure_future(self._fetch_requests_routine(), loop=loop) self._closed = False
def _sender_routine(self): """ Background task, that sends pending batches to leader nodes for batch's partition. This incapsulates same logic as Java's `Sender` background thread. Because we use asyncio this is more event based loop, rather than counting timeout till next possible even like in Java. The procedure: * Group pending batches by partition leaders (write nodes) * Ignore not ready (disconnected) and nodes, that already have a pending request. * If we have unknown leaders for partitions, we request a metadata update. * Wait for any event, that can change the above procedure, like new metadata or pending send is finished and a new one can be done. """ tasks = set() try: while True: batches, unknown_leaders_exist = \ self._message_accumulator.drain_by_nodes( ignore_nodes=self._in_flight) # create produce task for every batch for node_id, batches in batches.items(): task = ensure_future(self._send_produce_req( node_id, batches), loop=self._loop) self._in_flight.add(node_id) tasks.add(task) if unknown_leaders_exist: # we have at least one unknown partition's leader, # try to update cluster metadata and wait backoff time fut = self.client.force_metadata_update() waiters = tasks.union([fut]) else: fut = self._message_accumulator.data_waiter() waiters = tasks.union([fut]) # wait when: # * At least one of produce task is finished # * Data for new partition arrived # * Metadata update if partition leader unknown done, _ = yield from asyncio.wait( waiters, return_when=asyncio.FIRST_COMPLETED, loop=self._loop) # done tasks should never produce errors, if they are it's a # bug for task in done: task.result() tasks -= done except asyncio.CancelledError: # done tasks should never produce errors, if they are it's a bug for task in tasks: yield from task except Exception: # pragma: no cover log.error("Unexpected error in sender routine", exc_info=True)
async def test_fetcher__update_fetch_positions(self): client = AIOKafkaClient( loop=self.loop, bootstrap_servers=[]) subscriptions = SubscriptionState(loop=self.loop) fetcher = Fetcher(client, subscriptions, loop=self.loop) self.add_cleanup(fetcher.close) # Disable backgroud task fetcher._fetch_task.cancel() try: await fetcher._fetch_task except asyncio.CancelledError: pass fetcher._fetch_task = ensure_future( asyncio.sleep(1000000, loop=self.loop), loop=self.loop) partition = TopicPartition('test', 0) offsets = {partition: OffsetAndTimestamp(12, -1)} async def _proc_offset_request(node_id, topic_data): return offsets fetcher._proc_offset_request = mock.Mock() fetcher._proc_offset_request.side_effect = _proc_offset_request def reset_assignment(): subscriptions.assign_from_user({partition}) assignment = subscriptions.subscription.assignment tp_state = assignment.state_value(partition) return assignment, tp_state assignment, tp_state = reset_assignment() self.assertIsNone(tp_state._position) # CASE: reset from committed # In basic case we will need to wait for committed update_task = ensure_future( fetcher._update_fetch_positions(assignment, 0, [partition]), loop=self.loop ) await asyncio.sleep(0.1, loop=self.loop) self.assertFalse(update_task.done()) # Will continue only after committed is resolved tp_state.reset_committed(OffsetAndMetadata(4, "")) needs_wakeup = await update_task self.assertFalse(needs_wakeup) self.assertEqual(tp_state._position, 4) self.assertEqual(fetcher._proc_offset_request.call_count, 0) # CASE: reset for already valid position will have no effect tp_state.begin_commit() # reset commit, to make sure we don't wait it await fetcher._update_fetch_positions(assignment, 0, [partition]) self.assertEqual(tp_state._position, 4) self.assertEqual(fetcher._proc_offset_request.call_count, 0) # CASE: awaiting_reset for the partition tp_state.await_reset(OffsetResetStrategy.LATEST) self.assertIsNone(tp_state._position) await fetcher._update_fetch_positions(assignment, 0, [partition]) self.assertEqual(tp_state._position, 12) self.assertEqual(fetcher._proc_offset_request.call_count, 1) # CASE: seeked while waiting for committed to be resolved assignment, tp_state = reset_assignment() update_task = ensure_future( fetcher._update_fetch_positions(assignment, 0, [partition]), loop=self.loop ) await asyncio.sleep(0.1, loop=self.loop) self.assertFalse(update_task.done()) tp_state.seek(8) tp_state.reset_committed(OffsetAndMetadata(4, "")) await update_task self.assertEqual(tp_state._position, 8) self.assertEqual(fetcher._proc_offset_request.call_count, 1) # CASE: awaiting_reset during waiting assignment, tp_state = reset_assignment() update_task = ensure_future( fetcher._update_fetch_positions(assignment, 0, [partition]), loop=self.loop ) await asyncio.sleep(0.1, loop=self.loop) self.assertFalse(update_task.done()) tp_state.await_reset(OffsetResetStrategy.LATEST) tp_state.reset_committed(OffsetAndMetadata(4, "")) await update_task self.assertEqual(tp_state._position, 12) self.assertEqual(fetcher._proc_offset_request.call_count, 2) # CASE: reset using default strategy if committed offset undefined assignment, tp_state = reset_assignment() tp_state.reset_committed(OffsetAndMetadata(-1, "")) await fetcher._update_fetch_positions(assignment, 0, [partition]) self.assertEqual(tp_state._position, 12) self.assertEqual(fetcher._records, {}) # CASE: set error if _default_reset_strategy = OffsetResetStrategy.NONE assignment, tp_state = reset_assignment() tp_state.reset_committed(OffsetAndMetadata(-1, "")) fetcher._default_reset_strategy = OffsetResetStrategy.NONE needs_wakeup = await fetcher._update_fetch_positions( assignment, 0, [partition]) self.assertTrue(needs_wakeup) self.assertIsNone(tp_state._position) self.assertIsInstance(fetcher._records[partition], FetchError) fetcher._records.clear() # CASE: if _proc_offset_request errored, we will retry on another spin fetcher._proc_offset_request.side_effect = UnknownError() assignment, tp_state = reset_assignment() tp_state.await_reset(OffsetResetStrategy.LATEST) await fetcher._update_fetch_positions(assignment, 0, [partition]) self.assertIsNone(tp_state._position) self.assertTrue(tp_state.awaiting_reset) # CASE: reset 2 partitions separately, 1 will rese, 1 will get # committed fetcher._proc_offset_request.side_effect = _proc_offset_request partition2 = TopicPartition('test', 1) subscriptions.assign_from_user({partition, partition2}) assignment = subscriptions.subscription.assignment tp_state = assignment.state_value(partition) tp_state2 = assignment.state_value(partition2) tp_state.await_reset(OffsetResetStrategy.LATEST) tp_state2.reset_committed(OffsetAndMetadata(5, "")) await fetcher._update_fetch_positions( assignment, 0, [partition, partition2]) self.assertEqual(tp_state.position, 12) self.assertEqual(tp_state2.position, 5)
def test_basic(self): cluster = ClusterMetadata(metadata_max_age_ms=10000) ma = MessageAccumulator(cluster, 1000, None, 30, self.loop) data_waiter = ma.data_waiter() done, _ = yield from asyncio.wait( [data_waiter], timeout=0.2, loop=self.loop) self.assertFalse(bool(done)) # no data in accumulator yet... tp0 = TopicPartition("test-topic", 0) tp1 = TopicPartition("test-topic", 1) yield from ma.add_message(tp0, b'key', b'value', timeout=2) yield from ma.add_message(tp1, None, b'value without key', timeout=2) done, _ = yield from asyncio.wait( [data_waiter], timeout=0.2, loop=self.loop) self.assertTrue(bool(done)) batches, unknown_leaders_exist = ma.drain_by_nodes(ignore_nodes=[]) self.assertEqual(batches, {}) self.assertEqual(unknown_leaders_exist, True) def mocked_leader_for_partition(tp): if tp == tp0: return 0 if tp == tp1: return 1 return -1 cluster.leader_for_partition = mock.MagicMock() cluster.leader_for_partition.side_effect = mocked_leader_for_partition batches, unknown_leaders_exist = ma.drain_by_nodes(ignore_nodes=[]) self.assertEqual(len(batches), 2) self.assertEqual(unknown_leaders_exist, False) m_set0 = batches[0].get(tp0) self.assertEqual(type(m_set0), MessageBatch) m_set1 = batches[1].get(tp1) self.assertEqual(type(m_set1), MessageBatch) self.assertEqual(m_set0.expired(), False) data_waiter = ensure_future(ma.data_waiter(), loop=self.loop) done, _ = yield from asyncio.wait( [data_waiter], timeout=0.2, loop=self.loop) self.assertFalse(bool(done)) # no data in accumulator again... # testing batch overflow tp2 = TopicPartition("test-topic", 2) yield from ma.add_message( tp0, None, b'some short message', timeout=2) yield from ma.add_message( tp0, None, b'some other short message', timeout=2) yield from ma.add_message( tp1, None, b'0123456789' * 70, timeout=2) yield from ma.add_message( tp2, None, b'message to unknown leader', timeout=2) # next we try to add message with len=500, # as we have buffer_size=1000 coroutine will block until data will be # drained add_task = ensure_future( ma.add_message(tp1, None, b'0123456789' * 50, timeout=2), loop=self.loop) done, _ = yield from asyncio.wait( [add_task], timeout=0.2, loop=self.loop) self.assertFalse(bool(done)) batches, unknown_leaders_exist = ma.drain_by_nodes(ignore_nodes=[1, 2]) self.assertEqual(unknown_leaders_exist, True) m_set0 = batches[0].get(tp0) self.assertEqual(m_set0._builder._relative_offset, 2) m_set1 = batches[1].get(tp1) self.assertEqual(m_set1, None) done, _ = yield from asyncio.wait( [add_task], timeout=0.1, loop=self.loop) self.assertFalse(bool(done)) # we stil not drained data for tp1 batches, unknown_leaders_exist = ma.drain_by_nodes(ignore_nodes=[]) self.assertEqual(unknown_leaders_exist, True) m_set0 = batches[0].get(tp0) self.assertEqual(m_set0, None) m_set1 = batches[1].get(tp1) self.assertEqual(m_set1._builder._relative_offset, 1) done, _ = yield from asyncio.wait( [add_task], timeout=0.2, loop=self.loop) self.assertTrue(bool(done)) batches, unknown_leaders_exist = ma.drain_by_nodes(ignore_nodes=[]) self.assertEqual(unknown_leaders_exist, True) m_set1 = batches[1].get(tp1) self.assertEqual(m_set1._builder._relative_offset, 1)
def test_load_metadata(self): brokers = [ (0, 'broker_1', 4567), (1, 'broker_2', 5678) ] topics = [ (NO_ERROR, 'topic_1', [ (NO_ERROR, 0, 1, [1, 2], [1, 2]) ]), (NO_ERROR, 'topic_2', [ (NO_LEADER, 0, -1, [], []), (NO_LEADER, 1, 1, [], []), ]), (NO_LEADER, 'topic_no_partitions', []), (UNKNOWN_TOPIC_OR_PARTITION, 'topic_unknown', []), (NO_ERROR, 'topic_3', [ (NO_ERROR, 0, 0, [0, 1], [0, 1]), (NO_ERROR, 1, 1, [1, 0], [1, 0]), (NO_ERROR, 2, 0, [0, 1], [0, 1]) ]), (NO_ERROR, 'topic_4', [ (NO_ERROR, 0, 0, [0, 1], [0, 1]), (REPLICA_NOT_AVAILABLE, 1, 1, [1, 0], [1, 0]), ]), (INVALID_TOPIC, 'topic_5', []), # Just ignored (UNKNOWN_ERROR, 'topic_6', []), # Just ignored (TOPIC_AUTHORIZATION_FAILED, 'topic_auth_error', []), ] @asyncio.coroutine def send(request_id): return MetadataResponse(brokers, topics) mocked_conns = {(0, 0): mock.MagicMock()} mocked_conns[(0, 0)].send.side_effect = send client = AIOKafkaClient(loop=self.loop, bootstrap_servers=['broker_1:4567']) task = ensure_future(client._md_synchronizer(), loop=self.loop) client._conns = mocked_conns client.cluster.update_metadata(MetadataResponse(brokers[:1], [])) self.loop.run_until_complete(client.force_metadata_update()) task.cancel() md = client.cluster c_brokers = md.brokers() self.assertEqual(len(c_brokers), 2) expected_brokers = [ (0, 'broker_1', 4567, None), (1, 'broker_2', 5678, None) ] self.assertEqual(sorted(expected_brokers), sorted(list(c_brokers))) c_topics = md.topics() self.assertEqual(len(c_topics), 4) self.assertEqual(md.partitions_for_topic('topic_1'), set([0])) self.assertEqual(md.partitions_for_topic('topic_2'), set([0, 1])) self.assertEqual(md.partitions_for_topic('topic_3'), set([0, 1, 2])) self.assertEqual(md.partitions_for_topic('topic_4'), set([0, 1])) self.assertEqual( md.available_partitions_for_topic('topic_2'), set([1])) mocked_conns[(0, 0)].connected.return_value = False is_ready = self.loop.run_until_complete(client.ready(0)) self.assertEqual(is_ready, False) is_ready = self.loop.run_until_complete(client.ready(1)) self.assertEqual(is_ready, False) self.assertEqual(mocked_conns, {}) with self.assertRaises(NodeNotReadyError): self.loop.run_until_complete(client.send(0, None)) self.assertEqual(md.unauthorized_topics, {'topic_auth_error'})