1use std::collections::BTreeMap;
16
17use risingwave_common::bail;
18
19use super::simd_json_parser::DebeziumJsonAccessBuilder;
20use super::{DebeziumAvroAccessBuilder, DebeziumAvroParserConfig};
21use crate::error::ConnectorResult;
22use crate::parser::unified::debezium::DebeziumChangeEvent;
23use crate::parser::unified::json::TimestamptzHandling;
24use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer;
25use crate::parser::{
26 AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, ParseResult,
27 ParserFormat, ProtocolProperties, SourceStreamChunkRowWriter, SpecificParserConfig,
28};
29use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef};
30
31#[derive(Debug)]
32pub struct DebeziumParser {
33 key_builder: AccessBuilderImpl,
34 payload_builder: AccessBuilderImpl,
35 pub(crate) rw_columns: Vec<SourceColumnDesc>,
36 source_ctx: SourceContextRef,
37
38 props: DebeziumProps,
39}
40
41pub const DEBEZIUM_IGNORE_KEY: &str = "ignore_key";
42
43#[derive(Debug, Clone, Default)]
44pub struct DebeziumProps {
45 pub ignore_key: bool,
48}
49
50impl DebeziumProps {
51 pub fn from(props: &BTreeMap<String, String>) -> Self {
52 let ignore_key = props
53 .get(DEBEZIUM_IGNORE_KEY)
54 .map(|v| v.eq_ignore_ascii_case("true"))
55 .unwrap_or(false);
56 Self { ignore_key }
57 }
58}
59
60async fn build_accessor_builder(
61 config: EncodingProperties,
62 encoding_type: EncodingType,
63) -> ConnectorResult<AccessBuilderImpl> {
64 match config {
65 EncodingProperties::Avro(_) => {
66 let config = DebeziumAvroParserConfig::new(config).await?;
67 Ok(AccessBuilderImpl::DebeziumAvro(
68 DebeziumAvroAccessBuilder::new(config, encoding_type)?,
69 ))
70 }
71 EncodingProperties::Json(json_config) => Ok(AccessBuilderImpl::DebeziumJson(
72 DebeziumJsonAccessBuilder::new(
73 json_config
74 .timestamptz_handling
75 .unwrap_or(TimestamptzHandling::GuessNumberUnit),
76 )?,
77 )),
78 _ => bail!("unsupported encoding for Debezium"),
79 }
80}
81
82impl DebeziumParser {
83 pub async fn new(
84 props: SpecificParserConfig,
85 rw_columns: Vec<SourceColumnDesc>,
86 source_ctx: SourceContextRef,
87 ) -> ConnectorResult<Self> {
88 let key_builder =
89 build_accessor_builder(props.encoding_config.clone(), EncodingType::Key).await?;
90 let payload_builder =
91 build_accessor_builder(props.encoding_config, EncodingType::Value).await?;
92 let debezium_props = if let ProtocolProperties::Debezium(props) = props.protocol_config {
93 props
94 } else {
95 unreachable!(
96 "expecting Debezium protocol properties but got {:?}",
97 props.protocol_config
98 )
99 };
100 Ok(Self {
101 key_builder,
102 payload_builder,
103 rw_columns,
104 source_ctx,
105 props: debezium_props,
106 })
107 }
108
109 pub async fn new_for_test(rw_columns: Vec<SourceColumnDesc>) -> ConnectorResult<Self> {
110 use crate::parser::JsonProperties;
111
112 let props = SpecificParserConfig {
113 encoding_config: EncodingProperties::Json(JsonProperties {
114 use_schema_registry: false,
115 timestamptz_handling: None,
116 }),
117 protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()),
118 };
119 Self::new(props, rw_columns, SourceContext::dummy().into()).await
120 }
121
122 pub async fn parse_inner(
123 &mut self,
124 key: Option<Vec<u8>>,
125 payload: Option<Vec<u8>>,
126 mut writer: SourceStreamChunkRowWriter<'_>,
127 ) -> ConnectorResult<ParseResult> {
128 let meta = writer.source_meta();
129 let key_accessor = match (key, self.props.ignore_key) {
131 (None, false) => None,
132 (Some(data), false) => Some(self.key_builder.generate_accessor(data, meta).await?),
133 (_, true) => None,
134 };
135 let payload_accessor = match payload {
136 None => None,
137 Some(data) => Some(self.payload_builder.generate_accessor(data, meta).await?),
138 };
139 let row_op = DebeziumChangeEvent::new(key_accessor, payload_accessor);
140
141 match apply_row_operation_on_stream_chunk_writer(&row_op, &mut writer) {
142 Ok(_) => Ok(ParseResult::Rows),
143 Err(err) => {
144 if let Some(transaction_control) =
147 row_op.transaction_control(&self.source_ctx.connector_props)
148 {
149 Ok(ParseResult::TransactionControl(transaction_control))
150 } else {
151 Err(err)?
152 }
153 }
154 }
155 }
156}
157
158impl ByteStreamSourceParser for DebeziumParser {
159 fn columns(&self) -> &[SourceColumnDesc] {
160 &self.rw_columns
161 }
162
163 fn source_ctx(&self) -> &SourceContext {
164 &self.source_ctx
165 }
166
167 fn parser_format(&self) -> ParserFormat {
168 ParserFormat::Debezium
169 }
170
171 #[allow(clippy::unused_async)] async fn parse_one<'a>(
173 &'a mut self,
174 _key: Option<Vec<u8>>,
175 _payload: Option<Vec<u8>>,
176 _writer: SourceStreamChunkRowWriter<'a>,
177 ) -> ConnectorResult<()> {
178 unreachable!("should call `parse_one_with_txn` instead")
179 }
180
181 async fn parse_one_with_txn<'a>(
182 &'a mut self,
183 key: Option<Vec<u8>>,
184 payload: Option<Vec<u8>>,
185 writer: SourceStreamChunkRowWriter<'a>,
186 ) -> ConnectorResult<ParseResult> {
187 self.parse_inner(key, payload, writer).await
188 }
189}
190
191#[cfg(test)]
192mod tests {
193 use std::ops::Deref;
194 use std::sync::Arc;
195
196 use risingwave_common::catalog::{CDC_SOURCE_COLUMN_NUM, ColumnCatalog, ColumnDesc, ColumnId};
197 use risingwave_common::row::Row;
198 use risingwave_common::types::{DataType, Timestamptz};
199 use risingwave_pb::plan_common::{
200 AdditionalColumn, AdditionalColumnTimestamp, additional_column,
201 };
202
203 use super::*;
204 use crate::parser::{JsonProperties, SourceStreamChunkBuilder, TransactionControl};
205 use crate::source::{ConnectorProperties, SourceCtrlOpts};
206
207 #[tokio::test]
208 async fn test_parse_transaction_metadata() {
209 let schema = ColumnCatalog::debezium_cdc_source_cols();
210
211 let columns = schema
212 .iter()
213 .map(|c| SourceColumnDesc::from(&c.column_desc))
214 .collect::<Vec<_>>();
215
216 let props = SpecificParserConfig {
217 encoding_config: EncodingProperties::Json(JsonProperties {
218 use_schema_registry: false,
219 timestamptz_handling: None,
220 }),
221 protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()),
222 };
223 let source_ctx = SourceContext {
224 connector_props: ConnectorProperties::PostgresCdc(Box::default()),
225 ..SourceContext::dummy()
226 };
227 let mut parser = DebeziumParser::new(props, columns.clone(), Arc::new(source_ctx))
228 .await
229 .unwrap();
230 let mut dummy_builder = SourceStreamChunkBuilder::new(columns, SourceCtrlOpts::for_test());
231
232 let begin_msg = r#"{"schema":null,"payload":{"status":"BEGIN","id":"35352:3962948040","event_count":null,"data_collections":null,"ts_ms":1704269323180}}"#;
234 let commit_msg = r#"{"schema":null,"payload":{"status":"END","id":"35352:3962950064","event_count":11,"data_collections":[{"data_collection":"public.orders_tx","event_count":5},{"data_collection":"public.person","event_count":6}],"ts_ms":1704269323180}}"#;
235 let res = parser
236 .parse_one_with_txn(
237 None,
238 Some(begin_msg.as_bytes().to_vec()),
239 dummy_builder.row_writer(),
240 )
241 .await;
242 match res {
243 Ok(ParseResult::TransactionControl(TransactionControl::Begin { id })) => {
244 assert_eq!(id.deref(), "35352");
245 }
246 _ => panic!("unexpected parse result: {:?}", res),
247 }
248 let res = parser
249 .parse_one_with_txn(
250 None,
251 Some(commit_msg.as_bytes().to_vec()),
252 dummy_builder.row_writer(),
253 )
254 .await;
255 match res {
256 Ok(ParseResult::TransactionControl(TransactionControl::Commit { id })) => {
257 assert_eq!(id.deref(), "35352");
258 }
259 _ => panic!("unexpected parse result: {:?}", res),
260 }
261 }
262
263 #[tokio::test]
264 async fn test_parse_additional_columns() {
265 let columns = vec![
266 ColumnDesc::named("O_ORDERKEY", ColumnId::new(1), DataType::Int64),
267 ColumnDesc::named("O_CUSTKEY", ColumnId::new(2), DataType::Int64),
268 ColumnDesc::named("O_ORDERSTATUS", ColumnId::new(3), DataType::Varchar),
269 ColumnDesc::named("O_TOTALPRICE", ColumnId::new(4), DataType::Decimal),
270 ColumnDesc::named("O_ORDERDATE", ColumnId::new(5), DataType::Date),
271 ColumnDesc::named_with_additional_column(
272 "commit_ts",
273 ColumnId::new(6),
274 DataType::Timestamptz,
275 AdditionalColumn {
276 column_type: Some(additional_column::ColumnType::Timestamp(
277 AdditionalColumnTimestamp {},
278 )),
279 },
280 ),
281 ];
282
283 let columns = columns
284 .iter()
285 .map(SourceColumnDesc::from)
286 .collect::<Vec<_>>();
287
288 let props = SpecificParserConfig {
289 encoding_config: EncodingProperties::Json(JsonProperties {
290 use_schema_registry: false,
291 timestamptz_handling: None,
292 }),
293 protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()),
294 };
295 let source_ctx = SourceContext {
296 connector_props: ConnectorProperties::PostgresCdc(Box::default()),
297 ..SourceContext::dummy()
298 };
299 let mut parser = DebeziumParser::new(props, columns.clone(), Arc::new(source_ctx))
300 .await
301 .unwrap();
302 let mut builder = SourceStreamChunkBuilder::new(columns, SourceCtrlOpts::for_test());
303
304 let payload = r#"{ "payload": { "before": null, "after": { "O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" }, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null }, "op": "c", "ts_ms": 1695277757017, "transaction": null } }"#;
305
306 let res = parser
307 .parse_one_with_txn(
308 None,
309 Some(payload.as_bytes().to_vec()),
310 builder.row_writer(),
311 )
312 .await;
313 match res {
314 Ok(ParseResult::Rows) => {
315 builder.finish_current_chunk();
316 let chunk = builder.consume_ready_chunks().next().unwrap();
317 for (_, row) in chunk.rows() {
318 let commit_ts = row.datum_at(5).unwrap().into_timestamptz();
319 assert_eq!(commit_ts, Timestamptz::from_millis(1695277757000).unwrap());
320 }
321 }
322 _ => panic!("unexpected parse result: {:?}", res),
323 }
324 }
325
326 #[tokio::test]
327 async fn test_cdc_source_job_schema() {
328 let columns = ColumnCatalog::debezium_cdc_source_cols();
329 assert_eq!(CDC_SOURCE_COLUMN_NUM, columns.len() as u32);
331 }
332}