def pause(self, partitions: Sequence[Partition]) -> None: """ Pause the consumption of messages for the provided partitions. Raises an ``InvalidState`` if called on a closed consumer. """ if self.__state in { KafkaConsumerState.CLOSED, KafkaConsumerState.ERROR }: raise InvalidState(self.__state) if set(partitions) - self.__offsets.keys(): raise ConsumerError("cannot pause unassigned partitions") self.__consumer.pause([ ConfluentTopicPartition(partition.topic.name, partition.index) for partition in partitions ]) self.__paused.update(partitions) # XXX: Seeking to a specific partition offset and immediately pausing # that partition causes the seek to be ignored for some reason. self.seek({ partition: offset for partition, offset in self.__offsets.items() if partition in partitions })
def __seek(self, offsets: Mapping[TopicPartition, int]) -> None: if self.__state is KafkaConsumerState.ASSIGNING: # Calling ``seek`` on the Confluent consumer from an assignment # callback will throw an "Erroneous state" error. Instead, # partition offsets have to be initialized by calling ``assign``. self.__consumer.assign([ ConfluentTopicPartition(stream.topic, stream.partition, offset) for stream, offset in offsets.items() ]) else: for stream, offset in offsets.items(): self.__consumer.seek( ConfluentTopicPartition(stream.topic, stream.partition, offset)) self.__offsets.update(offsets)
def assignment_callback( consumer: ConfluentConsumer, partitions: Sequence[ConfluentTopicPartition] ) -> None: self.__state = KafkaConsumerState.ASSIGNING try: assignment: MutableSequence[ConfluentTopicPartition] = [] for partition in self.__consumer.committed(partitions): if partition.offset >= 0: assignment.append(partition) elif partition.offset == OFFSET_INVALID: assignment.append( self.__resolve_partition_starting_offset(partition) ) else: raise ValueError("received unexpected offset") offsets: MutableMapping[Partition, int] = { Partition(Topic(i.topic), i.partition): i.offset for i in assignment } self.__seek(offsets) # Ensure that all partitions are resumed on assignment to avoid # carrying over state from a previous assignment. self.__consumer.resume( [ ConfluentTopicPartition( partition.topic.name, partition.index, offset ) for partition, offset in offsets.items() ] ) for partition in offsets: self.__paused.discard(partition) except Exception: self.__state = KafkaConsumerState.ERROR raise try: if on_assign is not None: on_assign(offsets) finally: self.__state = KafkaConsumerState.CONSUMING
def __commit(self) -> Mapping[Partition, int]: if self.__state in { KafkaConsumerState.CLOSED, KafkaConsumerState.ERROR }: raise InvalidState(self.__state) result: Optional[Sequence[ConfluentTopicPartition]] if self.__staged_offsets: result = self.__consumer.commit( offsets=[ ConfluentTopicPartition(partition.topic.name, partition.index, offset) for partition, offset in self.__staged_offsets.items() ], asynchronous=False, ) else: result = [] assert result is not None # synchronous commit should return result immediately self.__staged_offsets.clear() offsets: MutableMapping[Partition, int] = {} for value in result: # The Confluent Kafka Consumer will include logical offsets in the # sequence of ``Partition`` objects returned by ``commit``. These # are an implementation detail of the Kafka Consumer, so we don't # expose them here. # NOTE: These should no longer be seen now that we are forcing # offsets to be set as part of the assignment callback. if value.offset in self.LOGICAL_OFFSETS: continue assert value.offset >= 0, "expected non-negative offset" offsets[Partition(Topic(value.topic), value.partition)] = value.offset return offsets
def resume(self, partitions: Sequence[Partition]) -> None: """ Resume the consumption of messages for the provided partitions. Raises an ``InvalidState`` if called on a closed consumer. """ if self.__state in { KafkaConsumerState.CLOSED, KafkaConsumerState.ERROR }: raise InvalidState(self.__state) if set(partitions) - self.__offsets.keys(): raise ConsumerError("cannot resume unassigned partitions") self.__consumer.resume([ ConfluentTopicPartition(partition.topic.name, partition.index) for partition in partitions ]) for partition in partitions: self.__paused.discard(partition)
def __resolve_partition_offset_latest( self, partition: ConfluentTopicPartition) -> ConfluentTopicPartition: low, high = self.__consumer.get_watermark_offsets(partition) return ConfluentTopicPartition(partition.topic, partition.partition, high)