risingwave_connector/connector_common/
connection.rs1use std::collections::{BTreeMap, HashMap};
16use std::time::Duration;
17
18use anyhow::Context;
19use opendal::Operator;
20use opendal::services::{Azblob, Gcs, S3};
21use phf::{Set, phf_set};
22use rdkafka::ClientConfig;
23use rdkafka::config::RDKafkaLogLevel;
24use rdkafka::consumer::{BaseConsumer, Consumer};
25use risingwave_common::bail;
26use risingwave_common::secret::LocalSecretManager;
27use risingwave_common::util::env_var::env_var_is_true;
28use risingwave_pb::catalog::PbConnection;
29use serde::Deserialize;
30use serde_with::serde_as;
31use tonic::async_trait;
32use url::Url;
33use with_options::WithOptions;
34
35use crate::connector_common::common::DISABLE_DEFAULT_CREDENTIAL;
36use crate::connector_common::{
37 AwsAuthProps, IcebergCommon, IcebergTableIdentifier, KafkaConnectionProps,
38 KafkaPrivateLinkCommon,
39};
40use crate::enforce_secret::EnforceSecret;
41use crate::error::ConnectorResult;
42use crate::schema::schema_registry::Client as ConfluentSchemaRegistryClient;
43use crate::sink::elasticsearch_opensearch::elasticsearch_opensearch_config::ElasticSearchOpenSearchConfig;
44use crate::source::build_connection;
45use crate::source::kafka::{KafkaContextCommon, RwConsumerContext};
46
47pub const SCHEMA_REGISTRY_CONNECTION_TYPE: &str = "schema_registry";
48
49#[async_trait]
51pub trait Connection: Send {
52 async fn validate_connection(&self) -> ConnectorResult<()>;
53}
54
55#[serde_as]
56#[derive(Debug, Clone, Deserialize, WithOptions, PartialEq)]
57#[serde(deny_unknown_fields)]
58pub struct KafkaConnection {
59 #[serde(flatten)]
60 pub inner: KafkaConnectionProps,
61 #[serde(flatten)]
62 pub kafka_private_link_common: KafkaPrivateLinkCommon,
63 #[serde(flatten)]
64 pub aws_auth_props: AwsAuthProps,
65}
66
67impl EnforceSecret for KafkaConnection {
68 fn enforce_secret<'a>(prop_iter: impl Iterator<Item = &'a str>) -> ConnectorResult<()> {
69 for prop in prop_iter {
70 KafkaConnectionProps::enforce_one(prop)?;
71 AwsAuthProps::enforce_one(prop)?;
72 }
73 Ok(())
74 }
75}
76
77pub async fn validate_connection(connection: &PbConnection) -> ConnectorResult<()> {
78 if let Some(ref info) = connection.info {
79 match info {
80 risingwave_pb::catalog::connection::Info::ConnectionParams(cp) => {
81 let options = cp.properties.clone().into_iter().collect();
82 let secret_refs = cp.secret_refs.clone().into_iter().collect();
83 let props_secret_resolved =
84 LocalSecretManager::global().fill_secrets(options, secret_refs)?;
85 let connection = build_connection(cp.connection_type(), props_secret_resolved)?;
86 connection.validate_connection().await?
87 }
88 risingwave_pb::catalog::connection::Info::PrivateLinkService(_) => unreachable!(),
89 }
90 }
91 Ok(())
92}
93
94#[async_trait]
95impl Connection for KafkaConnection {
96 async fn validate_connection(&self) -> ConnectorResult<()> {
97 let client = self.build_client().await?;
98 client.fetch_metadata(None, Duration::from_secs(10)).await?;
100 Ok(())
101 }
102}
103
104pub fn read_kafka_log_level() -> Option<RDKafkaLogLevel> {
105 let log_level = std::env::var("RISINGWAVE_KAFKA_LOG_LEVEL").ok()?;
106 match log_level.to_uppercase().as_str() {
107 "DEBUG" => Some(RDKafkaLogLevel::Debug),
108 "INFO" => Some(RDKafkaLogLevel::Info),
109 "WARN" => Some(RDKafkaLogLevel::Warning),
110 "ERROR" => Some(RDKafkaLogLevel::Error),
111 "CRITICAL" => Some(RDKafkaLogLevel::Critical),
112 "EMERG" => Some(RDKafkaLogLevel::Emerg),
113 "ALERT" => Some(RDKafkaLogLevel::Alert),
114 "NOTICE" => Some(RDKafkaLogLevel::Notice),
115 _ => None,
116 }
117}
118
119impl KafkaConnection {
120 async fn build_client(&self) -> ConnectorResult<BaseConsumer<RwConsumerContext>> {
121 let mut config = ClientConfig::new();
122 let bootstrap_servers = &self.inner.brokers;
123 let broker_rewrite_map = self.kafka_private_link_common.broker_rewrite_map.clone();
124 config.set("bootstrap.servers", bootstrap_servers);
125 self.inner.set_security_properties(&mut config);
126
127 let ctx_common = KafkaContextCommon::new(
129 broker_rewrite_map,
130 None,
131 None,
132 self.aws_auth_props.clone(),
133 self.inner.is_aws_msk_iam(),
134 )
135 .await?;
136 let client_ctx = RwConsumerContext::new(ctx_common);
137
138 if let Some(log_level) = read_kafka_log_level() {
139 config.set_log_level(log_level);
140 }
141 let client: BaseConsumer<RwConsumerContext> =
142 config.create_with_context(client_ctx).await?;
143 if self.inner.is_aws_msk_iam() {
144 #[cfg(not(madsim))]
145 client.poll(Duration::from_secs(10)); #[cfg(madsim)]
147 client.poll(Duration::from_secs(10)).await;
148 }
149 Ok(client)
150 }
151}
152
153#[serde_as]
154#[derive(Debug, Clone, PartialEq, Eq, Deserialize, WithOptions)]
155#[serde(deny_unknown_fields)]
156pub struct IcebergConnection {
157 #[serde(flatten)]
158 pub common: IcebergCommon,
159
160 #[serde(rename = "catalog.jdbc.user")]
161 pub jdbc_user: Option<String>,
162
163 #[serde(rename = "catalog.jdbc.password")]
164 pub jdbc_password: Option<String>,
165}
166
167impl EnforceSecret for IcebergConnection {
168 const ENFORCE_SECRET_PROPERTIES: Set<&'static str> = phf_set! {
169 "s3.access.key",
170 "s3.secret.key",
171 "gcs.credential",
172 "catalog.token",
173 };
174}
175
176#[async_trait]
177impl Connection for IcebergConnection {
178 async fn validate_connection(&self) -> ConnectorResult<()> {
179 let common = &self.common;
180
181 let info = match &common.warehouse_path {
182 Some(warehouse_path) => {
183 let is_s3_tables = warehouse_path.starts_with("arn:aws:s3tables");
184 let url = Url::parse(warehouse_path);
185 if (url.is_err() || is_s3_tables)
186 && matches!(common.catalog_type(), "rest" | "rest_rust")
187 {
188 None
192 } else {
193 let url =
194 url.with_context(|| format!("Invalid warehouse path: {}", warehouse_path))?;
195 let bucket = url
196 .host_str()
197 .with_context(|| {
198 format!("Invalid s3 path: {}, bucket is missing", warehouse_path)
199 })?
200 .to_owned();
201 let root = url.path().trim_start_matches('/').to_owned();
202 Some((url.scheme().to_owned(), bucket, root))
203 }
204 }
205 None => {
206 if matches!(common.catalog_type(), "rest" | "rest_rust") {
207 None
208 } else {
209 bail!("`warehouse.path` must be set");
210 }
211 }
212 };
213
214 if let Some((scheme, bucket, root)) = info {
216 match scheme.as_str() {
217 "s3" | "s3a" => {
218 let mut builder = S3::default();
219 if let Some(region) = &common.region {
220 builder = builder.region(region);
221 }
222 if let Some(endpoint) = &common.endpoint {
223 builder = builder.endpoint(endpoint);
224 }
225 if let Some(access_key) = &common.access_key {
226 builder = builder.access_key_id(access_key);
227 }
228 if let Some(secret_key) = &common.secret_key {
229 builder = builder.secret_access_key(secret_key);
230 }
231 builder = builder.root(root.as_str()).bucket(bucket.as_str());
232 let op = Operator::new(builder)?.finish();
233 op.check().await?;
234 }
235 "gs" | "gcs" => {
236 let mut builder = Gcs::default();
237 if let Some(credential) = &common.gcs_credential {
238 builder = builder.credential(credential);
239 }
240 builder = builder.root(root.as_str()).bucket(bucket.as_str());
241 let op = Operator::new(builder)?.finish();
242 op.check().await?;
243 }
244 "azblob" => {
245 let mut builder = Azblob::default();
246 if let Some(account_name) = &common.azblob_account_name {
247 builder = builder.account_name(account_name);
248 }
249 if let Some(azblob_account_key) = &common.azblob_account_key {
250 builder = builder.account_key(azblob_account_key);
251 }
252 if let Some(azblob_endpoint_url) = &common.azblob_endpoint_url {
253 builder = builder.endpoint(azblob_endpoint_url);
254 }
255 builder = builder.root(root.as_str()).container(bucket.as_str());
256 let op = Operator::new(builder)?.finish();
257 op.check().await?;
258 }
259 _ => {
260 bail!("Unsupported scheme: {}", scheme);
261 }
262 }
263 }
264
265 if env_var_is_true(DISABLE_DEFAULT_CREDENTIAL)
266 && matches!(common.enable_config_load, Some(true))
267 {
268 bail!("`enable_config_load` can't be enabled in this environment");
269 }
270
271 if common.hosted_catalog.unwrap_or(false) {
272 if common.catalog_type.is_some() {
274 bail!("`catalog.type` must not be set when `hosted_catalog` is set");
275 }
276 if common.catalog_uri.is_some() {
277 bail!("`catalog.uri` must not be set when `hosted_catalog` is set");
278 }
279 if common.catalog_name.is_some() {
280 bail!("`catalog.name` must not be set when `hosted_catalog` is set");
281 }
282 if self.jdbc_user.is_some() {
283 bail!("`catalog.jdbc.user` must not be set when `hosted_catalog` is set");
284 }
285 if self.jdbc_password.is_some() {
286 bail!("`catalog.jdbc.password` must not be set when `hosted_catalog` is set");
287 }
288 return Ok(());
289 }
290
291 if common.catalog_type.is_none() {
292 bail!("`catalog.type` must be set");
293 }
294
295 let iceberg_common = common.clone();
297
298 let mut java_map = HashMap::new();
299 if let Some(jdbc_user) = &self.jdbc_user {
300 java_map.insert("jdbc.user".to_owned(), jdbc_user.to_owned());
301 }
302 if let Some(jdbc_password) = &self.jdbc_password {
303 java_map.insert("jdbc.password".to_owned(), jdbc_password.to_owned());
304 }
305 let catalog = iceberg_common.create_catalog(&java_map).await?;
306 let test_table_ident = IcebergTableIdentifier {
308 database_name: Some("test_database".to_owned()),
309 table_name: "test_table".to_owned(),
310 }
311 .to_table_ident()?;
312 catalog.table_exists(&test_table_ident).await?;
313 Ok(())
314 }
315}
316
317#[serde_as]
318#[derive(Debug, Clone, Deserialize, WithOptions, PartialEq, Hash, Eq)]
319#[serde(deny_unknown_fields)]
320pub struct ConfluentSchemaRegistryConnection {
321 #[serde(rename = "schema.registry")]
322 pub url: String,
323 #[serde(rename = "schema.registry.username")]
325 pub username: Option<String>,
326 #[serde(rename = "schema.registry.password")]
327 pub password: Option<String>,
328}
329
330#[async_trait]
331impl Connection for ConfluentSchemaRegistryConnection {
332 async fn validate_connection(&self) -> ConnectorResult<()> {
333 let client = ConfluentSchemaRegistryClient::try_from(self)?;
335 client.validate_connection().await?;
336 Ok(())
337 }
338}
339
340impl EnforceSecret for ConfluentSchemaRegistryConnection {
341 const ENFORCE_SECRET_PROPERTIES: Set<&'static str> = phf_set! {
342 "schema.registry.password",
343 };
344}
345
346#[derive(Debug, Clone, Deserialize, PartialEq, Hash, Eq)]
347pub struct ElasticsearchConnection(pub BTreeMap<String, String>);
348
349#[async_trait]
350impl Connection for ElasticsearchConnection {
351 async fn validate_connection(&self) -> ConnectorResult<()> {
352 const CONNECTOR: &str = "elasticsearch";
353
354 let config = ElasticSearchOpenSearchConfig::try_from(self)?;
355 let client = config.build_client(CONNECTOR)?;
356 client.ping().await?;
357 Ok(())
358 }
359}
360
361impl EnforceSecret for ElasticsearchConnection {
362 const ENFORCE_SECRET_PROPERTIES: Set<&'static str> = phf_set! {
363 "elasticsearch.password",
364 };
365}