1
  2
  3
  4
  5
  6
  7
  8
  9
 10
 11
 12
 13
 14
 15
 16
 17
 18
 19
 20
 21
 22
 23
 24
 25
 26
 27
 28
 29
 30
 31
 32
 33
 34
 35
 36
 37
 38
 39
 40
 41
 42
 43
 44
 45
 46
 47
 48
 49
 50
 51
 52
 53
 54
 55
 56
 57
 58
 59
 60
 61
 62
 63
 64
 65
 66
 67
 68
 69
 70
 71
 72
 73
 74
 75
 76
 77
 78
 79
 80
 81
 82
 83
 84
 85
 86
 87
 88
 89
 90
 91
 92
 93
 94
 95
 96
 97
 98
 99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
//! Base trait and common functionality for all consumers.

use std::ptr;
use std::time::Duration;

use log::{error, trace};

use rdkafka_sys as rdsys;
use rdkafka_sys::types::*;

use crate::client::{ClientContext, NativeClient};
use crate::error::KafkaResult;
use crate::groups::GroupList;
use crate::message::BorrowedMessage;
use crate::metadata::Metadata;
use crate::topic_partition_list::{Offset, TopicPartitionList};
use crate::util::{cstr_to_owned, Timeout};

pub mod base_consumer;
pub mod stream_consumer;

// Re-exports.
pub use self::base_consumer::BaseConsumer;
pub use self::stream_consumer::{MessageStream, StreamConsumer};

/// Rebalance information.
#[derive(Clone, Debug)]
pub enum Rebalance<'a> {
    /// A new partition assignment is received.
    Assign(&'a TopicPartitionList),
    /// All partitions are revoked.
    Revoke,
    /// Unexpected error from Kafka.
    Error(String),
}

/// Consumer specific Context. This user-defined object can be used to provide custom callbacks to
/// consumer events. Refer to the list of methods to check which callbacks can be specified.
pub trait ConsumerContext: ClientContext {
    /// Implements the default rebalancing strategy and calls the `pre_rebalance` and
    /// `post_rebalance` methods. If this method is overridden, it will be responsibility
    /// of the user to call them if needed.
    fn rebalance(
        &self,
        native_client: &NativeClient,
        err: RDKafkaRespErr,
        tpl: &mut TopicPartitionList,
    ) {
        let rebalance = match err {
            RDKafkaRespErr::RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS => Rebalance::Assign(tpl),
            RDKafkaRespErr::RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS => Rebalance::Revoke,
            _ => {
                let error = unsafe { cstr_to_owned(rdsys::rd_kafka_err2str(err)) };
                error!("Error rebalancing: {}", error);
                Rebalance::Error(error)
            }
        };

        trace!("Running pre-rebalance with {:?}", rebalance);
        self.pre_rebalance(&rebalance);

        trace!("Running rebalance with {:?}", rebalance);
        // Execute rebalance
        unsafe {
            match err {
                RDKafkaRespErr::RD_KAFKA_RESP_ERR__ASSIGN_PARTITIONS => {
                    rdsys::rd_kafka_assign(native_client.ptr(), tpl.ptr());
                }
                _ => {
                    // Also for RD_KAFKA_RESP_ERR__REVOKE_PARTITIONS
                    rdsys::rd_kafka_assign(native_client.ptr(), ptr::null());
                }
            }
        }
        trace!("Running post-rebalance with {:?}", rebalance);
        self.post_rebalance(&rebalance);
    }

