risingwave_connector/
with_options.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::collections::{BTreeMap, HashMap};
16use std::marker::PhantomData;
17use std::time::Duration;
18
19use risingwave_pb::secret::PbSecretRef;
20
21use crate::sink::catalog::SinkFormatDesc;
22use crate::source::cdc::MYSQL_CDC_CONNECTOR;
23use crate::source::cdc::external::CdcTableType;
24use crate::source::iceberg::ICEBERG_CONNECTOR;
25use crate::source::{
26    AZBLOB_CONNECTOR, GCS_CONNECTOR, KAFKA_CONNECTOR, LEGACY_S3_CONNECTOR, OPENDAL_S3_CONNECTOR,
27    POSIX_FS_CONNECTOR, UPSTREAM_SOURCE_KEY,
28};
29
30/// Marker trait for `WITH` options. Only for `#[derive(WithOptions)]`, should not be used manually.
31///
32/// This is used to ensure the `WITH` options types have reasonable structure.
33///
34/// TODO: add this bound for sink. There's a `SourceProperties` trait for sources, but no similar
35/// things for sinks.
36pub trait WithOptions {
37    #[doc(hidden)]
38    #[inline(always)]
39    fn assert_receiver_is_with_options(&self) {}
40}
41
42// Currently CDC properties are handled specially.
43// - It simply passes HashMap to Java DBZ.
44// - It's not handled by serde.
45// - It contains fields other than WITH options.
46// TODO: remove the workaround here. And also use #[derive] for it.
47
48impl<T: crate::source::cdc::CdcSourceTypeTrait> WithOptions
49    for crate::source::cdc::CdcProperties<T>
50{
51}
52
53// impl the trait for value types
54
55impl<T: WithOptions> WithOptions for Option<T> {}
56impl WithOptions for Vec<String> {}
57impl WithOptions for Vec<u64> {}
58impl WithOptions for HashMap<String, String> {}
59impl WithOptions for BTreeMap<String, String> {}
60
61impl WithOptions for String {}
62impl WithOptions for bool {}
63impl WithOptions for usize {}
64impl WithOptions for u8 {}
65impl WithOptions for u16 {}
66impl WithOptions for u32 {}
67impl WithOptions for u64 {}
68impl WithOptions for i32 {}
69impl WithOptions for i64 {}
70impl WithOptions for f64 {}
71impl WithOptions for std::time::Duration {}
72impl WithOptions for crate::connector_common::MqttQualityOfService {}
73impl WithOptions for crate::sink::file_sink::opendal_sink::PathPartitionPrefix {}
74impl WithOptions for crate::sink::kafka::CompressionCodec {}
75impl WithOptions for crate::source::filesystem::file_common::CompressionFormat {}
76impl WithOptions for nexmark::config::RateShape {}
77impl WithOptions for nexmark::event::EventType {}
78impl<T> WithOptions for PhantomData<T> {}
79
80pub trait Get {
81    fn get(&self, key: &str) -> Option<&String>;
82}
83
84impl Get for HashMap<String, String> {
85    fn get(&self, key: &str) -> Option<&String> {
86        self.get(key)
87    }
88}
89
90impl Get for BTreeMap<String, String> {
91    fn get(&self, key: &str) -> Option<&String> {
92        self.get(key)
93    }
94}
95
96/// Utility methods for `WITH` properties (`HashMap` and `BTreeMap`).
97pub trait WithPropertiesExt: Get + Sized {
98    #[inline(always)]
99    fn get_connector(&self) -> Option<String> {
100        self.get(UPSTREAM_SOURCE_KEY).map(|s| s.to_lowercase())
101    }
102
103    #[inline(always)]
104    fn is_kafka_connector(&self) -> bool {
105        let Some(connector) = self.get_connector() else {
106            return false;
107        };
108        connector == KAFKA_CONNECTOR
109    }
110
111    #[inline(always)]
112    fn is_mysql_cdc_connector(&self) -> bool {
113        let Some(connector) = self.get_connector() else {
114            return false;
115        };
116        connector == MYSQL_CDC_CONNECTOR
117    }
118
119    #[inline(always)]
120    fn get_sync_call_timeout(&self) -> Option<Duration> {
121        const SYNC_CALL_TIMEOUT_KEY: &str = "properties.sync.call.timeout"; // only from kafka props, add more if needed
122        self.get(SYNC_CALL_TIMEOUT_KEY)
123            // ignore the error is ok here, because we will parse the field again when building the properties and has more precise error message
124            .and_then(|s| duration_str::parse_std(s).ok())
125    }
126
127    #[inline(always)]
128    fn is_cdc_connector(&self) -> bool {
129        let Some(connector) = self.get_connector() else {
130            return false;
131        };
132        connector.contains("-cdc")
133    }
134
135    /// It is shared when `CREATE SOURCE`, and not shared when `CREATE TABLE`. So called "shareable".
136    fn is_shareable_cdc_connector(&self) -> bool {
137        self.is_cdc_connector() && CdcTableType::from_properties(self).can_backfill()
138    }
139
140    /// Tables with MySQL and PostgreSQL connectors are maintained for backward compatibility.
141    /// The newly added SQL Server CDC connector is only supported when created as shared.
142    fn is_shareable_only_cdc_connector(&self) -> bool {
143        self.is_cdc_connector() && CdcTableType::from_properties(self).shareable_only()
144    }
145
146    fn enable_transaction_metadata(&self) -> bool {
147        CdcTableType::from_properties(self).enable_transaction_metadata()
148    }
149
150    fn is_shareable_non_cdc_connector(&self) -> bool {
151        self.is_kafka_connector()
152    }
153
154    #[inline(always)]
155    fn is_iceberg_connector(&self) -> bool {
156        let Some(connector) = self.get_connector() else {
157            return false;
158        };
159        connector == ICEBERG_CONNECTOR
160    }
161
162    fn connector_need_pk(&self) -> bool {
163        // Currently only iceberg connector doesn't need primary key
164        // introduced in https://github.com/risingwavelabs/risingwave/pull/14971
165        // XXX: This seems not the correct way. Iceberg doesn't necessarily lack a PK.
166        // "batch source" doesn't need a PK?
167        // For streaming, if it has a PK, do we want to use it? It seems not safe.
168        !self.is_iceberg_connector()
169    }
170
171    fn is_legacy_fs_connector(&self) -> bool {
172        self.get(UPSTREAM_SOURCE_KEY)
173            .map(|s| s.eq_ignore_ascii_case(LEGACY_S3_CONNECTOR))
174            .unwrap_or(false)
175    }
176
177    fn is_new_fs_connector(&self) -> bool {
178        self.get(UPSTREAM_SOURCE_KEY)
179            .map(|s| {
180                s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR)
181                    || s.eq_ignore_ascii_case(POSIX_FS_CONNECTOR)
182                    || s.eq_ignore_ascii_case(GCS_CONNECTOR)
183                    || s.eq_ignore_ascii_case(AZBLOB_CONNECTOR)
184            })
185            .unwrap_or(false)
186    }
187}
188
189impl<T: Get> WithPropertiesExt for T {}
190
191/// Options or properties extracted from the `WITH` clause of DDLs.
192#[derive(Default, Clone, Debug, PartialEq, Eq, Hash)]
193pub struct WithOptionsSecResolved {
194    inner: BTreeMap<String, String>,
195    secret_ref: BTreeMap<String, PbSecretRef>,
196}
197
198impl std::ops::Deref for WithOptionsSecResolved {
199    type Target = BTreeMap<String, String>;
200
201    fn deref(&self) -> &Self::Target {
202        &self.inner
203    }
204}
205
206impl std::ops::DerefMut for WithOptionsSecResolved {
207    fn deref_mut(&mut self) -> &mut Self::Target {
208        &mut self.inner
209    }
210}
211
212impl WithOptionsSecResolved {
213    /// Create a new [`WithOptions`] from a option [`BTreeMap`] and resolved secret ref.
214    pub fn new(inner: BTreeMap<String, String>, secret_ref: BTreeMap<String, PbSecretRef>) -> Self {
215        Self { inner, secret_ref }
216    }
217
218    /// Create a new [`WithOptions`] from a [`BTreeMap`].
219    pub fn without_secrets(inner: BTreeMap<String, String>) -> Self {
220        Self {
221            inner,
222            secret_ref: Default::default(),
223        }
224    }
225
226    /// Take the value of the option map and secret refs.
227    pub fn into_parts(self) -> (BTreeMap<String, String>, BTreeMap<String, PbSecretRef>) {
228        (self.inner, self.secret_ref)
229    }
230
231    pub fn value_eq_ignore_case(&self, key: &str, val: &str) -> bool {
232        if let Some(inner_val) = self.inner.get(key) {
233            if inner_val.eq_ignore_ascii_case(val) {
234                return true;
235            }
236        }
237        false
238    }
239}
240
241/// For `planner_test` crate so that it does not depend directly on `connector` crate just for `SinkFormatDesc`.
242impl TryFrom<&WithOptionsSecResolved> for Option<SinkFormatDesc> {
243    type Error = crate::sink::SinkError;
244
245    fn try_from(value: &WithOptionsSecResolved) -> std::result::Result<Self, Self::Error> {
246        let connector = value.get(crate::sink::CONNECTOR_TYPE_KEY);
247        let r#type = value.get(crate::sink::SINK_TYPE_OPTION);
248        match (connector, r#type) {
249            (Some(c), Some(t)) => SinkFormatDesc::from_legacy_type(c, t),
250            _ => Ok(None),
251        }
252    }
253}
254
255impl Get for WithOptionsSecResolved {
256    fn get(&self, key: &str) -> Option<&String> {
257        self.inner.get(key)
258    }
259}