risingwave_connector/
with_options.rs

1// Copyright 2023 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, BTreeSet, HashMap};
16use std::marker::PhantomData;
17use std::time::Duration;
18
19use risingwave_pb::id::SecretId;
20use risingwave_pb::secret::PbSecretRef;
21
22use crate::error::ConnectorResult;
23use crate::sink::catalog::SinkFormatDesc;
24use crate::source::cdc::MYSQL_CDC_CONNECTOR;
25use crate::source::cdc::external::ExternalCdcTableType;
26use crate::source::iceberg::ICEBERG_CONNECTOR;
27use crate::source::{
28    ADBC_SNOWFLAKE_CONNECTOR, AZBLOB_CONNECTOR, BATCH_POSIX_FS_CONNECTOR, GCS_CONNECTOR,
29    KAFKA_CONNECTOR, LEGACY_S3_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR,
30    PULSAR_CONNECTOR, UPSTREAM_SOURCE_KEY,
31};
32
33/// Marker trait for `WITH` options. Only for `#[derive(WithOptions)]`, should not be used manually.
34///
35/// This is used to ensure the `WITH` options types have reasonable structure.
36///
37/// TODO: add this bound for sink. There's a `SourceProperties` trait for sources, but no similar
38/// things for sinks.
39pub trait WithOptions {
40    #[doc(hidden)]
41    #[inline(always)]
42    fn assert_receiver_is_with_options(&self) {}
43}
44
45// Currently CDC properties are handled specially.
46// - It simply passes HashMap to Java DBZ.
47// - It's not handled by serde.
48// - It contains fields other than WITH options.
49// TODO: remove the workaround here. And also use #[derive] for it.
50
51impl<T: crate::source::cdc::CdcSourceTypeTrait> WithOptions
52    for crate::source::cdc::CdcProperties<T>
53{
54}
55
56// impl the trait for value types
57
58impl<T: WithOptions> WithOptions for Option<T> {}
59impl WithOptions for Vec<String> {}
60impl WithOptions for Vec<u64> {}
61impl WithOptions for HashMap<String, String> {}
62impl WithOptions for BTreeMap<String, String> {}
63
64impl WithOptions for String {}
65impl WithOptions for bool {}
66impl WithOptions for usize {}
67impl WithOptions for u8 {}
68impl WithOptions for u16 {}
69impl WithOptions for u32 {}
70impl WithOptions for u64 {}
71impl WithOptions for i32 {}
72impl WithOptions for i64 {}
73impl WithOptions for f64 {}
74impl WithOptions for std::time::Duration {}
75impl WithOptions for crate::connector_common::MqttQualityOfService {}
76impl WithOptions for crate::sink::file_sink::opendal_sink::PathPartitionPrefix {}
77impl WithOptions for crate::sink::kafka::CompressionCodec {}
78impl WithOptions for crate::source::filesystem::file_common::CompressionFormat {}
79impl WithOptions for nexmark::config::RateShape {}
80impl WithOptions for nexmark::event::EventType {}
81impl<T> WithOptions for PhantomData<T> {}
82
83pub trait Get {
84    fn get(&self, key: &str) -> Option<&String>;
85}
86
87pub trait GetKeyIter {
88    fn key_iter(&self) -> impl Iterator<Item = &str>;
89}
90
91impl GetKeyIter for HashMap<String, String> {
92    fn key_iter(&self) -> impl Iterator<Item = &str> {
93        self.keys().map(|s| s.as_str())
94    }
95}
96
97impl Get for HashMap<String, String> {
98    fn get(&self, key: &str) -> Option<&String> {
99        self.get(key)
100    }
101}
102
103impl Get for BTreeMap<String, String> {
104    fn get(&self, key: &str) -> Option<&String> {
105        self.get(key)
106    }
107}
108
109impl GetKeyIter for BTreeMap<String, String> {
110    fn key_iter(&self) -> impl Iterator<Item = &str> {
111        self.keys().map(|s| s.as_str())
112    }
113}
114
115/// Utility methods for `WITH` properties (`HashMap` and `BTreeMap`).
116pub trait WithPropertiesExt: Get + GetKeyIter + Sized {
117    #[inline(always)]
118    fn get_connector(&self) -> Option<String> {
119        self.get(UPSTREAM_SOURCE_KEY).map(|s| s.to_lowercase())
120    }
121
122    #[inline(always)]
123    fn is_kafka_connector(&self) -> bool {
124        let Some(connector) = self.get_connector() else {
125            return false;
126        };
127        connector == KAFKA_CONNECTOR
128    }
129
130    #[inline(always)]
131    fn is_pulsar_connector(&self) -> bool {
132        let Some(connector) = self.get_connector() else {
133            return false;
134        };
135        connector == PULSAR_CONNECTOR
136    }
137
138    #[inline(always)]
139    fn is_mysql_cdc_connector(&self) -> bool {
140        let Some(connector) = self.get_connector() else {
141            return false;
142        };
143        connector == MYSQL_CDC_CONNECTOR
144    }
145
146    #[inline(always)]
147    fn get_sync_call_timeout(&self) -> Option<Duration> {
148        const SYNC_CALL_TIMEOUT_KEY: &str = "properties.sync.call.timeout"; // only from kafka props, add more if needed
149        self.get(SYNC_CALL_TIMEOUT_KEY)
150            // ignore the error is ok here, because we will parse the field again when building the properties and has more precise error message
151            .and_then(|s| duration_str::parse_std(s).ok())
152    }
153
154    #[inline(always)]
155    fn is_cdc_connector(&self) -> bool {
156        let Some(connector) = self.get_connector() else {
157            return false;
158        };
159        connector.contains("-cdc")
160    }
161
162    /// It is shared when `CREATE SOURCE`, and not shared when `CREATE TABLE`. So called "shareable".
163    fn is_shareable_cdc_connector(&self) -> bool {
164        self.is_cdc_connector() && ExternalCdcTableType::from_properties(self).can_backfill()
165    }
166
167    fn enable_transaction_metadata(&self) -> bool {
168        ExternalCdcTableType::from_properties(self).enable_transaction_metadata()
169    }
170
171    fn is_shareable_non_cdc_connector(&self) -> bool {
172        self.is_kafka_connector()
173    }
174
175    #[inline(always)]
176    fn is_iceberg_connector(&self) -> bool {
177        let Some(connector) = self.get_connector() else {
178            return false;
179        };
180        connector == ICEBERG_CONNECTOR
181    }
182
183    fn connector_need_pk(&self) -> bool {
184        // Currently only iceberg connector doesn't need primary key
185        // introduced in https://github.com/risingwavelabs/risingwave/pull/14971
186        // XXX: This seems not the correct way. Iceberg doesn't necessarily lack a PK.
187        // "batch source" doesn't need a PK?
188        // For streaming, if it has a PK, do we want to use it? It seems not safe.
189        !self.is_iceberg_connector()
190    }
191
192    fn is_legacy_fs_connector(&self) -> bool {
193        self.get(UPSTREAM_SOURCE_KEY)
194            .map(|s| s.eq_ignore_ascii_case(LEGACY_S3_CONNECTOR))
195            .unwrap_or(false)
196    }
197
198    fn is_new_fs_connector(&self) -> bool {
199        self.get(UPSTREAM_SOURCE_KEY)
200            .map(|s| {
201                s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR)
202                    || s.eq_ignore_ascii_case(POSIX_FS_CONNECTOR)
203                    || s.eq_ignore_ascii_case(GCS_CONNECTOR)
204                    || s.eq_ignore_ascii_case(AZBLOB_CONNECTOR)
205            })
206            .unwrap_or(false)
207    }
208
209    fn is_batch_connector(&self) -> bool {
210        self.get(UPSTREAM_SOURCE_KEY)
211            .map(|s| {
212                s.eq_ignore_ascii_case(BATCH_POSIX_FS_CONNECTOR)
213                    || s.eq_ignore_ascii_case(ADBC_SNOWFLAKE_CONNECTOR)
214            })
215            .unwrap_or(false)
216    }
217
218    fn requires_singleton(&self) -> bool {
219        self.is_new_fs_connector() || self.is_iceberg_connector() || self.is_batch_connector()
220    }
221}
222
223impl<T: Get + GetKeyIter> WithPropertiesExt for T {}
224
225/// Options or properties extracted from the `WITH` clause of DDLs.
226#[derive(Default, Clone, Debug, PartialEq, Eq, Hash)]
227pub struct WithOptionsSecResolved {
228    inner: BTreeMap<String, String>,
229    secret_ref: BTreeMap<String, PbSecretRef>,
230}
231
232impl std::ops::Deref for WithOptionsSecResolved {
233    type Target = BTreeMap<String, String>;
234
235    fn deref(&self) -> &Self::Target {
236        &self.inner
237    }
238}
239
240impl std::ops::DerefMut for WithOptionsSecResolved {
241    fn deref_mut(&mut self) -> &mut Self::Target {
242        &mut self.inner
243    }
244}
245
246impl WithOptionsSecResolved {
247    /// Create a new [`WithOptions`] from a option [`BTreeMap`] and resolved secret ref.
248    pub fn new(inner: BTreeMap<String, String>, secret_ref: BTreeMap<String, PbSecretRef>) -> Self {
249        Self { inner, secret_ref }
250    }
251
252    pub fn as_plaintext(&self) -> &BTreeMap<String, String> {
253        &self.inner
254    }
255
256    pub fn as_secret(&self) -> &BTreeMap<String, PbSecretRef> {
257        &self.secret_ref
258    }
259
260    pub fn handle_update(
261        &mut self,
262        update_alter_props: BTreeMap<String, String>,
263        update_alter_secret_refs: BTreeMap<String, PbSecretRef>,
264    ) -> ConnectorResult<(Vec<SecretId>, Vec<SecretId>)> {
265        let old_secret_ids = self
266            .secret_ref
267            .values()
268            .map(|secret_ref| secret_ref.secret_id)
269            .collect::<BTreeSet<_>>();
270
271        // make sure the key in update_alter_props and update_alter_secret_refs not collide
272        for key in update_alter_props.keys() {
273            if update_alter_secret_refs.contains_key(key) {
274                return Err(
275                    anyhow::anyhow!("the key {} is set both in plaintext and secret", key).into(),
276                );
277            }
278        }
279
280        // remove legacy key if it's set in both plaintext and secret
281        // When a property changes from secret to plaintext, remove the old secret dependency
282        for k in update_alter_props.keys() {
283            self.secret_ref.remove(k);
284        }
285
286        // Handle secret ref updates
287        for k in update_alter_secret_refs.keys() {
288            // Remove any plaintext value for this key
289            self.inner.remove(k);
290        }
291
292        self.inner.extend(update_alter_props);
293        self.secret_ref.extend(update_alter_secret_refs);
294
295        let new_secret_ids = self
296            .secret_ref
297            .values()
298            .map(|secret_ref| secret_ref.secret_id)
299            .collect::<BTreeSet<_>>();
300        let to_add_secret_dep = new_secret_ids
301            .difference(&old_secret_ids)
302            .copied()
303            .collect();
304        let to_remove_secret_dep = old_secret_ids
305            .difference(&new_secret_ids)
306            .copied()
307            .collect();
308
309        Ok((to_add_secret_dep, to_remove_secret_dep))
310    }
311
312    /// Create a new [`WithOptions`] from a [`BTreeMap`].
313    pub fn without_secrets(inner: BTreeMap<String, String>) -> Self {
314        Self {
315            inner,
316            secret_ref: Default::default(),
317        }
318    }
319
320    /// Take the value of the option map and secret refs.
321    pub fn into_parts(self) -> (BTreeMap<String, String>, BTreeMap<String, PbSecretRef>) {
322        (self.inner, self.secret_ref)
323    }
324
325    pub fn value_eq_ignore_case(&self, key: &str, val: &str) -> bool {
326        if let Some(inner_val) = self.inner.get(key)
327            && inner_val.eq_ignore_ascii_case(val)
328        {
329            return true;
330        }
331        false
332    }
333}
334
335/// For `planner_test` crate so that it does not depend directly on `connector` crate just for `SinkFormatDesc`.
336impl TryFrom<&WithOptionsSecResolved> for Option<SinkFormatDesc> {
337    type Error = crate::sink::SinkError;
338
339    fn try_from(value: &WithOptionsSecResolved) -> std::result::Result<Self, Self::Error> {
340        let connector = value.get(crate::sink::CONNECTOR_TYPE_KEY);
341        let r#type = value.get(crate::sink::SINK_TYPE_OPTION);
342        match (connector, r#type) {
343            (Some(c), Some(t)) => SinkFormatDesc::from_legacy_type(c, t),
344            _ => Ok(None),
345        }
346    }
347}
348
349impl Get for WithOptionsSecResolved {
350    fn get(&self, key: &str) -> Option<&String> {
351        self.inner.get(key)
352    }
353}
354
355impl GetKeyIter for WithOptionsSecResolved {
356    fn key_iter(&self) -> impl Iterator<Item = &str> {
357        self.inner.keys().map(|s| s.as_str())
358    }
359}