    /// Pre-rebalance callback. This method will run before the rebalance and should
    /// terminate its execution quickly.
    #[allow(unused_variables)]
    fn pre_rebalance<'a>(&self, rebalance: &Rebalance<'a>) {}

    /// Post-rebalance callback. This method will run after the rebalance and should
    /// terminate its execution quickly.
    #[allow(unused_variables)]
    fn post_rebalance<'a>(&self, rebalance: &Rebalance<'a>) {}

    // TODO: convert pointer to structure
    /// Post commit callback. This method will run after a group of offsets was committed to the
    /// offset store.
    #[allow(unused_variables)]
    fn commit_callback(&self, result: KafkaResult<()>, offsets: &TopicPartitionList) {}

    /// Returns the minimum interval at which to poll the main queue, which
    /// services the logging, stats, and error callbacks.
    ///
    /// The main queue is polled once whenever [`Consumer.poll`] is called. If
    /// `Consumer.poll` is called with a timeout that is larger than this
    /// interval, then the main queue will be polled at that interval while the
    /// consumer queue is blocked.
    ///
    /// For example, if the main queue's minimum poll interval is 200ms and
    /// `Consumer.poll` is called with a timeout of 1s, then `Consumer.poll` may
    /// block for up to 1s waiting for a message, but it will poll the main
    /// queue every 200ms while it is waiting.
    ///
    /// By default, the minimum poll interval for the main queue is 1s.
    fn main_queue_min_poll_interval(&self) -> Timeout {
        Timeout::After(Duration::from_secs(1))
    }

    /// Message queue nonempty callback. This method will run when the
    /// consumer's message queue switches from empty to nonempty.
    fn message_queue_nonempty_callback(&self) {}
}

/// An empty consumer context that can be user when no context is needed.
#[derive(Clone)]
pub struct DefaultConsumerContext;

impl ClientContext for DefaultConsumerContext {}
impl ConsumerContext for DefaultConsumerContext {}

/// Specifies if the commit should be performed synchronously
/// or asynchronously.
pub enum CommitMode {
    /// Synchronous commit.
    Sync = 0,
    /// Asynchronous commit.
    Async = 1,
}

/// Common trait for all consumers.
///
/// # Note about object safety
///
/// Doing type erasure on consumers is expected to be rare (eg. `Box<Consumer>`). Therefore, the
/// API is optimised for the case where a concrete type is available. As a result, some methods are
/// not available on trait objects, since they are generic.
///
/// If there's still the need to erase the type, the generic methods can still be reached through
/// the [`get_base_consumer`](#method.get_base_consumer) method.
pub trait Consumer<C: ConsumerContext = DefaultConsumerContext> {
    /// Returns a reference to the BaseConsumer.
    fn get_base_consumer(&self) -> &BaseConsumer<C>;

    // Default implementations

    /// Subscribe the consumer to a list of topics.
    fn subscribe(&self, topics: &[&str]) -> KafkaResult<()> {
        self.get_base_consumer().subscribe(topics)
    }

    /// Unsubscribe the current subscription list.
    fn unsubscribe(&self) {
        self.get_base_consumer().unsubscribe();
    }

    /// Manually assign topics and partitions to the consumer. If used, automatic consumer
    /// rebalance won't be activated.
    fn assign(&self, assignment: &TopicPartitionList) -> KafkaResult<()> {
        self.get_base_consumer().assign(assignment)
    }

    /// Seek 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`.
    fn seek<T: Into<Timeout>>(
        &self,
        topic: &str,
        partition: i32,
        offset: Offset,
        timeout: T,
    ) -> KafkaResult<()> {
        self.get_base_consumer()
            .seek(topic, partition, offset, timeout)
    }

    /// 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.
    fn commit(
        &self,
        topic_partition_list: &TopicPartitionList,
        mode: CommitMode,
    ) -> KafkaResult<()> {
        self.get_base_consumer().commit(topic_partition_list, mode)
    }

    /// Commit 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.
    fn commit_consumer_state(&self, mode: CommitMode) -> KafkaResult<()> {
        self.get_base_consumer().commit_consumer_state(mode)
    }

    /// Commit the provided message. Note that this will also automatically commit every
    /// message with lower offset within the same partition.
    fn commit_message(&self, message: &BorrowedMessage, mode: CommitMode) -> KafkaResult<()> {
        self.get_base_consumer().commit_message(message, mode)
    }

