Skip to content
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 connector usage #19270

Draft
wants to merge 42 commits into
base: main
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
42 commits
Select commit Hold shift + click to select a range
b9b2f79
share kafka client on meta
tabVersion Oct 22, 2024
ab46672
check kafka connection identical
tabVersion Oct 22, 2024
3373a58
fix
tabVersion Oct 22, 2024
c5203d6
fix with props
tabVersion Oct 22, 2024
3b6a6a2
fix
tabVersion Oct 22, 2024
3b3f725
use connection hash as hashmap entry
tabVersion Oct 22, 2024
968ed08
fix
tabVersion Oct 22, 2024
a41f3fc
fix
Oct 23, 2024
6534ebf
rerun
tabVersion Oct 23, 2024
7249e78
Merge branch 'tab/share-kafka-client-enum' of https://github.com/risi…
tabVersion Oct 23, 2024
0024e1d
Merge remote-tracking branch 'origin' into tab/share-kafka-client-enum
tabVersion Oct 23, 2024
ad8b989
fix
tabVersion Oct 23, 2024
ae1b70a
fix
tabVersion Oct 23, 2024
58b5128
better with options
Oct 25, 2024
f115a0c
use kafka connection as hashkey
Oct 25, 2024
d128644
use moka
Oct 25, 2024
ae9df41
fix lint
Oct 25, 2024
45295bc
fix
Oct 25, 2024
a9e34c7
fix
Oct 25, 2024
d27ab90
Merge branch 'main' into tab/share-kafka-client-enum
tabVersion Oct 25, 2024
256485c
Merge remote-tracking branch 'origin' into tab/share-kafka-client-enum
Oct 28, 2024
62cb953
Merge branch 'tab/share-kafka-client-enum' of https://github.com/risi…
Oct 28, 2024
725e23c
remove get hash func
Oct 28, 2024
832f66f
migrate to Weak
Oct 28, 2024
73f0b7b
minor
Oct 28, 2024
ac1d63d
fix
Oct 28, 2024
35fb002
Merge remote-tracking branch 'origin' into tab/share-kafka-client-enum
tabVersion Oct 29, 2024
ec49096
test bump quanta to 0.12.3
tabVersion Oct 29, 2024
16d8c42
update patch
tabVersion Oct 29, 2024
b3efda6
moka 0.12.3
Oct 30, 2024
6a729f5
update proto
Nov 1, 2024
fbc9917
Merge remote-tracking branch 'origin' into tab/connection
Nov 1, 2024
a79d5da
stash
Nov 2, 2024
a561ea3
stash
Nov 2, 2024
2d7ec88
Merge remote-tracking branch 'origin' into tab/connection
Nov 2, 2024
3a18c4c
stash
Nov 3, 2024
60c09fd
basic
Nov 4, 2024
e9f8d72
handle secret ref
Nov 5, 2024
23b2011
Merge remote-tracking branch 'origin' into tab/connection
Nov 5, 2024
673bccb
stash
Nov 6, 2024
635975d
fix
Nov 6, 2024
0fd3972
Merge remote-tracking branch 'origin' into tab/connection
Nov 7, 2024
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 5 additions & 1 deletion Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -153,7 +153,11 @@ arrow-udf-flight = "0.4"
clap = { version = "4", features = ["cargo", "derive", "env"] }
# Use a forked version which removes the dependencies on dynamo db to reduce
# compile time and binary size.
deltalake = { version = "0.20.1", features = ["s3", "gcs", "datafusion"] }
deltalake = { version = "0.20.1", features = [
"s3",
"gcs",
"datafusion",
] }
itertools = "0.13.0"
jsonbb = "0.1.4"
lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "2682b85" }
Expand Down
24 changes: 24 additions & 0 deletions proto/catalog.proto
Original file line number Diff line number Diff line change
Expand Up @@ -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 Source {
Expand Down Expand Up @@ -123,6 +126,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;
Expand Down Expand Up @@ -156,6 +161,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;
}

// the catalog of the sink. There are two kind of schema here. The full schema is all columns
Expand All @@ -178,6 +186,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;
Expand Down Expand Up @@ -226,6 +236,19 @@ message Subscription {
SubscriptionState subscription_state = 19;
}

