-
Notifications
You must be signed in to change notification settings - Fork 591
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
tabversion
committed
Nov 18, 2024
1 parent
aaa6a34
commit 1d2cb3d
Showing
14 changed files
with
229 additions
and
70 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,125 @@ | ||
// Copyright 2024 RisingWave Labs | ||
// | ||
// Licensed under the Apache License, Version 2.0 (the "License"); | ||
// you may not use this file except in compliance with the License. | ||
// You may obtain a copy of the License at | ||
// | ||
// http://www.apache.org/licenses/LICENSE-2.0 | ||
// | ||
// Unless required by applicable law or agreed to in writing, software | ||
// distributed under the License is distributed on an "AS IS" BASIS, | ||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
// See the License for the specific language governing permissions and | ||
// limitations under the License. | ||
|
||
use std::time::Duration; | ||
|
||
use rdkafka::consumer::{BaseConsumer, Consumer}; | ||
use rdkafka::ClientConfig; | ||
use risingwave_common::secret::LocalSecretManager; | ||
use risingwave_pb::catalog::PbConnection; | ||
use serde_derive::Deserialize; | ||
use serde_with::serde_as; | ||
use tonic::async_trait; | ||
use with_options::WithOptions; | ||
|
||
use crate::connector_common::{AwsAuthProps, KafkaConnectionInner, KafkaPrivateLinkCommon}; | ||
use crate::error::ConnectorResult; | ||
use crate::source::kafka::{KafkaContextCommon, RwConsumerContext}; | ||
use crate::{dispatch_connection_impl, ConnectionImpl}; | ||
|
||
#[async_trait] | ||
pub trait ConnectionValidate { | ||
async fn test_connection(&self) -> ConnectorResult<()>; | ||
} | ||
|
||
#[serde_as] | ||
#[derive(Debug, Clone, Deserialize, WithOptions, PartialEq)] | ||
pub struct KafkaConnection { | ||
#[serde(flatten)] | ||
pub inner: KafkaConnectionInner, | ||
#[serde(flatten)] | ||
pub kafka_private_link_common: KafkaPrivateLinkCommon, | ||
#[serde(flatten)] | ||
pub aws_auth_props: AwsAuthProps, | ||
} | ||
|
||
pub async fn validate_connection(connection: &PbConnection) -> ConnectorResult<()> { | ||
if let Some(ref info) = connection.info { | ||
match info { | ||
risingwave_pb::catalog::connection::Info::ConnectionParams(cp) => { | ||
let options = cp.properties.clone().into_iter().collect(); | ||
let secret_refs = cp.secret_refs.clone().into_iter().collect(); | ||
let props_secret_resolved = | ||
LocalSecretManager::global().fill_secrets(options, secret_refs)?; | ||
let connection_impl = | ||
ConnectionImpl::from_proto(cp.connection_type(), props_secret_resolved)?; | ||
dispatch_connection_impl!(connection_impl, inner, inner.test_connection().await?) | ||
} | ||
risingwave_pb::catalog::connection::Info::PrivateLinkService(_) => unreachable!(), | ||
} | ||
} | ||
Ok(()) | ||
} | ||
|
||
#[async_trait] | ||
impl ConnectionValidate for KafkaConnection { | ||
async fn test_connection(&self) -> ConnectorResult<()> { | ||
let client = self.build_client().await?; | ||
// describe cluster here | ||
client.fetch_metadata(None, Duration::from_secs(10)).await?; | ||
Ok(()) | ||
} | ||
} | ||
|
||
impl KafkaConnection { | ||
async fn build_client(&self) -> ConnectorResult<BaseConsumer<RwConsumerContext>> { | ||
let mut config = ClientConfig::new(); | ||
let bootstrap_servers = &self.inner.brokers; | ||
let broker_rewrite_map = self.kafka_private_link_common.broker_rewrite_map.clone(); | ||
config.set("bootstrap.servers", bootstrap_servers); | ||
self.inner.set_security_properties(&mut config); | ||
|
||
// dup with Kafka Enumerator | ||
let ctx_common = KafkaContextCommon::new( | ||
broker_rewrite_map, | ||
None, | ||
None, | ||
self.aws_auth_props.clone(), | ||
self.inner.is_aws_msk_iam(), | ||
) | ||
.await?; | ||
let client_ctx = RwConsumerContext::new(ctx_common); | ||
let client: BaseConsumer<RwConsumerContext> = | ||
config.create_with_context(client_ctx).await?; | ||
if self.inner.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; | ||
} | ||
Ok(client) | ||
} | ||
} | ||
|
||
#[serde_as] | ||
#[derive(Debug, Clone, PartialEq, Eq, Deserialize, WithOptions)] | ||
pub struct IcebergConnection {} | ||
|
||
#[async_trait] | ||
impl ConnectionValidate for IcebergConnection { | ||
async fn test_connection(&self) -> ConnectorResult<()> { | ||
todo!() | ||
} | ||
} | ||
|
||
#[serde_as] | ||
#[derive(Debug, Clone, Deserialize, WithOptions, PartialEq, Hash, Eq)] | ||
pub struct SchemaRegistryConnection {} | ||
|
||
#[async_trait] | ||
impl ConnectionValidate for SchemaRegistryConnection { | ||
async fn test_connection(&self) -> ConnectorResult<()> { | ||
todo!() | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.