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 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403 404 405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593
//! High-level consumers with a [`Stream`](futures_util::Stream) interface.
use std::ffi::CString;
use std::marker::PhantomData;
use std::os::raw::c_void;
use std::pin::Pin;
use std::ptr;
use std::sync::{Arc, Mutex};
use std::task::{Context, Poll, Waker};
use std::time::Duration;
use crate::log::trace;
use futures_channel::oneshot;
use futures_util::future::{self, Either, FutureExt};
use futures_util::pin_mut;
use futures_util::stream::{Stream, StreamExt};
use slab::Slab;
use rdkafka_sys as rdsys;
use rdkafka_sys::types::*;
use crate::client::{Client, NativeQueue};
use crate::config::{ClientConfig, FromClientConfig, FromClientConfigAndContext};
use crate::consumer::base_consumer::BaseConsumer;
use crate::consumer::{
CommitMode, Consumer, ConsumerContext, ConsumerGroupMetadata, DefaultConsumerContext,
RebalanceProtocol,
};
use crate::error::{KafkaError, KafkaResult};
use crate::groups::GroupList;
use crate::message::BorrowedMessage;
use crate::metadata::Metadata;
use crate::topic_partition_list::{Offset, TopicPartitionList};
use crate::util::{AsyncRuntime, DefaultRuntime, NativePtr, Timeout};
unsafe extern "C" fn native_message_queue_nonempty_cb(_: *mut RDKafka, opaque_ptr: *mut c_void) {
let wakers = &*(opaque_ptr as *const WakerSlab);
wakers.wake_all();
}
unsafe fn enable_nonempty_callback(queue: &NativeQueue, wakers: &Arc<WakerSlab>) {
rdsys::rd_kafka_queue_cb_event_enable(
queue.ptr(),
Some(native_message_queue_nonempty_cb),
Arc::as_ptr(wakers) as *mut c_void,
)
}
unsafe fn disable_nonempty_callback(queue: &NativeQueue) {
rdsys::rd_kafka_queue_cb_event_enable(queue.ptr(), None, ptr::null_mut())
}
struct WakerSlab {
wakers: Mutex<Slab<Option<Waker>>>,
}
impl WakerSlab {
fn new() -> WakerSlab {
WakerSlab {
wakers: Mutex::new(Slab::new()),
}
}
fn wake_all(&self) {
let mut wakers = self.wakers.lock().unwrap();
for (_, waker) in wakers.iter_mut() {
if let Some(waker) = waker.take() {
waker.wake();
}
}
}
fn register(&self) -> usize {
let mut wakers = self.wakers.lock().expect("lock poisoned");
wakers.insert(None)
}
fn unregister(&self, slot: usize) {
let mut wakers = self.wakers.lock().expect("lock poisoned");
wakers.remove(slot);
}
fn set_waker(&self, slot: usize, waker: Waker) {
let mut wakers = self.wakers.lock().expect("lock poisoned");
wakers[slot] = Some(waker);
}
}
/// A stream of messages from a [`StreamConsumer`].
///
/// See the documentation of [`StreamConsumer::stream`] for details.
pub struct MessageStream<'a> {
wakers: &'a WakerSlab,
queue: &'a NativeQueue,
slot: usize,
}
impl<'a> MessageStream<'a> {
fn new(wakers: &'a WakerSlab, queue: &'a NativeQueue) -> MessageStream<'a> {
let slot = wakers.register();
MessageStream {
wakers,
queue,
slot,
}
}
fn poll(&self) -> Option<KafkaResult<BorrowedMessage<'a>>> {
unsafe {
NativePtr::from_ptr(rdsys::rd_kafka_consume_queue(self.queue.ptr(), 0))
.map(|p| BorrowedMessage::from_consumer(p, self.queue))
}
}
}
impl<'a> Stream for MessageStream<'a> {
type Item = KafkaResult<BorrowedMessage<'a>>;
fn poll_next(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
// If there is a message ready, yield it immediately to avoid the
// taking the lock in `self.set_waker`.
if let Some(message) = self.poll() {
return Poll::Ready(Some(message));
}
// Otherwise, we need to wait for a message to become available. Store
// the waker so that we are woken up if the queue flips from non-empty
// to empty. We have to store the waker repatedly in case this future
// migrates between tasks.
self.wakers.set_waker(self.slot, cx.waker().clone());
// Check whether a new message became available after we installed the
// waker. This avoids a race where `poll` returns None to indicate that
// the queue is empty, but the queue becomes non-empty before we've
// installed the waker.
match self.poll() {
None => Poll::Pending,
Some(message) => Poll::Ready(Some(message)),
}
}
}
impl<'a> Drop for MessageStream<'a> {
fn drop(&mut self) {
self.wakers.unregister(self.slot);
}
}
/// A high-level consumer with a [`Stream`](futures_util::Stream) interface.
///
/// This consumer doesn't need to be polled explicitly. Extracting an item from
/// the stream returned by the [`stream`](StreamConsumer::stream) will
/// implicitly poll the underlying Kafka consumer.
///
/// If you activate the consumer group protocol by calling
/// [`subscribe`](Consumer::subscribe), the stream consumer will integrate with
/// librdkafka's liveness detection as described in [KIP-62]. You must be sure
/// that you attempt to extract a message from the stream consumer at least
/// every `max.poll.interval.ms` milliseconds, or librdkafka will assume that
/// the processing thread is wedged and leave the consumer groups.
///
/// [KIP-62]: https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread
#[must_use = "Consumer polling thread will stop immediately if unused"]
pub struct StreamConsumer<C = DefaultConsumerContext, R = DefaultRuntime>
where
C: ConsumerContext + 'static,
{
base: BaseConsumer<C>,
wakers: Arc<WakerSlab>,
queue: NativeQueue,
_shutdown_trigger: oneshot::Sender<()>,
_runtime: PhantomData<R>,
}
impl<R> FromClientConfig for StreamConsumer<DefaultConsumerContext, R>
where
R: AsyncRuntime,
{
fn from_config(config: &ClientConfig) -> KafkaResult<Self> {
StreamConsumer::from_config_and_context(config, DefaultConsumerContext)
}
}
/// Creates a new `StreamConsumer` starting from a [`ClientConfig`].
impl<C, R> FromClientConfigAndContext<C> for StreamConsumer<C, R>
where
C: ConsumerContext + 'static,
R: AsyncRuntime,
{
fn from_config_and_context(config: &ClientConfig, context: C) -> KafkaResult<Self> {
let native_config = config.create_native_config()?;
let poll_interval = {
let millis: u64 = native_config
.get("max.poll.interval.ms")?
.parse()
.expect("librdkafka validated config value is valid u64");
Duration::from_millis(millis)
};
let base = BaseConsumer::new(config, native_config, context)?;
let native_ptr = base.client().native_ptr() as usize;
// Redirect rdkafka's main queue to the consumer queue so that we only
// need to listen to the consumer queue to observe events like
// rebalancings and stats.
unsafe { rdsys::rd_kafka_poll_set_consumer(base.client().native_ptr()) };
let queue = base.client().consumer_queue().ok_or_else(|| {
KafkaError::ClientCreation("librdkafka failed to create consumer queue".into())
})?;
let wakers = Arc::new(WakerSlab::new());
unsafe { enable_nonempty_callback(&queue, &wakers) }
// We need to make sure we poll the consumer at least once every max
// poll interval, *unless* the processing task has wedged. To accomplish
// this, we launch a background task that sends spurious wakeup
// notifications at half the max poll interval. An unwedged processing
// task will wake up and poll the consumer with plenty of time to spare,
// while a wedged processing task will not.
//
// The default max poll interval is 5m, so there is essentially no
// performance impact to these spurious wakeups.
let (shutdown_trigger, shutdown_tripwire) = oneshot::channel();
let mut shutdown_tripwire = shutdown_tripwire.fuse();
R::spawn({
let wakers = wakers.clone();
async move {
trace!("Starting stream consumer wake loop: 0x{:x}", native_ptr);
loop {
let delay = R::delay_for(poll_interval / 2).fuse();
pin_mut!(delay);
match future::select(&mut delay, &mut shutdown_tripwire).await {
Either::Left(_) => wakers.wake_all(),
Either::Right(_) => break,
}
}
trace!("Shut down stream consumer wake loop: 0x{:x}", native_ptr);
}
});
Ok(StreamConsumer {
base,
wakers,
queue,
_shutdown_trigger: shutdown_trigger,
_runtime: PhantomData,
})
}
}
impl<C, R> StreamConsumer<C, R>
where
C: ConsumerContext + 'static,
{
/// Constructs a stream that yields messages from this consumer.
///
/// It is legal to have multiple live message streams for the same consumer,
/// and to move those message streams across threads. Note, however, that
/// the message streams share the same underlying state. A message received
/// by the consumer will be delivered to only one of the live message
/// streams. If you seek the underlying consumer, all message streams
/// created from the consumer will begin to draw messages from the new
/// position of the consumer.
///
/// If you want multiple independent views of a Kafka topic, create multiple
/// consumers, not multiple message streams.
pub fn stream(&self) -> MessageStream<'_> {
MessageStream::new(&self.wakers, &self.queue)
}
/// Receives the next message from the stream.
///
/// This method will block until the next message is available or an error
/// occurs. It is legal to call `recv` from multiple threads simultaneously.
///
/// This method is [cancellation safe].
///
/// Note that this method is exactly as efficient as constructing a
/// single-use message stream and extracting one message from it:
///
/// ```
/// use futures::stream::StreamExt;
/// # use rdkafka::consumer::StreamConsumer;
///
/// # async fn example(consumer: StreamConsumer) {
/// consumer.stream().next().await.expect("MessageStream never returns None");
/// # }
/// ```
///
/// [cancellation safe]: https://docs.rs/tokio/latest/tokio/macro.select.html#cancellation-safety
pub async fn recv(&self) -> Result<BorrowedMessage<'_>, KafkaError> {
self.stream()
.next()
.await
.expect("kafka streams never terminate")
}
/// Splits messages for the specified partition into their own stream.
///
/// If the `topic` or `partition` is invalid, returns `None`.
///
/// After calling this method, newly-fetched messages for the specified
/// partition will be returned via [`StreamPartitionQueue::recv`] rather
/// than [`StreamConsumer::recv`]. Note that there may be buffered messages
/// for the specified partition that will continue to be returned by
/// `StreamConsumer::recv`. For best results, call `split_partition_queue`
/// before the first call to
/// `StreamConsumer::recv`.
///
/// You must periodically await `StreamConsumer::recv`, even if no messages
/// are expected, to serve callbacks. Consider using a background task like:
///
/// ```
/// # use rdkafka::consumer::StreamConsumer;
/// # use tokio::task::JoinHandle;
/// # async fn example(stream_consumer: StreamConsumer) -> JoinHandle<()> {
/// tokio::spawn(async move {
/// let message = stream_consumer.recv().await;
/// panic!("main stream consumer queue unexpectedly received message: {:?}", message);
/// })
/// # }
/// ```
///
/// 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.
pub fn split_partition_queue(
self: &Arc<Self>,
topic: &str,
partition: i32,
) -> Option<StreamPartitionQueue<C, R>> {
let topic = match CString::new(topic) {
Ok(topic) => topic,
Err(_) => return None,
};
let queue = unsafe {
NativeQueue::from_ptr(rdsys::rd_kafka_queue_get_partition(
self.base.client().native_ptr(),
topic.as_ptr(),
partition,
))
};
queue.map(|queue| {
let wakers = Arc::new(WakerSlab::new());
unsafe {
rdsys::rd_kafka_queue_forward(queue.ptr(), ptr::null_mut());
enable_nonempty_callback(&queue, &wakers);
}
StreamPartitionQueue {
queue,
wakers,
_consumer: self.clone(),
}
})
}
}
impl<C, R> Consumer<C> for StreamConsumer<C, R>
where
C: ConsumerContext,
{
fn client(&self) -> &Client<C> {
self.base.client()
}
fn group_metadata(&self) -> Option<ConsumerGroupMetadata> {
self.base.group_metadata()
}
fn subscribe(&self, topics: &[&str]) -> KafkaResult<()> {
self.base.subscribe(topics)
}
fn unsubscribe(&self) {
self.base.unsubscribe();
}
fn assign(&self, assignment: &TopicPartitionList) -> KafkaResult<()> {
self.base.assign(assignment)
}
fn seek<T: Into<Timeout>>(
&self,
topic: &str,
partition: i32,
offset: Offset,
timeout: T,
) -> KafkaResult<()> {
self.base.seek(topic, partition, offset, timeout)
}
fn commit(
&self,
topic_partition_list: &TopicPartitionList,
mode: CommitMode,
) -> KafkaResult<()> {
self.base.commit(topic_partition_list, mode)
}
fn commit_consumer_state(&self, mode: CommitMode) -> KafkaResult<()> {
self.base.commit_consumer_state(mode)
}
fn commit_message(&self, message: &BorrowedMessage<'_>, mode: CommitMode) -> KafkaResult<()> {
self.base.commit_message(message, mode)
}
fn store_offset(&self, topic: &str, partition: i32, offset: i64) -> KafkaResult<()> {
self.base.store_offset(topic, partition, offset)
}
fn store_offset_from_message(&self, message: &BorrowedMessage<'_>) -> KafkaResult<()> {
self.base.store_offset_from_message(message)
}
fn store_offsets(&self, tpl: &TopicPartitionList) -> KafkaResult<()> {
self.base.store_offsets(tpl)
}
fn subscription(&self) -> KafkaResult<TopicPartitionList> {
self.base.subscription()
}
fn assignment(&self) -> KafkaResult<TopicPartitionList> {
self.base.assignment()
}
fn committed<T>(&self, timeout: T) -> KafkaResult<TopicPartitionList>
where
T: Into<Timeout>,
Self: Sized,
{
self.base.committed(timeout)
}
fn committed_offsets<T>(
&self,
tpl: TopicPartitionList,
timeout: T,
) -> KafkaResult<TopicPartitionList>
where
T: Into<Timeout>,
{
self.base.committed_offsets(tpl, timeout)
}
fn offsets_for_timestamp<T>(
&self,
timestamp: i64,
timeout: T,
) -> KafkaResult<TopicPartitionList>
where
T: Into<Timeout>,
Self: Sized,
{
self.base.offsets_for_timestamp(timestamp, timeout)
}
fn offsets_for_times<T>(
&self,
timestamps: TopicPartitionList,
timeout: T,
) -> KafkaResult<TopicPartitionList>
where
T: Into<Timeout>,
Self: Sized,
{
self.base.offsets_for_times(timestamps, timeout)
}
fn position(&self) -> KafkaResult<TopicPartitionList> {
self.base.position()
}
fn fetch_metadata<T>(&self, topic: Option<&str>, timeout: T) -> KafkaResult<Metadata>
where
T: Into<Timeout>,
Self: Sized,
{
self.base.fetch_metadata(topic, timeout)
}
fn fetch_watermarks<T>(
&self,
topic: &str,
partition: i32,
timeout: T,
) -> KafkaResult<(i64, i64)>
where
T: Into<Timeout>,
Self: Sized,
{
self.base.fetch_watermarks(topic, partition, timeout)
}
fn fetch_group_list<T>(&self, group: Option<&str>, timeout: T) -> KafkaResult<GroupList>
where
T: Into<Timeout>,
Self: Sized,
{
self.base.fetch_group_list(group, timeout)
}
fn pause(&self, partitions: &TopicPartitionList) -> KafkaResult<()> {
self.base.pause(partitions)
}
fn resume(&self, partitions: &TopicPartitionList) -> KafkaResult<()> {
self.base.resume(partitions)
}
fn rebalance_protocol(&self) -> RebalanceProtocol {
self.base.rebalance_protocol()
}
}
/// A message queue for a single partition of a [`StreamConsumer`].
///
/// See the documentation of [`StreamConsumer::split_partition_queue`] for
/// details.
pub struct StreamPartitionQueue<C, R = DefaultRuntime>
where
C: ConsumerContext + 'static,
{
queue: NativeQueue,
wakers: Arc<WakerSlab>,
_consumer: Arc<StreamConsumer<C, R>>,
}
impl<C, R> StreamPartitionQueue<C, R>
where
C: ConsumerContext,
{
/// Constructs a stream that yields messages from this partition.
///
/// It is legal to have multiple live message streams for the same
/// partition, and to move those message streams across threads. Note,
/// however, that the message streams share the same underlying state. A
/// message received by the partition will be delivered to only one of the
/// live message streams. If you seek the underlying partition, all message
/// streams created from the partition will begin to draw messages from the
/// new position of the partition.
///
/// If you want multiple independent views of a Kafka partition, create
/// multiple consumers, not multiple partition streams.
pub fn stream(&self) -> MessageStream<'_> {
MessageStream::new(&self.wakers, &self.queue)
}
/// Receives the next message from the stream.
///
/// This method will block until the next message is available or an error
/// occurs. It is legal to call `recv` from multiple threads simultaneously.
///
/// This method is [cancellation safe].
///
/// Note that this method is exactly as efficient as constructing a
/// single-use message stream and extracting one message from it:
///
/// ```
/// use futures::stream::StreamExt;
/// # use rdkafka::consumer::ConsumerContext;
/// # use rdkafka::consumer::stream_consumer::StreamPartitionQueue;
//
/// # async fn example<C>(partition_queue: StreamPartitionQueue<C>)
/// # where
/// # C: ConsumerContext {
/// partition_queue.stream().next().await.expect("MessageStream never returns None");
/// # }
/// ```
///
/// [cancellation safe]: https://docs.rs/tokio/latest/tokio/macro.select.html#cancellation-safety
pub async fn recv(&self) -> Result<BorrowedMessage<'_>, KafkaError> {
self.stream()
.next()
.await
.expect("kafka streams never terminate")
}
}
impl<C, R> Drop for StreamPartitionQueue<C, R>
where
C: ConsumerContext,
{
fn drop(&mut self) {
unsafe { disable_nonempty_callback(&self.queue) }
}
}