pub struct BaseConsumer<C = DefaultConsumerContext>
where C: ConsumerContext + 'static,
{ /* private fields */ }
Expand description

A low-level consumer that requires manual polling.

This consumer must be periodically polled to make progress on rebalancing, callbacks and to receive messages.

Implementations§

source§

impl<C> BaseConsumer<C>
where C: ConsumerContext,

source

pub fn poll<T: Into<Timeout>>( &self, timeout: T ) -> Option<KafkaResult<BorrowedMessage<'_>>>

Polls the consumer for new messages.

It won’t block for more than the specified timeout. Use zero Duration for non-blocking call. With no timeout it blocks until an event is received.

This method should be called at regular intervals, even if no message is expected, to serve any queued callbacks waiting to be called. This is especially important for automatic consumer rebalance, as the rebalance function will be executed by the thread calling the poll() function.

Lifetime

The returned message lives in the memory of the consumer and cannot outlive it.

source

pub fn iter(&self) -> Iter<'_, C>

Returns an iterator over the available messages.

It repeatedly calls poll with no timeout.

Note that it’s also possible to iterate over the consumer directly.

Examples

All these are equivalent and will receive messages without timing out.

loop {
  let message = consumer.poll(None);
  // Handle the message
}
for message in consumer.iter() {
  // Handle the message
}
for message in &consumer {
  // Handle the message
}
source

pub fn split_partition_queue( self: &Arc<Self>, topic: &str, partition: i32 ) -> Option<PartitionQueue<C>>

Splits messages for the specified partition into their own queue.

If the topic or partition is invalid, returns None.

After calling this method, newly-fetched messages for the specified partition will be returned via PartitionQueue::poll rather than BaseConsumer::poll. Note that there may be buffered messages for the specified partition that will continue to be returned by BaseConsumer::poll. For best results, call split_partition_queue before the first call to BaseConsumer::poll.

You must continue to call BaseConsumer::poll, even if no messages are expected, to serve callbacks.

Note that calling Consumer::assign will deactivate any existing partition queues. You will need to call this method for every partition that should be split after every call to assign.

Beware that this method is implemented for &Arc<Self>, not &self. You will need to wrap your consumer in an Arc in order to call this method. This design permits moving the partition queue to another thread while ensuring the partition queue does not outlive the consumer.

Trait Implementations§

source§

impl<C> Consumer<C> for BaseConsumer<C>
where C: ConsumerContext,

source§

fn client(&self) -> &Client<C>

Returns the Client underlying this consumer.
source§

fn group_metadata(&self) -> Option<ConsumerGroupMetadata>

Returns the current consumer group metadata associated with the consumer. Read more
source§

fn subscribe(&self, topics: &[&str]) -> KafkaResult<()>

Subscribes the consumer to a list of topics.
source§

fn unsubscribe(&self)

Unsubscribes the current subscription list.
source§

fn assign(&self, assignment: &TopicPartitionList) -> KafkaResult<()>

Manually assigns topics and partitions to the consumer. If used, automatic consumer rebalance won’t be activated.
source§

fn seek<T: Into<Timeout>>( &self, topic: &str, partition: i32, offset: Offset, timeout: T ) -> KafkaResult<()>

Seeks to offset for the specified topic and partition. After a successful call to seek, the next poll of the consumer will return the message with offset.
source§

fn commit( &self, topic_partition_list: &TopicPartitionList, mode: CommitMode ) -> KafkaResult<()>

Commits the offset of the specified message. The commit can be sync (blocking), or async. Notice that when a specific offset is committed, all the previous offsets are considered committed as well. Use this method only if you are processing messages in order. Read more
source§

fn commit_consumer_state(&self, mode: CommitMode) -> KafkaResult<()>

Commits the current consumer state. Notice that if the consumer fails after a message has been received, but before the message has been processed by the user code, this might lead to data loss. Check the “at-least-once delivery” section in the readme for more information.
source§

