From 488aa6308755d54556c35f5155b9afb0e5c29313 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 27 Apr 2023 17:27:34 -0400 Subject: [PATCH 01/37] test(kafka source): integration tests for acknowledgement handling during shutdown and rebalance events MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit When running locally, this is most easily tested using an existing kafka topic pre-populated with, say 100k messages: ❯ kcat -b $BROKER_ADDR -P -t TestTopic -K : -l <(for i in $(seq 1 100000); do echo "${i}:{\"value\": ${i}}"; done); ...then running the tests, targeting that topic through environment variables. This can be a bit finicky with regard to timings, so KAFKA_SHUTDOWN_DELAY controls how long to run the first consumer before shutting down (drain at shutdown test), and KAFKA_CONSUMER_DELAY controls the time between starting new consumers during the rebalancing test. ❯ KAFKA_SEND_COUNT=0 \ KAFKA_EXPECT_COUNT=100000 \ KAFKA_TEST_TOPIC=TestTopic \ KAFKA_CONSUMER_DELAY=5000 \ KAFKA_SHUTDOWN_DELAY=5000 \ KAFKA_HOST=$BROKER_ADDR \ KAFKA_PORT=9092 \ cargo test --lib --no-default-features -F sources-kafka -F kafka-integration-tests drains_acknowledgement --- src/sources/kafka.rs | 295 ++++++++++++++++++++++++++++++++++++++----- 1 file changed, 261 insertions(+), 34 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 762c13f9c6cd9..df71f2debb039 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -5,6 +5,9 @@ use std::{ time::Duration, }; +#[cfg(test)] +use std::collections::HashSet; + use async_stream::stream; use bytes::Bytes; use chrono::{DateTime, TimeZone, Utc}; @@ -301,6 +304,8 @@ impl SourceConfig for KafkaSourceConfig { cx.shutdown, cx.out, acknowledgements, + #[cfg(test)] + false, log_namespace, ))) } @@ -374,6 +379,7 @@ async fn kafka_source( mut shutdown: ShutdownSignal, mut out: SourceSender, acknowledgements: bool, + #[cfg(test)] eof: bool, log_namespace: LogNamespace, ) -> Result<(), ()> { let consumer = Arc::new(consumer); @@ -390,6 +396,9 @@ async fn kafka_source( let mut stream = consumer.stream(); + #[cfg(test)] + let mut eof_partitions = HashSet::new(); + loop { tokio::select! { biased; @@ -404,8 +413,26 @@ async fn kafka_source( } }, message = stream.next() => match message { - None => break, // WHY? - Some(Err(error)) => emit!(KafkaReadError { error }), + None => unreachable!("MessageStream never returns Ready(None)"), + Some(Err(error)) => match error { + #[cfg(test)] + rdkafka::error::KafkaError::PartitionEOF(partition) if eof => { + // Optionally exit on partition EOF. This is used in tests to collect events + // when the precise number of events is not known (e.g. restarting the source after + // shutdown or a consumer rebalance). Note that the client should set "enable.partition.eof" + // to true, or the client won't get this notification + // Note also, this assumes a single topic is being consumed, since + // PartitionEOF does not include the topic name + eof_partitions.insert(partition); + if let Ok(assignment) = consumer.assignment() { + debug!("Reached end of partition {} ({} of {}).", partition, eof_partitions.len(), assignment.count()); + if eof_partitions.len() == assignment.count() { + break; + } + } + }, + _ => emit!(KafkaReadError { error }), + }, Some(Ok(msg)) => { emit!(KafkaBytesReceived { byte_size: msg.payload_len(), @@ -758,9 +785,17 @@ mod test { pub fn kafka_host() -> String { std::env::var("KAFKA_HOST").unwrap_or_else(|_| "localhost".into()) } + pub fn kafka_port() -> u16 { + let port = std::env::var("KAFKA_PORT").unwrap_or_else(|_| "9091".into()); + port.parse().expect("Invalid port number") + } - pub fn kafka_address(port: u16) -> String { - format!("{}:{}", kafka_host(), port) + pub fn kafka_address() -> String { + format!("{}:{}", kafka_host(), kafka_port()) + } + + fn kafka_max_bytes() -> String { + std::env::var("KAFKA_MAX_BYTES").unwrap_or_else(|_| "1024".into()) } #[test] @@ -773,13 +808,19 @@ mod test { group: &str, log_namespace: LogNamespace, ) -> KafkaSourceConfig { + let mut kafka_options = HashMap::new(); + kafka_options.insert("enable.partition.eof".into(), "true".into()); + // Fetch in small batches + kafka_options.insert("fetch.message.max.bytes".into(), kafka_max_bytes().into()); + KafkaSourceConfig { - bootstrap_servers: kafka_address(9091), + bootstrap_servers: kafka_address(), topics: vec![topic.into()], group_id: group.into(), auto_offset_reset: "beginning".into(), session_timeout_ms: Duration::from_millis(6000), commit_interval_ms: Duration::from_millis(1), + librdkafka_options: Some(kafka_options), key_field: default_key_field(), topic_key: default_topic_key(), partition_key: default_partition_key(), @@ -928,9 +969,14 @@ mod integration_test { const HEADER_KEY: &str = "my header"; const HEADER_VALUE: &str = "my header value"; + fn kafka_test_topic() -> String { + std::env::var("KAFKA_TEST_TOPIC") + .unwrap_or_else(|_| format!("test-topic-{}", random_string(10)).into()) + } + fn client_config(group: Option<&str>) -> T { let mut client = ClientConfig::new(); - client.set("bootstrap.servers", kafka_address(9091)); + client.set("bootstrap.servers", kafka_address()); client.set("produce.offset.report", "true"); client.set("message.timeout.ms", "5000"); client.set("auto.commit.interval.ms", "1"); @@ -940,25 +986,33 @@ mod integration_test { client.create().expect("Producer creation error") } - async fn send_events(topic: String, count: usize) -> DateTime { + async fn send_events(topic: String, partitions: i32, count: usize) -> DateTime { let now = Utc::now(); let timestamp = now.timestamp_millis(); - let producer: FutureProducer = client_config(None); - - for i in 0..count { - let text = format!("{} {:03}", TEXT, i); - let key = format!("{} {}", KEY, i); - let record = FutureRecord::to(&topic) - .payload(&text) - .key(&key) - .timestamp(timestamp) - .headers(OwnedHeaders::new().insert(Header { - key: HEADER_KEY, - value: Some(HEADER_VALUE), - })); - - if let Err(error) = producer.send(record, Timeout::Never).await { + let producer: &FutureProducer = &client_config(None); + let topic_name = topic.as_ref(); + + create_topic(topic_name, partitions).await; + + let writes = (0..count) + .map(|i| async move { + let text = format!("{} {:03}", TEXT, i); + let key = format!("{} {}", KEY, i); + let record = FutureRecord::to(topic_name) + .payload(&text) + .key(&key) + .timestamp(timestamp) + .headers(OwnedHeaders::new().insert(Header { + key: HEADER_KEY, + value: Some(HEADER_VALUE), + })); + producer.send(record, Timeout::Never).await + }) + .collect::>(); + + for res in writes { + if let Err(error) = res.await { panic!("Cannot send event to Kafka: {:?}", error); } } @@ -966,6 +1020,15 @@ mod integration_test { now } + async fn send_to_test_topic(partitions: i32, count: usize) -> (String, String, DateTime) { + let topic = kafka_test_topic(); + let group_id = format!("test-group-{}", random_string(10)); + + let sent_at = send_events(topic.clone(), partitions, count).await; + + (topic, group_id, sent_at) + } + #[tokio::test] async fn consumes_event_with_acknowledgements() { send_receive(true, |_| false, 10, LogNamespace::Legacy).await; @@ -1018,12 +1081,12 @@ mod integration_test { let group_id = format!("test-group-{}", random_string(10)); let config = make_config(&topic, &group_id, log_namespace); - let now = send_events(topic.clone(), 10).await; + let now = send_events(topic.clone(), 1, 10).await; let events = assert_source_compliance(&["protocol", "topic", "partition"], async move { let (tx, rx) = SourceSender::new_test_errors(error_at); let (trigger_shutdown, shutdown_done) = - spawn_kafka(tx, config, acknowledgements, log_namespace); + spawn_kafka(tx, config, acknowledgements, false, log_namespace); let events = collect_n(rx, SEND_COUNT).await; // Yield to the finalization task to let it collect the // batch status receivers before signalling the shutdown. @@ -1138,6 +1201,7 @@ mod integration_test { tx: SourceSender, config: KafkaSourceConfig, acknowledgements: bool, + eof: bool, log_namespace: LogNamespace, ) -> (Trigger, Tripwire) { let (trigger_shutdown, shutdown, shutdown_done) = ShutdownSignal::new_wired(); @@ -1157,6 +1221,7 @@ mod integration_test { shutdown, tx, acknowledgements, + eof, log_namespace, )); (trigger_shutdown, shutdown_done) @@ -1176,9 +1241,9 @@ mod integration_test { .offset() } - async fn create_topic(group_id: &str, topic: &str, partitions: i32) { - let client: AdminClient = client_config(Some(group_id)); - for result in client + async fn create_topic(topic: &str, partitions: i32) { + let client: AdminClient = client_config(None); + let topic_results = client .create_topics( [&NewTopic { name: topic, @@ -1189,9 +1254,14 @@ mod integration_test { &AdminOptions::default(), ) .await - .expect("create_topics failed") - { - result.expect("Creating a topic failed"); + .expect("create_topics failed"); + + for result in topic_results { + if let Err((topic, err)) = result { + if err != rdkafka::types::RDKafkaErrorCode::TopicAlreadyExists { + panic!("Creating a topic failed: {:?}", (topic, err)) + } + } } } @@ -1224,20 +1294,20 @@ mod integration_test { const DELAY: u64 = 100; let (topic, group_id, config) = make_rand_config(); - create_topic(&group_id, &topic, 2).await; + create_topic(&topic, 2).await; - let _send_start = send_events(topic.clone(), NEVENTS).await; + let _send_start = send_events(topic.clone(), 1, NEVENTS).await; let (tx, rx1) = delay_pipeline(1, Duration::from_millis(200), EventStatus::Delivered); let (trigger_shutdown1, shutdown_done1) = - spawn_kafka(tx, config.clone(), true, LogNamespace::Legacy); + spawn_kafka(tx, config.clone(), true, false, LogNamespace::Legacy); let events1 = tokio::spawn(collect_n(rx1, NEVENTS)); sleep(Duration::from_secs(1)).await; let (tx, rx2) = delay_pipeline(2, Duration::from_millis(DELAY), EventStatus::Delivered); let (trigger_shutdown2, shutdown_done2) = - spawn_kafka(tx, config, true, LogNamespace::Legacy); + spawn_kafka(tx, config, true, true, LogNamespace::Legacy); let events2 = tokio::spawn(collect_n(rx2, NEVENTS)); sleep(Duration::from_secs(5)).await; @@ -1281,6 +1351,163 @@ mod integration_test { // Assert they are all in sequential order and no dupes, TODO } + #[tokio::test] + async fn drains_acknowledgements_at_shutdown() { + // 1. Send N events (if running against a pre-populated kafka topic, use send_count=0 and expect_count=expected number of messages; otherwise just set send_count) + let send_count: usize = std::env::var("KAFKA_SEND_COUNT") + .unwrap_or_else(|_| "100".into()) + .parse() + .expect("Number of messages to send to kafka."); + let expect_count: usize = std::env::var("KAFKA_EXPECT_COUNT") + .unwrap_or_else(|_| format!("{}", send_count).into()) + .parse() + .expect("Number of messages to expect consumers to process."); + let delay_ms: u64 = std::env::var("KAFKA_SHUTDOWN_DELAY") + .unwrap_or_else(|_| "3000".into()) + .parse() + .expect("Number of milliseconds before shutting down first consumer."); + + let (topic, group_id, _) = send_to_test_topic(1, send_count).await; + + // 2. Run the kafka source to read some of the events + // 3. Send a shutdown signal (at some point before all events are read) + let events1 = { + let config = make_config(&topic, &group_id, LogNamespace::Legacy); + let (tx, rx) = SourceSender::new_test_errors(|_| false); + let (trigger_shutdown, shutdown_done) = + spawn_kafka(tx, config, true, false, LogNamespace::Legacy); + let (events, _) = tokio::join!(rx.collect::>(), async move { + sleep(Duration::from_millis(delay_ms)).await; + drop(trigger_shutdown); + }); + shutdown_done.await; + events + }; + + debug!("Consumer group.id: {}", &group_id); + debug!( + "First consumer read {} of {} messages.", + events1.len(), + expect_count + ); + + // 4. Run the kafka source again to finish reading the events + let events2 = { + let config = make_config(&topic, &group_id, LogNamespace::Legacy); + let (tx, rx) = SourceSender::new_test_errors(|_| false); + let (trigger_shutdown, shutdown_done) = + spawn_kafka(tx, config, true, true, LogNamespace::Legacy); + let events = rx.collect::>().await; + drop(trigger_shutdown); + shutdown_done.await; + events + }; + + debug!( + "Second consumer read {} of {} messages.", + events2.len(), + expect_count + ); + + // 5. Total number of events processed should equal the number sent + let total = events1.len() + events2.len(); + assert_ne!( + events1.len(), + 0, + "First batch of events is non-zero (increase KAFKA_SHUTDOWN_DELAY?)" + ); + assert_ne!(events2.len(), 0, "Second batch of events is non-zero (decrease KAFKA_SHUTDOWN_DELAY or increase KAFKA_SEND_COUNT?) "); + assert_eq!(total, expect_count); + } + + #[tokio::test] + async fn drains_acknowledgements_during_rebalance() { + // 1. Send N events (if running against a pre-populated kafka topic, use send_count=0 and expect_count=expected number of messages; otherwise just set send_count) + let send_count: usize = std::env::var("KAFKA_SEND_COUNT") + .unwrap_or_else(|_| "100".into()) + .parse() + .expect("Number of messages to send to kafka."); + let expect_count: usize = std::env::var("KAFKA_EXPECT_COUNT") + .unwrap_or_else(|_| format!("{}", send_count).into()) + .parse() + .expect("Number of messages to expect consumers to process."); + let delay_ms: u64 = std::env::var("KAFKA_CONSUMER_DELAY") + .unwrap_or_else(|_| "3000".into()) + .parse() + .expect("Number of milliseconds before shutting down first consumer."); + + let (topic, group_id, _) = send_to_test_topic(2, send_count).await; + debug!("Consumer group.id: {}", &group_id); + + // 2. Run the kafka source to read some of the events + // 3. Start 2nd & 3rd consumers using the same group.id, triggering rebalance events + let config1 = make_config(&topic, &group_id, LogNamespace::Legacy); + let config2 = make_config(&topic, &group_id, LogNamespace::Legacy); + let config3 = make_config(&topic, &group_id, LogNamespace::Legacy); + + let (events1, events2, events3) = tokio::join!( + async move { + let (tx, rx) = SourceSender::new_test_errors(|_| false); + let (_trigger_shutdown, _shutdown_done) = + spawn_kafka(tx, config1, true, true, LogNamespace::Legacy); + + rx.collect::>().await + }, + async move { + sleep(Duration::from_millis(delay_ms)).await; + let (tx, rx) = SourceSender::new_test_errors(|_| false); + let (_trigger_shutdown, _shutdown_done) = + spawn_kafka(tx, config2, true, true, LogNamespace::Legacy); + + rx.collect::>().await + }, + async move { + sleep(Duration::from_millis(delay_ms * 2)).await; + let (tx, rx) = SourceSender::new_test_errors(|_| false); + let (_trigger_shutdown, _shutdown_done) = + spawn_kafka(tx, config3, true, true, LogNamespace::Legacy); + + rx.collect::>().await + } + ); + + debug!( + "First consumer read {} of {} messages.", + events1.len(), + expect_count + ); + + debug!( + "Second consumer read {} of {} messages.", + events2.len(), + expect_count + ); + debug!( + "Third consumer read {} of {} messages.", + events3.len(), + expect_count + ); + + // 5. Total number of events processed should equal the number sent + let total = events1.len() + events2.len() + events3.len(); + assert_ne!( + events1.len(), + 0, + "First batch of events should be non-zero (increase delay?)" + ); + assert_ne!( + events2.len(), + 0, + "Second batch of events is non-zero (decrease delay or increase KAFKA_SEND_COUNT?) " + ); + assert_ne!( + events3.len(), + 0, + "Third batch of events is non-zero (decrease delay or increase KAFKA_SEND_COUNT?) " + ); + assert_eq!(total, expect_count); + } + fn map_logs(events: EventArray) -> impl Iterator { events.into_events().map(|event| { let log = event.into_log(); From 2b81df39ab77d4a757856723507af81b07055b82 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Tue, 16 May 2023 16:45:48 -0400 Subject: [PATCH 02/37] fix(kafka source): drain pending acknowledgements on shutdown and rebalance --- src/internal_events/kafka.rs | 18 ++ src/sources/kafka.rs | 511 ++++++++++++++++++++++++++++------- 2 files changed, 424 insertions(+), 105 deletions(-) diff --git a/src/internal_events/kafka.rs b/src/internal_events/kafka.rs index 92afe66b98a53..ee4a4889adb6b 100644 --- a/src/internal_events/kafka.rs +++ b/src/internal_events/kafka.rs @@ -111,6 +111,24 @@ impl InternalEvent for KafkaReadError { } } +#[derive(Debug)] +pub struct KafkaPauseResumeError { + pub error: rdkafka::error::KafkaError, +} + +impl InternalEvent for KafkaPauseResumeError { + fn emit(self) { + error!( + message = "Failed to pause or resume consumer.", + error = %self.error, + error_code = "pause_resume_consumer", + error_type = error_type::COMMAND_FAILED, + stage = error_stage::RECEIVING, + internal_log_rate_limit = true, + ) + } +} + #[derive(Debug)] pub struct KafkaStatisticsReceived<'a> { pub statistics: &'a rdkafka::Statistics, diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index df71f2debb039..ba71b1f03c797 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -1,13 +1,13 @@ use std::{ collections::{BTreeMap, HashMap}, io::Cursor, - sync::Arc, + sync::{ + mpsc::{sync_channel, SyncSender}, + Arc, RwLock, Weak, + }, time::Duration, }; -#[cfg(test)] -use std::collections::HashSet; - use async_stream::stream; use bytes::Bytes; use chrono::{DateTime, TimeZone, Utc}; @@ -15,19 +15,26 @@ use codecs::{ decoding::{DeserializerConfig, FramingConfig}, StreamDecodingError, }; -use futures::{Stream, StreamExt}; +use futures::{stream::BoxStream, Stream, StreamExt}; use lookup::{lookup_v2::OptionalValuePath, owned_value_path, path, OwnedValuePath}; use once_cell::sync::OnceCell; use rdkafka::{ consumer::{CommitMode, Consumer, ConsumerContext, Rebalance, StreamConsumer}, message::{BorrowedMessage, Headers as _, Message}, - ClientConfig, ClientContext, Statistics, + ClientConfig, ClientContext, Statistics, types::RDKafkaErrorCode, error::KafkaError, }; use serde_with::serde_as; use snafu::{ResultExt, Snafu}; +use tokio::{ + runtime::Handle, + sync::mpsc::{self, UnboundedReceiver, UnboundedSender}, + task::JoinHandle, + time::Instant, +}; +use tokio_stream::StreamMap; use tokio_util::codec::FramedRead; -use vector_common::finalizer::OrderedFinalizer; +use vector_common::{finalization::BatchStatusReceiver, finalizer::OrderedFinalizer}; use vector_config::configurable_component; use vector_core::{ config::{LegacyKey, LogNamespace}, @@ -43,8 +50,8 @@ use crate::{ }, event::{BatchNotifier, BatchStatus, Event, Value}, internal_events::{ - KafkaBytesReceived, KafkaEventsReceived, KafkaOffsetUpdateError, KafkaReadError, - StreamClosedError, + KafkaBytesReceived, KafkaEventsReceived, KafkaOffsetUpdateError, KafkaPauseResumeError, + KafkaReadError, StreamClosedError, }, kafka, serde::{bool_or_struct, default_decoding, default_framing_message_based}, @@ -292,10 +299,10 @@ impl SourceConfig for KafkaSourceConfig { async fn build(&self, cx: SourceContext) -> crate::Result { let log_namespace = cx.log_namespace(self.log_namespace); - let consumer = create_consumer(self)?; + let acknowledgements = cx.do_acknowledgements(self.acknowledgements); + let consumer = create_consumer(self, acknowledgements)?; let decoder = DecodingConfig::new(self.framing.clone(), self.decoding.clone(), log_namespace).build(); - let acknowledgements = cx.do_acknowledgements(self.acknowledgements); Ok(Box::pin(kafka_source( self.clone(), @@ -374,77 +381,56 @@ impl SourceConfig for KafkaSourceConfig { async fn kafka_source( config: KafkaSourceConfig, - consumer: StreamConsumer, + consumer: StreamConsumer, decoder: Decoder, - mut shutdown: ShutdownSignal, - mut out: SourceSender, + shutdown: ShutdownSignal, + out: SourceSender, acknowledgements: bool, #[cfg(test)] eof: bool, log_namespace: LogNamespace, ) -> Result<(), ()> { let consumer = Arc::new(consumer); - let (finalizer, mut ack_stream) = - OrderedFinalizer::::maybe_new(acknowledgements, Some(shutdown.clone())); - let finalizer = finalizer.map(Arc::new); - if let Some(finalizer) = &finalizer { - consumer - .context() - .finalizer - .set(Arc::clone(finalizer)) - .expect("Finalizer is only set once"); - } - let mut stream = consumer.stream(); + consumer + .context() + .consumer + .set(Arc::downgrade(&consumer)) + .expect("Error setting up consumer context."); - #[cfg(test)] - let mut eof_partitions = HashSet::new(); + let mut ack_task = None; + if acknowledgements { + let consumer = Arc::clone(&consumer); + let (callback_sender, callback_rx) = mpsc::unbounded_channel(); - loop { - tokio::select! { - biased; - _ = &mut shutdown => break, - entry = ack_stream.next() => if let Some((status, entry)) = entry { - if status == BatchStatus::Delivered { - if let Err(error) = - consumer.store_offset(&entry.topic, entry.partition, entry.offset) - { - emit!(KafkaOffsetUpdateError { error }); - } - } - }, - message = stream.next() => match message { - None => unreachable!("MessageStream never returns Ready(None)"), - Some(Err(error)) => match error { - #[cfg(test)] - rdkafka::error::KafkaError::PartitionEOF(partition) if eof => { - // Optionally exit on partition EOF. This is used in tests to collect events - // when the precise number of events is not known (e.g. restarting the source after - // shutdown or a consumer rebalance). Note that the client should set "enable.partition.eof" - // to true, or the client won't get this notification - // Note also, this assumes a single topic is being consumed, since - // PartitionEOF does not include the topic name - eof_partitions.insert(partition); - if let Ok(assignment) = consumer.assignment() { - debug!("Reached end of partition {} ({} of {}).", partition, eof_partitions.len(), assignment.count()); - if eof_partitions.len() == assignment.count() { - break; - } - } - }, - _ => emit!(KafkaReadError { error }), - }, - Some(Ok(msg)) => { - emit!(KafkaBytesReceived { - byte_size: msg.payload_len(), - protocol: "tcp", - topic: msg.topic(), - partition: msg.partition(), - }); + consumer + .context() + .callbacks + .set(callback_sender) + .expect("Error setting up consumer callback channel."); - parse_message(msg, decoder.clone(), config.keys(), &finalizer, &mut out, &consumer, log_namespace).await; - } - }, - } + ack_task = Some(handle_acks( + consumer, + callback_rx, + config.session_timeout_ms, + )); + } + + let msg_task = handle_messages( + config, + Arc::clone(&consumer), + decoder, + shutdown, + out, + log_namespace, + #[cfg(test)] + eof, + ); + + if let Some(ack_task) = ack_task { + _ = tokio::join!(msg_task, ack_task); + // _ = tokio::join!(ack_task); + } else { + _ = tokio::join!(msg_task); } // Since commits are async internally, we try one last sync commit inside the interval @@ -457,33 +443,197 @@ async fn kafka_source( Ok(()) } +fn handle_acks( + consumer: Arc>, + mut callbacks: UnboundedReceiver, + max_drain_ms: Duration, +) -> JoinHandle<()> { + let mut drain_signal: Option> = None; + + fn handle_ack( + consumer: &Arc>, + status: BatchStatus, + entry: FinalizerEntry, + ) { + if status == BatchStatus::Delivered { + if let Err(error) = consumer.store_offset(&entry.topic, entry.partition, entry.offset) { + emit!(KafkaOffsetUpdateError { error }); + } + } + } + + tokio::spawn(async move { + let mut ack_streams: StreamMap = StreamMap::new(); + let mut draining_acks: StreamMap = StreamMap::new(); + + let mut shutting_down = false; + let drain_deadline = tokio::time::sleep(max_drain_ms); + tokio::pin!(drain_deadline); + + loop { + let has_acks = !draining_acks.is_empty() || !ack_streams.is_empty(); + let next_ack = if drain_signal.is_some() { + draining_acks.next() + } else { + ack_streams.next() + }; + + tokio::select! { + entry = next_ack, if has_acks || drain_signal.is_some() => match entry { + Some((_key, (delivery_status, entry))) => { + handle_ack(&consumer, delivery_status, entry); + if let Some(ref signal) = drain_signal { + _ = signal.send(()); + } + }, + // Nothing left in the acknowledgement stream; check for exit conditions + None => match drain_signal.take() { + Some(signal) => { + if let Err(e) = signal.send(()) { + warn!("Error signaling client task: {:?}", e); + } + }, + None if shutting_down => { + break + }, + None => { + // ack stream is empty, but we're not shutting down or rebalancing (probably starting up, or just finished a rebalance) + } + } + }, + + _ = &mut drain_deadline, if drain_signal.is_some() => { + debug!("Acknowledgement drain deadline reached. Dropping any pending ack streams for revoked partitions."); + draining_acks.clear(); + if let Err(e) = drain_signal.take().unwrap().send(()) { + warn!("Error sending to drain signal: {}.", e); + } + }, + + callback = callbacks.recv() => match callback { + Some(KafkaCallback::ShuttingDown) => { + shutting_down = true; + }, + Some(KafkaCallback::PartitionsAssigned(mut assigned_streams)) => { + for (tp, acks) in assigned_streams.drain(0..) { + ack_streams.insert(tp, acks); + } + }, + Some(KafkaCallback::PartitionsRevoked(revoked_partitions, drain)) => { + drain_deadline.as_mut().reset(Instant::now() + max_drain_ms); + if drain_signal.replace(drain).is_some() { + unreachable!("Concurrent rebalance callbacks should not be possible."); + } + + for tp in revoked_partitions.iter() { + // Move streams for partitions being revoked to the + // draining_acks map to prioritize processing them. + // If the stream has already been drained and + // removed from the StreamMap internally, remove will return None + if let Some(acks) = ack_streams.remove(tp) { + draining_acks.insert(tp.clone(), acks); + } + } + }, + None => {}, + } + } + } + }) +} + +fn handle_messages( + config: KafkaSourceConfig, + consumer: Arc>, + decoder: Decoder, + mut shutdown: ShutdownSignal, + mut out: SourceSender, + log_namespace: LogNamespace, + #[cfg(test)] eof: bool, +) -> JoinHandle<()> { + tokio::task::spawn_blocking(move || { + let mut stream = consumer.stream(); + let handle = Handle::current(); + let mut done: bool = false; + + #[cfg(test)] + let mut eof_partitions = std::collections::HashSet::new(); + + while !done { + handle.block_on(async { + tokio::select! { + _ = &mut shutdown => { + consumer.context().shutdown(); + + done = true + }, + message = stream.next() => match message { + None => unreachable!("MessageStream never returns Ready(None)"), + Some(Err(error)) => match error { + #[cfg(test)] + rdkafka::error::KafkaError::PartitionEOF(partition) if eof => { + // Optionally exit on partition EOF. This is used in tests to collect events + // when the precise number of events is not known (e.g. restarting the source after + // shutdown or a consumer rebalance). Note that the client should set "enable.partition.eof" + // to true, or the client won't get this notification + // Note also: this assumes a single topic is being consumed, since PartitionEOF + // does not include the topic name + // Note also (also): we use a Set to track which partitions have reached EOF; due to + // rebalances we might get notified about an EOF partition more than once + eof_partitions.insert(partition); + if let Ok(assignment) = consumer.assignment() { + if eof_partitions.len() == assignment.count() { + consumer.context().shutdown(); + done = true + } + } + }, + _ => emit!(KafkaReadError { error }), + }, + Some(Ok(msg)) => { + emit!(KafkaBytesReceived { + byte_size: msg.payload_len(), + protocol: "tcp", + topic: msg.topic(), + partition: msg.partition(), + }); + + parse_message(msg, decoder.clone(), config.keys(), &mut out, &consumer, log_namespace).await; + } + }, + } + }) + } + }) +} + async fn parse_message( msg: BorrowedMessage<'_>, decoder: Decoder, keys: Keys<'_>, - finalizer: &Option>>, out: &mut SourceSender, - consumer: &Arc>, + consumer: &Arc>, log_namespace: LogNamespace, ) { + let context = consumer.context(); + if let Some((count, mut stream)) = parse_stream(&msg, decoder, keys, log_namespace) { - match finalizer { - Some(finalizer) => { - let (batch, receiver) = BatchNotifier::new_with_receiver(); - let mut stream = stream.map(|event| event.with_batch_notifier(&batch)); - match out.send_event_stream(&mut stream).await { - Err(error) => { - emit!(StreamClosedError { error, count }); - } - Ok(_) => { - // Drop stream to avoid borrowing `msg`: "[...] borrow might be used - // here, when `stream` is dropped and runs the destructor [...]". - drop(stream); - finalizer.add(msg.into(), receiver); - } + if context.acknowledgements { + let (batch, receiver) = BatchNotifier::new_with_receiver(); + let mut stream = stream.map(|event| event.with_batch_notifier(&batch)); + match out.send_event_stream(&mut stream).await { + Err(error) => { + emit!(StreamClosedError { error, count }); + } + Ok(_) => { + // Drop stream to avoid borrowing `msg`: "[...] borrow might be used + // here, when `stream` is dropped and runs the destructor [...]". + drop(stream); + context.add_finalizer_entry(msg.into(), receiver); } } - None => match out.send_event_stream(&mut stream).await { + } else { + match out.send_event_stream(&mut stream).await { Err(error) => { emit!(StreamClosedError { error, count }); } @@ -494,7 +644,7 @@ async fn parse_message( emit!(KafkaOffsetUpdateError { error }); } } - }, + } } } } @@ -697,7 +847,10 @@ impl<'a> From> for FinalizerEntry { } } -fn create_consumer(config: &KafkaSourceConfig) -> crate::Result> { +fn create_consumer( + config: &KafkaSourceConfig, + acknowledgements: bool, +) -> crate::Result> { let mut client_config = ClientConfig::new(); client_config .set("group.id", &config.group_id) @@ -734,8 +887,9 @@ fn create_consumer(config: &KafkaSourceConfig) -> crate::Result>(CustomContext::new( + .create_with_context::<_, StreamConsumer<_>>(KafkaSourceContext::new( config.metrics.topic_lag_metric, + acknowledgements, )) .context(KafkaCreateSnafu)?; let topics: Vec<&str> = config.topics.iter().map(|s| s.as_str()).collect(); @@ -744,32 +898,167 @@ fn create_consumer(config: &KafkaSourceConfig) -> crate::Result; + +enum KafkaCallback { + PartitionsAssigned(Vec<(TopicPartition, AckStream)>), + PartitionsRevoked(Vec, SyncSender<()>), + ShuttingDown, +} + #[derive(Default)] -struct CustomContext { +struct KafkaSourceContext { + acknowledgements: bool, stats: kafka::KafkaStatisticsContext, - finalizer: OnceCell>>, + /// A callback channel used to coordinate between the main consumer task and the acknowledgement task + callbacks: OnceCell>, + + /// Use a finalizer stream for each partition being consumed, so that when a partition + /// is revoked during a consumer rebalance, acknowledgements can be prioritized (and + /// dropped after a time limit) without affecting acks for retained partitions. + finalizers: RwLock>>, + + /// A weak reference to the consumer, so that we can commit offsets during a rebalance operation + consumer: OnceCell>>, } -impl CustomContext { - fn new(expose_lag_metrics: bool) -> Self { +impl KafkaSourceContext { + fn new(expose_lag_metrics: bool, acknowledgements: bool) -> Self { Self { stats: kafka::KafkaStatisticsContext { expose_lag_metrics }, + acknowledgements, ..Default::default() } } + + pub fn add_finalizer_entry(&self, mut entry: FinalizerEntry, recv: BatchStatusReceiver) { + if let Ok(fin) = self.finalizers.read() { + let key = (entry.topic, entry.partition); + if let Some(entries) = fin.get(&key) { + entry.topic = key.0; // Slightly awkward, but avoids cloning the topic string for every entry added + entries.add(entry, recv); + } + } + } + + pub fn shutdown(&self) { + if let Some(w) = self.consumer.get() { + if let Some(consumer) = w.upgrade() { + if let Ok(topics) = consumer.subscription() { + if let Err(error) = consumer.pause(&topics) { + emit!(KafkaPauseResumeError { error }); + } + } + } + } + + if let Ok(mut fin) = self.finalizers.write() { + fin.clear(); + } + + if let Some(tx) = self.callbacks.get() { + let _ = tx.send(KafkaCallback::ShuttingDown); + } + } + + fn add_finalizerset(&self, key: TopicPartition) -> Option<(TopicPartition, AckStream)> { + if let Ok(fin) = self.finalizers.read() { + if fin.contains_key(&key) { + trace!("Finalizer entry already exists for {}:{}.", key.0, key.1); + return None; + } + } + + let (finalizer, ack_stream) = OrderedFinalizer::::new(None); + if let Ok(mut fin) = self.finalizers.write() { + fin.insert(key.clone(), finalizer); + Some((key, ack_stream)) + } else { + // error getting the RwLock, i.e. the lock is poisoned (!!) + None + } + } + + fn rm_finalizerset(&self, key: &TopicPartition) { + if let Ok(mut fin) = self.finalizers.write() { + fin.remove(key); + } + } + + fn commit_consumer_state(&self) { + if let Some(w) = self.consumer.get() { + if let Some(consumer) = w.upgrade() { + match consumer.commit_consumer_state(CommitMode::Sync) { + Ok(_) => {}, + Err(KafkaError::ConsumerCommit(RDKafkaErrorCode::NoOffset)) => { + trace!("No offsets to commit."); + }, + Err(error) => emit!(KafkaOffsetUpdateError { error }) + } + } + } + } } -impl ClientContext for CustomContext { +impl ClientContext for KafkaSourceContext { fn stats(&self, statistics: Statistics) { self.stats.stats(statistics) } } -impl ConsumerContext for CustomContext { - fn post_rebalance(&self, rebalance: &Rebalance) { - if matches!(rebalance, Rebalance::Revoke(_)) { - if let Some(finalizer) = self.finalizer.get() { - finalizer.flush(); +impl ConsumerContext for KafkaSourceContext { + fn pre_rebalance(&self, rebalance: &Rebalance) { + if let Some(tx) = self.callbacks.get() { + match rebalance { + Rebalance::Assign(tpl) => { + // Partitions are being assigned to this consumer! + // 1. Create a finalizer set for the new partitions + // 2. `self` keeps a reference to the sender (entry point for pending acks) for the finalizer set + // 3. Hand a reference to the receiver stream to the acknowledgement task via callback channel + let ack_streams: Vec<(TopicPartition, AckStream)> = tpl + .elements() + .iter() + .filter_map(|el| self.add_finalizerset((el.topic().into(), el.partition()))) + .collect(); + + if !ack_streams.is_empty() { + let _ = tx.send(KafkaCallback::PartitionsAssigned(ack_streams)); + } + } + Rebalance::Revoke(tpl) => { + // Partitions are being revoked from this consumer! + // 1. Close the sending side for new acknowledgement entries. + // 2. Notify the acknowledgement task, and provide a rendezvous channel; wait for that channel to close + // to indicate when acks for revoked partitions are drained. + // 3. Commit consumer offsets and return, allowing the rebalance to complete + let revoked: Vec = tpl + .elements() + .iter() + .map(|el| { + let key = (el.topic().into(), el.partition()); + self.rm_finalizerset(&key); + key + }) + .collect(); + + if !revoked.is_empty() { + let (send, rendezvous) = sync_channel(0); + match tx.send(KafkaCallback::PartitionsRevoked(revoked, send)) { + // The ack task will signal on this channel when it has drained the + // revoked partitions (or when it times out, to prevent the consumer being kicked from the group) + Ok(_) => while rendezvous.recv().is_ok() { } + // If the ack task has already hung up, that implies we're exiting, + // and acks are already drained (or at least, no more will be processed) + _ => (), + } + + self.commit_consumer_state(); + } + } + Rebalance::Error(message) => { + error!("Error during rebalance: {}.", message); + } } } } @@ -920,7 +1209,7 @@ mod test { #[tokio::test] async fn consumer_create_ok() { let config = make_config("topic", "group", LogNamespace::Legacy); - assert!(create_consumer(&config).is_ok()); + assert!(create_consumer(&config, true).is_ok()); } #[tokio::test] @@ -929,7 +1218,7 @@ mod test { auto_offset_reset: "incorrect-auto-offset-reset".to_string(), ..make_config("topic", "group", LogNamespace::Legacy) }; - assert!(create_consumer(&config).is_err()); + assert!(create_consumer(&config, true).is_err()); } } @@ -1205,7 +1494,7 @@ mod integration_test { log_namespace: LogNamespace, ) -> (Trigger, Tripwire) { let (trigger_shutdown, shutdown, shutdown_done) = ShutdownSignal::new_wired(); - let consumer = create_consumer(&config).unwrap(); + let consumer = create_consumer(&config, acknowledgements).unwrap(); let decoder = DecodingConfig::new( config.framing.clone(), @@ -1444,6 +1733,7 @@ mod integration_test { let config1 = make_config(&topic, &group_id, LogNamespace::Legacy); let config2 = make_config(&topic, &group_id, LogNamespace::Legacy); let config3 = make_config(&topic, &group_id, LogNamespace::Legacy); + let config4 = make_config(&topic, &group_id, LogNamespace::Legacy); let (events1, events2, events3) = tokio::join!( async move { @@ -1471,6 +1761,14 @@ mod integration_test { } ); + let unconsumed = async move { + let (tx, rx) = SourceSender::new_test_errors(|_| false); + let (_trigger_shutdown, _shutdown_done) = + spawn_kafka(tx, config4, true, true, LogNamespace::Legacy); + + rx.collect::>().await + }.await; + debug!( "First consumer read {} of {} messages.", events1.len(), @@ -1505,6 +1803,9 @@ mod integration_test { 0, "Third batch of events is non-zero (decrease delay or increase KAFKA_SEND_COUNT?) " ); + assert_eq!( + unconsumed.len(), 0, "The first set of consumer should consume and ack all messages." + ); assert_eq!(total, expect_count); } From 03aedbd0d7710ae254779dd5ad2e5c89a4068dc9 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 25 May 2023 11:41:37 -0400 Subject: [PATCH 03/37] fix(kafka source): performance improvements for acknowledgement handling on many partitions Instead of tokio StreamMap, which gets very slow when more than a handful of partitions are involved, use a task and forwarding channel for each partition. Introduces a little bookkeeping, but scales well to at least hundreds of partitions --- src/internal_events/kafka.rs | 18 -- src/sources/kafka.rs | 440 +++++++++++++++++++++++++---------- 2 files changed, 311 insertions(+), 147 deletions(-) diff --git a/src/internal_events/kafka.rs b/src/internal_events/kafka.rs index ee4a4889adb6b..92afe66b98a53 100644 --- a/src/internal_events/kafka.rs +++ b/src/internal_events/kafka.rs @@ -111,24 +111,6 @@ impl InternalEvent for KafkaReadError { } } -#[derive(Debug)] -pub struct KafkaPauseResumeError { - pub error: rdkafka::error::KafkaError, -} - -impl InternalEvent for KafkaPauseResumeError { - fn emit(self) { - error!( - message = "Failed to pause or resume consumer.", - error = %self.error, - error_code = "pause_resume_consumer", - error_type = error_type::COMMAND_FAILED, - stage = error_stage::RECEIVING, - internal_log_rate_limit = true, - ) - } -} - #[derive(Debug)] pub struct KafkaStatisticsReceived<'a> { pub statistics: &'a rdkafka::Statistics, diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index ba71b1f03c797..29d1865c908df 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -1,5 +1,5 @@ use std::{ - collections::{BTreeMap, HashMap}, + collections::{BTreeMap, HashMap, HashSet}, io::Cursor, sync::{ mpsc::{sync_channel, SyncSender}, @@ -20,18 +20,18 @@ use lookup::{lookup_v2::OptionalValuePath, owned_value_path, path, OwnedValuePat use once_cell::sync::OnceCell; use rdkafka::{ consumer::{CommitMode, Consumer, ConsumerContext, Rebalance, StreamConsumer}, + error::KafkaError, message::{BorrowedMessage, Headers as _, Message}, - ClientConfig, ClientContext, Statistics, types::RDKafkaErrorCode, error::KafkaError, + types::RDKafkaErrorCode, + ClientConfig, ClientContext, Statistics, }; use serde_with::serde_as; use snafu::{ResultExt, Snafu}; use tokio::{ runtime::Handle, - sync::mpsc::{self, UnboundedReceiver, UnboundedSender}, + sync::mpsc::{self, Receiver, Sender, UnboundedReceiver, UnboundedSender}, task::JoinHandle, - time::Instant, }; -use tokio_stream::StreamMap; use tokio_util::codec::FramedRead; use vector_common::{finalization::BatchStatusReceiver, finalizer::OrderedFinalizer}; @@ -50,8 +50,8 @@ use crate::{ }, event::{BatchNotifier, BatchStatus, Event, Value}, internal_events::{ - KafkaBytesReceived, KafkaEventsReceived, KafkaOffsetUpdateError, KafkaPauseResumeError, - KafkaReadError, StreamClosedError, + KafkaBytesReceived, KafkaEventsReceived, KafkaOffsetUpdateError, KafkaReadError, + StreamClosedError, }, kafka, serde::{bool_or_struct, default_decoding, default_framing_message_based}, @@ -428,18 +428,12 @@ async fn kafka_source( if let Some(ack_task) = ack_task { _ = tokio::join!(msg_task, ack_task); - // _ = tokio::join!(ack_task); } else { _ = tokio::join!(msg_task); } - // Since commits are async internally, we try one last sync commit inside the interval - // in case there have been acks. - if let Ok(current_assignment) = consumer.assignment() { - // not logging on error because it will error if there are no offsets stored for a partition, - // and this is best-effort cleanup anyway - _ = consumer.commit(¤t_assignment, CommitMode::Sync); - } + consumer.context().commit_consumer_state(); + Ok(()) } @@ -462,80 +456,233 @@ fn handle_acks( } } - tokio::spawn(async move { - let mut ack_streams: StreamMap = StreamMap::new(); - let mut draining_acks: StreamMap = StreamMap::new(); + async fn revoke_timeout(t: Duration) { + tokio::time::sleep(t).await; + } - let mut shutting_down = false; - let drain_deadline = tokio::time::sleep(max_drain_ms); - tokio::pin!(drain_deadline); + // Wrap acks for each partition in this enum, so that we can let the receiver + // know when it has seen the last one for the current assignment of this partition + enum ForwardedAck { + Entry(BatchStatus, FinalizerEntry), + Drained(TopicPartition, i64), + } - loop { - let has_acks = !draining_acks.is_empty() || !ack_streams.is_empty(); - let next_ack = if drain_signal.is_some() { - draining_acks.next() - } else { - ack_streams.next() + struct KafkaPartitionState { + /// The sender for ack forwarding tasks to use + ack_tx: Option>, + + /// Tasks forwarding acknowledgement entries for each partition to a main channel. + /// There will be one task per assigned partition, and this allows us, during rebalances, + /// to precisely manage when acks for any revoked partitions are complete + ack_forwarders: tokio::task::JoinSet, + + /// Abort handles for each forwarding task, indexed by the (topic, partition) pair. This + /// allows for precise task cancellation when a partition is revoked but acks can't be processed + /// before a timeout; only pending acks for revoked partitions will be cancelled/dropped. + abort_handles: HashMap, + + /// The Set of partitions expected to drain during a shutdown or rebalance that revokes partitions + expect_drain: HashSet, + + /// The set of partitions we have observed and stored the final acknowledgement for. Ack streams + /// can complete before we get a rebalance callback, so "observed complete" (based on seeing the end of the stream) + /// and "expect to complete" (based on seeing a rebalance callback with revoked partition info) are tracked separately. + observed_drain: HashSet, + } + impl KafkaPartitionState { + fn new() -> (Self, Receiver) { + let (ack_tx, all_acks) = mpsc::channel(16); // arbitrary size 16 + let state = KafkaPartitionState { + ack_tx: Some(ack_tx), + ack_forwarders: tokio::task::JoinSet::new(), + abort_handles: HashMap::new(), + expect_drain: HashSet::new(), + observed_drain: HashSet::new(), }; + (state, all_acks) + } + pub fn assign_partition(&mut self, tp: TopicPartition, acks: AckStream) { + if let Some(ref ack_tx) = self.ack_tx { + self.abort_handles.insert( + tp.clone(), + self.ack_forwarders + .spawn(forward_acks(tp, acks, ack_tx.clone())), + ); + } + } + pub fn revoke_partition(&mut self, tp: TopicPartition) { + self.expect_drain.insert(tp); + } + + pub fn forwarder_complete(&mut self, tp: &TopicPartition) { + self.abort_handles.remove(tp); + } + pub fn has_forwarders(&self) -> bool { + !self.ack_forwarders.is_empty() + } + pub fn abort_pending_forwarders(&mut self) { + for tp in self.expect_drain.drain() { + // If the handle isn't here anymore (None case) it just means the task already completed + if let Some(handle) = self.abort_handles.remove(&tp) { + handle.abort(); + } + } + } + pub fn observed_last_ack(&mut self, tp: TopicPartition) { + self.observed_drain.insert(tp); + } + + pub fn is_drain_complete(&self) -> bool { + self.observed_drain == self.expect_drain + } + + pub fn clear(&mut self) { + self.expect_drain.clear(); + self.observed_drain.clear(); + } + + pub fn close(&mut self) { + let _ = self.ack_tx.take(); + } + } + + async fn forward_acks( + tp: TopicPartition, + mut acks: AckStream, + forward_to: mpsc::Sender, + ) -> TopicPartition { + let mut last_offset = -1; + while let Some((status, entry)) = acks.next().await { + last_offset = entry.offset; + if let Err(e) = forward_to.send(ForwardedAck::Entry(status, entry)).await { + warn!("Error sending to main ack task: {}", e); + } + } + let _ = forward_to + .send(ForwardedAck::Drained(tp.clone(), last_offset)) + .await; + tp + } + + tokio::spawn(async move { + /* + Ok how does this work and where are the nuances in here? + We have: + - the consumer task: a task talking to Kafka, reading messages and getting rebalance notifications, etc. + - Finalizer entries are added to partition-specific channels from this task + - During rebalances, send notifications about assigned/revoked partitions + - When a partition is revoked, or the client is shutting down, close the + finalizer channels so they can be drained by the ack task. Wait on a channel coordinating with + the acknowledgement task to avoid proceeding with rebalance before pending offsets are written + Nuance: the kafka task runs blocking code in the rebalance callbacks, so must live + on a separate thread from acknowledgement handling, otherwise everything deadlocks + + - the acknowledgement task: + - Nuance: rebalancing may revoke a subset of our partitions, depending on the strategy-- to handle this correctly + we use a finalizer stream per partition + - When a partition is assigned, spawn a task dedicated to reading the finalizer channel. Tasks and channels + perform significantly better than a tokio StreamMap when dealing with more than a few partitions. + - As finalizers become ready, forward them to the main ack task to be stored. + - When a finalizer channel closes, the final message in the forwarding channel + is a marker indicating the partition is drained. Nuance: acks for the partition + are considered drained when this marker is _processed by the main ack task_, not + when the forwarding task ends + Additional nuance: finalizer channels can end before we even get a rebalance notification! + - During a rebalance, we track expected and observed drain markers, as well as a timeout. + As soon as the expected partition streams are drained, or the timeout is reached, signal back + to the consumer task to proceed with rebalancing. + In the case of a timeout, drop any remaining acks on revoked partitions. + - Rebalance bookkeeping is done by the ForwardedAck enum and KafkaPartitionState struct. + + */ + + let mut drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); + let (mut partition_state, mut all_acks) = KafkaPartitionState::new(); + loop { tokio::select! { - entry = next_ack, if has_acks || drain_signal.is_some() => match entry { - Some((_key, (delivery_status, entry))) => { - handle_ack(&consumer, delivery_status, entry); - if let Some(ref signal) = drain_signal { - _ = signal.send(()); + Some(callback) = callbacks.recv() => match callback { + KafkaCallback::PartitionsAssigned(mut assigned_streams) => { + for (tp, acks) in assigned_streams.drain(0..) { + partition_state.assign_partition(tp, acks); } }, - // Nothing left in the acknowledgement stream; check for exit conditions - None => match drain_signal.take() { - Some(signal) => { - if let Err(e) = signal.send(()) { - warn!("Error signaling client task: {:?}", e); - } - }, - None if shutting_down => { - break - }, - None => { - // ack stream is empty, but we're not shutting down or rebalancing (probably starting up, or just finished a rebalance) + KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => { + drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); + + for tp in revoked_partitions.drain(0..) { + partition_state.revoke_partition(tp); } - } + + if partition_state.is_drain_complete() { + partition_state.clear(); + drop(drain); + } else if drain_signal.replace(drain).is_some() { + unreachable!("Concurrent rebalance callbacks should not be possible."); + } + }, + KafkaCallback::ShuttingDown(drain) => { + // Shutting down is just like a full assignment revoke, but we also close the ack senders and callback + // channels, since we don't expect additional assignments or rebalances + if let Ok(tpl) = consumer.assignment() { + tpl.elements() + .iter() + .for_each(|el| { + partition_state.revoke_partition((el.topic().into(), el.partition())); + }); + } + + drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); + if partition_state.is_drain_complete() { + partition_state.clear(); + drop(drain); + } else if drain_signal.replace(drain).is_some() { + unreachable!("Shutdown callback happened somehow during an ongoing rebalance...?") + } + + // No new partitions will be assigned, so drop our handle to the ack sender and close the callback channel + partition_state.close(); + callbacks.close(); + }, + }, + + // As partition-specific sender tasks complete (revoked partitions + // during a rebalance, shutdown, or eof during tests), handle the results here + Some(Ok(finished_partition)) = partition_state.ack_forwarders.join_next(), if partition_state.has_forwarders() => { + partition_state.forwarder_complete(&finished_partition); }, _ = &mut drain_deadline, if drain_signal.is_some() => { debug!("Acknowledgement drain deadline reached. Dropping any pending ack streams for revoked partitions."); - draining_acks.clear(); + partition_state.abort_pending_forwarders(); + partition_state.clear(); + if let Err(e) = drain_signal.take().unwrap().send(()) { warn!("Error sending to drain signal: {}.", e); } }, - callback = callbacks.recv() => match callback { - Some(KafkaCallback::ShuttingDown) => { - shutting_down = true; - }, - Some(KafkaCallback::PartitionsAssigned(mut assigned_streams)) => { - for (tp, acks) in assigned_streams.drain(0..) { - ack_streams.insert(tp, acks); - } - }, - Some(KafkaCallback::PartitionsRevoked(revoked_partitions, drain)) => { - drain_deadline.as_mut().reset(Instant::now() + max_drain_ms); - if drain_signal.replace(drain).is_some() { - unreachable!("Concurrent rebalance callbacks should not be possible."); + Some(entry) = all_acks.recv() => match entry { + ForwardedAck::Drained(tp, _offset) => { + partition_state.observed_last_ack(tp); + + if drain_signal.is_some() { + _ = drain_signal.as_ref().map(|sig| _ = sig.send(()) ); } - for tp in revoked_partitions.iter() { - // Move streams for partitions being revoked to the - // draining_acks map to prioritize processing them. - // If the stream has already been drained and - // removed from the StreamMap internally, remove will return None - if let Some(acks) = ack_streams.remove(tp) { - draining_acks.insert(tp.clone(), acks); - } + if partition_state.is_drain_complete() { + partition_state.clear(); + drain_signal.take(); } }, - None => {}, + ForwardedAck::Entry(delivery_status, entry) => { + handle_ack(&consumer, delivery_status, entry); + }, + }, + + // acks and callbacks are all done + else => { + break } } } @@ -564,7 +711,6 @@ fn handle_messages( tokio::select! { _ = &mut shutdown => { consumer.context().shutdown(); - done = true }, message = stream.next() => match message { @@ -572,19 +718,19 @@ fn handle_messages( Some(Err(error)) => match error { #[cfg(test)] rdkafka::error::KafkaError::PartitionEOF(partition) if eof => { - // Optionally exit on partition EOF. This is used in tests to collect events - // when the precise number of events is not known (e.g. restarting the source after - // shutdown or a consumer rebalance). Note that the client should set "enable.partition.eof" - // to true, or the client won't get this notification - // Note also: this assumes a single topic is being consumed, since PartitionEOF - // does not include the topic name - // Note also (also): we use a Set to track which partitions have reached EOF; due to - // rebalances we might get notified about an EOF partition more than once + // NB this is not production ready EOF detection! Hence cfg(test) on this branch + // Used only in tests when we can be certain only one topic is being consumed, + // and new messages are not written after EOF is seen + // Also: RdKafka only tells us the partition, so + // we are assuming single-topic consumers when using this. + // Also also: due to rebalances, we might get notified about an EOF partition + // more than once, so we use a Set to exit once we've seen EOF on all currently-assigned partitions eof_partitions.insert(partition); if let Ok(assignment) = consumer.assignment() { - if eof_partitions.len() == assignment.count() { + // All currently assigned partitions have reached EOF + if assignment.elements().iter().all(|tp| eof_partitions.contains(&tp.partition())) { consumer.context().shutdown(); - done = true + done = true; } } }, @@ -830,7 +976,7 @@ impl ReceivedMessage { } } -#[derive(Debug)] +#[derive(Debug, Eq, PartialEq, Hash)] struct FinalizerEntry { topic: String, partition: i32, @@ -904,13 +1050,14 @@ type AckStream = BoxStream<'static, (BatchStatus, FinalizerEntry)>; enum KafkaCallback { PartitionsAssigned(Vec<(TopicPartition, AckStream)>), PartitionsRevoked(Vec, SyncSender<()>), - ShuttingDown, + ShuttingDown(SyncSender<()>), } #[derive(Default)] struct KafkaSourceContext { acknowledgements: bool, stats: kafka::KafkaStatisticsContext, + /// A callback channel used to coordinate between the main consumer task and the acknowledgement task callbacks: OnceCell>, @@ -943,22 +1090,17 @@ impl KafkaSourceContext { } pub fn shutdown(&self) { - if let Some(w) = self.consumer.get() { - if let Some(consumer) = w.upgrade() { - if let Ok(topics) = consumer.subscription() { - if let Err(error) = consumer.pause(&topics) { - emit!(KafkaPauseResumeError { error }); - } - } - } - } - if let Ok(mut fin) = self.finalizers.write() { fin.clear(); } if let Some(tx) = self.callbacks.get() { - let _ = tx.send(KafkaCallback::ShuttingDown); + let (send, rendezvous) = sync_channel(0); + if let Ok(_) = tx.send(KafkaCallback::ShuttingDown(send)) { + while rendezvous.recv().is_ok() { + self.commit_consumer_state(); + } + } } } @@ -990,11 +1132,10 @@ impl KafkaSourceContext { if let Some(w) = self.consumer.get() { if let Some(consumer) = w.upgrade() { match consumer.commit_consumer_state(CommitMode::Sync) { - Ok(_) => {}, - Err(KafkaError::ConsumerCommit(RDKafkaErrorCode::NoOffset)) => { - trace!("No offsets to commit."); - }, - Err(error) => emit!(KafkaOffsetUpdateError { error }) + Ok(_) | Err(KafkaError::ConsumerCommit(RDKafkaErrorCode::NoOffset)) => { + /* Success, or nothing to do \0/ */ + } + Err(error) => emit!(KafkaOffsetUpdateError { error }), } } } @@ -1014,7 +1155,7 @@ impl ConsumerContext for KafkaSourceContext { Rebalance::Assign(tpl) => { // Partitions are being assigned to this consumer! // 1. Create a finalizer set for the new partitions - // 2. `self` keeps a reference to the sender (entry point for pending acks) for the finalizer set + // 2. `self` keeps a reference to the sender - this is the entry point for pending acks on this partition // 3. Hand a reference to the receiver stream to the acknowledgement task via callback channel let ack_streams: Vec<(TopicPartition, AckStream)> = tpl .elements() @@ -1022,6 +1163,7 @@ impl ConsumerContext for KafkaSourceContext { .filter_map(|el| self.add_finalizerset((el.topic().into(), el.partition()))) .collect(); + trace!("Partition(s) assigned: {}", ack_streams.len()); if !ack_streams.is_empty() { let _ = tx.send(KafkaCallback::PartitionsAssigned(ack_streams)); } @@ -1042,12 +1184,18 @@ impl ConsumerContext for KafkaSourceContext { }) .collect(); + trace!("Partition(s) revoked: {}", revoked.len()); if !revoked.is_empty() { let (send, rendezvous) = sync_channel(0); match tx.send(KafkaCallback::PartitionsRevoked(revoked, send)) { - // The ack task will signal on this channel when it has drained the - // revoked partitions (or when it times out, to prevent the consumer being kicked from the group) - Ok(_) => while rendezvous.recv().is_ok() { } + // The ack task will signal on this channel when it has drained a revoked partition + // and will close the channel when it has drained all revoked partitions, + // or when it times out, to prevent the consumer being kicked from the group. + Ok(_) => { + while rendezvous.recv().is_ok() { + self.commit_consumer_state(); + } + } // If the ack task has already hung up, that implies we're exiting, // and acks are already drained (or at least, no more will be processed) _ => (), @@ -1083,7 +1231,7 @@ mod test { format!("{}:{}", kafka_host(), kafka_port()) } - fn kafka_max_bytes() -> String { + pub fn kafka_max_bytes() -> String { std::env::var("KAFKA_MAX_BYTES").unwrap_or_else(|_| "1024".into()) } @@ -1096,12 +1244,8 @@ mod test { topic: &str, group: &str, log_namespace: LogNamespace, + librdkafka_options: Option>, ) -> KafkaSourceConfig { - let mut kafka_options = HashMap::new(); - kafka_options.insert("enable.partition.eof".into(), "true".into()); - // Fetch in small batches - kafka_options.insert("fetch.message.max.bytes".into(), kafka_max_bytes().into()); - KafkaSourceConfig { bootstrap_servers: kafka_address(), topics: vec![topic.into()], @@ -1109,7 +1253,7 @@ mod test { auto_offset_reset: "beginning".into(), session_timeout_ms: Duration::from_millis(6000), commit_interval_ms: Duration::from_millis(1), - librdkafka_options: Some(kafka_options), + librdkafka_options: librdkafka_options, key_field: default_key_field(), topic_key: default_topic_key(), partition_key: default_partition_key(), @@ -1124,7 +1268,7 @@ mod test { #[test] fn test_output_schema_definition_vector_namespace() { - let definitions = make_config("topic", "group", LogNamespace::Vector) + let definitions = make_config("topic", "group", LogNamespace::Vector, None) .outputs(LogNamespace::Vector) .remove(0) .schema_definition(true); @@ -1172,7 +1316,7 @@ mod test { #[test] fn test_output_schema_definition_legacy_namespace() { - let definitions = make_config("topic", "group", LogNamespace::Legacy) + let definitions = make_config("topic", "group", LogNamespace::Legacy, None) .outputs(LogNamespace::Legacy) .remove(0) .schema_definition(true); @@ -1208,7 +1352,7 @@ mod test { #[tokio::test] async fn consumer_create_ok() { - let config = make_config("topic", "group", LogNamespace::Legacy); + let config = make_config("topic", "group", LogNamespace::Legacy, None); assert!(create_consumer(&config, true).is_ok()); } @@ -1216,7 +1360,7 @@ mod test { async fn consumer_create_incorrect_auto_offset_reset() { let config = KafkaSourceConfig { auto_offset_reset: "incorrect-auto-offset-reset".to_string(), - ..make_config("topic", "group", LogNamespace::Legacy) + ..make_config("topic", "group", LogNamespace::Legacy, None) }; assert!(create_consumer(&config, true).is_err()); } @@ -1368,7 +1512,7 @@ mod integration_test { let topic = format!("test-topic-{}", random_string(10)); let group_id = format!("test-group-{}", random_string(10)); - let config = make_config(&topic, &group_id, log_namespace); + let config = make_config(&topic, &group_id, log_namespace, None); let now = send_events(topic.clone(), 1, 10).await; @@ -1460,7 +1604,7 @@ mod integration_test { fn make_rand_config() -> (String, String, KafkaSourceConfig) { let topic = format!("test-topic-{}", random_string(10)); let group_id = format!("test-group-{}", random_string(10)); - let config = make_config(&topic, &group_id, LogNamespace::Legacy); + let config = make_config(&topic, &group_id, LogNamespace::Legacy, None); (topic, group_id, config) } @@ -1660,8 +1804,12 @@ mod integration_test { // 2. Run the kafka source to read some of the events // 3. Send a shutdown signal (at some point before all events are read) + let mut opts = HashMap::new(); + // Set options to get partition EOF notifications, and fetch data in small/configurable size chunks + opts.insert("enable.partition.eof".into(), "true".into()); + opts.insert("fetch.message.max.bytes".into(), kafka_max_bytes().into()); let events1 = { - let config = make_config(&topic, &group_id, LogNamespace::Legacy); + let config = make_config(&topic, &group_id, LogNamespace::Legacy, Some(opts.clone())); let (tx, rx) = SourceSender::new_test_errors(|_| false); let (trigger_shutdown, shutdown_done) = spawn_kafka(tx, config, true, false, LogNamespace::Legacy); @@ -1682,7 +1830,7 @@ mod integration_test { // 4. Run the kafka source again to finish reading the events let events2 = { - let config = make_config(&topic, &group_id, LogNamespace::Legacy); + let config = make_config(&topic, &group_id, LogNamespace::Legacy, Some(opts)); let (tx, rx) = SourceSender::new_test_errors(|_| false); let (trigger_shutdown, shutdown_done) = spawn_kafka(tx, config, true, true, LogNamespace::Legacy); @@ -1709,8 +1857,7 @@ mod integration_test { assert_eq!(total, expect_count); } - #[tokio::test] - async fn drains_acknowledgements_during_rebalance() { + async fn consume_with_rebalance(rebalance_strategy: String) { // 1. Send N events (if running against a pre-populated kafka topic, use send_count=0 and expect_count=expected number of messages; otherwise just set send_count) let send_count: usize = std::env::var("KAFKA_SEND_COUNT") .unwrap_or_else(|_| "100".into()) @@ -1726,14 +1873,34 @@ mod integration_test { .expect("Number of milliseconds before shutting down first consumer."); let (topic, group_id, _) = send_to_test_topic(2, send_count).await; - debug!("Consumer group.id: {}", &group_id); + println!("Topic: {}", &topic); + println!("Consumer group.id: {}", &group_id); // 2. Run the kafka source to read some of the events // 3. Start 2nd & 3rd consumers using the same group.id, triggering rebalance events - let config1 = make_config(&topic, &group_id, LogNamespace::Legacy); - let config2 = make_config(&topic, &group_id, LogNamespace::Legacy); - let config3 = make_config(&topic, &group_id, LogNamespace::Legacy); - let config4 = make_config(&topic, &group_id, LogNamespace::Legacy); + let mut kafka_options = HashMap::new(); + kafka_options.insert("enable.partition.eof".into(), "true".into()); + kafka_options.insert("fetch.message.max.bytes".into(), kafka_max_bytes().into()); + kafka_options.insert("partition.assignment.strategy".into(), rebalance_strategy); + let config1 = make_config( + &topic, + &group_id, + LogNamespace::Legacy, + Some(kafka_options.clone()), + ); + let config2 = make_config( + &topic, + &group_id, + LogNamespace::Legacy, + Some(kafka_options.clone()), + ); + let config3 = make_config( + &topic, + &group_id, + LogNamespace::Legacy, + Some(kafka_options.clone()), + ); + let config4 = make_config(&topic, &group_id, LogNamespace::Legacy, Some(kafka_options)); let (events1, events2, events3) = tokio::join!( async move { @@ -1762,12 +1929,13 @@ mod integration_test { ); let unconsumed = async move { - let (tx, rx) = SourceSender::new_test_errors(|_| false); - let (_trigger_shutdown, _shutdown_done) = - spawn_kafka(tx, config4, true, true, LogNamespace::Legacy); + let (tx, rx) = SourceSender::new_test_errors(|_| false); + let (_trigger_shutdown, _shutdown_done) = + spawn_kafka(tx, config4, true, true, LogNamespace::Legacy); - rx.collect::>().await - }.await; + rx.collect::>().await + } + .await; debug!( "First consumer read {} of {} messages.", @@ -1804,11 +1972,25 @@ mod integration_test { "Third batch of events is non-zero (decrease delay or increase KAFKA_SEND_COUNT?) " ); assert_eq!( - unconsumed.len(), 0, "The first set of consumer should consume and ack all messages." + unconsumed.len(), + 0, + "The first set of consumer should consume and ack all messages." ); assert_eq!(total, expect_count); } + #[tokio::test] + async fn drains_acknowledgements_during_rebalance_default_assignments() { + // the default, eager rebalance strategies generally result in more revocations + consume_with_rebalance("range,roundrobin".into()).await; + } + #[tokio::test] + async fn drains_acknowledgements_during_rebalance_sticky_assignments() { + // Cooperative rebalance strategies generally result in fewer revokes, + // as only reassigned partitions are revoked + consume_with_rebalance("cooperative-sticky".into()).await; + } + fn map_logs(events: EventArray) -> impl Iterator { events.into_events().map(|event| { let log = event.into_log(); From 01914976da6eff0cf33afae22921ab9cca534926 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 25 May 2023 18:18:44 -0400 Subject: [PATCH 04/37] clippy fixes, and remove unnecessary last_offset tracking --- src/sources/kafka.rs | 29 ++++++++++++----------------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 29d1865c908df..ef4b34617f949 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -464,7 +464,7 @@ fn handle_acks( // know when it has seen the last one for the current assignment of this partition enum ForwardedAck { Entry(BatchStatus, FinalizerEntry), - Drained(TopicPartition, i64), + Drained(TopicPartition), } struct KafkaPartitionState { @@ -551,15 +551,13 @@ fn handle_acks( mut acks: AckStream, forward_to: mpsc::Sender, ) -> TopicPartition { - let mut last_offset = -1; while let Some((status, entry)) = acks.next().await { - last_offset = entry.offset; if let Err(e) = forward_to.send(ForwardedAck::Entry(status, entry)).await { warn!("Error sending to main ack task: {}", e); } } let _ = forward_to - .send(ForwardedAck::Drained(tp.clone(), last_offset)) + .send(ForwardedAck::Drained(tp.clone())) .await; tp } @@ -663,7 +661,7 @@ fn handle_acks( }, Some(entry) = all_acks.recv() => match entry { - ForwardedAck::Drained(tp, _offset) => { + ForwardedAck::Drained(tp) => { partition_state.observed_last_ack(tp); if drain_signal.is_some() { @@ -1096,7 +1094,7 @@ impl KafkaSourceContext { if let Some(tx) = self.callbacks.get() { let (send, rendezvous) = sync_channel(0); - if let Ok(_) = tx.send(KafkaCallback::ShuttingDown(send)) { + if tx.send(KafkaCallback::ShuttingDown(send)).is_ok() { while rendezvous.recv().is_ok() { self.commit_consumer_state(); } @@ -1187,18 +1185,15 @@ impl ConsumerContext for KafkaSourceContext { trace!("Partition(s) revoked: {}", revoked.len()); if !revoked.is_empty() { let (send, rendezvous) = sync_channel(0); - match tx.send(KafkaCallback::PartitionsRevoked(revoked, send)) { - // The ack task will signal on this channel when it has drained a revoked partition - // and will close the channel when it has drained all revoked partitions, - // or when it times out, to prevent the consumer being kicked from the group. - Ok(_) => { - while rendezvous.recv().is_ok() { - self.commit_consumer_state(); - } + // The ack task will signal on this channel when it has drained a revoked partition + // and will close the channel when it has drained all revoked partitions, + // or when it times out, to prevent the consumer being kicked from the group. + // This send will return Err if the ack task has already exited; in that case we + // proceed without waiting + if tx.send(KafkaCallback::PartitionsRevoked(revoked, send)).is_ok() { + while rendezvous.recv().is_ok() { + self.commit_consumer_state(); } - // If the ack task has already hung up, that implies we're exiting, - // and acks are already drained (or at least, no more will be processed) - _ => (), } self.commit_consumer_state(); From bf4a5c31a75c5a24e8ed5300d83123a7dd8c0a89 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 25 May 2023 20:33:28 -0400 Subject: [PATCH 05/37] cargo fmt again --- src/sources/kafka.rs | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index ef4b34617f949..1ff7d39103406 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -556,9 +556,7 @@ fn handle_acks( warn!("Error sending to main ack task: {}", e); } } - let _ = forward_to - .send(ForwardedAck::Drained(tp.clone())) - .await; + let _ = forward_to.send(ForwardedAck::Drained(tp.clone())).await; tp } @@ -1190,7 +1188,10 @@ impl ConsumerContext for KafkaSourceContext { // or when it times out, to prevent the consumer being kicked from the group. // This send will return Err if the ack task has already exited; in that case we // proceed without waiting - if tx.send(KafkaCallback::PartitionsRevoked(revoked, send)).is_ok() { + if tx + .send(KafkaCallback::PartitionsRevoked(revoked, send)) + .is_ok() + { while rendezvous.recv().is_ok() { self.commit_consumer_state(); } @@ -1226,10 +1227,6 @@ mod test { format!("{}:{}", kafka_host(), kafka_port()) } - pub fn kafka_max_bytes() -> String { - std::env::var("KAFKA_MAX_BYTES").unwrap_or_else(|_| "1024".into()) - } - #[test] fn generate_config() { crate::test_util::test_generate_config::(); @@ -1401,6 +1398,10 @@ mod integration_test { std::env::var("KAFKA_TEST_TOPIC") .unwrap_or_else(|_| format!("test-topic-{}", random_string(10)).into()) } + fn kafka_max_bytes() -> String { + std::env::var("KAFKA_MAX_BYTES").unwrap_or_else(|_| "1024".into()) + } + fn client_config(group: Option<&str>) -> T { let mut client = ClientConfig::new(); From 9d32a820a9cc96054bf22bb3b51d650476ae1aec Mon Sep 17 00:00:00 2001 From: John Chesley Date: Fri, 26 May 2023 12:20:57 -0400 Subject: [PATCH 06/37] fmt --- src/sources/kafka.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 1ff7d39103406..d03c83458d866 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -1402,7 +1402,6 @@ mod integration_test { std::env::var("KAFKA_MAX_BYTES").unwrap_or_else(|_| "1024".into()) } - fn client_config(group: Option<&str>) -> T { let mut client = ClientConfig::new(); client.set("bootstrap.servers", kafka_address()); From 3582940c9d3b78e6aef653f3dba38c2a81f2dd15 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 29 Jun 2023 10:35:29 -0400 Subject: [PATCH 07/37] clean up handle_messages loop and add a tracing span for metrics collection --- src/sources/kafka.rs | 119 ++++++++++++++++++++++--------------------- 1 file changed, 60 insertions(+), 59 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index d03c83458d866..f5ebce02b433f 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -415,16 +415,21 @@ async fn kafka_source( )); } - let msg_task = handle_messages( - config, - Arc::clone(&consumer), - decoder, - shutdown, - out, - log_namespace, - #[cfg(test)] - eof, - ); + let msg_consumer = Arc::clone(&consumer); + let span = info_span!("kafka_source"); + let msg_task = tokio::task::spawn_blocking(move || { + let _enter = span.enter(); + handle_messages( + config, + msg_consumer, + decoder, + shutdown, + out, + log_namespace, + #[cfg(test)] + eof, + ); + }); if let Some(ack_task) = ack_task { _ = tokio::join!(msg_task, ack_task); @@ -693,60 +698,56 @@ fn handle_messages( mut out: SourceSender, log_namespace: LogNamespace, #[cfg(test)] eof: bool, -) -> JoinHandle<()> { - tokio::task::spawn_blocking(move || { +) -> () { + #[cfg(test)] + let mut eof_partitions = std::collections::HashSet::new(); + + Handle::current().block_on(async move { let mut stream = consumer.stream(); - let handle = Handle::current(); - let mut done: bool = false; - - #[cfg(test)] - let mut eof_partitions = std::collections::HashSet::new(); - - while !done { - handle.block_on(async { - tokio::select! { - _ = &mut shutdown => { - consumer.context().shutdown(); - done = true - }, - message = stream.next() => match message { - None => unreachable!("MessageStream never returns Ready(None)"), - Some(Err(error)) => match error { - #[cfg(test)] - rdkafka::error::KafkaError::PartitionEOF(partition) if eof => { - // NB this is not production ready EOF detection! Hence cfg(test) on this branch - // Used only in tests when we can be certain only one topic is being consumed, - // and new messages are not written after EOF is seen - // Also: RdKafka only tells us the partition, so - // we are assuming single-topic consumers when using this. - // Also also: due to rebalances, we might get notified about an EOF partition - // more than once, so we use a Set to exit once we've seen EOF on all currently-assigned partitions - eof_partitions.insert(partition); - if let Ok(assignment) = consumer.assignment() { - // All currently assigned partitions have reached EOF - if assignment.elements().iter().all(|tp| eof_partitions.contains(&tp.partition())) { - consumer.context().shutdown(); - done = true; - } + loop { + tokio::select! { + _ = &mut shutdown => { + consumer.context().shutdown(); + break + }, + + message = stream.next() => match message { + None => unreachable!("MessageStream never returns Ready(None)"), + Some(Err(error)) => match error { + #[cfg(test)] + rdkafka::error::KafkaError::PartitionEOF(partition) if eof => { + // NB this is not production ready EOF detection! Hence cfg(test) on this branch + // Used only in tests when we can be certain only one topic is being consumed, + // and new messages are not written after EOF is seen + // Also: RdKafka only tells us the partition, so + // we are assuming single-topic consumers when using this. + // Also also: due to rebalances, we might get notified about an EOF partition + // more than once, so we use a Set to exit once we've seen EOF on all currently-assigned partitions + eof_partitions.insert(partition); + if let Ok(assignment) = consumer.assignment() { + // All currently assigned partitions have reached EOF + if assignment.elements().iter().all(|tp| eof_partitions.contains(&tp.partition())) { + consumer.context().shutdown(); + break } - }, - _ => emit!(KafkaReadError { error }), + } }, - Some(Ok(msg)) => { - emit!(KafkaBytesReceived { - byte_size: msg.payload_len(), - protocol: "tcp", - topic: msg.topic(), - partition: msg.partition(), - }); - - parse_message(msg, decoder.clone(), config.keys(), &mut out, &consumer, log_namespace).await; - } + _ => emit!(KafkaReadError { error }), }, - } - }) + Some(Ok(msg)) => { + emit!(KafkaBytesReceived { + byte_size: msg.payload_len(), + protocol: "tcp", + topic: msg.topic(), + partition: msg.partition(), + }); + + parse_message(msg, decoder.clone(), config.keys(), &mut out, &consumer, log_namespace).await; + } + }, + } } - }) + }); } async fn parse_message( From c21e1742ace9204a02812ae33c56d12fbb74b7f3 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 29 Jun 2023 11:36:56 -0400 Subject: [PATCH 08/37] fixup changes lost after merging master --- src/sources/kafka.rs | 29 ++++++++++++++--------------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 9cc913df33417..34ea0aa4e5ddf 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -765,23 +765,22 @@ async fn parse_message( let context = consumer.context(); if let Some((count, mut stream)) = parse_stream(&msg, decoder, keys, log_namespace) { - match finalizer { - Some(finalizer) => { - let (batch, receiver) = BatchNotifier::new_with_receiver(); - let mut stream = stream.map(|event| event.with_batch_notifier(&batch)); - match out.send_event_stream(&mut stream).await { - Err(error) => { - emit!(StreamClosedError { count }); - } - Ok(_) => { - // Drop stream to avoid borrowing `msg`: "[...] borrow might be used - // here, when `stream` is dropped and runs the destructor [...]". - drop(stream); - finalizer.add(msg.into(), receiver); - } + if context.acknowledgements { + let (batch, receiver) = BatchNotifier::new_with_receiver(); + let mut stream = stream.map(|event| event.with_batch_notifier(&batch)); + match out.send_event_stream(&mut stream).await { + Err(_) => { + emit!(StreamClosedError { count }); + } + Ok(_) => { + // Drop stream to avoid borrowing `msg`: "[...] borrow might be used + // here, when `stream` is dropped and runs the destructor [...]". + drop(stream); + context.add_finalizer_entry(msg.into(), receiver); } } - None => match out.send_event_stream(&mut stream).await { + } else { + match out.send_event_stream(&mut stream).await { Err(_) => { emit!(StreamClosedError { count }); } From 28b2b444faa6b73ffc94ce541c18d4736ccb4430 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Mon, 3 Jul 2023 16:58:21 -0400 Subject: [PATCH 09/37] clippy warning --- src/sources/kafka.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 34ea0aa4e5ddf..8bc317f97338c 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -702,7 +702,7 @@ fn handle_messages( mut out: SourceSender, log_namespace: LogNamespace, #[cfg(test)] eof: bool, -) -> () { +) { #[cfg(test)] let mut eof_partitions = std::collections::HashSet::new(); From ab243cbdd50ff3144991677cbecefbc83a2d404a Mon Sep 17 00:00:00 2001 From: John Chesley Date: Mon, 28 Aug 2023 17:38:04 -0400 Subject: [PATCH 10/37] enhancement(kafka source): kafka source uses a dedicated task per partition to consume & acknowledge messages --- src/sources/kafka.rs | 821 ++++++++++++++++++++----------------------- 1 file changed, 384 insertions(+), 437 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 8bc317f97338c..9990d9f2b4cbc 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -3,7 +3,7 @@ use std::{ io::Cursor, sync::{ mpsc::{sync_channel, SyncSender}, - Arc, RwLock, Weak, + Arc, Weak, }, time::Duration, }; @@ -15,26 +15,31 @@ use codecs::{ decoding::{DeserializerConfig, FramingConfig}, StreamDecodingError, }; -use futures::{stream::BoxStream, Stream, StreamExt}; +use futures::{Stream, StreamExt}; use lookup::{lookup_v2::OptionalValuePath, owned_value_path, path, OwnedValuePath}; use once_cell::sync::OnceCell; use rdkafka::{ - consumer::{CommitMode, Consumer, ConsumerContext, Rebalance, StreamConsumer}, + consumer::{ + stream_consumer::StreamPartitionQueue, CommitMode, Consumer, ConsumerContext, Rebalance, + StreamConsumer, + }, error::KafkaError, message::{BorrowedMessage, Headers as _, Message}, types::RDKafkaErrorCode, - ClientConfig, ClientContext, Statistics, + ClientConfig, ClientContext, Statistics, TopicPartitionList, }; use serde_with::serde_as; use snafu::{ResultExt, Snafu}; use tokio::{ runtime::Handle, - sync::mpsc::{self, Receiver, Sender, UnboundedReceiver, UnboundedSender}, - task::JoinHandle, + sync::{ + mpsc::{self, UnboundedReceiver, UnboundedSender}, + oneshot, + }, }; use tokio_util::codec::FramedRead; -use vector_common::{finalization::BatchStatusReceiver, finalizer::OrderedFinalizer}; +use vector_common::finalizer::OrderedFinalizer; use vector_config::configurable_component; use vector_core::{ config::{LegacyKey, LogNamespace}, @@ -304,18 +309,16 @@ impl SourceConfig for KafkaSourceConfig { let log_namespace = cx.log_namespace(self.log_namespace); let acknowledgements = cx.do_acknowledgements(self.acknowledgements); - let consumer = create_consumer(self, acknowledgements)?; let decoder = DecodingConfig::new(self.framing.clone(), self.decoding.clone(), log_namespace).build(); + let consumer = create_consumer(self, acknowledgements)?; Ok(Box::pin(kafka_source( self.clone(), consumer, decoder, - cx.shutdown, cx.out, - acknowledgements, - #[cfg(test)] + cx.shutdown, false, log_namespace, ))) @@ -387,12 +390,12 @@ async fn kafka_source( config: KafkaSourceConfig, consumer: StreamConsumer, decoder: Decoder, - shutdown: ShutdownSignal, out: SourceSender, - acknowledgements: bool, - #[cfg(test)] eof: bool, + shutdown: ShutdownSignal, + eof: bool, log_namespace: LogNamespace, ) -> Result<(), ()> { + let span = info_span!("kafka_source"); let consumer = Arc::new(consumer); consumer @@ -401,311 +404,320 @@ async fn kafka_source( .set(Arc::downgrade(&consumer)) .expect("Error setting up consumer context."); - let mut ack_task = None; - if acknowledgements { + let (callback_sender, callback_rx) = mpsc::unbounded_channel(); + consumer + .context() + .callbacks + .set(callback_sender) + .expect("Error setting up consumer callback channel."); + + // EOF signal allowing the coordination task to tell the kafka client task when all partitions have reached EOF + let (eof_tx, eof_rx) = tokio::sync::oneshot::channel::<()>(); + + let coordination_task = { + let span = span.clone(); let consumer = Arc::clone(&consumer); - let (callback_sender, callback_rx) = mpsc::unbounded_channel(); + let session_timeout_ms = config.session_timeout_ms; + let partition_state = KafkaPartitionState::new(config, decoder, out, log_namespace); + tokio::spawn(async move { + let _enter = span.enter(); + coordinate_kafka_callbacks( + consumer, + callback_rx, + partition_state, + session_timeout_ms, + if eof { Some(eof_tx) } else { None }, + ) + .await; + }) + }; - consumer - .context() - .callbacks - .set(callback_sender) - .expect("Error setting up consumer callback channel."); + let client_task = { + let consumer = Arc::clone(&consumer); + tokio::task::spawn_blocking(move || { + let _enter = span.enter(); + drive_kafka_consumer(consumer, shutdown, eof_rx); + }) + }; - ack_task = Some(handle_acks( - consumer, - callback_rx, - config.session_timeout_ms, - )); - } + _ = tokio::join!(client_task, coordination_task); + consumer.context().commit_consumer_state(); - let msg_consumer = Arc::clone(&consumer); - let span = info_span!("kafka_source"); - let msg_task = tokio::task::spawn_blocking(move || { - let _enter = span.enter(); - handle_messages( + Ok(()) +} + +/// KafkaPartitionState holds all the pieces that are needed to consume a kafka +/// partition stream, and track the state needed in order to correctly manage +/// rebalance events +struct KafkaPartitionState { + config: KafkaSourceConfig, + decoder: Decoder, + out: SourceSender, + log_namespace: LogNamespace, + + /// The Set of partitions expected to drain during a shutdown or rebalance that revokes partitions + expect_drain: HashSet, + + /// The set of partitions we have observed and stored the final acknowledgement for. Ack streams + /// can complete before we get a rebalance callback, so "observed complete" (based on seeing the end of the stream) + /// and "expect to complete" (based on seeing a rebalance callback with revoked partition info) are tracked separately. + observed_drain: HashSet, +} + +impl KafkaPartitionState { + fn new( + config: KafkaSourceConfig, + decoder: Decoder, + out: SourceSender, + log_namespace: LogNamespace, + ) -> Self { + Self { config, - msg_consumer, decoder, - shutdown, out, log_namespace, - #[cfg(test)] - eof, - ); - }); + expect_drain: HashSet::new(), + observed_drain: HashSet::new(), + } + } - if let Some(ack_task) = ack_task { - _ = tokio::join!(msg_task, ack_task); - } else { - _ = tokio::join!(msg_task); + /// Spawn a task on the provided JoinSet to consume the kafka StreamPartitionQueue, and handle acknowledgements for the messages consumed + /// Returns a channel sender that can be used to signal that the consumer should stop and drain pending acknowledgements, + /// and an AbortHandle that can be used to forcefully end the task. + fn consume_partition( + &self, + joinset: &mut tokio::task::JoinSet, + tp: TopicPartition, + consumer: Arc>, + p: StreamPartitionQueue, + acknowledgements: bool, + exit_eof: bool, + ) -> (oneshot::Sender<()>, tokio::task::AbortHandle) { + // TODO there's probably a better way to pass the keys() config around than cloning the entire config object? But they are tied to the lifetime of the config object, so..this avoids lifetime messes for now + let conf = self.config.clone(); + let decoder = self.decoder.clone(); + let log_namespace = self.log_namespace; + let mut out = self.out.clone(); + + let (end_tx, mut end_signal) = tokio::sync::oneshot::channel::<()>(); + + let handle = joinset.spawn(async move { + let mut messages = p.stream(); + let (finalizer, mut ack_stream) = OrderedFinalizer::::new(None); + + // finalizer is the entry point for new pending acknowledgements; + // when it is dropped, no new messages will be consumed, and the + // task will end when it reaches the end of ack_stream + let mut finalizer = Some(finalizer); + + loop { + tokio::select!( + _ = &mut end_signal, if finalizer.is_some() => { + finalizer.take(); + }, + message = messages.next(), if finalizer.is_some() => match message { + None => unreachable!("MessageStream never calls Ready(None)"), + Some(Err(error)) => match error { + rdkafka::error::KafkaError::PartitionEOF(partition) if exit_eof => { + debug!("EOF for partition {}.", partition); + finalizer.take(); + }, + _ => emit!(KafkaReadError { error }), + }, + Some(Ok(msg)) => { + emit!(KafkaBytesReceived { + byte_size: msg.payload_len(), + protocol: "tcp", + topic: msg.topic(), + partition: msg.partition(), + }); + parse_message(msg, decoder.clone(), conf.keys(), &mut out, acknowledgements, &finalizer, log_namespace).await; + } + }, + + ack = ack_stream.next() => match ack { + Some((status, entry)) => { + if status == BatchStatus::Delivered { + if let Err(error) = consumer.store_offset(&entry.topic, entry.partition, entry.offset) { + emit!(KafkaOffsetUpdateError { error }); + } + } + } + None if finalizer.is_none() => { + debug!("acknowledgement stream complete for partition {}:{}.", &tp.0, tp.1); + break + } + None => { + debug!("acknowledgement stream empty for {}:{}", &tp.0, tp.1); + } + } + ) + } + tp + }); + (end_tx, handle) } - consumer.context().commit_consumer_state(); + pub fn revoke_partition(&mut self, tp: TopicPartition) { + self.expect_drain.insert(tp); + } - Ok(()) + pub fn observed_last_ack(&mut self, tp: TopicPartition) { + self.observed_drain.insert(tp); + } + + pub fn is_drain_complete(&self) -> bool { + self.observed_drain == self.expect_drain + } + + pub fn clear(&mut self) { + self.expect_drain.clear(); + self.observed_drain.clear(); + } } -fn handle_acks( +async fn coordinate_kafka_callbacks( consumer: Arc>, mut callbacks: UnboundedReceiver, + mut partition_state: KafkaPartitionState, max_drain_ms: Duration, -) -> JoinHandle<()> { + mut eof: Option>, +) { let mut drain_signal: Option> = None; - fn handle_ack( - consumer: &Arc>, - status: BatchStatus, - entry: FinalizerEntry, - ) { - if status == BatchStatus::Delivered { - if let Err(error) = consumer.store_offset(&entry.topic, entry.partition, entry.offset) { - emit!(KafkaOffsetUpdateError { error }); - } - } - } - async fn revoke_timeout(t: Duration) { tokio::time::sleep(t).await; } + let mut drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); - // Wrap acks for each partition in this enum, so that we can let the receiver - // know when it has seen the last one for the current assignment of this partition - enum ForwardedAck { - Entry(BatchStatus, FinalizerEntry), - Drained(TopicPartition), - } + // A oneshot channel is used for each consumed partition, so that we can + // signal to that task to stop consuming, drain pending acks, and exit + let mut end_signals: HashMap> = HashMap::new(); - struct KafkaPartitionState { - /// The sender for ack forwarding tasks to use - ack_tx: Option>, + // The set of consumer tasks, each consuming a specific partition. The task + // is both consuming the messages (passing them to the output stream) _and_ + // processing the corresponding acknowledgement stream. A consumer task + // should completely drain its acknowledgement stream after receiving an end signal + let mut partition_consumers: tokio::task::JoinSet = Default::default(); - /// Tasks forwarding acknowledgement entries for each partition to a main channel. - /// There will be one task per assigned partition, and this allows us, during rebalances, - /// to precisely manage when acks for any revoked partitions are complete - ack_forwarders: tokio::task::JoinSet, + // Handles that will let us end any consumer task that exceeds a drain deadline + let mut abort_handles: HashMap = HashMap::new(); - /// Abort handles for each forwarding task, indexed by the (topic, partition) pair. This - /// allows for precise task cancellation when a partition is revoked but acks can't be processed - /// before a timeout; only pending acks for revoked partitions will be cancelled/dropped. - abort_handles: HashMap, + let exit_eof = eof.is_some(); - /// The Set of partitions expected to drain during a shutdown or rebalance that revokes partitions - expect_drain: HashSet, + loop { + tokio::select! { + Some(Ok(finished_partition)) = partition_consumers.join_next(), if !partition_consumers.is_empty() => { + debug!("Partition consumer finished for {}:{}", &finished_partition.0, finished_partition.1); + abort_handles.remove(&finished_partition); + partition_state.observed_last_ack(finished_partition); - /// The set of partitions we have observed and stored the final acknowledgement for. Ack streams - /// can complete before we get a rebalance callback, so "observed complete" (based on seeing the end of the stream) - /// and "expect to complete" (based on seeing a rebalance callback with revoked partition info) are tracked separately. - observed_drain: HashSet, - } - impl KafkaPartitionState { - fn new() -> (Self, Receiver) { - let (ack_tx, all_acks) = mpsc::channel(16); // arbitrary size 16 - let state = KafkaPartitionState { - ack_tx: Some(ack_tx), - ack_forwarders: tokio::task::JoinSet::new(), - abort_handles: HashMap::new(), - expect_drain: HashSet::new(), - observed_drain: HashSet::new(), - }; - (state, all_acks) - } - pub fn assign_partition(&mut self, tp: TopicPartition, acks: AckStream) { - if let Some(ref ack_tx) = self.ack_tx { - self.abort_handles.insert( - tp.clone(), - self.ack_forwarders - .spawn(forward_acks(tp, acks, ack_tx.clone())), - ); - } - } - pub fn revoke_partition(&mut self, tp: TopicPartition) { - self.expect_drain.insert(tp); - } + // Signal the client task that at least one partition has completed + _ = drain_signal.as_ref().map(|sig| _ = sig.send(()) ); - pub fn forwarder_complete(&mut self, tp: &TopicPartition) { - self.abort_handles.remove(tp); - } - pub fn has_forwarders(&self) -> bool { - !self.ack_forwarders.is_empty() - } - pub fn abort_pending_forwarders(&mut self) { - for tp in self.expect_drain.drain() { - // If the handle isn't here anymore (None case) it just means the task already completed - if let Some(handle) = self.abort_handles.remove(&tp) { - handle.abort(); + if partition_state.is_drain_complete() { + debug!("All expected partitions have drained."); + partition_state.clear(); + // Signal the client task that all partitions that are expected to drain have completed + drain_signal.take(); } - } - } - pub fn observed_last_ack(&mut self, tp: TopicPartition) { - self.observed_drain.insert(tp); - } - - pub fn is_drain_complete(&self) -> bool { - self.observed_drain == self.expect_drain - } - pub fn clear(&mut self) { - self.expect_drain.clear(); - self.observed_drain.clear(); - } - - pub fn close(&mut self) { - let _ = self.ack_tx.take(); - } - } - - async fn forward_acks( - tp: TopicPartition, - mut acks: AckStream, - forward_to: mpsc::Sender, - ) -> TopicPartition { - while let Some((status, entry)) = acks.next().await { - if let Err(e) = forward_to.send(ForwardedAck::Entry(status, entry)).await { - warn!("Error sending to main ack task: {}", e); - } - } - let _ = forward_to.send(ForwardedAck::Drained(tp.clone())).await; - tp - } - - tokio::spawn(async move { - /* - Ok how does this work and where are the nuances in here? - We have: - - the consumer task: a task talking to Kafka, reading messages and getting rebalance notifications, etc. - - Finalizer entries are added to partition-specific channels from this task - - During rebalances, send notifications about assigned/revoked partitions - - When a partition is revoked, or the client is shutting down, close the - finalizer channels so they can be drained by the ack task. Wait on a channel coordinating with - the acknowledgement task to avoid proceeding with rebalance before pending offsets are written - Nuance: the kafka task runs blocking code in the rebalance callbacks, so must live - on a separate thread from acknowledgement handling, otherwise everything deadlocks - - - the acknowledgement task: - - Nuance: rebalancing may revoke a subset of our partitions, depending on the strategy-- to handle this correctly - we use a finalizer stream per partition - - When a partition is assigned, spawn a task dedicated to reading the finalizer channel. Tasks and channels - perform significantly better than a tokio StreamMap when dealing with more than a few partitions. - - As finalizers become ready, forward them to the main ack task to be stored. - - When a finalizer channel closes, the final message in the forwarding channel - is a marker indicating the partition is drained. Nuance: acks for the partition - are considered drained when this marker is _processed by the main ack task_, not - when the forwarding task ends - Additional nuance: finalizer channels can end before we even get a rebalance notification! - - During a rebalance, we track expected and observed drain markers, as well as a timeout. - As soon as the expected partition streams are drained, or the timeout is reached, signal back - to the consumer task to proceed with rebalancing. - In the case of a timeout, drop any remaining acks on revoked partitions. - - Rebalance bookkeeping is done by the ForwardedAck enum and KafkaPartitionState struct. - - */ - - let mut drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); - let (mut partition_state, mut all_acks) = KafkaPartitionState::new(); + // NB this is not production-ready EOF/end-of-partition detection, + // and is only enabled under specific scenarios in testing + if exit_eof && partition_consumers.is_empty() { + debug!("All partitions have exited or reached EOF"); + let _ = eof.take().map(|e| e.send(())); + } + }, + Some(callback) = callbacks.recv() => match callback { + KafkaCallback::PartitionsAssigned(mut assigned_partitions, done) => { + let acks = consumer.context().acknowledgements; + for tp in assigned_partitions.drain(0..) { + let topic = tp.0.as_str(); + let partition = tp.1; + // It _should_ be impossible for this expect() to panic, since we receive the topic/partition pair from the rebalance callback + let pq = consumer.split_partition_queue(topic, partition).expect("Failed to get partition queue: invalid topic or partition."); + + debug!("Consuming partition {}:{}", &tp.0, tp.1); + let (end_tx, handle) = partition_state.consume_partition(&mut partition_consumers, tp.clone(), Arc::clone(&consumer), pq, acks, exit_eof); + abort_handles.insert(tp.clone(), handle); + end_signals.insert(tp, end_tx); + } + drop(done); // Implied, probably..but just to be explicit about when this is dropped, since it is important + }, + KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => { + drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); - loop { - tokio::select! { - Some(callback) = callbacks.recv() => match callback { - KafkaCallback::PartitionsAssigned(mut assigned_streams) => { - for (tp, acks) in assigned_streams.drain(0..) { - partition_state.assign_partition(tp, acks); + for tp in revoked_partitions.drain(0..) { + if let Some(end) = end_signals.remove(&tp) { + let _ = end.send(()); } - }, - KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => { - drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); + debug!("Revoking partition {}:{}", &tp.0, tp.1); + partition_state.revoke_partition(tp); + } - for tp in revoked_partitions.drain(0..) { + if partition_state.is_drain_complete() { + partition_state.clear(); + drop(drain); + } else if drain_signal.replace(drain).is_some() { + unreachable!("Concurrent rebalance callbacks should not be possible."); + } + }, + KafkaCallback::ShuttingDown(drain) => { + // Shutting down is just like a full assignment revoke, but we also close the + // callback channels, since we don't expect additional assignments or rebalances + if let Ok(tpl) = consumer.assignment() { + tpl.elements() + .iter() + .for_each(|el| { + + let tp: TopicPartition = (el.topic().into(), el.partition()); + if let Some(end) = end_signals.remove(&tp) { + let _ = end.send(()); + } partition_state.revoke_partition(tp); - } - - if partition_state.is_drain_complete() { - partition_state.clear(); - drop(drain); - } else if drain_signal.replace(drain).is_some() { - unreachable!("Concurrent rebalance callbacks should not be possible."); - } - }, - KafkaCallback::ShuttingDown(drain) => { - // Shutting down is just like a full assignment revoke, but we also close the ack senders and callback - // channels, since we don't expect additional assignments or rebalances - if let Ok(tpl) = consumer.assignment() { - tpl.elements() - .iter() - .for_each(|el| { - partition_state.revoke_partition((el.topic().into(), el.partition())); - }); - } - - drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); - if partition_state.is_drain_complete() { - partition_state.clear(); - drop(drain); - } else if drain_signal.replace(drain).is_some() { - unreachable!("Shutdown callback happened somehow during an ongoing rebalance...?") - } + }); + } - // No new partitions will be assigned, so drop our handle to the ack sender and close the callback channel - partition_state.close(); - callbacks.close(); - }, - }, + drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); + if partition_state.is_drain_complete() { + partition_state.clear(); + drop(drain); + } else if drain_signal.replace(drain).is_some() { + unreachable!("Shutdown callback happened somehow during an ongoing rebalance...?") + } - // As partition-specific sender tasks complete (revoked partitions - // during a rebalance, shutdown, or eof during tests), handle the results here - Some(Ok(finished_partition)) = partition_state.ack_forwarders.join_next(), if partition_state.has_forwarders() => { - partition_state.forwarder_complete(&finished_partition); + callbacks.close(); }, + }, - _ = &mut drain_deadline, if drain_signal.is_some() => { - debug!("Acknowledgement drain deadline reached. Dropping any pending ack streams for revoked partitions."); - partition_state.abort_pending_forwarders(); - partition_state.clear(); - - if let Err(e) = drain_signal.take().unwrap().send(()) { - warn!("Error sending to drain signal: {}.", e); + _ = &mut drain_deadline, if drain_signal.is_some() => { + debug!("Acknowledgement drain deadline reached. Dropping any pending ack streams for revoked partitions."); + for tp in partition_state.expect_drain.drain() { + if let Some(handle) = abort_handles.remove(&tp) { + handle.abort(); } - }, - - Some(entry) = all_acks.recv() => match entry { - ForwardedAck::Drained(tp) => { - partition_state.observed_last_ack(tp); - - if drain_signal.is_some() { - _ = drain_signal.as_ref().map(|sig| _ = sig.send(()) ); - } - - if partition_state.is_drain_complete() { - partition_state.clear(); - drain_signal.take(); - } - }, - ForwardedAck::Entry(delivery_status, entry) => { - handle_ack(&consumer, delivery_status, entry); - }, - }, + } + partition_state.clear(); - // acks and callbacks are all done - else => { - break + if let Err(e) = drain_signal.take().unwrap().send(()) { + warn!("Error sending to drain signal: {}.", e); } + }, + + // Consumers are done, and callback channel is closed + else => { + break } } - }) + } } -fn handle_messages( - config: KafkaSourceConfig, +fn drive_kafka_consumer( consumer: Arc>, - decoder: Decoder, mut shutdown: ShutdownSignal, - mut out: SourceSender, - log_namespace: LogNamespace, - #[cfg(test)] eof: bool, + mut eof: tokio::sync::oneshot::Receiver<()>, ) { - #[cfg(test)] - let mut eof_partitions = std::collections::HashSet::new(); - Handle::current().block_on(async move { let mut stream = consumer.stream(); loop { @@ -715,38 +727,18 @@ fn handle_messages( break }, + _ = &mut eof => { + consumer.context().shutdown(); + break + }, + + // NB: messages are not received on this thread, however we poll + // the consumer to serve client callbacks, such as rebalance notifications message = stream.next() => match message { None => unreachable!("MessageStream never returns Ready(None)"), - Some(Err(error)) => match error { - #[cfg(test)] - rdkafka::error::KafkaError::PartitionEOF(partition) if eof => { - // NB this is not production ready EOF detection! Hence cfg(test) on this branch - // Used only in tests when we can be certain only one topic is being consumed, - // and new messages are not written after EOF is seen - // Also: RdKafka only tells us the partition, so - // we are assuming single-topic consumers when using this. - // Also also: due to rebalances, we might get notified about an EOF partition - // more than once, so we use a Set to exit once we've seen EOF on all currently-assigned partitions - eof_partitions.insert(partition); - if let Ok(assignment) = consumer.assignment() { - // All currently assigned partitions have reached EOF - if assignment.elements().iter().all(|tp| eof_partitions.contains(&tp.partition())) { - consumer.context().shutdown(); - break - } - } - }, - _ => emit!(KafkaReadError { error }), - }, - Some(Ok(msg)) => { - emit!(KafkaBytesReceived { - byte_size: msg.payload_len(), - protocol: "tcp", - topic: msg.topic(), - partition: msg.partition(), - }); - - parse_message(msg, decoder.clone(), config.keys(), &mut out, &consumer, log_namespace).await; + Some(Err(error)) => emit!(KafkaReadError { error }), + Some(Ok(_msg)) => { + unreachable!("Messages are consumed in dedicated tasks for each partition.") } }, } @@ -759,37 +751,32 @@ async fn parse_message( decoder: Decoder, keys: Keys<'_>, out: &mut SourceSender, - consumer: &Arc>, + acknowledgements: bool, + finalizer: &Option>, log_namespace: LogNamespace, ) { - let context = consumer.context(); - - if let Some((count, mut stream)) = parse_stream(&msg, decoder, keys, log_namespace) { - if context.acknowledgements { - let (batch, receiver) = BatchNotifier::new_with_receiver(); - let mut stream = stream.map(|event| event.with_batch_notifier(&batch)); - match out.send_event_stream(&mut stream).await { - Err(_) => { - emit!(StreamClosedError { count }); - } - Ok(_) => { - // Drop stream to avoid borrowing `msg`: "[...] borrow might be used - // here, when `stream` is dropped and runs the destructor [...]". - drop(stream); - context.add_finalizer_entry(msg.into(), receiver); - } + if let Some((count, stream)) = parse_stream(&msg, decoder, keys, log_namespace) { + let (batch, receiver) = BatchNotifier::new_with_receiver(); + let mut stream = stream.map(|event| { + // All acknowledgements flow through the normal Finalizer stream so + // that they can be handled in one place, but are only tied to the + // batch when acknowledgements are enabled + if acknowledgements { + event.with_batch_notifier(&batch) + } else { + event } - } else { - match out.send_event_stream(&mut stream).await { - Err(_) => { - emit!(StreamClosedError { count }); - } - Ok(_) => { - if let Err(error) = - consumer.store_offset(msg.topic(), msg.partition(), msg.offset()) - { - emit!(KafkaOffsetUpdateError { error }); - } + }); + match out.send_event_stream(&mut stream).await { + Err(_) => { + emit!(StreamClosedError { count }); + } + Ok(_) => { + // Drop stream to avoid borrowing `msg`: "[...] borrow might be used + // here, when `stream` is dropped and runs the destructor [...]". + drop(stream); + if let Some(f) = finalizer.as_ref() { + f.add(msg.into(), receiver) } } } @@ -1046,15 +1033,13 @@ fn create_consumer( } type TopicPartition = (String, i32); -type AckStream = BoxStream<'static, (BatchStatus, FinalizerEntry)>; enum KafkaCallback { - PartitionsAssigned(Vec<(TopicPartition, AckStream)>), + PartitionsAssigned(Vec, SyncSender<()>), PartitionsRevoked(Vec, SyncSender<()>), ShuttingDown(SyncSender<()>), } -#[derive(Default)] struct KafkaSourceContext { acknowledgements: bool, stats: kafka::KafkaStatisticsContext, @@ -1062,11 +1047,6 @@ struct KafkaSourceContext { /// A callback channel used to coordinate between the main consumer task and the acknowledgement task callbacks: OnceCell>, - /// Use a finalizer stream for each partition being consumed, so that when a partition - /// is revoked during a consumer rebalance, acknowledgements can be prioritized (and - /// dropped after a time limit) without affecting acks for retained partitions. - finalizers: RwLock>>, - /// A weak reference to the consumer, so that we can commit offsets during a rebalance operation consumer: OnceCell>>, } @@ -1076,25 +1056,12 @@ impl KafkaSourceContext { Self { stats: kafka::KafkaStatisticsContext { expose_lag_metrics }, acknowledgements, - ..Default::default() - } - } - - pub fn add_finalizer_entry(&self, mut entry: FinalizerEntry, recv: BatchStatusReceiver) { - if let Ok(fin) = self.finalizers.read() { - let key = (entry.topic, entry.partition); - if let Some(entries) = fin.get(&key) { - entry.topic = key.0; // Slightly awkward, but avoids cloning the topic string for every entry added - entries.add(entry, recv); - } + callbacks: OnceCell::default(), + consumer: OnceCell::default(), } } pub fn shutdown(&self) { - if let Ok(mut fin) = self.finalizers.write() { - fin.clear(); - } - if let Some(tx) = self.callbacks.get() { let (send, rendezvous) = sync_channel(0); if tx.send(KafkaCallback::ShuttingDown(send)).is_ok() { @@ -1105,39 +1072,67 @@ impl KafkaSourceContext { } } - fn add_finalizerset(&self, key: TopicPartition) -> Option<(TopicPartition, AckStream)> { - if let Ok(fin) = self.finalizers.read() { - if fin.contains_key(&key) { - trace!("Finalizer entry already exists for {}:{}.", key.0, key.1); - return None; - } - } + /// Emit a PartitionsAssigned callback with the topic-partitions to be consumed, + /// and block until confirmation is received that a stream and consumer for + /// each topic-partition has been set up. This function blocks until the + /// rendezvous channel sender is dropped by the callback handler. + fn consume_partitions(&self, tpl: &TopicPartitionList) { + let callbacks = self + .callbacks + .get() + .expect("Callbacks handler was not initialized."); + + let (send, rendezvous) = sync_channel(0); + let _ = callbacks.send(KafkaCallback::PartitionsAssigned( + tpl.elements() + .iter() + .map(|tp| (tp.topic().into(), tp.partition())) + .collect(), + send, + )); - let (finalizer, ack_stream) = OrderedFinalizer::::new(None); - if let Ok(mut fin) = self.finalizers.write() { - fin.insert(key.clone(), finalizer); - Some((key, ack_stream)) - } else { - // error getting the RwLock, i.e. the lock is poisoned (!!) - None + while rendezvous.recv().is_ok() { + // no-op: wait for partition assignment handler to complete } } - fn rm_finalizerset(&self, key: &TopicPartition) { - if let Ok(mut fin) = self.finalizers.write() { - fin.remove(key); + /// Emit a PartitionsRevoked callback and block until confirmation is + /// received that acknowledgements have been processed for each of them. + /// The rendezvous channel used in the callback can send multiple times to + /// signal individual partitions completing. This function blocks until the + /// sender is dropped by the callback handler. + fn revoke_partitions(&self, tpl: &TopicPartitionList) { + let callbacks = self + .callbacks + .get() + .expect("Callbacks handler was not initialized."); + + let (send, rendezvous) = sync_channel(0); + let _ = callbacks.send(KafkaCallback::PartitionsRevoked( + tpl.elements() + .iter() + .map(|tp| (tp.topic().into(), tp.partition())) + .collect(), + send, + )); + + while rendezvous.recv().is_ok() { + self.commit_consumer_state(); } } fn commit_consumer_state(&self) { - if let Some(w) = self.consumer.get() { - if let Some(consumer) = w.upgrade() { - match consumer.commit_consumer_state(CommitMode::Sync) { - Ok(_) | Err(KafkaError::ConsumerCommit(RDKafkaErrorCode::NoOffset)) => { - /* Success, or nothing to do \0/ */ - } - Err(error) => emit!(KafkaOffsetUpdateError { error }), + if let Some(consumer) = self + .consumer + .get() + .expect("Consumer reference was not initialized.") + .upgrade() + { + match consumer.commit_consumer_state(CommitMode::Sync) { + Ok(_) | Err(KafkaError::ConsumerCommit(RDKafkaErrorCode::NoOffset)) => { + /* Success, or nothing to do - yay \0/ */ } + Err(error) => emit!(KafkaOffsetUpdateError { error }), } } } @@ -1151,63 +1146,16 @@ impl ClientContext for KafkaSourceContext { impl ConsumerContext for KafkaSourceContext { fn pre_rebalance(&self, rebalance: &Rebalance) { - if let Some(tx) = self.callbacks.get() { - match rebalance { - Rebalance::Assign(tpl) => { - // Partitions are being assigned to this consumer! - // 1. Create a finalizer set for the new partitions - // 2. `self` keeps a reference to the sender - this is the entry point for pending acks on this partition - // 3. Hand a reference to the receiver stream to the acknowledgement task via callback channel - let ack_streams: Vec<(TopicPartition, AckStream)> = tpl - .elements() - .iter() - .filter_map(|el| self.add_finalizerset((el.topic().into(), el.partition()))) - .collect(); - - trace!("Partition(s) assigned: {}", ack_streams.len()); - if !ack_streams.is_empty() { - let _ = tx.send(KafkaCallback::PartitionsAssigned(ack_streams)); - } - } - Rebalance::Revoke(tpl) => { - // Partitions are being revoked from this consumer! - // 1. Close the sending side for new acknowledgement entries. - // 2. Notify the acknowledgement task, and provide a rendezvous channel; wait for that channel to close - // to indicate when acks for revoked partitions are drained. - // 3. Commit consumer offsets and return, allowing the rebalance to complete - let revoked: Vec = tpl - .elements() - .iter() - .map(|el| { - let key = (el.topic().into(), el.partition()); - self.rm_finalizerset(&key); - key - }) - .collect(); - - trace!("Partition(s) revoked: {}", revoked.len()); - if !revoked.is_empty() { - let (send, rendezvous) = sync_channel(0); - // The ack task will signal on this channel when it has drained a revoked partition - // and will close the channel when it has drained all revoked partitions, - // or when it times out, to prevent the consumer being kicked from the group. - // This send will return Err if the ack task has already exited; in that case we - // proceed without waiting - if tx - .send(KafkaCallback::PartitionsRevoked(revoked, send)) - .is_ok() - { - while rendezvous.recv().is_ok() { - self.commit_consumer_state(); - } - } + match rebalance { + Rebalance::Assign(tpl) => self.consume_partitions(tpl), - self.commit_consumer_state(); - } - } - Rebalance::Error(message) => { - error!("Error during rebalance: {}.", message); - } + Rebalance::Revoke(tpl) => { + self.revoke_partitions(tpl); + self.commit_consumer_state(); + } + + Rebalance::Error(message) => { + error!("Error during Kafka consumer group rebalance: {}.", message); } } } @@ -1631,14 +1579,13 @@ mod integration_test { } fn spawn_kafka( - tx: SourceSender, + out: SourceSender, config: KafkaSourceConfig, acknowledgements: bool, eof: bool, log_namespace: LogNamespace, ) -> (Trigger, Tripwire) { let (trigger_shutdown, shutdown, shutdown_done) = ShutdownSignal::new_wired(); - let consumer = create_consumer(&config, acknowledgements).unwrap(); let decoder = DecodingConfig::new( config.framing.clone(), @@ -1646,14 +1593,14 @@ mod integration_test { log_namespace, ) .build(); + let consumer = create_consumer(&config, acknowledgements).unwrap(); tokio::spawn(kafka_source( config, consumer, decoder, + out, shutdown, - tx, - acknowledgements, eof, log_namespace, )); @@ -1873,8 +1820,8 @@ mod integration_test { .expect("Number of milliseconds before shutting down first consumer."); let (topic, group_id, _) = send_to_test_topic(2, send_count).await; - println!("Topic: {}", &topic); - println!("Consumer group.id: {}", &group_id); + debug!("Topic: {}", &topic); + debug!("Consumer group.id: {}", &group_id); // 2. Run the kafka source to read some of the events // 3. Start 2nd & 3rd consumers using the same group.id, triggering rebalance events From 485524f363fce4dd601c44eef244e4df7725f64b Mon Sep 17 00:00:00 2001 From: John Chesley Date: Mon, 28 Aug 2023 18:00:46 -0400 Subject: [PATCH 11/37] make the spelling checker happy, maybe? --- src/sources/kafka.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index b5929c9548233..33768e5b55b06 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -488,7 +488,7 @@ impl KafkaPartitionState { /// and an AbortHandle that can be used to forcefully end the task. fn consume_partition( &self, - joinset: &mut tokio::task::JoinSet, + join_set: &mut tokio::task::JoinSet, tp: TopicPartition, consumer: Arc>, p: StreamPartitionQueue, @@ -503,7 +503,7 @@ impl KafkaPartitionState { let (end_tx, mut end_signal) = tokio::sync::oneshot::channel::<()>(); - let handle = joinset.spawn(async move { + let handle = join_set.spawn(async move { let mut messages = p.stream(); let (finalizer, mut ack_stream) = OrderedFinalizer::::new(None); From dc723a75e23f08d7d38be0974eb53d348df2c2de Mon Sep 17 00:00:00 2001 From: John Chesley Date: Tue, 29 Aug 2023 12:57:17 -0400 Subject: [PATCH 12/37] emit a debug log instead of panicking if a shutdown happens during a consumer rebalance --- src/sources/kafka.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 33768e5b55b06..439ff308671b1 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -686,7 +686,7 @@ async fn coordinate_kafka_callbacks( partition_state.clear(); drop(drain); } else if drain_signal.replace(drain).is_some() { - unreachable!("Shutdown callback happened somehow during an ongoing rebalance...?") + debug!("Kafka consumer shutting down mid-rebalance."); } callbacks.close(); From 1f49643bbce48c91b12511b1a06a74e0c38c26a7 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Tue, 29 Aug 2023 14:09:57 -0400 Subject: [PATCH 13/37] improved partition eof handling --- src/sources/kafka.rs | 39 ++++++++++++++++++++++++++++----------- 1 file changed, 28 insertions(+), 11 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 439ff308671b1..66ef714f45642 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -488,7 +488,7 @@ impl KafkaPartitionState { /// and an AbortHandle that can be used to forcefully end the task. fn consume_partition( &self, - join_set: &mut tokio::task::JoinSet, + join_set: &mut tokio::task::JoinSet<(TopicPartition, PartitionConsumerStatus)>, tp: TopicPartition, consumer: Arc>, p: StreamPartitionQueue, @@ -512,6 +512,8 @@ impl KafkaPartitionState { // task will end when it reaches the end of ack_stream let mut finalizer = Some(finalizer); + let mut status = PartitionConsumerStatus::NormalExit; + loop { tokio::select!( _ = &mut end_signal, if finalizer.is_some() => { @@ -522,6 +524,7 @@ impl KafkaPartitionState { Some(Err(error)) => match error { rdkafka::error::KafkaError::PartitionEOF(partition) if exit_eof => { debug!("EOF for partition {}.", partition); + status = PartitionConsumerStatus::PartitionEOF; finalizer.take(); }, _ => emit!(KafkaReadError { error }), @@ -555,7 +558,7 @@ impl KafkaPartitionState { } ) } - tp + (tp, status) }); (end_tx, handle) } @@ -600,7 +603,7 @@ async fn coordinate_kafka_callbacks( // is both consuming the messages (passing them to the output stream) _and_ // processing the corresponding acknowledgement stream. A consumer task // should completely drain its acknowledgement stream after receiving an end signal - let mut partition_consumers: tokio::task::JoinSet = Default::default(); + let mut partition_consumers: tokio::task::JoinSet<(TopicPartition, PartitionConsumerStatus)> = Default::default(); // Handles that will let us end any consumer task that exceeds a drain deadline let mut abort_handles: HashMap = HashMap::new(); @@ -609,7 +612,7 @@ async fn coordinate_kafka_callbacks( loop { tokio::select! { - Some(Ok(finished_partition)) = partition_consumers.join_next(), if !partition_consumers.is_empty() => { + Some(Ok((finished_partition, status))) = partition_consumers.join_next(), if !partition_consumers.is_empty() => { debug!("Partition consumer finished for {}:{}", &finished_partition.0, finished_partition.1); abort_handles.remove(&finished_partition); partition_state.observed_last_ack(finished_partition); @@ -624,11 +627,16 @@ async fn coordinate_kafka_callbacks( drain_signal.take(); } - // NB this is not production-ready EOF/end-of-partition detection, - // and is only enabled under specific scenarios in testing - if exit_eof && partition_consumers.is_empty() { - debug!("All partitions have exited or reached EOF"); - let _ = eof.take().map(|e| e.send(())); + match status { + // PartitionConsumerStatus differentiates between a task that exited after + // being signaled to end, and one that reached the end of its partition and + // was configured to exit. After the last such task ends, we signal the kafka + // driver task to shut down the main consumer too. Note this is only used in tests. + PartitionConsumerStatus::PartitionEOF if exit_eof && partition_consumers.is_empty() => { + debug!("All partitions have exited or reached EOF."); + let _ = eof.take().map(|e| e.send(())); + }, + _ => {} } }, Some(callback) = callbacks.recv() => match callback { @@ -640,12 +648,13 @@ async fn coordinate_kafka_callbacks( // It _should_ be impossible for this expect() to panic, since we receive the topic/partition pair from the rebalance callback let pq = consumer.split_partition_queue(topic, partition).expect("Failed to get partition queue: invalid topic or partition."); - debug!("Consuming partition {}:{}", &tp.0, tp.1); + debug!("Consuming partition {}:{}.", &tp.0, tp.1); let (end_tx, handle) = partition_state.consume_partition(&mut partition_consumers, tp.clone(), Arc::clone(&consumer), pq, acks, exit_eof); abort_handles.insert(tp.clone(), handle); end_signals.insert(tp, end_tx); } - drop(done); // Implied, probably..but just to be explicit about when this is dropped, since it is important + // ensure this is retained until all individual queues are set up + drop(done); }, KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => { drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); @@ -1038,6 +1047,14 @@ fn create_consumer( type TopicPartition = (String, i32); +/// Status returned by partition consumer tasks, allowing the coordination task +/// to differentiate between a consumer exiting normally (after receiving an end +/// signal) and exiting when it reaches the end of a partition +enum PartitionConsumerStatus { + NormalExit, + PartitionEOF, +} + enum KafkaCallback { PartitionsAssigned(Vec, SyncSender<()>), PartitionsRevoked(Vec, SyncSender<()>), From a69b2c4bf645cb1b5977247365436c312acdd5dc Mon Sep 17 00:00:00 2001 From: John Chesley Date: Wed, 6 Sep 2023 19:27:19 -0400 Subject: [PATCH 14/37] add OptionFuture to drain deadline and EOF handling, and use is_subset to detect when all expected partitions are finished draining --- src/sources/kafka.rs | 58 ++++++++++++++++++++++++++------------------ 1 file changed, 34 insertions(+), 24 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 66ef714f45642..b4a89ffbc85c4 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -16,6 +16,7 @@ use codecs::{ StreamDecodingError, }; use futures::{Stream, StreamExt}; +use futures_util::future::OptionFuture; use lookup::{lookup_v2::OptionalValuePath, owned_value_path, path, OwnedValuePath}; use once_cell::sync::OnceCell; use rdkafka::{ @@ -36,6 +37,7 @@ use tokio::{ mpsc::{self, UnboundedReceiver, UnboundedSender}, oneshot, }, + task::JoinSet, }; use tokio_util::codec::FramedRead; @@ -414,7 +416,12 @@ async fn kafka_source( .expect("Error setting up consumer callback channel."); // EOF signal allowing the coordination task to tell the kafka client task when all partitions have reached EOF - let (eof_tx, eof_rx) = tokio::sync::oneshot::channel::<()>(); + let (eof_tx, eof_rx) = if eof { + let (tx, rx) = oneshot::channel::<()>(); + (Some(tx), Some(rx)) + } else { + (None, None) + }; let coordination_task = { let span = span.clone(); @@ -428,7 +435,7 @@ async fn kafka_source( callback_rx, partition_state, session_timeout_ms, - if eof { Some(eof_tx) } else { None }, + eof_tx, ) .await; }) @@ -488,7 +495,7 @@ impl KafkaPartitionState { /// and an AbortHandle that can be used to forcefully end the task. fn consume_partition( &self, - join_set: &mut tokio::task::JoinSet<(TopicPartition, PartitionConsumerStatus)>, + join_set: &mut JoinSet<(TopicPartition, PartitionConsumerStatus)>, tp: TopicPartition, consumer: Arc>, p: StreamPartitionQueue, @@ -501,7 +508,7 @@ impl KafkaPartitionState { let log_namespace = self.log_namespace; let mut out = self.out.clone(); - let (end_tx, mut end_signal) = tokio::sync::oneshot::channel::<()>(); + let (end_tx, mut end_signal) = oneshot::channel::<()>(); let handle = join_set.spawn(async move { let mut messages = p.stream(); @@ -572,12 +579,13 @@ impl KafkaPartitionState { } pub fn is_drain_complete(&self) -> bool { - self.observed_drain == self.expect_drain + self.expect_drain.is_subset(&self.observed_drain) } pub fn clear(&mut self) { - self.expect_drain.clear(); - self.observed_drain.clear(); + for item in self.expect_drain.drain() { + self.observed_drain.remove(&item); + } } } @@ -586,24 +594,20 @@ async fn coordinate_kafka_callbacks( mut callbacks: UnboundedReceiver, mut partition_state: KafkaPartitionState, max_drain_ms: Duration, - mut eof: Option>, + mut eof: Option>, ) { let mut drain_signal: Option> = None; - - async fn revoke_timeout(t: Duration) { - tokio::time::sleep(t).await; - } - let mut drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); + let mut drain_deadline: OptionFuture<_> = None.into(); // A oneshot channel is used for each consumed partition, so that we can // signal to that task to stop consuming, drain pending acks, and exit - let mut end_signals: HashMap> = HashMap::new(); + let mut end_signals: HashMap> = HashMap::new(); // The set of consumer tasks, each consuming a specific partition. The task // is both consuming the messages (passing them to the output stream) _and_ // processing the corresponding acknowledgement stream. A consumer task // should completely drain its acknowledgement stream after receiving an end signal - let mut partition_consumers: tokio::task::JoinSet<(TopicPartition, PartitionConsumerStatus)> = Default::default(); + let mut partition_consumers: JoinSet<(TopicPartition, PartitionConsumerStatus)> = Default::default(); // Handles that will let us end any consumer task that exceeds a drain deadline let mut abort_handles: HashMap = HashMap::new(); @@ -614,17 +618,20 @@ async fn coordinate_kafka_callbacks( tokio::select! { Some(Ok((finished_partition, status))) = partition_consumers.join_next(), if !partition_consumers.is_empty() => { debug!("Partition consumer finished for {}:{}", &finished_partition.0, finished_partition.1); + // If this task ended on its own, the end_signal for it will still be in here. + end_signals.remove(&finished_partition); abort_handles.remove(&finished_partition); partition_state.observed_last_ack(finished_partition); // Signal the client task that at least one partition has completed _ = drain_signal.as_ref().map(|sig| _ = sig.send(()) ); - if partition_state.is_drain_complete() { + if drain_signal.is_some() && partition_state.is_drain_complete() { debug!("All expected partitions have drained."); partition_state.clear(); // Signal the client task that all partitions that are expected to drain have completed drain_signal.take(); + drain_deadline = None.into(); } match status { @@ -657,7 +664,7 @@ async fn coordinate_kafka_callbacks( drop(done); }, KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => { - drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); + drain_deadline = Some(Box::pin(tokio::time::sleep(max_drain_ms))).into(); for tp in revoked_partitions.drain(0..) { if let Some(end) = end_signals.remove(&tp) { @@ -690,7 +697,7 @@ async fn coordinate_kafka_callbacks( }); } - drain_deadline = tokio::spawn(revoke_timeout(max_drain_ms)); + drain_deadline = Some(Box::pin(tokio::time::sleep(max_drain_ms))).into(); if partition_state.is_drain_complete() { partition_state.clear(); drop(drain); @@ -702,7 +709,7 @@ async fn coordinate_kafka_callbacks( }, }, - _ = &mut drain_deadline, if drain_signal.is_some() => { + Some(_) = &mut drain_deadline => { debug!("Acknowledgement drain deadline reached. Dropping any pending ack streams for revoked partitions."); for tp in partition_state.expect_drain.drain() { if let Some(handle) = abort_handles.remove(&tp) { @@ -710,9 +717,11 @@ async fn coordinate_kafka_callbacks( } } partition_state.clear(); - - if let Err(e) = drain_signal.take().unwrap().send(()) { - warn!("Error sending to drain signal: {}.", e); + drain_deadline = None.into(); + if let Some(s) = drain_signal.take() { + if let Err(e) = s.send(()) { + warn!("Error sending to drain signal: {}.", e); + } } }, @@ -727,9 +736,10 @@ async fn coordinate_kafka_callbacks( fn drive_kafka_consumer( consumer: Arc>, mut shutdown: ShutdownSignal, - mut eof: tokio::sync::oneshot::Receiver<()>, + eof: Option>, ) { Handle::current().block_on(async move { + let mut eof: OptionFuture<_> = eof.into(); let mut stream = consumer.stream(); loop { tokio::select! { @@ -738,7 +748,7 @@ fn drive_kafka_consumer( break }, - _ = &mut eof => { + Some(_) = &mut eof => { consumer.context().shutdown(); break }, From 0b6d17e35bbf4054164f027575a06224a1de0164 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 7 Sep 2023 10:29:15 -0400 Subject: [PATCH 15/37] replace OnceCell with OnceLock --- src/sources/kafka.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index b4a89ffbc85c4..b4397234b91a5 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -3,7 +3,7 @@ use std::{ io::Cursor, sync::{ mpsc::{sync_channel, SyncSender}, - Arc, Weak, + Arc, OnceLock, Weak, }, time::Duration, }; @@ -18,7 +18,6 @@ use codecs::{ use futures::{Stream, StreamExt}; use futures_util::future::OptionFuture; use lookup::{lookup_v2::OptionalValuePath, owned_value_path, path, OwnedValuePath}; -use once_cell::sync::OnceCell; use rdkafka::{ consumer::{ stream_consumer::StreamPartitionQueue, CommitMode, Consumer, ConsumerContext, Rebalance, @@ -1076,10 +1075,10 @@ struct KafkaSourceContext { stats: kafka::KafkaStatisticsContext, /// A callback channel used to coordinate between the main consumer task and the acknowledgement task - callbacks: OnceCell>, + callbacks: OnceLock>, /// A weak reference to the consumer, so that we can commit offsets during a rebalance operation - consumer: OnceCell>>, + consumer: OnceLock>>, } impl KafkaSourceContext { @@ -1087,8 +1086,8 @@ impl KafkaSourceContext { Self { stats: kafka::KafkaStatisticsContext { expose_lag_metrics }, acknowledgements, - callbacks: OnceCell::default(), - consumer: OnceCell::default(), + callbacks: OnceLock::default(), + consumer: OnceLock::default(), } } From 68654702f4db781df806e9bb0c81936e7286600d Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 7 Sep 2023 11:33:05 -0400 Subject: [PATCH 16/37] cargo fmt --- src/sources/kafka.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index be92321c179c1..6c09853438302 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -603,7 +603,8 @@ async fn coordinate_kafka_callbacks( // is both consuming the messages (passing them to the output stream) _and_ // processing the corresponding acknowledgement stream. A consumer task // should completely drain its acknowledgement stream after receiving an end signal - let mut partition_consumers: JoinSet<(TopicPartition, PartitionConsumerStatus)> = Default::default(); + let mut partition_consumers: JoinSet<(TopicPartition, PartitionConsumerStatus)> = + Default::default(); // Handles that will let us end any consumer task that exceeds a drain deadline let mut abort_handles: HashMap = HashMap::new(); From 84dfe9216259636c60e69f11286aff3600dccb87 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 7 Sep 2023 19:06:48 -0400 Subject: [PATCH 17/37] create clear distinction between consuming and draining states --- src/sources/kafka.rs | 263 ++++++++++++++++++++++++++++--------------- 1 file changed, 171 insertions(+), 92 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 6c09853438302..f519517bbe800 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -1,6 +1,7 @@ use std::{ collections::{BTreeMap, HashMap, HashSet}, io::Cursor, + pin::Pin, sync::{ mpsc::{sync_channel, SyncSender}, Arc, OnceLock, Weak, @@ -37,6 +38,7 @@ use tokio::{ oneshot, }, task::JoinSet, + time::Sleep, }; use tokio_util::codec::FramedRead; @@ -423,13 +425,14 @@ async fn kafka_source( let span = span.clone(); let consumer = Arc::clone(&consumer); let session_timeout_ms = config.session_timeout_ms; - let partition_state = KafkaPartitionState::new(config, decoder, out, log_namespace); + let consumer_state = + ConsumerStateInner::::new(config, decoder, out, log_namespace); tokio::spawn(async move { let _enter = span.enter(); coordinate_kafka_callbacks( consumer, callback_rx, - partition_state, + consumer_state, session_timeout_ms, eof_tx, ) @@ -454,11 +457,16 @@ async fn kafka_source( /// KafkaPartitionState holds all the pieces that are needed to consume a kafka /// partition stream, and track the state needed in order to correctly manage /// rebalance events -struct KafkaPartitionState { +struct ConsumerStateInner { config: KafkaSourceConfig, decoder: Decoder, out: SourceSender, log_namespace: LogNamespace, + consumer_state: S, +} +struct Consuming; +struct Draining { + signal: SyncSender<()>, /// The Set of partitions expected to drain during a shutdown or rebalance that revokes partitions expect_drain: HashSet, @@ -468,9 +476,26 @@ struct KafkaPartitionState { /// and "expect to complete" (based on seeing a rebalance callback with revoked partition info) are tracked separately. observed_drain: HashSet, } +enum ConsumerState { + Consuming(ConsumerStateInner), + Draining(ConsumerStateInner), +} +impl Draining { + fn new(signal: SyncSender<()>) -> Self { + Self { + signal, + expect_drain: HashSet::new(), + observed_drain: HashSet::new(), + } + } + + fn is_complete(&self) -> bool { + self.expect_drain.is_subset(&self.observed_drain) + } +} -impl KafkaPartitionState { - fn new( +impl ConsumerStateInner { + const fn new( config: KafkaSourceConfig, decoder: Decoder, out: SourceSender, @@ -481,8 +506,7 @@ impl KafkaPartitionState { decoder, out, log_namespace, - expect_drain: HashSet::new(), - observed_drain: HashSet::new(), + consumer_state: Consuming, } } @@ -552,11 +576,11 @@ impl KafkaPartitionState { } } None if finalizer.is_none() => { - debug!("acknowledgement stream complete for partition {}:{}.", &tp.0, tp.1); + debug!("Acknowledgement stream complete for partition {}:{}.", &tp.0, tp.1); break } None => { - debug!("acknowledgement stream empty for {}:{}", &tp.0, tp.1); + debug!("Acknowledgement stream empty for {}:{}", &tp.0, tp.1); } } ) @@ -566,21 +590,57 @@ impl KafkaPartitionState { (end_tx, handle) } + /// Consume self, and return a "Draining" ConsumerState, along with a Future + /// representing a drain deadline, based on max_drain_ms + pub fn begin_drain( + self, + max_drain_ms: Duration, + sig: SyncSender<()>, + ) -> (Pin>, ConsumerStateInner) { + let deadline = Box::pin(tokio::time::sleep(max_drain_ms)); + + let draining = ConsumerStateInner { + config: self.config, + decoder: self.decoder, + out: self.out, + log_namespace: self.log_namespace, + consumer_state: Draining::new(sig), + }; + + (deadline, draining) + } +} + +impl ConsumerStateInner { + /// Mark the given TopicPartition as being revoked, adding it to the set of + /// partitions expected to drain pub fn revoke_partition(&mut self, tp: TopicPartition) { - self.expect_drain.insert(tp); + self.consumer_state.expect_drain.insert(tp); } - pub fn observed_last_ack(&mut self, tp: TopicPartition) { - self.observed_drain.insert(tp); + /// Add the given TopicPartition to the set of known "drained" partitions, + /// i.e. the consumer has drained the acknowledgement channel. A signal is + /// sent on the signal channel, indicating to the client that offsets may be committed + pub fn partition_drained(&mut self, tp: TopicPartition) { + _ = self.consumer_state.signal.send(()); + self.consumer_state.observed_drain.insert(tp); } + /// Return true if the set of expected drained partitions is a subset of the + /// partitions that have been observed to be finished pub fn is_drain_complete(&self) -> bool { - self.expect_drain.is_subset(&self.observed_drain) + self.consumer_state.is_complete() } - pub fn clear(&mut self) { - for item in self.expect_drain.drain() { - self.observed_drain.remove(&item); + /// Finish partition drain mode. Consumes self and the drain deadline + /// future, and returns a "Consuming" ConsumerState + pub fn finish(self, _deadline: OptionFuture>>) -> ConsumerStateInner { + ConsumerStateInner { + config: self.config, + decoder: self.decoder, + out: self.out, + log_namespace: self.log_namespace, + consumer_state: Consuming, } } } @@ -588,12 +648,12 @@ impl KafkaPartitionState { async fn coordinate_kafka_callbacks( consumer: Arc>, mut callbacks: UnboundedReceiver, - mut partition_state: KafkaPartitionState, + consumer_state: ConsumerStateInner, max_drain_ms: Duration, mut eof: Option>, ) { - let mut drain_signal: Option> = None; let mut drain_deadline: OptionFuture<_> = None.into(); + let mut consumer_state = ConsumerState::Consuming(consumer_state); // A oneshot channel is used for each consumed partition, so that we can // signal to that task to stop consuming, drain pending acks, and exit @@ -618,18 +678,28 @@ async fn coordinate_kafka_callbacks( // If this task ended on its own, the end_signal for it will still be in here. end_signals.remove(&finished_partition); abort_handles.remove(&finished_partition); - partition_state.observed_last_ack(finished_partition); - - // Signal the client task that at least one partition has completed - _ = drain_signal.as_ref().map(|sig| _ = sig.send(()) ); - if drain_signal.is_some() && partition_state.is_drain_complete() { - debug!("All expected partitions have drained."); - partition_state.clear(); - // Signal the client task that all partitions that are expected to drain have completed - drain_signal.take(); - drain_deadline = None.into(); - } + consumer_state = match consumer_state { + ConsumerState::Consuming(s) => { + // If we are here, it is likely because the consumer tasks are set up to exit upon reaching the end of the partition + if !exit_eof { + warn!("Partition consumer task finished, while not in draining mode."); + } + ConsumerState::Consuming(s) + }, + ConsumerState::Draining(mut state) => { + state.partition_drained(finished_partition); + + if state.is_drain_complete() { + debug!("All expected partitions have drained."); + let state = state.finish(drain_deadline); + drain_deadline = None.into(); + ConsumerState::Consuming(state) + } else { + ConsumerState::Draining(state) + } + } + }; match status { // PartitionConsumerStatus differentiates between a task that exited after @@ -644,81 +714,90 @@ async fn coordinate_kafka_callbacks( } }, Some(callback) = callbacks.recv() => match callback { - KafkaCallback::PartitionsAssigned(mut assigned_partitions, done) => { - let acks = consumer.context().acknowledgements; - for tp in assigned_partitions.drain(0..) { - let topic = tp.0.as_str(); - let partition = tp.1; - // It _should_ be impossible for this expect() to panic, since we receive the topic/partition pair from the rebalance callback - let pq = consumer.split_partition_queue(topic, partition).expect("Failed to get partition queue: invalid topic or partition."); - - debug!("Consuming partition {}:{}.", &tp.0, tp.1); - let (end_tx, handle) = partition_state.consume_partition(&mut partition_consumers, tp.clone(), Arc::clone(&consumer), pq, acks, exit_eof); - abort_handles.insert(tp.clone(), handle); - end_signals.insert(tp, end_tx); - } - // ensure this is retained until all individual queues are set up - drop(done); - }, - KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => { - drain_deadline = Some(Box::pin(tokio::time::sleep(max_drain_ms))).into(); - - for tp in revoked_partitions.drain(0..) { - if let Some(end) = end_signals.remove(&tp) { - let _ = end.send(()); + KafkaCallback::PartitionsAssigned(mut assigned_partitions, done) => match consumer_state { + ConsumerState::Draining(_) => error!("Kafka client is draining revoked partitions, invalid assignment?"), + ConsumerState::Consuming(ref consumer_state) => { + let acks = consumer.context().acknowledgements; + for tp in assigned_partitions.drain(0..) { + let topic = tp.0.as_str(); + let partition = tp.1; + // It _should_ be impossible for this expect() to panic, since we receive the topic/partition pair from the rebalance callback + let pq = consumer.split_partition_queue(topic, partition).expect("Failed to get partition queue: invalid topic or partition."); + + debug!("Consuming partition {}:{}.", &tp.0, tp.1); + let (end_tx, handle) = consumer_state.consume_partition(&mut partition_consumers, tp.clone(), Arc::clone(&consumer), pq, acks, exit_eof); + abort_handles.insert(tp.clone(), handle); + end_signals.insert(tp, end_tx); } - debug!("Revoking partition {}:{}", &tp.0, tp.1); - partition_state.revoke_partition(tp); - } - - if partition_state.is_drain_complete() { - partition_state.clear(); - drop(drain); - } else if drain_signal.replace(drain).is_some() { - unreachable!("Concurrent rebalance callbacks should not be possible."); + // ensure this is retained until all individual queues are set up + drop(done); } }, - KafkaCallback::ShuttingDown(drain) => { - // Shutting down is just like a full assignment revoke, but we also close the - // callback channels, since we don't expect additional assignments or rebalances - if let Ok(tpl) = consumer.assignment() { - tpl.elements() - .iter() - .for_each(|el| { - - let tp: TopicPartition = (el.topic().into(), el.partition()); + KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => consumer_state = match consumer_state { + ConsumerState::Draining(d) => { + error!("Kafka client is already draining revoked partitions."); + ConsumerState::Draining(d) + }, + ConsumerState::Consuming(state) => { + let (deadline, mut state) = state.begin_drain(max_drain_ms, drain); + drain_deadline = Some(deadline).into(); + + for tp in revoked_partitions.drain(0..) { if let Some(end) = end_signals.remove(&tp) { let _ = end.send(()); } - partition_state.revoke_partition(tp); - }); - } + debug!("Revoking partition {}:{}", &tp.0, tp.1); + state.revoke_partition(tp); + } - drain_deadline = Some(Box::pin(tokio::time::sleep(max_drain_ms))).into(); - if partition_state.is_drain_complete() { - partition_state.clear(); - drop(drain); - } else if drain_signal.replace(drain).is_some() { - debug!("Kafka consumer shutting down mid-rebalance."); + ConsumerState::Draining(state) + } + }, + KafkaCallback::ShuttingDown(drain) => consumer_state = match consumer_state { + ConsumerState::Draining(d) => { + error!("Kafka client handled a shutdown signal while a rebalance was in progress."); + ConsumerState::Draining(d) + }, + ConsumerState::Consuming(state) => { + // Shutting down is just like a full assignment revoke, but we also close the + // callback channels, since we don't expect additional assignments or rebalances + let (deadline, mut state) = state.begin_drain(max_drain_ms, drain); + drain_deadline = Some(deadline).into(); + + if let Ok(tpl) = consumer.assignment() { + tpl.elements() + .iter() + .for_each(|el| { + + let tp: TopicPartition = (el.topic().into(), el.partition()); + if let Some(end) = end_signals.remove(&tp) { + let _ = end.send(()); + } + state.revoke_partition(tp); + }); + } + callbacks.close(); + ConsumerState::Draining(state) } - - callbacks.close(); }, }, - Some(_) = &mut drain_deadline => { - debug!("Acknowledgement drain deadline reached. Dropping any pending ack streams for revoked partitions."); - for tp in partition_state.expect_drain.drain() { - if let Some(handle) = abort_handles.remove(&tp) { - handle.abort(); - } - } - partition_state.clear(); - drain_deadline = None.into(); - if let Some(s) = drain_signal.take() { - if let Err(e) = s.send(()) { - warn!("Error sending to drain signal: {}.", e); + Some(_) = &mut drain_deadline => consumer_state = match consumer_state { + ConsumerState::Consuming(s) => { + warn!("A drain deadline fired outside of draining mode."); + drain_deadline = None.into(); + ConsumerState::Consuming(s) + }, + ConsumerState::Draining(mut draining) => { + debug!("Acknowledgement drain deadline reached. Dropping any pending ack streams for revoked partitions."); + for tp in draining.consumer_state.expect_drain.drain() { + if let Some(handle) = abort_handles.remove(&tp) { + handle.abort(); + } } + let state = draining.finish(drain_deadline); + drain_deadline = None.into(); + ConsumerState::Consuming(state) } }, From d8238200e24bc8458691dec64606d145b76f5f7c Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 7 Sep 2023 22:56:57 -0400 Subject: [PATCH 18/37] add "complete" as a terminal state, and "keep_consuming", "keep_draining", and "complete" methods for descriptive state (non)transitions --- src/sources/kafka.rs | 161 ++++++++++++++++++++++++++++--------------- 1 file changed, 106 insertions(+), 55 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index f519517bbe800..c8e6975a52958 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -465,6 +465,7 @@ struct ConsumerStateInner { consumer_state: S, } struct Consuming; +struct Complete; struct Draining { signal: SyncSender<()>, @@ -475,15 +476,21 @@ struct Draining { /// can complete before we get a rebalance callback, so "observed complete" (based on seeing the end of the stream) /// and "expect to complete" (based on seeing a rebalance callback with revoked partition info) are tracked separately. observed_drain: HashSet, + + /// Whether the client is shutting down after draining + shutdown: bool, } +type OptionDeadline = OptionFuture>>; enum ConsumerState { Consuming(ConsumerStateInner), Draining(ConsumerStateInner), + Complete(ConsumerStateInner), } impl Draining { - fn new(signal: SyncSender<()>) -> Self { + fn new(signal: SyncSender<()>, shutdown: bool) -> Self { Self { signal, + shutdown, expect_drain: HashSet::new(), observed_drain: HashSet::new(), } @@ -494,6 +501,21 @@ impl Draining { } } +impl ConsumerStateInner { + pub fn complete(self, _deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) { + ( + None.into(), + ConsumerState::Complete(ConsumerStateInner { + config: self.config, + decoder: self.decoder, + out: self.out, + log_namespace: self.log_namespace, + consumer_state: Complete, + }), + ) + } +} + impl ConsumerStateInner { const fn new( config: KafkaSourceConfig, @@ -596,7 +618,8 @@ impl ConsumerStateInner { self, max_drain_ms: Duration, sig: SyncSender<()>, - ) -> (Pin>, ConsumerStateInner) { + shutdown: bool, + ) -> (OptionDeadline, ConsumerStateInner) { let deadline = Box::pin(tokio::time::sleep(max_drain_ms)); let draining = ConsumerStateInner { @@ -604,10 +627,14 @@ impl ConsumerStateInner { decoder: self.decoder, out: self.out, log_namespace: self.log_namespace, - consumer_state: Draining::new(sig), + consumer_state: Draining::new(sig, shutdown), }; - (deadline, draining) + (Some(deadline).into(), draining) + } + + pub const fn keep_consuming(self, deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) { + (deadline, ConsumerState::Consuming(self)) } } @@ -633,16 +660,27 @@ impl ConsumerStateInner { } /// Finish partition drain mode. Consumes self and the drain deadline - /// future, and returns a "Consuming" ConsumerState - pub fn finish(self, _deadline: OptionFuture>>) -> ConsumerStateInner { - ConsumerStateInner { - config: self.config, - decoder: self.decoder, - out: self.out, - log_namespace: self.log_namespace, - consumer_state: Consuming, + /// future, and returns a "Consuming" or "Complete" ConsumerState + pub fn finish_drain(self, deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) { + if self.consumer_state.shutdown { + self.complete(deadline) + } else { + ( + None.into(), + ConsumerState::Consuming(ConsumerStateInner { + config: self.config, + decoder: self.decoder, + out: self.out, + log_namespace: self.log_namespace, + consumer_state: Consuming, + }), + ) } } + + pub const fn keep_draining(self, deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) { + (deadline, ConsumerState::Draining(self)) + } } async fn coordinate_kafka_callbacks( @@ -679,26 +717,27 @@ async fn coordinate_kafka_callbacks( end_signals.remove(&finished_partition); abort_handles.remove(&finished_partition); - consumer_state = match consumer_state { - ConsumerState::Consuming(s) => { - // If we are here, it is likely because the consumer tasks are set up to exit upon reaching the end of the partition - if !exit_eof { - warn!("Partition consumer task finished, while not in draining mode."); - } - ConsumerState::Consuming(s) - }, + (drain_deadline, consumer_state) = match consumer_state { + ConsumerState::Complete(_) => unreachable!("Partition consumer finished after completion."), ConsumerState::Draining(mut state) => { state.partition_drained(finished_partition); if state.is_drain_complete() { debug!("All expected partitions have drained."); - let state = state.finish(drain_deadline); - drain_deadline = None.into(); - ConsumerState::Consuming(state) + state.finish_drain(drain_deadline) } else { - ConsumerState::Draining(state) + state.keep_draining(drain_deadline) } - } + }, + ConsumerState::Consuming(state) => { + // If we are here, it is likely because the consumer + // tasks are set up to exit upon reaching the end of the + // partition. + if !exit_eof { + debug!("Partition consumer task finished, while not in draining mode."); + } + state.keep_consuming(drain_deadline) + }, }; match status { @@ -715,6 +754,7 @@ async fn coordinate_kafka_callbacks( }, Some(callback) = callbacks.recv() => match callback { KafkaCallback::PartitionsAssigned(mut assigned_partitions, done) => match consumer_state { + ConsumerState::Complete(_) => unreachable!("Partition assignment received after completion."), ConsumerState::Draining(_) => error!("Kafka client is draining revoked partitions, invalid assignment?"), ConsumerState::Consuming(ref consumer_state) => { let acks = consumer.context().acknowledgements; @@ -734,13 +774,17 @@ async fn coordinate_kafka_callbacks( } }, KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => consumer_state = match consumer_state { + ConsumerState::Complete(_) => unreachable!("Partitions revoked after completion."), ConsumerState::Draining(d) => { - error!("Kafka client is already draining revoked partitions."); + // NB: This would only happen if the task driving the kafka client (i.e. rebalance handlers) + // is not handling shutdown signals, and a revoke happens during a shutdown drain; otherwise + // this is unreachable code. + warn!("Kafka client is already draining revoked partitions."); ConsumerState::Draining(d) }, ConsumerState::Consuming(state) => { - let (deadline, mut state) = state.begin_drain(max_drain_ms, drain); - drain_deadline = Some(deadline).into(); + let (deadline, mut state) = state.begin_drain(max_drain_ms, drain, false); + drain_deadline = deadline; for tp in revoked_partitions.drain(0..) { if let Some(end) = end_signals.remove(&tp) { @@ -753,40 +797,49 @@ async fn coordinate_kafka_callbacks( ConsumerState::Draining(state) } }, - KafkaCallback::ShuttingDown(drain) => consumer_state = match consumer_state { - ConsumerState::Draining(d) => { + KafkaCallback::ShuttingDown(drain) => (drain_deadline, consumer_state) = match consumer_state { + ConsumerState::Complete(_) => unreachable!("Shutdown received after completion."), + // Shutting down is just like a full assignment revoke, but we also close the + // callback channels, since we don't expect additional assignments or rebalances + ConsumerState::Draining(state) => { + // NB: This would only happen if the task driving the kafka client is + // not handling shutdown signals; otherwise this is unreachable code error!("Kafka client handled a shutdown signal while a rebalance was in progress."); - ConsumerState::Draining(d) + callbacks.close(); + state.keep_draining(drain_deadline) }, ConsumerState::Consuming(state) => { - // Shutting down is just like a full assignment revoke, but we also close the - // callback channels, since we don't expect additional assignments or rebalances - let (deadline, mut state) = state.begin_drain(max_drain_ms, drain); - drain_deadline = Some(deadline).into(); - - if let Ok(tpl) = consumer.assignment() { - tpl.elements() - .iter() - .for_each(|el| { - - let tp: TopicPartition = (el.topic().into(), el.partition()); - if let Some(end) = end_signals.remove(&tp) { - let _ = end.send(()); - } - state.revoke_partition(tp); - }); - } callbacks.close(); - ConsumerState::Draining(state) + if partition_consumers.is_empty() { + // Skip the draining phase if this shutdown was initiated by all tasks reaching PartitionEOF + debug!("All consumer tasks have already finished (reached EOF); shutting down."); + state.complete(drain_deadline) + } else { + debug!("Signaling {} consumer tasks to shut down.", partition_consumers.len()); + let (deadline, mut state) = state.begin_drain(max_drain_ms, drain, true); + if let Ok(tpl) = consumer.assignment() { + tpl.elements() + .iter() + .for_each(|el| { + + let tp: TopicPartition = (el.topic().into(), el.partition()); + if let Some(end) = end_signals.remove(&tp) { + let _ = end.send(()); + } + state.revoke_partition(tp); + }); + } + (deadline, ConsumerState::Draining(state)) + } } }, }, - Some(_) = &mut drain_deadline => consumer_state = match consumer_state { + Some(_) = &mut drain_deadline => (drain_deadline, consumer_state) = match consumer_state { + ConsumerState::Complete(_) => unreachable!("Drain deadline received after completion."), ConsumerState::Consuming(s) => { warn!("A drain deadline fired outside of draining mode."); - drain_deadline = None.into(); - ConsumerState::Consuming(s) + (None.into(), ConsumerState::Consuming(s)) }, ConsumerState::Draining(mut draining) => { debug!("Acknowledgement drain deadline reached. Dropping any pending ack streams for revoked partitions."); @@ -795,9 +848,7 @@ async fn coordinate_kafka_callbacks( handle.abort(); } } - let state = draining.finish(drain_deadline); - drain_deadline = None.into(); - ConsumerState::Consuming(state) + draining.finish_drain(drain_deadline) } }, From c6082f37ab9c59453b4ecdb29d7e826d33024314 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Fri, 8 Sep 2023 10:30:52 -0400 Subject: [PATCH 19/37] use state transition methods consistently for all state transitions --- src/sources/kafka.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index c8e6975a52958..dab39be95ddbb 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -773,18 +773,17 @@ async fn coordinate_kafka_callbacks( drop(done); } }, - KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => consumer_state = match consumer_state { + KafkaCallback::PartitionsRevoked(mut revoked_partitions, drain) => (drain_deadline, consumer_state) = match consumer_state { ConsumerState::Complete(_) => unreachable!("Partitions revoked after completion."), ConsumerState::Draining(d) => { // NB: This would only happen if the task driving the kafka client (i.e. rebalance handlers) // is not handling shutdown signals, and a revoke happens during a shutdown drain; otherwise // this is unreachable code. warn!("Kafka client is already draining revoked partitions."); - ConsumerState::Draining(d) + d.keep_draining(drain_deadline) }, ConsumerState::Consuming(state) => { let (deadline, mut state) = state.begin_drain(max_drain_ms, drain, false); - drain_deadline = deadline; for tp in revoked_partitions.drain(0..) { if let Some(end) = end_signals.remove(&tp) { @@ -794,7 +793,7 @@ async fn coordinate_kafka_callbacks( state.revoke_partition(tp); } - ConsumerState::Draining(state) + state.keep_draining(deadline) } }, KafkaCallback::ShuttingDown(drain) => (drain_deadline, consumer_state) = match consumer_state { @@ -829,7 +828,7 @@ async fn coordinate_kafka_callbacks( state.revoke_partition(tp); }); } - (deadline, ConsumerState::Draining(state)) + state.keep_draining(deadline) } } }, From 444b3e1660868f30b8340b259b1e48bd639c284d Mon Sep 17 00:00:00 2001 From: John Chesley Date: Fri, 8 Sep 2023 10:31:42 -0400 Subject: [PATCH 20/37] slightly clearer assertion messages about what is expected --- src/sources/kafka.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index dab39be95ddbb..c5155c28ba9aa 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -1957,9 +1957,9 @@ mod integration_test { assert_ne!( events1.len(), 0, - "First batch of events is non-zero (increase KAFKA_SHUTDOWN_DELAY?)" + "First batch of events should be non-zero (increase KAFKA_SHUTDOWN_DELAY?)" ); - assert_ne!(events2.len(), 0, "Second batch of events is non-zero (decrease KAFKA_SHUTDOWN_DELAY or increase KAFKA_SEND_COUNT?) "); + assert_ne!(events2.len(), 0, "Second batch of events should be non-zero (decrease KAFKA_SHUTDOWN_DELAY or increase KAFKA_SEND_COUNT?) "); assert_eq!(total, expect_count); } @@ -2070,17 +2070,17 @@ mod integration_test { assert_ne!( events2.len(), 0, - "Second batch of events is non-zero (decrease delay or increase KAFKA_SEND_COUNT?) " + "Second batch of events should be non-zero (decrease delay or increase KAFKA_SEND_COUNT?) " ); assert_ne!( events3.len(), 0, - "Third batch of events is non-zero (decrease delay or increase KAFKA_SEND_COUNT?) " + "Third batch of events should be non-zero (decrease delay or increase KAFKA_SEND_COUNT?) " ); assert_eq!( unconsumed.len(), 0, - "The first set of consumer should consume and ack all messages." + "The first set of consumers should consume and ack all messages." ); assert_eq!(total, expect_count); } From 88cd2e5068492df80b95309bdb906e1b3a8b14bd Mon Sep 17 00:00:00 2001 From: John Chesley Date: Fri, 8 Sep 2023 10:56:42 -0400 Subject: [PATCH 21/37] update obsolete comment, make coordinator loop condition explicit --- src/sources/kafka.rs | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index c5155c28ba9aa..7867259f638e8 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -454,9 +454,23 @@ async fn kafka_source( Ok(()) } -/// KafkaPartitionState holds all the pieces that are needed to consume a kafka -/// partition stream, and track the state needed in order to correctly manage -/// rebalance events +/// ConsumerStateInner implements a small struct/enum-based state machine. +/// +/// With a ConsumerStateInner, the client is able to spawn new tasks +/// when partitions are assigned. When a shutdown signal is received, or +/// partitions are being revoked, the Consuming state is traded for a Draining +/// state (and associated drain deadline future) via the `begin_drain` method +/// +/// A ConsumerStateInner keeps track of partitions that are expected +/// to complete, and also owns the signal that, when dropped, indicates to the +/// client driver task that it is safe to proceed with the rebalance or shutdown. +/// When draining is complete, or the deadline is reached, Draining is traded in for +/// either a Consuming (after a revoke) or Complete (in the case of shutdown) state, +/// via the `finish_drain` method. +/// +/// A ConsumerStateInner is the final state, reached after a shutdown +/// signal is received. This can not be traded for another state, and the +/// coordination task should exit when this state is reached. struct ConsumerStateInner { config: KafkaSourceConfig, decoder: Decoder, @@ -709,7 +723,7 @@ async fn coordinate_kafka_callbacks( let exit_eof = eof.is_some(); - loop { + while let ConsumerState::Consuming(_) | ConsumerState::Draining(_) = consumer_state { tokio::select! { Some(Ok((finished_partition, status))) = partition_consumers.join_next(), if !partition_consumers.is_empty() => { debug!("Partition consumer finished for {}:{}", &finished_partition.0, finished_partition.1); From db999e28000ad3f7b7c072c66bc32b635c3fe2e5 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Fri, 8 Sep 2023 11:59:46 -0400 Subject: [PATCH 22/37] use keep_consuming from the drain_timeout while consuming handler --- src/sources/kafka.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 7867259f638e8..4279073ee44e4 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -850,9 +850,9 @@ async fn coordinate_kafka_callbacks( Some(_) = &mut drain_deadline => (drain_deadline, consumer_state) = match consumer_state { ConsumerState::Complete(_) => unreachable!("Drain deadline received after completion."), - ConsumerState::Consuming(s) => { + ConsumerState::Consuming(state) => { warn!("A drain deadline fired outside of draining mode."); - (None.into(), ConsumerState::Consuming(s)) + state.keep_consuming(None.into()) }, ConsumerState::Draining(mut draining) => { debug!("Acknowledgement drain deadline reached. Dropping any pending ack streams for revoked partitions."); From 1edb2703f6a1804ba5d8b84f26b4279abc5b6b2c Mon Sep 17 00:00:00 2001 From: John Chesley Date: Mon, 11 Sep 2023 18:08:52 -0400 Subject: [PATCH 23/37] rely solely on adding/removing entries in expect_drain to detect when draining is complete --- src/sources/kafka.rs | 36 +++++++++++++++++++----------------- 1 file changed, 19 insertions(+), 17 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 4279073ee44e4..73841b74e2a0b 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -483,14 +483,12 @@ struct Complete; struct Draining { signal: SyncSender<()>, - /// The Set of partitions expected to drain during a shutdown or rebalance that revokes partitions + /// The set of topic-partition tasks that have completed, populated at the + /// beginning of a rebalance or shutdown. Partitions that are not being + /// actively consumed (e.g. due to the consumer task exiting early) should not + /// be added. The draining phase is considered complete when this set is empty. expect_drain: HashSet, - /// The set of partitions we have observed and stored the final acknowledgement for. Ack streams - /// can complete before we get a rebalance callback, so "observed complete" (based on seeing the end of the stream) - /// and "expect to complete" (based on seeing a rebalance callback with revoked partition info) are tracked separately. - observed_drain: HashSet, - /// Whether the client is shutting down after draining shutdown: bool, } @@ -506,12 +504,11 @@ impl Draining { signal, shutdown, expect_drain: HashSet::new(), - observed_drain: HashSet::new(), } } fn is_complete(&self) -> bool { - self.expect_drain.is_subset(&self.observed_drain) + self.expect_drain.is_empty() } } @@ -655,7 +652,11 @@ impl ConsumerStateInner { impl ConsumerStateInner { /// Mark the given TopicPartition as being revoked, adding it to the set of /// partitions expected to drain - pub fn revoke_partition(&mut self, tp: TopicPartition) { + pub fn revoke_partition(&mut self, tp: TopicPartition, end_signal: oneshot::Sender<()>) { + // Note that if this send() returns Err, it means the task has already + // ended, but the completion has not been processed yet (otherwise we wouldn't have access to the end_signal), + // so we should still add it to the "expect to drain" set + _ = end_signal.send(()); self.consumer_state.expect_drain.insert(tp); } @@ -664,11 +665,10 @@ impl ConsumerStateInner { /// sent on the signal channel, indicating to the client that offsets may be committed pub fn partition_drained(&mut self, tp: TopicPartition) { _ = self.consumer_state.signal.send(()); - self.consumer_state.observed_drain.insert(tp); + self.consumer_state.expect_drain.remove(&tp); } - /// Return true if the set of expected drained partitions is a subset of the - /// partitions that have been observed to be finished + /// Return true if all expected partitions have drained pub fn is_drain_complete(&self) -> bool { self.consumer_state.is_complete() } @@ -801,10 +801,11 @@ async fn coordinate_kafka_callbacks( for tp in revoked_partitions.drain(0..) { if let Some(end) = end_signals.remove(&tp) { - let _ = end.send(()); + debug!("Revoking partition {}:{}", &tp.0, tp.1); + state.revoke_partition(tp, end); + } else { + debug!("Consumer task for partition {}:{} already finished.", &tp.0, tp.1); } - debug!("Revoking partition {}:{}", &tp.0, tp.1); - state.revoke_partition(tp); } state.keep_draining(deadline) @@ -837,9 +838,10 @@ async fn coordinate_kafka_callbacks( let tp: TopicPartition = (el.topic().into(), el.partition()); if let Some(end) = end_signals.remove(&tp) { - let _ = end.send(()); + state.revoke_partition(tp, end); + } else { + debug!("Consumer task for partition {}:{} already finished.", &tp.0, tp.1); } - state.revoke_partition(tp); }); } state.keep_draining(deadline) From d2cdeb104577f8eb015a1078658ba00001757879 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Mon, 11 Sep 2023 22:50:20 -0400 Subject: [PATCH 24/37] fix comment :P --- src/sources/kafka.rs | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 73841b74e2a0b..d1fc141741ee1 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -483,13 +483,16 @@ struct Complete; struct Draining { signal: SyncSender<()>, - /// The set of topic-partition tasks that have completed, populated at the - /// beginning of a rebalance or shutdown. Partitions that are not being - /// actively consumed (e.g. due to the consumer task exiting early) should not - /// be added. The draining phase is considered complete when this set is empty. + /// The set of topic-partition tasks that are required to complete during + /// the draining phase, populated at the beginning of a rebalance or shutdown. + /// Partitions that are being revoked, but not being actively consumed + /// (e.g. due to the consumer task exiting early) should not be included. + /// The draining phase is considered complete when this set is empty. expect_drain: HashSet, - /// Whether the client is shutting down after draining + /// Whether the client is shutting down after draining. If set to true, + /// the `finish_drain` method will return a Complete state, otherwise + /// a Consuming state. shutdown: bool, } type OptionDeadline = OptionFuture>>; From e4e57b9b9f15c54686cc76ce5171610a40fdff83 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Tue, 12 Sep 2023 11:19:31 -0400 Subject: [PATCH 25/37] clippy/fmt fixes --- src/sources/kafka.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index d1fc141741ee1..a78b51a59883b 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -1374,7 +1374,7 @@ mod test { auto_offset_reset: "beginning".into(), session_timeout_ms: Duration::from_millis(6000), commit_interval_ms: Duration::from_millis(1), - librdkafka_options: librdkafka_options, + librdkafka_options, key_field: default_key_field(), topic_key: default_topic_key(), partition_key: default_partition_key(), @@ -1525,7 +1525,7 @@ mod integration_test { fn kafka_test_topic() -> String { std::env::var("KAFKA_TEST_TOPIC") - .unwrap_or_else(|_| format!("test-topic-{}", random_string(10)).into()) + .unwrap_or_else(|_| format!("test-topic-{}", random_string(10))) } fn kafka_max_bytes() -> String { std::env::var("KAFKA_MAX_BYTES").unwrap_or_else(|_| "1024".into()) @@ -1917,7 +1917,7 @@ mod integration_test { .parse() .expect("Number of messages to send to kafka."); let expect_count: usize = std::env::var("KAFKA_EXPECT_COUNT") - .unwrap_or_else(|_| format!("{}", send_count).into()) + .unwrap_or_else(|_| format!("{}", send_count)) .parse() .expect("Number of messages to expect consumers to process."); let delay_ms: u64 = std::env::var("KAFKA_SHUTDOWN_DELAY") @@ -1932,7 +1932,7 @@ mod integration_test { let mut opts = HashMap::new(); // Set options to get partition EOF notifications, and fetch data in small/configurable size chunks opts.insert("enable.partition.eof".into(), "true".into()); - opts.insert("fetch.message.max.bytes".into(), kafka_max_bytes().into()); + opts.insert("fetch.message.max.bytes".into(), kafka_max_bytes()); let events1 = { let config = make_config(&topic, &group_id, LogNamespace::Legacy, Some(opts.clone())); let (tx, rx) = SourceSender::new_test_errors(|_| false); @@ -1989,7 +1989,7 @@ mod integration_test { .parse() .expect("Number of messages to send to kafka."); let expect_count: usize = std::env::var("KAFKA_EXPECT_COUNT") - .unwrap_or_else(|_| format!("{}", send_count).into()) + .unwrap_or_else(|_| format!("{}", send_count)) .parse() .expect("Number of messages to expect consumers to process."); let delay_ms: u64 = std::env::var("KAFKA_CONSUMER_DELAY") @@ -2005,7 +2005,7 @@ mod integration_test { // 3. Start 2nd & 3rd consumers using the same group.id, triggering rebalance events let mut kafka_options = HashMap::new(); kafka_options.insert("enable.partition.eof".into(), "true".into()); - kafka_options.insert("fetch.message.max.bytes".into(), kafka_max_bytes().into()); + kafka_options.insert("fetch.message.max.bytes".into(), kafka_max_bytes()); kafka_options.insert("partition.assignment.strategy".into(), rebalance_strategy); let config1 = make_config( &topic, From 7999fa7c3afb3d7a1d0eab1655bce7f91a019972 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Wed, 13 Sep 2023 09:56:08 -0400 Subject: [PATCH 26/37] minor cleanup: during shutdown, use is_drain_complete to detect the already-finished state --- src/sources/kafka.rs | 43 +++++++++++++++++++------------------------ 1 file changed, 19 insertions(+), 24 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index a78b51a59883b..43475baa5c61f 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -827,26 +827,26 @@ async fn coordinate_kafka_callbacks( }, ConsumerState::Consuming(state) => { callbacks.close(); - if partition_consumers.is_empty() { - // Skip the draining phase if this shutdown was initiated by all tasks reaching PartitionEOF - debug!("All consumer tasks have already finished (reached EOF); shutting down."); - state.complete(drain_deadline) + let (deadline, mut state) = state.begin_drain(max_drain_ms, drain, true); + if let Ok(tpl) = consumer.assignment() { + tpl.elements() + .iter() + .for_each(|el| { + + let tp: TopicPartition = (el.topic().into(), el.partition()); + if let Some(end) = end_signals.remove(&tp) { + debug!("Shutting down and revoking partition {}:{}", &tp.0, tp.1); + state.revoke_partition(tp, end); + } else { + debug!("Consumer task for partition {}:{} already finished.", &tp.0, tp.1); + } + }); + } + // If shutdown was initiated by partition EOF mode, the drain phase + // will already be complete and would time out if not accounted for here + if state.is_drain_complete() { + state.finish_drain(deadline) } else { - debug!("Signaling {} consumer tasks to shut down.", partition_consumers.len()); - let (deadline, mut state) = state.begin_drain(max_drain_ms, drain, true); - if let Ok(tpl) = consumer.assignment() { - tpl.elements() - .iter() - .for_each(|el| { - - let tp: TopicPartition = (el.topic().into(), el.partition()); - if let Some(end) = end_signals.remove(&tp) { - state.revoke_partition(tp, end); - } else { - debug!("Consumer task for partition {}:{} already finished.", &tp.0, tp.1); - } - }); - } state.keep_draining(deadline) } } @@ -869,11 +869,6 @@ async fn coordinate_kafka_callbacks( draining.finish_drain(drain_deadline) } }, - - // Consumers are done, and callback channel is closed - else => { - break - } } } } From 78c7a0e8027a21ac06ad446844a39e07a4a545b7 Mon Sep 17 00:00:00 2001 From: j chesley <42561540+jches@users.noreply.github.com> Date: Wed, 20 Sep 2023 17:53:05 -0400 Subject: [PATCH 27/37] integration test uses `FuturesUnordered` for better performance Co-authored-by: Doug Smith --- src/sources/kafka.rs | 44 ++++++++++++++++---------------------------- 1 file changed, 16 insertions(+), 28 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 43475baa5c61f..2950e158bf1b2 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -772,7 +772,7 @@ async fn coordinate_kafka_callbacks( Some(callback) = callbacks.recv() => match callback { KafkaCallback::PartitionsAssigned(mut assigned_partitions, done) => match consumer_state { ConsumerState::Complete(_) => unreachable!("Partition assignment received after completion."), - ConsumerState::Draining(_) => error!("Kafka client is draining revoked partitions, invalid assignment?"), + ConsumerState::Draining(_) => error!("Partition assignment received while draining revoked partitions, invalid assignment?"), ConsumerState::Consuming(ref consumer_state) => { let acks = consumer.context().acknowledgements; for tp in assigned_partitions.drain(0..) { @@ -1547,7 +1547,7 @@ mod integration_test { create_topic(topic_name, partitions).await; - let writes = (0..count) + (0..count) .map(|i| async move { let text = format!("{} {:03}", TEXT, i); let key = format!("{} {}", KEY, i); @@ -1559,15 +1559,13 @@ mod integration_test { key: HEADER_KEY, value: Some(HEADER_VALUE), })); - producer.send(record, Timeout::Never).await + if let Err(error) = producer.send(record, Timeout::Never).await { + panic!("Cannot send event to Kafka: {:?}", error); + } }) - .collect::>(); - - for res in writes { - if let Err(error) = res.await { - panic!("Cannot send event to Kafka: {:?}", error); - } - } + .collect::>() + .collect::>() + .await; now } @@ -1860,7 +1858,7 @@ mod integration_test { let (tx, rx2) = delay_pipeline(2, Duration::from_millis(DELAY), EventStatus::Delivered); let (trigger_shutdown2, shutdown_done2) = - spawn_kafka(tx, config, true, true, LogNamespace::Legacy); + spawn_kafka(tx, config, true, false, LogNamespace::Legacy); let events2 = tokio::spawn(collect_n(rx2, NEVENTS)); sleep(Duration::from_secs(5)).await; @@ -1908,7 +1906,7 @@ mod integration_test { async fn drains_acknowledgements_at_shutdown() { // 1. Send N events (if running against a pre-populated kafka topic, use send_count=0 and expect_count=expected number of messages; otherwise just set send_count) let send_count: usize = std::env::var("KAFKA_SEND_COUNT") - .unwrap_or_else(|_| "100".into()) + .unwrap_or_else(|_| "250000".into()) .parse() .expect("Number of messages to send to kafka."); let expect_count: usize = std::env::var("KAFKA_EXPECT_COUNT") @@ -1916,7 +1914,7 @@ mod integration_test { .parse() .expect("Number of messages to expect consumers to process."); let delay_ms: u64 = std::env::var("KAFKA_SHUTDOWN_DELAY") - .unwrap_or_else(|_| "3000".into()) + .unwrap_or_else(|_| "2000".into()) .parse() .expect("Number of milliseconds before shutting down first consumer."); @@ -1980,7 +1978,7 @@ mod integration_test { async fn consume_with_rebalance(rebalance_strategy: String) { // 1. Send N events (if running against a pre-populated kafka topic, use send_count=0 and expect_count=expected number of messages; otherwise just set send_count) let send_count: usize = std::env::var("KAFKA_SEND_COUNT") - .unwrap_or_else(|_| "100".into()) + .unwrap_or_else(|_| "250000".into()) .parse() .expect("Number of messages to send to kafka."); let expect_count: usize = std::env::var("KAFKA_EXPECT_COUNT") @@ -1988,7 +1986,7 @@ mod integration_test { .parse() .expect("Number of messages to expect consumers to process."); let delay_ms: u64 = std::env::var("KAFKA_CONSUMER_DELAY") - .unwrap_or_else(|_| "3000".into()) + .unwrap_or_else(|_| "2000".into()) .parse() .expect("Number of milliseconds before shutting down first consumer."); @@ -2008,19 +2006,9 @@ mod integration_test { LogNamespace::Legacy, Some(kafka_options.clone()), ); - let config2 = make_config( - &topic, - &group_id, - LogNamespace::Legacy, - Some(kafka_options.clone()), - ); - let config3 = make_config( - &topic, - &group_id, - LogNamespace::Legacy, - Some(kafka_options.clone()), - ); - let config4 = make_config(&topic, &group_id, LogNamespace::Legacy, Some(kafka_options)); + let config2 = config1.clone(); + let config3 = config1.clone(); + let config4 = config1.clone(); let (events1, events2, events3) = tokio::join!( async move { From c88e68878282cec89514def22454511cc8256fca Mon Sep 17 00:00:00 2001 From: John Chesley Date: Wed, 20 Sep 2023 18:04:25 -0400 Subject: [PATCH 28/37] use FuturesUnordered --- src/sources/kafka.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 2950e158bf1b2..80404a70d79ad 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -1489,6 +1489,7 @@ mod integration_test { use chrono::{DateTime, SubsecRound, Utc}; use futures::Stream; + use futures_util::stream::FuturesUnordered; use rdkafka::{ admin::{AdminClient, AdminOptions, NewTopic, TopicReplication}, client::DefaultClientContext, From b3cef9bea6add81adbb1965d80f7f985376f1c99 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Wed, 20 Sep 2023 19:44:21 -0400 Subject: [PATCH 29/37] use 6 partitions for integration test --- src/sources/kafka.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 80404a70d79ad..83c9edc5d1a4c 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -1991,7 +1991,7 @@ mod integration_test { .parse() .expect("Number of milliseconds before shutting down first consumer."); - let (topic, group_id, _) = send_to_test_topic(2, send_count).await; + let (topic, group_id, _) = send_to_test_topic(6, send_count).await; debug!("Topic: {}", &topic); debug!("Consumer group.id: {}", &group_id); From 61784a3e0ccbbe96c12d185dca62d35118e785de Mon Sep 17 00:00:00 2001 From: John Chesley Date: Thu, 21 Sep 2023 15:14:29 -0400 Subject: [PATCH 30/37] integration test using 125k messages --- src/sources/kafka.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 83c9edc5d1a4c..4693205ae3af8 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -1907,7 +1907,7 @@ mod integration_test { async fn drains_acknowledgements_at_shutdown() { // 1. Send N events (if running against a pre-populated kafka topic, use send_count=0 and expect_count=expected number of messages; otherwise just set send_count) let send_count: usize = std::env::var("KAFKA_SEND_COUNT") - .unwrap_or_else(|_| "250000".into()) + .unwrap_or_else(|_| "125000".into()) .parse() .expect("Number of messages to send to kafka."); let expect_count: usize = std::env::var("KAFKA_EXPECT_COUNT") @@ -1979,7 +1979,7 @@ mod integration_test { async fn consume_with_rebalance(rebalance_strategy: String) { // 1. Send N events (if running against a pre-populated kafka topic, use send_count=0 and expect_count=expected number of messages; otherwise just set send_count) let send_count: usize = std::env::var("KAFKA_SEND_COUNT") - .unwrap_or_else(|_| "250000".into()) + .unwrap_or_else(|_| "125000".into()) .parse() .expect("Number of messages to send to kafka."); let expect_count: usize = std::env::var("KAFKA_EXPECT_COUNT") From 89dbaa979ec6e6a578382672075e2e9b362a36a5 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Fri, 22 Sep 2023 10:54:53 -0400 Subject: [PATCH 31/37] add drain_timeout_ms option for kafka source --- src/sources/kafka.rs | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 4693205ae3af8..5614939ab818b 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -128,6 +128,21 @@ pub struct KafkaSourceConfig { #[configurable(metadata(docs::human_name = "Session Timeout"))] session_timeout_ms: Duration, + /// Timeout to drain pending acknowledgements during shutdown or a Kafka + /// consumer group rebalance. + /// + /// When Vector shuts down or the Kafka consumer group revokes partitions from this + /// consumer, wait a maximum of `drain_timeout_ms` for the source to + /// process pending acknowledgements. Must be less than `session_timeout_ms` + /// to ensure the consumer is not excluded from the group during a rebalance. + /// + /// Default value is half of `session_timeout_ms`. + #[serde(skip_serializing_if = "Option::is_none")] + #[configurable(metadata(docs::examples = 2500, docs::examples = 5000))] + #[configurable(metadata(docs::advanced))] + #[configurable(metadata(docs::human_name = "Drain Timeout"))] + drain_timeout_ms: Option, + /// Timeout for network requests. #[serde_as(as = "serde_with::DurationMilliSeconds")] #[configurable(metadata(docs::examples = 30000, docs::examples = 60000))] @@ -424,7 +439,10 @@ async fn kafka_source( let coordination_task = { let span = span.clone(); let consumer = Arc::clone(&consumer); - let session_timeout_ms = config.session_timeout_ms; + let drain_timeout_ms = match config.drain_timeout_ms { + Some(d) => Duration::from_millis(d), + None => config.session_timeout_ms / 2, + }; let consumer_state = ConsumerStateInner::::new(config, decoder, out, log_namespace); tokio::spawn(async move { @@ -433,7 +451,7 @@ async fn kafka_source( consumer, callback_rx, consumer_state, - session_timeout_ms, + drain_timeout_ms, eof_tx, ) .await; From bb4b544f30c6f130e63313ee6534e44a874c0b53 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Fri, 22 Sep 2023 12:23:04 -0400 Subject: [PATCH 32/37] enforce drain_timeout_ms < session_timeout_ms when building kafka source --- src/sources/kafka.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 5614939ab818b..e7c3c291bc78d 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -69,6 +69,8 @@ use crate::{ #[derive(Debug, Snafu)] enum BuildError { + #[snafu(display("The drain_timeout_ms ({}) must be less than session_timeout_ms ({})", value, session_timeout_ms.as_millis()))] + KafkaInvalidDrainTimeoutError { value: u64, session_timeout_ms: Duration }, #[snafu(display("Could not create Kafka consumer: {}", source))] KafkaCreateError { source: rdkafka::error::KafkaError }, #[snafu(display("Could not subscribe to Kafka topics: {}", source))] @@ -328,6 +330,10 @@ impl SourceConfig for KafkaSourceConfig { .build()?; let acknowledgements = cx.do_acknowledgements(self.acknowledgements); + if let Some(d) = self.drain_timeout_ms { + snafu::ensure!(Duration::from_millis(d) <= self.session_timeout_ms, KafkaInvalidDrainTimeoutSnafu { value:d, session_timeout_ms: self.session_timeout_ms }); + } + let consumer = create_consumer(self, acknowledgements)?; Ok(Box::pin(kafka_source( From afb7fa1f89c08e2c578854b7537d6c1f9dd7e508 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Fri, 22 Sep 2023 15:35:58 -0400 Subject: [PATCH 33/37] generate component docs --- .../reference/components/sources/base/kafka.cue | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/website/cue/reference/components/sources/base/kafka.cue b/website/cue/reference/components/sources/base/kafka.cue index 3834952d75297..597411063803e 100644 --- a/website/cue/reference/components/sources/base/kafka.cue +++ b/website/cue/reference/components/sources/base/kafka.cue @@ -192,6 +192,21 @@ base: components: sources: kafka: configuration: { } } } + drain_timeout_ms: { + description: """ + Timeout to drain pending acknowledgements during shutdown or a Kafka + consumer group rebalance. + + When Vector shuts down or the Kafka consumer group revokes partitions from this + consumer, wait a maximum of `drain_timeout_ms` for the source to + process pending acknowledgements. Must be less than `session_timeout_ms` + to ensure the consumer is not excluded from the group during a rebalance. + + Default value is half of `session_timeout_ms`. + """ + required: false + type: uint: examples: [2500, 5000] + } fetch_wait_max_ms: { description: "Maximum time the broker may wait to fill the response." required: false From e455ce01d47eb2b37b039d9a9f17e732bc6b7841 Mon Sep 17 00:00:00 2001 From: j chesley <42561540+jches@users.noreply.github.com> Date: Tue, 3 Oct 2023 09:46:29 -0400 Subject: [PATCH 34/37] use Option::{unzip, map_or} methods Co-authored-by: Bruce Guenter --- src/sources/kafka.rs | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index e7c3c291bc78d..9358d8e18ab4b 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -435,20 +435,12 @@ async fn kafka_source( .expect("Error setting up consumer callback channel."); // EOF signal allowing the coordination task to tell the kafka client task when all partitions have reached EOF - let (eof_tx, eof_rx) = if eof { - let (tx, rx) = oneshot::channel::<()>(); - (Some(tx), Some(rx)) - } else { - (None, None) - }; + let (eof_tx, eof_rx) = eof.then(|| oneshot::channel::<()>()).unzip(); let coordination_task = { let span = span.clone(); let consumer = Arc::clone(&consumer); - let drain_timeout_ms = match config.drain_timeout_ms { - Some(d) => Duration::from_millis(d), - None => config.session_timeout_ms / 2, - }; + let drain_timeout_ms = config.drain_timeout_ms.map_or(config.session_timeout_ms / 2, Duration::from_millis); let consumer_state = ConsumerStateInner::::new(config, decoder, out, log_namespace); tokio::spawn(async move { @@ -796,7 +788,7 @@ async fn coordinate_kafka_callbacks( Some(callback) = callbacks.recv() => match callback { KafkaCallback::PartitionsAssigned(mut assigned_partitions, done) => match consumer_state { ConsumerState::Complete(_) => unreachable!("Partition assignment received after completion."), - ConsumerState::Draining(_) => error!("Partition assignment received while draining revoked partitions, invalid assignment?"), + ConsumerState::Draining(_) => error!("Partition assignment received while draining revoked partitions, maybe an invalid assignment."), ConsumerState::Consuming(ref consumer_state) => { let acks = consumer.context().acknowledgements; for tp in assigned_partitions.drain(0..) { From 4ad434b6e747e67ed7e9564c72f59e877e85d8d4 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Tue, 3 Oct 2023 21:34:59 -0400 Subject: [PATCH 35/37] remove OnceLock on callback channel sender, and other review cleanups - avoid panic in case split_partition_queue returns None - remove `pub` markers that are not needed - add comments around drain coordination signalling - cargo fmt --- src/sources/kafka.rs | 120 ++++++++++++++++++++++--------------------- 1 file changed, 62 insertions(+), 58 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 9358d8e18ab4b..7928b83abae0a 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -70,7 +70,10 @@ use crate::{ #[derive(Debug, Snafu)] enum BuildError { #[snafu(display("The drain_timeout_ms ({}) must be less than session_timeout_ms ({})", value, session_timeout_ms.as_millis()))] - KafkaInvalidDrainTimeoutError { value: u64, session_timeout_ms: Duration }, + KafkaInvalidDrainTimeoutError { + value: u64, + session_timeout_ms: Duration, + }, #[snafu(display("Could not create Kafka consumer: {}", source))] KafkaCreateError { source: rdkafka::error::KafkaError }, #[snafu(display("Could not subscribe to Kafka topics: {}", source))] @@ -331,14 +334,21 @@ impl SourceConfig for KafkaSourceConfig { let acknowledgements = cx.do_acknowledgements(self.acknowledgements); if let Some(d) = self.drain_timeout_ms { - snafu::ensure!(Duration::from_millis(d) <= self.session_timeout_ms, KafkaInvalidDrainTimeoutSnafu { value:d, session_timeout_ms: self.session_timeout_ms }); + snafu::ensure!( + Duration::from_millis(d) <= self.session_timeout_ms, + KafkaInvalidDrainTimeoutSnafu { + value: d, + session_timeout_ms: self.session_timeout_ms + } + ); } - let consumer = create_consumer(self, acknowledgements)?; + let (consumer, callback_rx) = create_consumer(self, acknowledgements)?; Ok(Box::pin(kafka_source( self.clone(), consumer, + callback_rx, decoder, cx.out, cx.shutdown, @@ -412,6 +422,7 @@ impl SourceConfig for KafkaSourceConfig { async fn kafka_source( config: KafkaSourceConfig, consumer: StreamConsumer, + callback_rx: UnboundedReceiver, decoder: Decoder, out: SourceSender, shutdown: ShutdownSignal, @@ -427,12 +438,6 @@ async fn kafka_source( .set(Arc::downgrade(&consumer)) .expect("Error setting up consumer context."); - let (callback_sender, callback_rx) = mpsc::unbounded_channel(); - consumer - .context() - .callbacks - .set(callback_sender) - .expect("Error setting up consumer callback channel."); // EOF signal allowing the coordination task to tell the kafka client task when all partitions have reached EOF let (eof_tx, eof_rx) = eof.then(|| oneshot::channel::<()>()).unzip(); @@ -440,7 +445,9 @@ async fn kafka_source( let coordination_task = { let span = span.clone(); let consumer = Arc::clone(&consumer); - let drain_timeout_ms = config.drain_timeout_ms.map_or(config.session_timeout_ms / 2, Duration::from_millis); + let drain_timeout_ms = config + .drain_timeout_ms + .map_or(config.session_timeout_ms / 2, Duration::from_millis); let consumer_state = ConsumerStateInner::::new(config, decoder, out, log_namespace); tokio::spawn(async move { @@ -497,6 +504,10 @@ struct ConsumerStateInner { struct Consuming; struct Complete; struct Draining { + /// The rendezvous channel sender from the revoke or shutdown callback. Sending on this channel + /// indicates to the kafka client task that one or more partitions have been drained, while + /// closing this channel indicates that all expected partitions have drained, or the drain + /// timeout has been reached. signal: SyncSender<()>, /// The set of topic-partition tasks that are required to complete during @@ -532,7 +543,7 @@ impl Draining { } impl ConsumerStateInner { - pub fn complete(self, _deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) { + fn complete(self, _deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) { ( None.into(), ConsumerState::Complete(ConsumerStateInner { @@ -595,6 +606,7 @@ impl ConsumerStateInner { loop { tokio::select!( + // is_some() checks prevent polling end_signal after it completes _ = &mut end_signal, if finalizer.is_some() => { finalizer.take(); }, @@ -644,7 +656,7 @@ impl ConsumerStateInner { /// Consume self, and return a "Draining" ConsumerState, along with a Future /// representing a drain deadline, based on max_drain_ms - pub fn begin_drain( + fn begin_drain( self, max_drain_ms: Duration, sig: SyncSender<()>, @@ -671,7 +683,7 @@ impl ConsumerStateInner { impl ConsumerStateInner { /// Mark the given TopicPartition as being revoked, adding it to the set of /// partitions expected to drain - pub fn revoke_partition(&mut self, tp: TopicPartition, end_signal: oneshot::Sender<()>) { + fn revoke_partition(&mut self, tp: TopicPartition, end_signal: oneshot::Sender<()>) { // Note that if this send() returns Err, it means the task has already // ended, but the completion has not been processed yet (otherwise we wouldn't have access to the end_signal), // so we should still add it to the "expect to drain" set @@ -682,19 +694,21 @@ impl ConsumerStateInner { /// Add the given TopicPartition to the set of known "drained" partitions, /// i.e. the consumer has drained the acknowledgement channel. A signal is /// sent on the signal channel, indicating to the client that offsets may be committed - pub fn partition_drained(&mut self, tp: TopicPartition) { + fn partition_drained(&mut self, tp: TopicPartition) { + // This send() will only return Err if the receiver has already been disconnected (i.e. the + // kafka client task is no longer running) _ = self.consumer_state.signal.send(()); self.consumer_state.expect_drain.remove(&tp); } /// Return true if all expected partitions have drained - pub fn is_drain_complete(&self) -> bool { + fn is_drain_complete(&self) -> bool { self.consumer_state.is_complete() } /// Finish partition drain mode. Consumes self and the drain deadline /// future, and returns a "Consuming" or "Complete" ConsumerState - pub fn finish_drain(self, deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) { + fn finish_drain(self, deadline: OptionDeadline) -> (OptionDeadline, ConsumerState) { if self.consumer_state.shutdown { self.complete(deadline) } else { @@ -773,16 +787,13 @@ async fn coordinate_kafka_callbacks( }, }; - match status { - // PartitionConsumerStatus differentiates between a task that exited after - // being signaled to end, and one that reached the end of its partition and - // was configured to exit. After the last such task ends, we signal the kafka - // driver task to shut down the main consumer too. Note this is only used in tests. - PartitionConsumerStatus::PartitionEOF if exit_eof && partition_consumers.is_empty() => { - debug!("All partitions have exited or reached EOF."); - let _ = eof.take().map(|e| e.send(())); - }, - _ => {} + // PartitionConsumerStatus differentiates between a task that exited after + // being signaled to end, and one that reached the end of its partition and + // was configured to exit. After the last such task ends, we signal the kafka + // driver task to shut down the main consumer too. Note this is only used in tests. + if exit_eof && status == PartitionConsumerStatus::PartitionEOF && partition_consumers.is_empty() { + debug!("All partitions have exited or reached EOF."); + let _ = eof.take().map(|e| e.send(())); } }, Some(callback) = callbacks.recv() => match callback { @@ -794,13 +805,14 @@ async fn coordinate_kafka_callbacks( for tp in assigned_partitions.drain(0..) { let topic = tp.0.as_str(); let partition = tp.1; - // It _should_ be impossible for this expect() to panic, since we receive the topic/partition pair from the rebalance callback - let pq = consumer.split_partition_queue(topic, partition).expect("Failed to get partition queue: invalid topic or partition."); - - debug!("Consuming partition {}:{}.", &tp.0, tp.1); - let (end_tx, handle) = consumer_state.consume_partition(&mut partition_consumers, tp.clone(), Arc::clone(&consumer), pq, acks, exit_eof); - abort_handles.insert(tp.clone(), handle); - end_signals.insert(tp, end_tx); + if let Some(pq) = consumer.split_partition_queue(topic, partition) { + debug!("Consuming partition {}:{}.", &tp.0, tp.1); + let (end_tx, handle) = consumer_state.consume_partition(&mut partition_consumers, tp.clone(), Arc::clone(&consumer), pq, acks, exit_eof); + abort_handles.insert(tp.clone(), handle); + end_signals.insert(tp, end_tx); + } else { + warn!("Failed to get queue for assigned partition {}:{}.", &tp.0, tp.1); + } } // ensure this is retained until all individual queues are set up drop(done); @@ -1163,7 +1175,7 @@ impl<'a> From> for FinalizerEntry { fn create_consumer( config: &KafkaSourceConfig, acknowledgements: bool, -) -> crate::Result> { +) -> crate::Result<(StreamConsumer, UnboundedReceiver)> { let mut client_config = ClientConfig::new(); client_config .set("group.id", &config.group_id) @@ -1199,16 +1211,18 @@ fn create_consumer( } } + let (callbacks, callback_rx) = mpsc::unbounded_channel(); let consumer = client_config .create_with_context::<_, StreamConsumer<_>>(KafkaSourceContext::new( config.metrics.topic_lag_metric, acknowledgements, + callbacks, )) .context(KafkaCreateSnafu)?; let topics: Vec<&str> = config.topics.iter().map(|s| s.as_str()).collect(); consumer.subscribe(&topics).context(KafkaSubscribeSnafu)?; - Ok(consumer) + Ok((consumer, callback_rx)) } type TopicPartition = (String, i32); @@ -1216,6 +1230,7 @@ type TopicPartition = (String, i32); /// Status returned by partition consumer tasks, allowing the coordination task /// to differentiate between a consumer exiting normally (after receiving an end /// signal) and exiting when it reaches the end of a partition +#[derive(PartialEq)] enum PartitionConsumerStatus { NormalExit, PartitionEOF, @@ -1232,29 +1247,27 @@ struct KafkaSourceContext { stats: kafka::KafkaStatisticsContext, /// A callback channel used to coordinate between the main consumer task and the acknowledgement task - callbacks: OnceLock>, + callbacks: UnboundedSender, /// A weak reference to the consumer, so that we can commit offsets during a rebalance operation consumer: OnceLock>>, } impl KafkaSourceContext { - fn new(expose_lag_metrics: bool, acknowledgements: bool) -> Self { + fn new(expose_lag_metrics: bool, acknowledgements: bool, callbacks: UnboundedSender) -> Self { Self { stats: kafka::KafkaStatisticsContext { expose_lag_metrics }, acknowledgements, - callbacks: OnceLock::default(), consumer: OnceLock::default(), + callbacks, } } - pub fn shutdown(&self) { - if let Some(tx) = self.callbacks.get() { - let (send, rendezvous) = sync_channel(0); - if tx.send(KafkaCallback::ShuttingDown(send)).is_ok() { - while rendezvous.recv().is_ok() { - self.commit_consumer_state(); - } + fn shutdown(&self) { + let (send, rendezvous) = sync_channel(0); + if self.callbacks.send(KafkaCallback::ShuttingDown(send)).is_ok() { + while rendezvous.recv().is_ok() { + self.commit_consumer_state(); } } } @@ -1264,13 +1277,8 @@ impl KafkaSourceContext { /// each topic-partition has been set up. This function blocks until the /// rendezvous channel sender is dropped by the callback handler. fn consume_partitions(&self, tpl: &TopicPartitionList) { - let callbacks = self - .callbacks - .get() - .expect("Callbacks handler was not initialized."); - let (send, rendezvous) = sync_channel(0); - let _ = callbacks.send(KafkaCallback::PartitionsAssigned( + let _ = self.callbacks.send(KafkaCallback::PartitionsAssigned( tpl.elements() .iter() .map(|tp| (tp.topic().into(), tp.partition())) @@ -1289,13 +1297,8 @@ impl KafkaSourceContext { /// signal individual partitions completing. This function blocks until the /// sender is dropped by the callback handler. fn revoke_partitions(&self, tpl: &TopicPartitionList) { - let callbacks = self - .callbacks - .get() - .expect("Callbacks handler was not initialized."); - let (send, rendezvous) = sync_channel(0); - let _ = callbacks.send(KafkaCallback::PartitionsRevoked( + let _ = self.callbacks.send(KafkaCallback::PartitionsRevoked( tpl.elements() .iter() .map(|tp| (tp.topic().into(), tp.partition())) @@ -1781,11 +1784,12 @@ mod integration_test { .build() .unwrap(); - let consumer = create_consumer(&config, acknowledgements).unwrap(); + let (consumer, callback_rx) = create_consumer(&config, acknowledgements).unwrap(); tokio::spawn(kafka_source( config, consumer, + callback_rx, decoder, out, shutdown, From 6353bbcd99ba011dbd3d6bafb50033a161429418 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Wed, 4 Oct 2023 16:42:15 -0400 Subject: [PATCH 36/37] members of Keys struct are cloned once per consumed partition, instead of entire config object --- src/sources/kafka.rs | 37 ++++++++++++++++++------------------- 1 file changed, 18 insertions(+), 19 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index 7928b83abae0a..d0d3a2811e723 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -585,8 +585,7 @@ impl ConsumerStateInner { acknowledgements: bool, exit_eof: bool, ) -> (oneshot::Sender<()>, tokio::task::AbortHandle) { - // TODO there's probably a better way to pass the keys() config around than cloning the entire config object? But they are tied to the lifetime of the config object, so..this avoids lifetime messes for now - let conf = self.config.clone(); + let keys = self.config.keys(); let decoder = self.decoder.clone(); let log_namespace = self.log_namespace; let mut out = self.out.clone(); @@ -627,7 +626,7 @@ impl ConsumerStateInner { topic: msg.topic(), partition: msg.partition(), }); - parse_message(msg, decoder.clone(), conf.keys(), &mut out, acknowledgements, &finalizer, log_namespace).await; + parse_message(msg, decoder.clone(), &keys, &mut out, acknowledgements, &finalizer, log_namespace).await; } }, @@ -938,7 +937,7 @@ fn drive_kafka_consumer( async fn parse_message( msg: BorrowedMessage<'_>, decoder: Decoder, - keys: Keys<'_>, + keys: &'_ Keys, out: &mut SourceSender, acknowledgements: bool, finalizer: &Option>, @@ -976,7 +975,7 @@ async fn parse_message( fn parse_stream<'a>( msg: &BorrowedMessage<'a>, decoder: Decoder, - keys: Keys<'a>, + keys: &'a Keys, log_namespace: LogNamespace, ) -> Option<(usize, impl Stream + 'a)> { let payload = msg.payload()?; // skip messages with empty payload @@ -1017,24 +1016,24 @@ fn parse_stream<'a>( } #[derive(Clone, Debug)] -struct Keys<'a> { +struct Keys { timestamp: Option, - key_field: &'a Option, - topic: &'a Option, - partition: &'a Option, - offset: &'a Option, - headers: &'a Option, + key_field: Option, + topic: Option, + partition: Option, + offset: Option, + headers: Option, } -impl<'a> Keys<'a> { - fn from(schema: &'a LogSchema, config: &'a KafkaSourceConfig) -> Self { +impl Keys { + fn from(schema: &LogSchema, config: &KafkaSourceConfig) -> Self { Self { timestamp: schema.timestamp_key().cloned(), - key_field: &config.key_field.path, - topic: &config.topic_key.path, - partition: &config.partition_key.path, - offset: &config.offset_key.path, - headers: &config.headers_key.path, + key_field: config.key_field.path.clone(), + topic: config.topic_key.path.clone(), + partition: config.partition_key.path.clone(), + offset: config.offset_key.path.clone(), + headers: config.headers_key.path.clone(), } } } @@ -1083,7 +1082,7 @@ impl ReceivedMessage { } } - fn apply(&self, keys: &Keys<'_>, event: &mut Event, log_namespace: LogNamespace) { + fn apply(&self, keys: &Keys, event: &mut Event, log_namespace: LogNamespace) { if let Event::Log(ref mut log) = event { match log_namespace { LogNamespace::Vector => { From ff086d70c7553285ec5340e2bfa8eee9714c3594 Mon Sep 17 00:00:00 2001 From: John Chesley Date: Wed, 4 Oct 2023 18:02:13 -0400 Subject: [PATCH 37/37] cargo fmt and fix clippy warnings --- src/sources/kafka.rs | 35 +++++++++++++++++++++++------------ 1 file changed, 23 insertions(+), 12 deletions(-) diff --git a/src/sources/kafka.rs b/src/sources/kafka.rs index d0d3a2811e723..6bcba0de51682 100644 --- a/src/sources/kafka.rs +++ b/src/sources/kafka.rs @@ -70,14 +70,14 @@ use crate::{ #[derive(Debug, Snafu)] enum BuildError { #[snafu(display("The drain_timeout_ms ({}) must be less than session_timeout_ms ({})", value, session_timeout_ms.as_millis()))] - KafkaInvalidDrainTimeoutError { + InvalidDrainTimeout { value: u64, session_timeout_ms: Duration, }, #[snafu(display("Could not create Kafka consumer: {}", source))] - KafkaCreateError { source: rdkafka::error::KafkaError }, + CreateError { source: rdkafka::error::KafkaError }, #[snafu(display("Could not subscribe to Kafka topics: {}", source))] - KafkaSubscribeError { source: rdkafka::error::KafkaError }, + SubscribeError { source: rdkafka::error::KafkaError }, } /// Metrics configuration. @@ -336,7 +336,7 @@ impl SourceConfig for KafkaSourceConfig { if let Some(d) = self.drain_timeout_ms { snafu::ensure!( Duration::from_millis(d) <= self.session_timeout_ms, - KafkaInvalidDrainTimeoutSnafu { + InvalidDrainTimeoutSnafu { value: d, session_timeout_ms: self.session_timeout_ms } @@ -419,6 +419,7 @@ impl SourceConfig for KafkaSourceConfig { } } +#[allow(clippy::too_many_arguments)] async fn kafka_source( config: KafkaSourceConfig, consumer: StreamConsumer, @@ -438,9 +439,8 @@ async fn kafka_source( .set(Arc::downgrade(&consumer)) .expect("Error setting up consumer context."); - // EOF signal allowing the coordination task to tell the kafka client task when all partitions have reached EOF - let (eof_tx, eof_rx) = eof.then(|| oneshot::channel::<()>()).unzip(); + let (eof_tx, eof_rx) = eof.then(oneshot::channel::<()>).unzip(); let coordination_task = { let span = span.clone(); @@ -997,7 +997,7 @@ fn parse_stream<'a>( partition: rmsg.partition, }); for mut event in events { - rmsg.apply(&keys, &mut event, log_namespace); + rmsg.apply(keys, &mut event, log_namespace); yield event; } }, @@ -1174,7 +1174,10 @@ impl<'a> From> for FinalizerEntry { fn create_consumer( config: &KafkaSourceConfig, acknowledgements: bool, -) -> crate::Result<(StreamConsumer, UnboundedReceiver)> { +) -> crate::Result<( + StreamConsumer, + UnboundedReceiver, +)> { let mut client_config = ClientConfig::new(); client_config .set("group.id", &config.group_id) @@ -1217,9 +1220,9 @@ fn create_consumer( acknowledgements, callbacks, )) - .context(KafkaCreateSnafu)?; + .context(CreateSnafu)?; let topics: Vec<&str> = config.topics.iter().map(|s| s.as_str()).collect(); - consumer.subscribe(&topics).context(KafkaSubscribeSnafu)?; + consumer.subscribe(&topics).context(SubscribeSnafu)?; Ok((consumer, callback_rx)) } @@ -1253,7 +1256,11 @@ struct KafkaSourceContext { } impl KafkaSourceContext { - fn new(expose_lag_metrics: bool, acknowledgements: bool, callbacks: UnboundedSender) -> Self { + fn new( + expose_lag_metrics: bool, + acknowledgements: bool, + callbacks: UnboundedSender, + ) -> Self { Self { stats: kafka::KafkaStatisticsContext { expose_lag_metrics }, acknowledgements, @@ -1264,7 +1271,11 @@ impl KafkaSourceContext { fn shutdown(&self) { let (send, rendezvous) = sync_channel(0); - if self.callbacks.send(KafkaCallback::ShuttingDown(send)).is_ok() { + if self + .callbacks + .send(KafkaCallback::ShuttingDown(send)) + .is_ok() + { while rendezvous.recv().is_ok() { self.commit_consumer_state(); }