log_store/kafka/
client_manager.rs1use std::collections::HashMap;
16use std::sync::Arc;
17
18use common_wal::config::kafka::common::DEFAULT_BACKOFF_CONFIG;
19use common_wal::config::kafka::DatanodeKafkaConfig;
20use dashmap::DashMap;
21use rskafka::client::partition::{Compression, PartitionClient, UnknownTopicHandling};
22use rskafka::client::ClientBuilder;
23use snafu::ResultExt;
24use store_api::logstore::provider::KafkaProvider;
25use tokio::sync::{Mutex, RwLock};
26
27use crate::error::{BuildClientSnafu, BuildPartitionClientSnafu, Result, TlsConfigSnafu};
28use crate::kafka::index::{GlobalIndexCollector, NoopCollector};
29use crate::kafka::log_store::TopicStat;
30use crate::kafka::producer::{OrderedBatchProducer, OrderedBatchProducerRef};
31
32pub const DEFAULT_PARTITION: i32 = 0;
35
36pub(crate) type ClientManagerRef = Arc<ClientManager>;
38
39#[derive(Debug, Clone)]
41pub(crate) struct Client {
42 client: Arc<PartitionClient>,
43 producer: OrderedBatchProducerRef,
44}
45
46impl Client {
47 pub(crate) fn client(&self) -> &Arc<PartitionClient> {
48 &self.client
49 }
50
51 pub(crate) fn producer(&self) -> &OrderedBatchProducerRef {
52 &self.producer
53 }
54}
55
56#[derive(Debug)]
58pub(crate) struct ClientManager {
59 client: rskafka::client::Client,
60 mutex: Mutex<()>,
62 instances: RwLock<HashMap<Arc<KafkaProvider>, Client>>,
63 global_index_collector: Option<GlobalIndexCollector>,
64
65 flush_batch_size: usize,
66 compression: Compression,
67
68 topic_stats: Arc<DashMap<Arc<KafkaProvider>, TopicStat>>,
70}
71
72impl ClientManager {
73 pub(crate) async fn try_new(
75 config: &DatanodeKafkaConfig,
76 global_index_collector: Option<GlobalIndexCollector>,
77 topic_stats: Arc<DashMap<Arc<KafkaProvider>, TopicStat>>,
78 ) -> Result<Self> {
79 let mut builder = ClientBuilder::new(config.connection.broker_endpoints.clone())
81 .backoff_config(DEFAULT_BACKOFF_CONFIG);
82 if let Some(sasl) = &config.connection.sasl {
83 builder = builder.sasl_config(sasl.config.clone().into_sasl_config());
84 };
85 if let Some(tls) = &config.connection.tls {
86 builder = builder.tls_config(tls.to_tls_config().await.context(TlsConfigSnafu)?)
87 };
88
89 let client = builder.build().await.with_context(|_| BuildClientSnafu {
90 broker_endpoints: config.connection.broker_endpoints.clone(),
91 })?;
92
93 Ok(Self {
94 client,
95 mutex: Mutex::new(()),
96 instances: RwLock::new(HashMap::new()),
97 flush_batch_size: config.max_batch_bytes.as_bytes() as usize,
98 compression: Compression::Lz4,
99 global_index_collector,
100 topic_stats,
101 })
102 }
103
104 async fn try_insert(&self, provider: &Arc<KafkaProvider>) -> Result<Client> {
105 let _guard = self.mutex.lock().await;
106
107 let client = self.instances.read().await.get(provider).cloned();
108 match client {
109 Some(client) => Ok(client),
110 None => {
111 let client = self.try_create_client(provider).await?;
112 self.instances
113 .write()
114 .await
115 .insert(provider.clone(), client.clone());
116 self.topic_stats
117 .insert(provider.clone(), TopicStat::default());
118 Ok(client)
119 }
120 }
121 }
122
123 pub(crate) async fn get_or_insert(&self, provider: &Arc<KafkaProvider>) -> Result<Client> {
126 let client = self.instances.read().await.get(provider).cloned();
127 match client {
128 Some(client) => Ok(client),
129 None => self.try_insert(provider).await,
130 }
131 }
132
133 async fn try_create_client(&self, provider: &Arc<KafkaProvider>) -> Result<Client> {
134 let client = self
138 .client
139 .partition_client(
140 provider.topic.as_str(),
141 DEFAULT_PARTITION,
142 UnknownTopicHandling::Retry,
143 )
144 .await
145 .context(BuildPartitionClientSnafu {
146 topic: &provider.topic,
147 partition: DEFAULT_PARTITION,
148 })
149 .map(Arc::new)?;
150
151 let (tx, rx) = OrderedBatchProducer::channel();
152 let index_collector = if let Some(global_collector) = self.global_index_collector.as_ref() {
153 global_collector
154 .provider_level_index_collector(provider.clone(), tx.clone())
155 .await
156 } else {
157 Box::new(NoopCollector)
158 };
159 let producer = Arc::new(OrderedBatchProducer::new(
160 (tx, rx),
161 provider.clone(),
162 client.clone(),
163 self.compression,
164 self.flush_batch_size,
165 index_collector,
166 self.topic_stats.clone(),
167 ));
168
169 Ok(Client { client, producer })
170 }
171
172 pub(crate) fn global_index_collector(&self) -> Option<&GlobalIndexCollector> {
173 self.global_index_collector.as_ref()
174 }
175
176 pub(crate) async fn list_topics(&self) -> Vec<Arc<KafkaProvider>> {
178 self.instances.read().await.keys().cloned().collect()
179 }
180
181 #[cfg(test)]
182 pub(crate) fn topic_stats(&self) -> &Arc<DashMap<Arc<KafkaProvider>, TopicStat>> {
183 &self.topic_stats
184 }
185}
186
187#[cfg(test)]
188impl ClientManager {
189 pub(crate) fn controller_client(&self) -> rskafka::client::controller::ControllerClient {
191 self.client.controller_client().unwrap()
192 }
193}
194
195#[cfg(test)]
196mod tests {
197 use common_wal::maybe_skip_kafka_integration_test;
198 use common_wal::test_util::get_kafka_endpoints;
199 use tokio::sync::Barrier;
200
201 use super::*;
202 use crate::kafka::test_util::prepare;
203
204 #[tokio::test]
206 async fn test_sequential() {
207 maybe_skip_kafka_integration_test!();
208 let broker_endpoints = get_kafka_endpoints();
209 let (manager, topics) = prepare("test_sequential", 128, broker_endpoints).await;
210 let region_topic = (0..512)
212 .map(|region_id| (region_id, &topics[region_id % topics.len()]))
213 .collect::<HashMap<_, _>>();
214
215 for (_, topic) in region_topic {
217 let provider = Arc::new(KafkaProvider::new(topic.to_string()));
218 manager.get_or_insert(&provider).await.unwrap();
219 }
220
221 let client_pool = manager.instances.read().await;
223 let all_exist = topics.iter().all(|topic| {
224 let provider = Arc::new(KafkaProvider::new(topic.to_string()));
225 client_pool.contains_key(&provider)
226 });
227 assert!(all_exist);
228 }
229
230 #[tokio::test(flavor = "multi_thread")]
232 async fn test_parallel() {
233 maybe_skip_kafka_integration_test!();
234 let broker_endpoints = get_kafka_endpoints();
235 let (manager, topics) = prepare("test_parallel", 128, broker_endpoints).await;
236 let region_topic = (0..512)
238 .map(|region_id| (region_id, topics[region_id % topics.len()].clone()))
239 .collect::<HashMap<_, _>>();
240
241 let manager = Arc::new(manager);
243 let barrier = Arc::new(Barrier::new(region_topic.len()));
244 let tasks = region_topic
245 .into_values()
246 .map(|topic| {
247 let manager = manager.clone();
248 let barrier = barrier.clone();
249
250 tokio::spawn(async move {
251 barrier.wait().await;
252 let provider = Arc::new(KafkaProvider::new(topic));
253 assert!(manager.get_or_insert(&provider).await.is_ok());
254 })
255 })
256 .collect::<Vec<_>>();
257 futures::future::try_join_all(tasks).await.unwrap();
258
259 let client_pool = manager.instances.read().await;
261 let all_exist = topics.iter().all(|topic| {
262 let provider = Arc::new(KafkaProvider::new(topic.to_string()));
263 client_pool.contains_key(&provider)
264 });
265 assert!(all_exist);
266 }
267}