-
Notifications
You must be signed in to change notification settings - Fork 595
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: Connection for Kafka source & sink #19270
Changes from all 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
6a729f5
fbc9917
a79d5da
a561ea3
2d7ec88
3a18c4c
60c09fd
e9f8d72
23b2011
673bccb
635975d
0fd3972
0e01a05
01363ec
94f730e
5baccf3
f587e8b
1da1d1a
403868e
aaa6a34
1d2cb3d
45a9b8d
cc75ea7
a446707
e871ab7
eb371d5
99b2094
795a79d
c26c05b
64a4e26
28bb651
9c0e9df
e312ef2
d24fafb
dc3fd25
401660b
08fb575
48a28a2
d5be997
2f0a4a8
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 |
---|---|---|
@@ -0,0 +1,107 @@ | ||
control substitution on | ||
|
||
# for non-shared source | ||
statement ok | ||
set streaming_use_shared_source to false; | ||
|
||
statement ok | ||
create secret sec_broker with (backend = 'meta') as '${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}'; | ||
|
||
statement error unknown field `foo` | ||
create connection conn with (type = 'kafka', properties.bootstrap.server = secret sec_broker, foo = 'bar'); | ||
|
||
statement error Connection type "kinesis" is not supported | ||
create connection conn with (type = 'kinesis'); | ||
|
||
statement ok | ||
create connection conn with (type = 'kafka', properties.bootstrap.server = secret sec_broker, properties.security.protocol = 'plaintext'); | ||
|
||
query TTT | ||
select "name", "type_" from rw_catalog.rw_connections; | ||
---- | ||
conn CONNECTION_TYPE_KAFKA | ||
|
||
# unstable test serialization due to iter on hashmap | ||
# the "connectiond_params" looks like: | ||
# {"properties.bootstrap.server":"SECRET sec_broker AS TEXT","properties.security.protocol":"plaintext"} | ||
|
||
statement error Permission denied: PermissionDenied: secret used by 1 other objects. | ||
drop secret sec_broker; | ||
|
||
statement error Duplicated key in both WITH clause and Connection catalog: properties.security.protocol | ||
create table t1 (a int, b varchar) with ( | ||
connector = 'kafka', | ||
connection = conn, | ||
topic = 'connection_ddl_1', | ||
properties.security.protocol = 'plaintext') | ||
format plain encode json; | ||
|
||
statement error connector kinesis and connection type Kafka are not compatible | ||
create table t1 (a int, b varchar) with ( | ||
connector = 'kinesis', | ||
connection = conn, | ||
stream = 'connection_ddl_1', | ||
region = 'us-east-1') | ||
format plain encode json; | ||
|
||
system ok | ||
rpk topic create connection_ddl_1 -p 1 | ||
|
||
statement ok | ||
create table t1 (a int, b varchar) with ( | ||
connector = 'kafka', | ||
connection = conn, | ||
topic = 'connection_ddl_1') | ||
format plain encode json; | ||
|
||
statement error Permission denied: PermissionDenied: connection used by 1 other objects. | ||
drop connection conn; | ||
|
||
# Connection & Source & Sink will have independent rely on the secret | ||
statement error Permission denied: PermissionDenied: secret used by 2 other objects. | ||
drop secret sec_broker; | ||
|
||
statement ok | ||
create table data_table (a int, b varchar); | ||
|
||
statement ok | ||
insert into data_table values (1, 'a'), (2, 'b'), (3, 'c'); | ||
|
||
statement ok | ||
flush; | ||
|
||
statement ok | ||
create sink sink_kafka from data_table with ( | ||
connector = 'kafka', | ||
connection = conn, | ||
topic = 'connection_ddl_1' | ||
) format plain encode json ( | ||
force_append_only='true' | ||
); | ||
|
||
sleep 3s | ||
|
||
query IT rowsort | ||
select a, b from t1; | ||
---- | ||
1 a | ||
2 b | ||
3 c | ||
|
||
statement ok | ||
drop sink sink_kafka | ||
|
||
statement ok | ||
drop table data_table; | ||
|
||
statement ok | ||
drop table t1; | ||
|
||
statement ok | ||
drop connection conn; | ||
|
||
statement ok | ||
drop secret sec_broker; | ||
|
||
statement ok | ||
set streaming_use_shared_source to true; |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -92,6 +92,9 @@ message StreamSourceInfo { | |
// Handle the source relies on any sceret. The key is the propertity name and the value is the secret id and type. | ||
// For format and encode options. | ||
map<string, secret.SecretRef> format_encode_secret_refs = 16; | ||
|
||
// ref connection for schema registry | ||
optional uint32 connection_id = 17; | ||
} | ||
|
||
message WebhookSourceInfo { | ||
|
@@ -128,6 +131,8 @@ message Source { | |
uint32 associated_table_id = 12; | ||
} | ||
string definition = 13; | ||
|
||
// ref connection for connector | ||
optional uint32 connection_id = 14; | ||
|
||
optional uint64 initialized_at_epoch = 15; | ||
|
@@ -161,6 +166,9 @@ message SinkFormatDesc { | |
optional plan_common.EncodeType key_encode = 4; | ||
// Secret used for format encode options. | ||
map<string, secret.SecretRef> secret_refs = 5; | ||
|
||
// ref connection for schema registry | ||
optional uint32 connection_id = 6; | ||
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 think this should be added to 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. Let me clear the logic for the related proto defs
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 see. So here we expand all the connections into concrete parameters on creation, right? This is okay to me, but in this way, I think (It seems 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. Similar to the source side, the connector part and schema registry part will have independent connection, just like individual secret_refs. we are offering syntax like
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. Yeah, I know. What I mean is: should it be resolved and eliminated before converting into a proto message?
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.
We resolve the props in the frontend and the catalog is persisted in the meta. We have to reserve a field to keep the connection_id to maintain the dependency. And to keep the design simple and align with the secret ref, I think adding & We are going to support 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 see. You keep both
IIRC, secret ref doesn't keep the resolved plaintext secret at all. It always resolves whenever using a secret. 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.
Oh, here is a some gap on |
||
} | ||
|
||
// the catalog of the sink. There are two kind of schema here. The full schema is all columns | ||
|
@@ -183,6 +191,8 @@ message Sink { | |
uint32 owner = 11; | ||
map<string, string> properties = 12; | ||
string definition = 13; | ||
|
||
// ref connection for connector | ||
optional uint32 connection_id = 14; | ||
optional uint64 initialized_at_epoch = 15; | ||
optional uint64 created_at_epoch = 16; | ||
|
@@ -231,6 +241,19 @@ message Subscription { | |
SubscriptionState subscription_state = 19; | ||
} | ||
|
||
message ConnectionParams { | ||
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. This is kind of a weak-typed implementation, as the properties are kept as a On the contrary, we may optionally make it strong-typed by defining connection's fields as a I tend to prefer the latter one, but I am not sure how to handle secrets. Comments are welcomed. 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 had this idea before. Current impl overcomes the prior problem by merging everything into hashmap and keeps the original path to |
||
enum ConnectionType { | ||
CONNECTION_TYPE_UNSPECIFIED = 0; | ||
CONNECTION_TYPE_KAFKA = 1; | ||
CONNECTION_TYPE_ICEBERG = 2; | ||
CONNECTION_TYPE_SCHEMA_REGISTRY = 3; | ||
} | ||
|
||
ConnectionType connection_type = 1; | ||
map<string, string> properties = 2; | ||
map<string, secret.SecretRef> secret_refs = 3; | ||
} | ||
|
||
message Connection { | ||
message PrivateLinkService { | ||
enum PrivateLinkProvider { | ||
|
@@ -251,6 +274,7 @@ message Connection { | |
string name = 4; | ||
oneof info { | ||
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. No need to keep the 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. We are still using the proto in meta backup. Removing the 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. Well, then let's place |
||
PrivateLinkService private_link_service = 5 [deprecated = true]; | ||
ConnectionParams connection_params = 7; | ||
} | ||
uint32 owner = 6; | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,128 @@ | ||
// 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, KafkaConnectionProps, KafkaPrivateLinkCommon}; | ||
use crate::error::ConnectorResult; | ||
use crate::source::kafka::{KafkaContextCommon, RwConsumerContext}; | ||
use crate::{dispatch_connection_impl, ConnectionImpl}; | ||
|
||
#[async_trait] | ||
pub trait Connection { | ||
async fn test_connection(&self) -> ConnectorResult<()>; | ||
} | ||
|
||
#[serde_as] | ||
#[derive(Debug, Clone, Deserialize, WithOptions, PartialEq)] | ||
#[serde(deny_unknown_fields)] | ||
pub struct KafkaConnection { | ||
#[serde(flatten)] | ||
pub inner: KafkaConnectionProps, | ||
#[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 Connection 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)] | ||
#[serde(deny_unknown_fields)] | ||
pub struct IcebergConnection {} | ||
|
||
#[async_trait] | ||
impl Connection for IcebergConnection { | ||
async fn test_connection(&self) -> ConnectorResult<()> { | ||
todo!() | ||
} | ||
} | ||
|
||
#[serde_as] | ||
#[derive(Debug, Clone, Deserialize, WithOptions, PartialEq, Hash, Eq)] | ||
#[serde(deny_unknown_fields)] | ||
pub struct SchemaRegistryConnection {} | ||
|
||
#[async_trait] | ||
impl Connection for SchemaRegistryConnection { | ||
async fn test_connection(&self) -> ConnectorResult<()> { | ||
todo!() | ||
} | ||
} |
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.
So we just reuse
Source.connection_id
?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 and same for the schema registry part and sink.