def _on_join_leader(self, response): """ Perform leader synchronization and send back the assignment for the group via SyncGroupRequest Arguments: response (JoinResponse): broker response to parse Returns: Future: resolves to member assignment encoded-bytes """ try: group_assignment = \ yield from self._coordinator._perform_assignment( response.leader_id, response.group_protocol, response.members) except Exception as e: raise Errors.KafkaError(repr(e)) assignment_req = [] for member_id, assignment in group_assignment.items(): if not isinstance(assignment, bytes): assignment = assignment.encode() assignment_req.append((member_id, assignment)) request = SyncGroupRequest(self.group_id, self._coordinator.generation, self._coordinator.member_id, assignment_req) log.debug( "Sending leader SyncGroup for group %s to coordinator %s: %s", self.group_id, self.coordinator_id, request) return (yield from self._send_sync_group_request(request))
def _contains_abort_marker(self, next_batch): # Control Marker is used to specify when we can stop # aborting batches try: control_record = next(next_batch) except StopIteration: # pragma: no cover raise Errors.KafkaError( "Control batch did not contain any records") return ControlRecord.parse(control_record.key) == ABORT_MARKER
def pick_best(self, request_versions): api_key = request_versions[0].API_KEY supported_versions = self._versions.get(api_key) if supported_versions is None: return request_versions[0] else: for req_klass in reversed(request_versions): if supported_versions[0] <= req_klass.API_VERSION and \ req_klass.API_VERSION <= supported_versions[1]: return req_klass raise Errors.KafkaError( "Could not pick a version for API_KEY={} from {}. ".format( api_key, supported_versions))
def _fetch_requests_routine(self): """ Implements a background task to populate internal fetch queue ``self._records`` with prefetched messages. This helps isolate the ``getall/getone`` calls from actual calls to broker. This way we don't need to think of what happens if user calls get in 2 tasks, etc. The loop is quite complicated due to a large set of events that can allow new fetches to be send. Those include previous fetches, offset resets, metadata updates to discover new leaders for partitions, data consumed for partition. Previously the offset reset was performed separately, but it did not perform too reliably. In ``kafka-python`` and Java client the reset is perform in ``poll()`` before each fetch, which works good for sync systems. But with ``aiokafka`` the user can actually break such behaviour quite easily by performing actions from different tasks. """ try: assignment = None 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) while True: # If we lose assignment we just cancel all current tasks, # wait for new assignment and restart the loop if assignment is None or not assignment.active: for task in self._pending_tasks: # Those tasks should have proper handling for # cancellation if not task.done(): task.cancel() yield from task self._pending_tasks.clear() self._records.clear() subscription = self._subscriptions.subscription if subscription is None or \ subscription.assignment is None: try: waiter = self._subscriptions.wait_for_assignment() yield from waiter except Errors.KafkaError: # Critical coordination waiters will be passed # to user, but fetcher can just ignore those continue assignment = self._subscriptions.subscription.assignment assert assignment is not None and assignment.active # Reset consuming signal future. self._wait_consume_future = create_future(loop=self._loop) # Determine what action to take per node (fetch_requests, reset_requests, timeout, invalid_metadata, resume_futures) = self._get_actions_per_node(assignment) # Start fetch tasks for node_id, request in fetch_requests: start_pending_task(self._proc_fetch_request( assignment, node_id, request), node_id=node_id) # Start update position tasks for node_id, tps in reset_requests.items(): start_pending_task(self._update_fetch_positions( assignment, node_id, tps), node_id=node_id) # Apart from pending requests we also need to react to other # events to send new fetches as soon as possible other_futs = [ self._wait_consume_future, assignment.unassign_future ] if invalid_metadata: fut = self._client.force_metadata_update() other_futs.append(fut) done_set, _ = yield from asyncio.wait( chain(self._pending_tasks, other_futs, resume_futures), loop=self._loop, timeout=timeout, return_when=asyncio.FIRST_COMPLETED) # Process fetch tasks results if any done_pending = self._pending_tasks.intersection(done_set) if done_pending: has_new_data = any(fut.result() for fut in done_pending) if has_new_data: for waiter in self._fetch_waiters: # we added some messages to self._records, # wake up waiters self._notify(waiter) self._pending_tasks -= done_pending except asyncio.CancelledError: pass except Exception: # pragma: no cover yield from self.close() raise Errors.KafkaError("Unexpected error during data retrieval")
async def _merge_routine(self): """ The main routine, reading data and committing offsets """ done = () try: # init the rocessors self.build_processors() while not self._closed: # wait for rebalancing to finish if self._rebalancing: await self._rebalancing # time to wait for the next autocommit wait_timeout = await self._maybe_do_autocommit() # two concurrent tasks: read data and wakeup processors if not self._pending_tasks: self._pending_tasks = [ self._loop.create_task(self._sleep()), self._loop.create_task(self._readmany()), ] # the tasks can be cancelled, by this potion of code # should finish before rebalancing in order to have a # correct final commit try: self._before_rebalancing = asyncio.Future(loop=self._loop) # run the two tasks concurently, see who wins done, self._pending_tasks = await asyncio.wait( self._pending_tasks, return_when=asyncio.FIRST_COMPLETED, timeout=wait_timeout, loop=self._loop) # execute the coroutine of the first done, cancel the other one if done: if self._pending_tasks: for task in self._pending_tasks: task.cancel() await asyncio.wait(self._pending_tasks) for task in done: task = task.result() # can be None if has been cancelled for rebalancing if task: await asyncio.shield(task) self._pending_tasks = () done = () # else, timed out for autocommit, execute it next turn finally: self._before_rebalancing.set_result(None) self._before_rebalancing = None except asyncio.CancelledError: pass except Exception: log.error("Unexpected error in merge routine", exc_info=True) raise Errors.KafkaError("Unexpected error during merge") pending = [] for task in self._pending_tasks: if task.done(): task = task.result() if task: pending.append(task) else: task.cancel() pending.append(task) for task in done: task = task.result() if task and not task.done(): pending.append(task) # print ('self._merge_routine pending', pending, self._pending_tasks, done) self._pending_tasks = pending