message ConnectionParams {
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 {
Expand All @@ -246,6 +269,7 @@ message Connection {
string name = 4;
oneof info {
PrivateLinkService private_link_service = 5 [deprecated = true];
ConnectionParams connection_params = 7;
}
uint32 owner = 6;
}
Expand Down
3 changes: 2 additions & 1 deletion proto/ddl_service.proto
Original file line number Diff line number Diff line change
Expand Up @@ -424,7 +424,8 @@ message CreateConnectionRequest {
uint32 database_id = 2;
uint32 schema_id = 3;
oneof payload {
PrivateLink private_link = 4;
PrivateLink private_link = 4 [deprecated = true];
catalog.ConnectionParams connection_params = 6;
}
uint32 owner_id = 5;
}
Expand Down
13 changes: 13 additions & 0 deletions src/connector/src/connector_common/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -26,3 +26,16 @@ pub use common::{

mod iceberg;
pub use iceberg::IcebergCommon;

// #[derive(Debug, Clone, Deserialize)]
// pub enum ConnectionImpl {
// Kafka(KafkaConnection),
// }

// macro_rules! impl_connection_enum {
// ($impl:expr, $inner_name:ident, $prop_type_name:ident, $body:expr) => {
// impl ConnectionImpl {
// pub fn get_
// }
// };
// }
1 change: 1 addition & 0 deletions src/connector/src/sink/catalog/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -221,6 +221,7 @@ impl SinkFormatDesc {
options,
key_encode,
secret_refs: self.secret_refs.clone(),
connection_id: None,
}
}
}
Expand Down
7 changes: 7 additions & 0 deletions src/ctl/src/cmd_impl/meta/connection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,13 @@ pub async fn list_connections(context: &CtlContext) -> anyhow::Result<()> {
"PrivateLink: service_name: {}, endpoint_id: {}, dns_entries: {:?}",
svc.service_name, svc.endpoint_id, svc.dns_entries,
),
Some(Info::ConnectionParams(params)) => {
format!(
"CONNECTION_PARAMS_{}: {}",
params.get_connection_type().unwrap().as_str_name(),
serde_json::to_string(&params.get_properties()).unwrap()
)
}
None => "None".to_string(),
}
);
Expand Down
2 changes: 2 additions & 0 deletions src/frontend/src/catalog/connection_catalog.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,12 +30,14 @@ impl ConnectionCatalog {
pub fn connection_type(&self) -> &str {
match &self.info {
Info::PrivateLinkService(srv) => srv.get_provider().unwrap().as_str_name(),
Info::ConnectionParams(params) => params.get_connection_type().unwrap().as_str_name(),
}
}

pub fn provider(&self) -> &str {
match &self.info {
Info::PrivateLinkService(_) => "PRIVATELINK",
Info::ConnectionParams(_) => panic!("ConnectionParams is not supported as provider."),
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,23 +28,39 @@ struct RwConnection {
type_: String,
provider: String,
acl: String,
connection_params: String,
}

#[system_catalog(table, "rw_catalog.rw_connections")]
fn read_rw_connections(reader: &SysCatalogReaderImpl) -> Result<Vec<RwConnection>> {
let catalog_reader = reader.catalog_reader.read_guard();
let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?;

// todo: redesign the internal table for connection params
Ok(schemas
.flat_map(|schema| {
schema.iter_connections().map(|conn| RwConnection {
id: conn.id as i32,
name: conn.name.clone(),
schema_id: schema.id() as i32,
owner: conn.owner as i32,
type_: conn.connection_type().into(),
provider: conn.provider().into(),
acl: "".into(),
schema.iter_connections().map(|conn| {
let mut rw_connection = RwConnection {
id: conn.id as i32,
name: conn.name.clone(),
schema_id: schema.id() as i32,
owner: conn.owner as i32,
type_: conn.connection_type().into(),
provider: "".to_string(),
acl: "".into(),
connection_params: "".to_string(),
};
match &conn.info {
risingwave_pb::catalog::connection::Info::PrivateLinkService(_) => {
rw_connection.provider = conn.provider().into();
}
risingwave_pb::catalog::connection::Info::ConnectionParams(params) => {
rw_connection.connection_params =
serde_json::to_string(&params.get_properties()).unwrap();
}
};

rw_connection
})
})
.collect())
Expand Down
66 changes: 45 additions & 21 deletions src/frontend/src/handler/create_connection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,10 @@
use std::collections::BTreeMap;

use pgwire::pg_response::{PgResponse, StatementType};
use risingwave_connector::source::kafka::PRIVATELINK_CONNECTION;
use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR;
use risingwave_connector::source::kafka::{KAFKA_CONNECTOR, PRIVATELINK_CONNECTION};
use risingwave_pb::catalog::connection_params::ConnectionType;
use risingwave_pb::catalog::ConnectionParams;
use risingwave_pb::ddl_service::create_connection_request;
use risingwave_sqlparser::ast::CreateConnectionStatement;

Expand All @@ -24,36 +27,58 @@ use crate::binder::Binder;
use crate::error::ErrorCode::ProtocolError;
use crate::error::{ErrorCode, Result, RwError};
use crate::handler::HandlerArgs;
use crate::session::SessionImpl;
use crate::utils::resolve_secret_ref_in_with_options;
use crate::WithOptions;

pub(crate) const CONNECTION_TYPE_PROP: &str = "type";

#[inline(always)]
fn get_connection_property_required(
with_properties: &BTreeMap<String, String>,
with_properties: &mut BTreeMap<String, String>,
property: &str,
) -> Result<String> {
with_properties
.get(property)
.map(|s| s.to_lowercase())
.ok_or_else(|| {
RwError::from(ProtocolError(format!(
"Required property \"{property}\" is not provided"
)))
})
with_properties.remove(property).ok_or_else(|| {
RwError::from(ProtocolError(format!(
"Required property \"{property}\" is not provided"
)))
})
}
fn resolve_create_connection_payload(
with_properties: &BTreeMap<String, String>,
with_properties: WithOptions,
session: &SessionImpl,
) -> Result<create_connection_request::Payload> {
let connection_type = get_connection_property_required(with_properties, CONNECTION_TYPE_PROP)?;
match connection_type.as_str() {
PRIVATELINK_CONNECTION => Err(RwError::from(ErrorCode::Deprecated(
if !with_properties.connection_ref().is_empty() {
return Err(RwError::from(ErrorCode::InvalidParameterValue(
"Connection reference is not allowed in options in CREATE CONNECTION".to_string(),
)));
}

let (mut props, secret_refs) =
resolve_secret_ref_in_with_options(with_properties, session)?.into_parts();
let connection_type = get_connection_property_required(&mut props, CONNECTION_TYPE_PROP)?;
let connection_type = match connection_type.as_str() {
PRIVATELINK_CONNECTION => {
return Err(RwError::from(ErrorCode::Deprecated(
"CREATE CONNECTION to Private Link".to_string(),
"RisingWave Cloud Portal (Please refer to the doc https://docs.risingwave.com/cloud/create-a-connection/)".to_string(),
))),
_ => Err(RwError::from(ProtocolError(format!(
"Connection type \"{connection_type}\" is not supported"
)))),
}
)));
}
KAFKA_CONNECTOR => ConnectionType::Kafka,
ICEBERG_CONNECTOR => ConnectionType::Iceberg,
_ => {
return Err(RwError::from(ProtocolError(format!(
"Connection type \"{connection_type}\" is not supported"
))));
}
};
Ok(create_connection_request::Payload::ConnectionParams(
ConnectionParams {
connection_type: connection_type as i32,
properties: props.into_iter().collect(),
secret_refs: secret_refs.into_iter().collect(),
},
))
}

pub async fn handle_create_connection(
Expand All @@ -79,8 +104,7 @@ pub async fn handle_create_connection(
}
let (database_id, schema_id) = session.get_database_and_schema_id_for_create(schema_name)?;
let with_properties = handler_args.with_options.clone().into_connector_props();

let create_connection_payload = resolve_create_connection_payload(&with_properties)?;
let create_connection_payload = resolve_create_connection_payload(with_properties, &session)?;

let catalog_writer = session.catalog_writer()?;
catalog_writer
Expand Down
Loading
Loading