risingwave_connector/source/cdc/
mod.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
15pub mod enumerator;
16pub mod external;
17pub mod jni_source;
18pub mod source;
19pub mod split;
20
21use std::collections::{BTreeMap, HashMap};
22use std::marker::PhantomData;
23
24pub use enumerator::*;
25use itertools::Itertools;
26use risingwave_pb::catalog::PbSource;
27use risingwave_pb::connector_service::{PbSourceType, PbTableSchema, SourceType, TableSchema};
28use risingwave_pb::plan_common::ExternalTableDesc;
29use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn;
30use simd_json::prelude::ArrayTrait;
31pub use source::*;
32
33use crate::error::ConnectorResult;
34use crate::source::{SourceProperties, SplitImpl, TryFromBTreeMap};
35use crate::{for_all_classified_sources, impl_cdc_source_type};
36
37pub const CDC_CONNECTOR_NAME_SUFFIX: &str = "-cdc";
38pub const CDC_SNAPSHOT_MODE_KEY: &str = "debezium.snapshot.mode";
39pub const CDC_SNAPSHOT_BACKFILL: &str = "rw_cdc_backfill";
40pub const CDC_SHARING_MODE_KEY: &str = "rw.sharing.mode.enable";
41// User can set snapshot='false' to disable cdc backfill
42pub const CDC_BACKFILL_ENABLE_KEY: &str = "snapshot";
43pub const CDC_BACKFILL_SNAPSHOT_INTERVAL_KEY: &str = "snapshot.interval";
44pub const CDC_BACKFILL_SNAPSHOT_BATCH_SIZE_KEY: &str = "snapshot.batch_size";
45// We enable transaction for shared cdc source by default
46pub const CDC_TRANSACTIONAL_KEY: &str = "transactional";
47pub const CDC_WAIT_FOR_STREAMING_START_TIMEOUT: &str = "cdc.source.wait.streaming.start.timeout";
48pub const CDC_AUTO_SCHEMA_CHANGE_KEY: &str = "auto.schema.change";
49
50// User can set strong-schema='true' to enable strong schema for mongo cdc source
51pub const CDC_MONGODB_STRONG_SCHEMA_KEY: &str = "strong_schema";
52
53pub const MYSQL_CDC_CONNECTOR: &str = Mysql::CDC_CONNECTOR_NAME;
54pub const POSTGRES_CDC_CONNECTOR: &str = Postgres::CDC_CONNECTOR_NAME;
55pub const CITUS_CDC_CONNECTOR: &str = Citus::CDC_CONNECTOR_NAME;
56pub const MONGODB_CDC_CONNECTOR: &str = Mongodb::CDC_CONNECTOR_NAME;
57pub const SQL_SERVER_CDC_CONNECTOR: &str = SqlServer::CDC_CONNECTOR_NAME;
58
59/// Build a unique CDC table identifier from a source ID and external table name
60pub fn build_cdc_table_id(source_id: u32, external_table_name: &str) -> String {
61    format!("{}.{}", source_id, external_table_name)
62}
63
64pub trait CdcSourceTypeTrait: Send + Sync + Clone + std::fmt::Debug + 'static {
65    const CDC_CONNECTOR_NAME: &'static str;
66    fn source_type() -> CdcSourceType;
67}
68
69for_all_classified_sources!(impl_cdc_source_type);
70
71impl<'a> From<&'a str> for CdcSourceType {
72    fn from(name: &'a str) -> Self {
73        match name {
74            MYSQL_CDC_CONNECTOR => CdcSourceType::Mysql,
75            POSTGRES_CDC_CONNECTOR => CdcSourceType::Postgres,
76            CITUS_CDC_CONNECTOR => CdcSourceType::Citus,
77            MONGODB_CDC_CONNECTOR => CdcSourceType::Mongodb,
78            SQL_SERVER_CDC_CONNECTOR => CdcSourceType::SqlServer,
79            _ => CdcSourceType::Unspecified,
80        }
81    }
82}
83
84impl CdcSourceType {
85    pub fn as_str_name(&self) -> &str {
86        match self {
87            CdcSourceType::Mysql => "MySQL",
88            CdcSourceType::Postgres => "Postgres",
89            CdcSourceType::Citus => "Citus",
90            CdcSourceType::Mongodb => "MongoDB",
91            CdcSourceType::SqlServer => "SQL Server",
92            CdcSourceType::Unspecified => "Unspecified",
93        }
94    }
95}
96
97#[derive(Clone, Debug, Default)]
98pub struct CdcProperties<T: CdcSourceTypeTrait> {
99    /// Properties specified in the WITH clause by user
100    pub properties: BTreeMap<String, String>,
101
102    /// Schema of the source specified by users
103    pub table_schema: TableSchema,
104
105    /// Whether it is created by a cdc source job
106    pub is_cdc_source_job: bool,
107
108    /// For validation purpose, mark if the table is a backfill cdc table
109    pub is_backfill_table: bool,
110
111    pub _phantom: PhantomData<T>,
112}
113
114pub fn table_schema_exclude_additional_columns(table_schema: &TableSchema) -> TableSchema {
115    TableSchema {
116        columns: table_schema
117            .columns
118            .iter()
119            .filter(|col| {
120                col.additional_column
121                    .as_ref()
122                    .is_some_and(|val| val.column_type.is_none())
123            })
124            .cloned()
125            .collect(),
126        pk_indices: table_schema.pk_indices.clone(),
127    }
128}
129
130impl<T: CdcSourceTypeTrait> TryFromBTreeMap for CdcProperties<T> {
131    fn try_from_btreemap(
132        properties: BTreeMap<String, String>,
133        _deny_unknown_fields: bool,
134    ) -> ConnectorResult<Self> {
135        let is_share_source: bool = properties
136            .get(CDC_SHARING_MODE_KEY)
137            .is_some_and(|v| v == "true");
138        Ok(CdcProperties {
139            properties,
140            table_schema: Default::default(),
141            // TODO(siyuan): use serde to deserialize input hashmap
142            is_cdc_source_job: is_share_source,
143            is_backfill_table: false,
144            _phantom: PhantomData,
145        })
146    }
147}
148
149impl<T: CdcSourceTypeTrait> SourceProperties for CdcProperties<T>
150where
151    DebeziumCdcSplit<T>: TryFrom<SplitImpl, Error = crate::error::ConnectorError> + Into<SplitImpl>,
152    DebeziumSplitEnumerator<T>: ListCdcSplits<CdcSourceType = T>,
153{
154    type Split = DebeziumCdcSplit<T>;
155    type SplitEnumerator = DebeziumSplitEnumerator<T>;
156    type SplitReader = CdcSplitReader<T>;
157
158    const SOURCE_NAME: &'static str = T::CDC_CONNECTOR_NAME;
159
160    fn init_from_pb_source(&mut self, source: &PbSource) {
161        let pk_indices = source
162            .pk_column_ids
163            .iter()
164            .map(|&id| {
165                source
166                    .columns
167                    .iter()
168                    .position(|col| col.column_desc.as_ref().unwrap().column_id == id)
169                    .unwrap() as u32
170            })
171            .collect_vec();
172
173        let table_schema = PbTableSchema {
174            columns: source
175                .columns
176                .iter()
177                .flat_map(|col| &col.column_desc)
178                .filter(|col| {
179                    !matches!(
180                        col.generated_or_default_column,
181                        Some(GeneratedOrDefaultColumn::GeneratedColumn(_))
182                    )
183                })
184                .cloned()
185                .collect(),
186            pk_indices,
187        };
188        self.table_schema = table_schema;
189        if let Some(info) = source.info.as_ref() {
190            self.is_cdc_source_job = info.is_shared();
191        }
192    }
193
194    fn init_from_pb_cdc_table_desc(&mut self, table_desc: &ExternalTableDesc) {
195        let table_schema = TableSchema {
196            columns: table_desc
197                .columns
198                .iter()
199                .filter(|col| {
200                    !matches!(
201                        col.generated_or_default_column,
202                        Some(GeneratedOrDefaultColumn::GeneratedColumn(_))
203                    )
204                })
205                .cloned()
206                .collect(),
207            pk_indices: table_desc.stream_key.clone(),
208        };
209
210        self.table_schema = table_schema;
211        self.is_cdc_source_job = false;
212        self.is_backfill_table = true;
213    }
214}
215
216impl<T: CdcSourceTypeTrait> crate::source::UnknownFields for CdcProperties<T> {
217    fn unknown_fields(&self) -> HashMap<String, String> {
218        // FIXME: CDC does not handle unknown fields yet
219        HashMap::new()
220    }
221}
222
223impl<T: CdcSourceTypeTrait> CdcProperties<T> {
224    pub fn get_source_type_pb(&self) -> SourceType {
225        SourceType::from(T::source_type())
226    }
227}