-
Notifications
You must be signed in to change notification settings - Fork 590
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
feat: share kafka client on meta #19058
Changes from 13 commits
b9b2f79
ab46672
3373a58
c5203d6
3b6a6a2
3b3f725
968ed08
a41f3fc
6534ebf
7249e78
0024e1d
ad8b989
ae1b70a
58b5128
f115a0c
d128644
ae9df41
45295bc
a9e34c7
d27ab90
256485c
62cb953
725e23c
832f66f
73f0b7b
ac1d63d
35fb002
ec49096
16d8c42
b3efda6
51eca61
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -13,6 +13,7 @@ | |
// limitations under the License. | ||
|
||
use std::collections::HashMap; | ||
use std::sync::{Arc, LazyLock}; | ||
use std::time::Duration; | ||
|
||
use anyhow::{anyhow, Context}; | ||
|
@@ -32,6 +33,14 @@ use crate::source::kafka::{ | |
}; | ||
use crate::source::SourceEnumeratorContextRef; | ||
|
||
pub static SHARED_KAFKA_CLIENT: LazyLock<tokio::sync::Mutex<HashMap<u64, SharedKafkaItem>>> = | ||
tabVersion marked this conversation as resolved.
Show resolved
Hide resolved
|
||
LazyLock::new(|| tokio::sync::Mutex::new(HashMap::new())); | ||
|
||
pub struct SharedKafkaItem { | ||
pub client: Arc<BaseConsumer<RwConsumerContext>>, | ||
pub ref_count: i32, | ||
} | ||
|
||
#[derive(Debug, Copy, Clone, Eq, PartialEq)] | ||
pub enum KafkaEnumeratorOffset { | ||
Earliest, | ||
|
@@ -44,7 +53,7 @@ pub struct KafkaSplitEnumerator { | |
context: SourceEnumeratorContextRef, | ||
broker_address: String, | ||
topic: String, | ||
client: BaseConsumer<RwConsumerContext>, | ||
client: Arc<BaseConsumer<RwConsumerContext>>, | ||
start_offset: KafkaEnumeratorOffset, | ||
|
||
// maybe used in the future for batch processing | ||
|
@@ -68,12 +77,14 @@ impl SplitEnumerator for KafkaSplitEnumerator { | |
let mut config = rdkafka::ClientConfig::new(); | ||
let common_props = &properties.common; | ||
|
||
let broker_address = common_props.brokers.clone(); | ||
let broker_address = properties.connection.brokers.clone(); | ||
|
||
let connection_hash = properties.connection.get_hash(); | ||
let broker_rewrite_map = properties.privatelink_common.broker_rewrite_map.clone(); | ||
let topic = common_props.topic.clone(); | ||
config.set("bootstrap.servers", &broker_address); | ||
config.set("isolation.level", KAFKA_ISOLATION_LEVEL); | ||
common_props.set_security_properties(&mut config); | ||
properties.connection.set_security_properties(&mut config); | ||
properties.set_client(&mut config); | ||
let mut scan_start_offset = match properties | ||
.scan_startup_mode | ||
|
@@ -94,36 +105,67 @@ impl SplitEnumerator for KafkaSplitEnumerator { | |
scan_start_offset = KafkaEnumeratorOffset::Timestamp(time_offset) | ||
} | ||
|
||
// don't need kafka metrics from enumerator | ||
let ctx_common = KafkaContextCommon::new( | ||
broker_rewrite_map, | ||
None, | ||
None, | ||
properties.aws_auth_props, | ||
common_props.is_aws_msk_iam(), | ||
) | ||
.await?; | ||
let client_ctx = RwConsumerContext::new(ctx_common); | ||
let client: BaseConsumer<RwConsumerContext> = | ||
config.create_with_context(client_ctx).await?; | ||
|
||
// Note that before any SASL/OAUTHBEARER broker connection can succeed the application must call | ||
// rd_kafka_oauthbearer_set_token() once – either directly or, more typically, by invoking either | ||
// rd_kafka_poll(), rd_kafka_consumer_poll(), rd_kafka_queue_poll(), etc, in order to cause retrieval | ||
// of an initial token to occur. | ||
// https://docs.confluent.io/platform/current/clients/librdkafka/html/rdkafka_8h.html#a988395722598f63396d7a1bedb22adaf | ||
if common_props.is_aws_msk_iam() { | ||
#[cfg(not(madsim))] | ||
client.poll(Duration::from_secs(10)); // note: this is a blocking call | ||
#[cfg(madsim)] | ||
client.poll(Duration::from_secs(10)).await; | ||
let kafka_client: Arc<BaseConsumer<RwConsumerContext>>; | ||
let mut shared_client_guard = SHARED_KAFKA_CLIENT.lock().await; | ||
if let Some(item) = shared_client_guard.get_mut(&connection_hash) { | ||
tracing::info!( | ||
"reusing kafka client for connection hash {}, to broker {}", | ||
connection_hash, | ||
broker_address | ||
); | ||
kafka_client = item.client.clone(); | ||
item.ref_count += 1; | ||
drop(shared_client_guard); | ||
} else { | ||
// drop the guard and acquire a new one to avoid a 10s blocking call | ||
drop(shared_client_guard); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will this happen?
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I would recommend to use use moka::future::Cache; Particularly, a caching structure should handle these concurrent gets correctly by letting the caller B blocks until caller A completes its operation and insert back the cached item i.e. the Kafka Client. |
||
|
||
// don't need kafka metrics from enumerator | ||
let ctx_common = KafkaContextCommon::new( | ||
broker_rewrite_map, | ||
None, | ||
None, | ||
properties.aws_auth_props, | ||
properties.connection.is_aws_msk_iam(), | ||
) | ||
.await?; | ||
let client_ctx = RwConsumerContext::new(ctx_common); | ||
let client: BaseConsumer<RwConsumerContext> = | ||
config.create_with_context(client_ctx).await?; | ||
|
||
// Note that before any SASL/OAUTHBEARER broker connection can succeed the application must call | ||
// rd_kafka_oauthbearer_set_token() once – either directly or, more typically, by invoking either | ||
// rd_kafka_poll(), rd_kafka_consumer_poll(), rd_kafka_queue_poll(), etc, in order to cause retrieval | ||
// of an initial token to occur. | ||
// https://docs.confluent.io/platform/current/clients/librdkafka/html/rdkafka_8h.html#a988395722598f63396d7a1bedb22adaf | ||
if properties.connection.is_aws_msk_iam() { | ||
#[cfg(not(madsim))] | ||
client.poll(Duration::from_secs(10)); // note: this is a blocking call | ||
#[cfg(madsim)] | ||
client.poll(Duration::from_secs(10)).await; | ||
} | ||
|
||
kafka_client = Arc::new(client); | ||
tracing::debug!( | ||
"created kafka client for connection hash {} to broker {}", | ||
connection_hash, | ||
broker_address | ||
); | ||
let mut shared_client_guard = SHARED_KAFKA_CLIENT.lock().await; | ||
shared_client_guard.insert( | ||
connection_hash, | ||
SharedKafkaItem { | ||
client: kafka_client.clone(), | ||
ref_count: 1, | ||
}, | ||
); | ||
} | ||
|
||
Ok(Self { | ||
context, | ||
broker_address, | ||
topic, | ||
client, | ||
client: kafka_client, | ||
start_offset: scan_start_offset, | ||
stop_offset: KafkaEnumeratorOffset::None, | ||
sync_call_timeout: properties.common.sync_call_timeout, | ||
|
@@ -148,7 +190,7 @@ impl SplitEnumerator for KafkaSplitEnumerator { | |
.fetch_stop_offset(topic_partitions.as_ref(), &watermarks) | ||
.await?; | ||
|
||
let ret = topic_partitions | ||
let ret: Vec<_> = topic_partitions | ||
.into_iter() | ||
.map(|partition| KafkaSplit { | ||
topic: self.topic.clone(), | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems we will be able to use this same struct for
CREATE CONNECTION
later? 🤔There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes.