async def test__commit__IllegalStateError(self, *, cthread, _consumer): cthread._consumer = _consumer cthread.assignment = Mock() exc = _consumer.commit.side_effect = IllegalStateError('xx') cthread.crash = AsyncMock() assert not (await cthread._commit({TP1: 1001})) cthread.crash.assert_called_once_with(exc)
def seek_to_end(self, *partitions): """Seek to the most recent available offset for partitions. Arguments: *partitions: Optionally provide specific TopicPartitions, otherwise default to all assigned partitions. Raises: IllegalStateError: If any partition is not currently assigned TypeError: If partitions are not instances of TopicPartition .. versionadded:: 0.3.0 """ if not all([isinstance(p, TopicPartition) for p in partitions]): raise TypeError('partitions must be TopicPartition instances') yield from self._coordinator.ensure_partitions_assigned() if not partitions: partitions = self._subscription.assigned_partitions() assert partitions, 'No partitions are currently assigned' else: not_assigned = ( set(partitions) - self._subscription.assigned_partitions() ) if not_assigned: raise IllegalStateError( "Partitions {} are not assigned".format(not_assigned)) for tp in partitions: log.debug("Seeking to end of partition %s", tp) self._subscription.need_offset_reset( tp, OffsetResetStrategy.LATEST) yield from self._fetcher.update_fetch_positions(partitions)
def seek_to_committed(self, *partitions): """ Seek to the committed offset for partitions. Arguments: *partitions: Optionally provide specific TopicPartitions, otherwise default to all assigned partitions. Raises: IllegalStateError: If any partition is not currently assigned IllegalOperation: If used with ``group_id == None`` .. versionchanged:: 0.3.0 Changed ``AssertionError`` to ``IllegalStateError`` in case of unassigned partition """ if not all([isinstance(p, TopicPartition) for p in partitions]): raise TypeError('partitions must be TopicPartition instances') if not partitions: partitions = self._subscription.assigned_partitions() assert partitions, 'No partitions are currently assigned' else: not_assigned = (set(partitions) - self._subscription.assigned_partitions()) if not_assigned: raise IllegalStateError( "Partitions {} are not assigned".format(not_assigned)) for tp in partitions: offset = yield from self.committed(tp) log.debug("Seeking to committed of partition %s %s", tp, offset) if offset and offset > 0: self._fetcher.seek_to(tp, offset)
def seek_to_beginning(self, *partitions): """ Seek to the oldest available offset for partitions. Arguments: *partitions: Optionally provide specific TopicPartitions, otherwise default to all assigned partitions. Raises: IllegalStateError: If any partition is not currently assigned TypeError: If partitions are not instances of TopicPartition .. versionadded:: 0.3.0 """ if not all([isinstance(p, TopicPartition) for p in partitions]): raise TypeError('partitions must be TopicPartition instances') if not partitions: partitions = self._subscription.assigned_partitions() assert partitions, 'No partitions are currently assigned' else: not_assigned = (set(partitions) - self._subscription.assigned_partitions()) if not_assigned: raise IllegalStateError( "Partitions {} are not assigned".format(not_assigned)) for tp in partitions: log.debug("Seeking to beginning of partition %s", tp) yield from self._fetcher.request_offset_reset( partitions, OffsetResetStrategy.EARLIEST)
def _set_subscription_type(self, subscription_type: SubscriptionType): if (self._subscription_type == SubscriptionType.NONE or self._subscription_type == subscription_type): self._subscription_type = subscription_type else: raise IllegalStateError( "Subscription to topics, partitions and pattern are mutually " "exclusive")
def _assigned_state(self, tp: TopicPartition) -> "TopicPartitionState": assert self._subscription is not None assert self._subscription.assignment is not None tp_state = self._subscription.assignment.state_value(tp) if tp_state is None: raise IllegalStateError( "No current assignment for partition {}".format(tp)) return tp_state
def position(self, partition): """ Get the offset of the *next record* that will be fetched (if a record with that offset exists on broker). Arguments: partition (TopicPartition): partition to check Returns: int: offset Raises: IllegalStateError: partition is not assigned .. versionchanged:: 0.4.0 Changed ``AssertionError`` to ``IllegalStateError`` in case of unassigned partition """ while True: if not self._subscription.is_assigned(partition): raise IllegalStateError( 'Partition {} is not assigned'.format(partition)) assignment = self._subscription.subscription.assignment tp_state = assignment.state_value(partition) if not tp_state.has_valid_position: yield from asyncio.wait( [tp_state.wait_for_position(), assignment.unassign_future], return_when=asyncio.FIRST_COMPLETED, loop=self._loop, ) if not tp_state.has_valid_position: if self._subscription.subscription is None: raise IllegalStateError( 'Partition {} is not assigned'.format(partition)) if self._subscription.subscription.assignment is None: yield from self._subscription.wait_for_assignment() continue return tp_state.position
def commit(self, offsets=None): """ Commit offsets to Kafka. This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API should not be used. Currently only supports kafka-topic offset storage (not zookeeper) When explicitly passing ``offsets`` use either offset of next record, or tuple of offset and metadata:: tp = TopicPartition(msg.topic, msg.partition) metadata = "Some utf-8 metadata" # Either await consumer.commit({tp: msg.offset + 1}) # Or position directly await consumer.commit({tp: (msg.offset + 1, metadata)}) .. note:: If you want `fire and forget` commit, like ``commit_async()`` in *kafka-python*, just run it in a task. Something like:: fut = loop.create_task(consumer.commit()) fut.add_done_callback(on_commit_done) Arguments: offsets (dict, optional): {TopicPartition: (offset, metadata)} dict to commit with the configured ``group_id``. Defaults to current consumed offsets for all subscribed partitions. Raises: IllegalOperation: If used with ``group_id == None``. IllegalStateError: If partitions not assigned. ValueError: If offsets is of wrong format. CommitFailedError: If membership already changed on broker. KafkaError: If commit failed on broker side. This could be due to invalid offset, too long metadata, authorization failure, etc. .. versionchanged:: 0.4.0 Changed ``AssertionError`` to ``IllegalStateError`` in case of unassigned partition. .. versionchanged:: 0.4.0 Will now raise ``CommitFailedError`` in case membership changed, as (posibly) this partition is handled by another consumer. """ if self._group_id is None: raise IllegalOperation("Requires group_id") subscription = self._subscription.subscription if subscription is None: raise IllegalStateError("Not subscribed to any topics") assignment = subscription.assignment if assignment is None: raise IllegalStateError("No partitions assigned") if offsets is None: offsets = assignment.all_consumed_offsets() else: # validate `offsets` structure if not offsets or not isinstance(offsets, dict): raise ValueError(offsets) formatted_offsets = {} for tp, offset_and_metadata in offsets.items(): if not isinstance(tp, TopicPartition): raise ValueError("Key should be TopicPartition instance") if tp not in assignment.tps: raise IllegalStateError( "Partition {} is not assigned".format(tp)) if isinstance(offset_and_metadata, int): offset, metadata = offset_and_metadata, "" else: try: offset, metadata = offset_and_metadata except Exception: raise ValueError(offsets) if not isinstance(metadata, str): raise ValueError("Metadata should be a string") formatted_offsets[tp] = OffsetAndMetadata(offset, metadata) offsets = formatted_offsets yield from self._coordinator.commit_offsets(assignment, offsets)