    /// Store offset for this message to be used on the next (auto)commit.
    /// When using this `enable.auto.offset.store` should be set to `false` in the config.
    fn store_offset(&self, message: &BorrowedMessage) -> KafkaResult<()> {
        self.get_base_consumer().store_offset(message)
    }

    /// 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.
    fn store_offsets(&self, tpl: &TopicPartitionList) -> KafkaResult<()> {
        self.get_base_consumer().store_offsets(tpl)
    }

    /// Returns the current topic subscription.
    fn subscription(&self) -> KafkaResult<TopicPartitionList> {
        self.get_base_consumer().subscription()
    }

    /// Returns the current partition assignment.
    fn assignment(&self) -> KafkaResult<TopicPartitionList> {
        self.get_base_consumer().assignment()
    }

    /// Retrieve committed offsets for topics and partitions.
    fn committed<T>(&self, timeout: T) -> KafkaResult<TopicPartitionList>
    where
        T: Into<Timeout>,
        Self: Sized,
    {
        self.get_base_consumer().committed(timeout)
    }

    /// Retrieve committed offsets for specified topics and partitions.
    fn committed_offsets<T>(
        &self,
        tpl: TopicPartitionList,
        timeout: T,
    ) -> KafkaResult<TopicPartitionList>
    where
        T: Into<Timeout>,
    {
        self.get_base_consumer().committed_offsets(tpl, timeout)
    }

    /// Lookup the offsets for this consumer's partitions by timestamp.
    fn offsets_for_timestamp<T>(
        &self,
        timestamp: i64,
        timeout: T,
    ) -> KafkaResult<TopicPartitionList>
    where
        T: Into<Timeout>,
        Self: Sized,
    {
        self.get_base_consumer()
            .offsets_for_timestamp(timestamp, timeout)
    }

    /// Look up the offsets for the specified partitions by timestamp.
    fn offsets_for_times<T>(
        &self,
        timestamps: TopicPartitionList,
        timeout: T,
    ) -> KafkaResult<TopicPartitionList>
    where
        T: Into<Timeout>,
        Self: Sized,
    {
        self.get_base_consumer()
            .offsets_for_times(timestamps, timeout)
    }

    /// Retrieve current positions (offsets) for topics and partitions.
    fn position(&self) -> KafkaResult<TopicPartitionList> {
        self.get_base_consumer().position()
    }

    /// Returns the metadata information for the specified topic, or for all topics in the cluster
    /// if no topic is specified.
    fn fetch_metadata<T>(&self, topic: Option<&str>, timeout: T) -> KafkaResult<Metadata>
    where
        T: Into<Timeout>,
        Self: Sized,
    {
        self.get_base_consumer().fetch_metadata(topic, timeout)
    }

    /// Returns the metadata information for all the topics in the cluster.
    fn fetch_watermarks<T>(
        &self,
        topic: &str,
        partition: i32,
        timeout: T,
    ) -> KafkaResult<(i64, i64)>
    where
        T: Into<Timeout>,
        Self: Sized,
    {
        self.get_base_consumer()
            .fetch_watermarks(topic, partition, timeout)
    }

    /// Returns the group membership information for the given group. If no group is
    /// specified, all groups will be returned.
    fn fetch_group_list<T>(&self, group: Option<&str>, timeout: T) -> KafkaResult<GroupList>
    where
        T: Into<Timeout>,
        Self: Sized,
    {
        self.get_base_consumer().fetch_group_list(group, timeout)
    }

    /// Pause consumption for the provided list of partitions.
    fn pause(&self, partitions: &TopicPartitionList) -> KafkaResult<()> {
        self.get_base_consumer().pause(partitions)
    }

    /// Resume consumption for the provided list of partitions.
    fn resume(&self, partitions: &TopicPartitionList) -> KafkaResult<()> {
        self.get_base_consumer().resume(partitions)
    }
}