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::{TimeHandling, TimestampHandling, 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 json_config
77 .timestamp_handling
78 .unwrap_or(TimestampHandling::GuessNumberUnit),
79 json_config.time_handling.unwrap_or(TimeHandling::Micro),
80 )?,
81 )),
82 _ => bail!("unsupported encoding for Debezium"),
83 }
84}
85
86impl DebeziumParser {
87 pub async fn new(
88 props: SpecificParserConfig,
89 rw_columns: Vec<SourceColumnDesc>,
90 source_ctx: SourceContextRef,
91 ) -> ConnectorResult<Self> {
92 let key_builder =
93 build_accessor_builder(props.encoding_config.clone(), EncodingType::Key).await?;
94 let payload_builder =
95 build_accessor_builder(props.encoding_config, EncodingType::Value).await?;
96 let debezium_props = if let ProtocolProperties::Debezium(props) = props.protocol_config {
97 props
98 } else {
99 unreachable!(
100 "expecting Debezium protocol properties but got {:?}",
101 props.protocol_config
102 )
103 };
104 Ok(Self {
105 key_builder,
106 payload_builder,
107 rw_columns,
108 source_ctx,
109 props: debezium_props,
110 })
111 }
112
113 pub async fn new_for_test(rw_columns: Vec<SourceColumnDesc>) -> ConnectorResult<Self> {
114 use crate::parser::JsonProperties;
115
116 let props = SpecificParserConfig {
117 encoding_config: EncodingProperties::Json(JsonProperties {
118 use_schema_registry: false,
119 timestamptz_handling: None,
120 timestamp_handling: None,
121 time_handling: None,
122 }),
123 protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()),
124 };
125 Self::new(props, rw_columns, SourceContext::dummy().into()).await
126 }
127
128 pub async fn parse_inner(
129 &mut self,
130 key: Option<Vec<u8>>,
131 payload: Option<Vec<u8>>,
132 mut writer: SourceStreamChunkRowWriter<'_>,
133 ) -> ConnectorResult<ParseResult> {
134 let meta = writer.source_meta();
135 let key_accessor = match (key, self.props.ignore_key) {
137 (None, false) => None,
138 (Some(data), false) => Some(self.key_builder.generate_accessor(data, meta).await?),
139 (_, true) => None,
140 };
141 let payload_accessor = match payload {
142 None => None,
143 Some(data) => Some(self.payload_builder.generate_accessor(data, meta).await?),
144 };
145 let row_op = DebeziumChangeEvent::new(key_accessor, payload_accessor);
146
147 match apply_row_operation_on_stream_chunk_writer(&row_op, &mut writer) {
148 Ok(_) => Ok(ParseResult::Rows),
149 Err(err) => {
150 if let Some(transaction_control) =
153 row_op.transaction_control(&self.source_ctx.connector_props)
154 {
155 Ok(ParseResult::TransactionControl(transaction_control))
156 } else {
157 Err(err)?
158 }
159 }
160 }
161 }
162}
163
164impl ByteStreamSourceParser for DebeziumParser {
165 fn columns(&self) -> &[SourceColumnDesc] {
166 &self.rw_columns
167 }
168
169 fn source_ctx(&self) -> &SourceContext {
170 &self.source_ctx
171 }
172
173 fn parser_format(&self) -> ParserFormat {
174 ParserFormat::Debezium
175 }
176
177 #[allow(clippy::unused_async)] async fn parse_one<'a>(
179 &'a mut self,
180 _key: Option<Vec<u8>>,
181 _payload: Option<Vec<u8>>,
182 _writer: SourceStreamChunkRowWriter<'a>,
183 ) -> ConnectorResult<()> {
184 unreachable!("should call `parse_one_with_txn` instead")
185 }
186
187 async fn parse_one_with_txn<'a>(
188 &'a mut self,
189 key: Option<Vec<u8>>,
190 payload: Option<Vec<u8>>,
191 writer: SourceStreamChunkRowWriter<'a>,
192 ) -> ConnectorResult<ParseResult> {
193 self.parse_inner(key, payload, writer).await
194 }
195}
196
197#[cfg(test)]
198mod tests {
199 use std::ops::Deref;
200 use std::sync::Arc;
201
202 use risingwave_common::catalog::{CDC_SOURCE_COLUMN_NUM, ColumnCatalog, ColumnDesc, ColumnId};
203 use risingwave_common::row::Row;
204 use risingwave_common::types::{DataType, Timestamptz};
205 use risingwave_pb::plan_common::{
206 AdditionalColumn, AdditionalColumnTimestamp, additional_column,
207 };
208
209 use super::*;
210 use crate::parser::{JsonProperties, SourceStreamChunkBuilder, TransactionControl};
211 use crate::source::{ConnectorProperties, SourceCtrlOpts};
212
213 #[tokio::test]
214 async fn test_parse_transaction_metadata() {
215 let schema = ColumnCatalog::debezium_cdc_source_cols();
216
217 let columns = schema
218 .iter()
219 .map(|c| SourceColumnDesc::from(&c.column_desc))
220 .collect::<Vec<_>>();
221
222 let props = SpecificParserConfig {
223 encoding_config: EncodingProperties::Json(JsonProperties {
224 use_schema_registry: false,
225 timestamptz_handling: None,
226 timestamp_handling: None,
227 time_handling: None,
228 }),
229 protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()),
230 };
231 let source_ctx = SourceContext {
232 connector_props: ConnectorProperties::PostgresCdc(Box::default()),
233 ..SourceContext::dummy()
234 };
235 let mut parser = DebeziumParser::new(props, columns.clone(), Arc::new(source_ctx))
236 .await
237 .unwrap();
238 let mut dummy_builder = SourceStreamChunkBuilder::new(columns, SourceCtrlOpts::for_test());
239
240 let begin_msg = r#"{"schema":null,"payload":{"status":"BEGIN","id":"35352:3962948040","event_count":null,"data_collections":null,"ts_ms":1704269323180}}"#;
242 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}}"#;
243 let res = parser
244 .parse_one_with_txn(
245 None,
246 Some(begin_msg.as_bytes().to_vec()),
247 dummy_builder.row_writer(),
248 )
249 .await;
250 match res {
251 Ok(ParseResult::TransactionControl(TransactionControl::Begin { id })) => {
252 assert_eq!(id.deref(), "35352");
253 }
254 _ => panic!("unexpected parse result: {:?}", res),
255 }
256 let res = parser
257 .parse_one_with_txn(
258 None,
259 Some(commit_msg.as_bytes().to_vec()),
260 dummy_builder.row_writer(),
261 )
262 .await;
263 match res {
264 Ok(ParseResult::TransactionControl(TransactionControl::Commit { id })) => {
265 assert_eq!(id.deref(), "35352");
266 }
267 _ => panic!("unexpected parse result: {:?}", res),
268 }
269 }
270
271 #[tokio::test]
272 async fn test_parse_additional_columns() {
273 let columns = vec![
274 ColumnDesc::named("O_ORDERKEY", ColumnId::new(1), DataType::Int64),
275 ColumnDesc::named("O_CUSTKEY", ColumnId::new(2), DataType::Int64),
276 ColumnDesc::named("O_ORDERSTATUS", ColumnId::new(3), DataType::Varchar),
277 ColumnDesc::named("O_TOTALPRICE", ColumnId::new(4), DataType::Decimal),
278 ColumnDesc::named("O_ORDERDATE", ColumnId::new(5), DataType::Date),
279 ColumnDesc::named_with_additional_column(
280 "commit_ts",
281 ColumnId::new(6),
282 DataType::Timestamptz,
283 AdditionalColumn {
284 column_type: Some(additional_column::ColumnType::Timestamp(
285 AdditionalColumnTimestamp {},
286 )),
287 },
288 ),
289 ];
290
291 let columns = columns
292 .iter()
293 .map(SourceColumnDesc::from)
294 .collect::<Vec<_>>();
295
296 let props = SpecificParserConfig {
297 encoding_config: EncodingProperties::Json(JsonProperties {
298 use_schema_registry: false,
299 timestamptz_handling: None,
300 timestamp_handling: None,
301 time_handling: None,
302 }),
303 protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()),
304 };
305 let source_ctx = SourceContext {
306 connector_props: ConnectorProperties::PostgresCdc(Box::default()),
307 ..SourceContext::dummy()
308 };
309 let mut parser = DebeziumParser::new(props, columns.clone(), Arc::new(source_ctx))
310 .await
311 .unwrap();
312 let mut builder = SourceStreamChunkBuilder::new(columns, SourceCtrlOpts::for_test());
313
314 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 } }"#;
315
316 let res = parser
317 .parse_one_with_txn(
318 None,
319 Some(payload.as_bytes().to_vec()),
320 builder.row_writer(),
321 )
322 .await;
323 match res {
324 Ok(ParseResult::Rows) => {
325 builder.finish_current_chunk();
326 let chunk = builder.consume_ready_chunks().next().unwrap();
327 for (_, row) in chunk.rows() {
328 let commit_ts = row.datum_at(5).unwrap().into_timestamptz();
329 assert_eq!(commit_ts, Timestamptz::from_millis(1695277757000).unwrap());
330 }
331 }
332 _ => panic!("unexpected parse result: {:?}", res),
333 }
334 }
335
336 #[tokio::test]
337 async fn test_cdc_source_job_schema() {
338 let columns = ColumnCatalog::debezium_cdc_source_cols();
339 assert_eq!(CDC_SOURCE_COLUMN_NUM, columns.len() as u32);
341 }
342}