fn commit_message( &self, message: &BorrowedMessage<'_>, mode: CommitMode ) -> KafkaResult<()>

Commit the provided message. Note that this will also automatically commit every message with lower offset within the same partition. Read more
source§

fn store_offset( &self, topic: &str, partition: i32, offset: i64 ) -> KafkaResult<()>

Stores offset to be used on the next (auto)commit. When using this enable.auto.offset.store should be set to false in the config.
source§

fn store_offset_from_message( &self, message: &BorrowedMessage<'_> ) -> KafkaResult<()>

Like Consumer::store_offset, but the offset to store is derived from the provided message.
source§

fn store_offsets(&self, tpl: &TopicPartitionList) -> KafkaResult<()>

Store offsets to be used on the next (auto)commit. When using this enable.auto.offset.store should be set to false in the config.
source§

fn subscription(&self) -> KafkaResult<TopicPartitionList>

Returns the current topic subscription.
source§

fn assignment(&self) -> KafkaResult<TopicPartitionList>

Returns the current partition assignment.
source§

fn committed<T: Into<Timeout>>( &self, timeout: T ) -> KafkaResult<TopicPartitionList>

Retrieves the committed offsets for topics and partitions.
source§

fn committed_offsets<T: Into<Timeout>>( &self, tpl: TopicPartitionList, timeout: T ) -> KafkaResult<TopicPartitionList>

Retrieves the committed offsets for specified topics and partitions.
source§

fn offsets_for_timestamp<T: Into<Timeout>>( &self, timestamp: i64, timeout: T ) -> KafkaResult<TopicPartitionList>

Looks up the offsets for this consumer’s partitions by timestamp.
source§

fn offsets_for_times<T: Into<Timeout>>( &self, timestamps: TopicPartitionList, timeout: T ) -> KafkaResult<TopicPartitionList>

Looks up the offsets for the specified partitions by timestamp.
source§

fn position(&self) -> KafkaResult<TopicPartitionList>

Retrieve current positions (offsets) for topics and partitions.
source§

fn fetch_metadata<T: Into<Timeout>>( &self, topic: Option<&str>, timeout: T ) -> KafkaResult<Metadata>

Returns the metadata information for the specified topic, or for all topics in the cluster if no topic is specified.
source§

fn fetch_watermarks<T: Into<Timeout>>( &self, topic: &str, partition: i32, timeout: T ) -> KafkaResult<(i64, i64)>

Returns the low and high watermarks for a specific topic and partition.
source§

fn fetch_group_list<T: Into<Timeout>>( &self, group: Option<&str>, timeout: T ) -> KafkaResult<GroupList>

Returns the group membership information for the given group. If no group is specified, all groups will be returned.
source§

fn pause(&self, partitions: &TopicPartitionList) -> KafkaResult<()>

Pauses consumption for the provided list of partitions.
source§

fn resume(&self, partitions: &TopicPartitionList) -> KafkaResult<()>

Resumes consumption for the provided list of partitions.
source§

fn rebalance_protocol(&self) -> RebalanceProtocol

Reports the rebalance protocol in use.
source§

fn context(&self) -> &Arc<C>

Returns a reference to the ConsumerContext used to create this consumer.
source§

impl FromClientConfig for BaseConsumer

source§

fn from_config(config: &ClientConfig) -> KafkaResult<BaseConsumer>

Creates a client from a client configuration. The default client context will be used.
source§

impl<C: ConsumerContext> FromClientConfigAndContext<C> for BaseConsumer<C>

Creates a new BaseConsumer starting from a ClientConfig.

source§

fn from_config_and_context( config: &ClientConfig, context: C ) -> KafkaResult<BaseConsumer<C>>

Creates a client from a client configuration and a client context.
source§

impl<'a, C> IntoIterator for &'a BaseConsumer<C>
where C: ConsumerContext,

§

type Item = Result<BorrowedMessage<'a>, KafkaError>

The type of the elements being iterated over.
§

type IntoIter = Iter<'a, C>

Which kind of iterator are we turning this into?
source§

fn into_iter(self) -> Self::IntoIter

Creates an iterator from a value. Read more

Auto Trait Implementations§

§

impl<C> RefUnwindSafe for BaseConsumer<C>
where C: RefUnwindSafe,

§

impl<C> Send for BaseConsumer<C>

§

impl<C> Sync for BaseConsumer<C>

§

impl<C> Unpin for BaseConsumer<C>

§

impl<C> UnwindSafe for BaseConsumer<C>
where C: RefUnwindSafe,

Blanket Implementations§

source§

impl<T> Any for T
where T: 'static + ?Sized,

source§

fn type_id(&self) -> TypeId

Gets the TypeId of self. Read more
source§

impl<T> Borrow<T> for T
where T: ?Sized,

source§

fn borrow(&self) -> &T

Immutably borrows from an owned value. Read more
source§

impl<T> BorrowMut<T> for T
where T: ?Sized,

source§

fn borrow_mut(&mut self) -> &mut T

Mutably borrows from an owned value. Read more
source§

impl<T> From<T> for T

source§

fn from(t: T) -> T

Returns the argument unchanged.

source§

impl<T> Instrument for T

source§

fn instrument(self, span: Span) -> Instrumented<Self>

Instruments this type with the provided Span, returning an Instrumented wrapper. Read more
source§

fn in_current_span(self) -> Instrumented<Self>

Instruments this type with the current Span, returning an Instrumented wrapper. Read more
source§

impl<T, U> Into<U> for T
where U: From<T>,

source§

fn into(self) -> U

Calls U::from(self).

That is, this conversion is whatever the implementation of From<T> for U chooses to do.

source§

impl<T, U> TryFrom<U> for T
where U: Into<T>,

§

type Error = Infallible

The type returned in the event of a conversion error.
source§

fn try_from(value: U) -> Result<T, <T as TryFrom<U>>::Error>

Performs the conversion.
source§

impl<T, U> TryInto<U> for T
where U: TryFrom<T>,

§

type Error = <U as TryFrom<T>>::Error

The type returned in the event of a conversion error.
source§

fn try_into(self) -> Result<U, <U as TryFrom<T>>::Error>

Performs the conversion.
source§

impl<T> WithSubscriber for T

source§

fn with_subscriber<S>(self, subscriber: S) -> WithDispatch<Self>
where S: Into<Dispatch>,

Attaches the provided Subscriber to this type, returning a WithDispatch wrapper. Read more
source§

fn with_current_subscriber(self) -> WithDispatch<Self>

Attaches the current default Subscriber to this type, returning a WithDispatch wrapper. Read more