risingwave_connector/parser/unified/
maxwell.rs1use risingwave_common::types::{DataType, DatumCow, ScalarRefImpl, ToDatumRef};
16
17use super::{Access, ChangeEvent};
18use crate::parser::unified::ChangeEventOperation;
19use crate::source::SourceColumnDesc;
20
21pub const MAXWELL_INSERT_OP: &str = "insert";
22pub const MAXWELL_UPDATE_OP: &str = "update";
23pub const MAXWELL_DELETE_OP: &str = "delete";
24
25pub struct MaxwellChangeEvent<A>(A);
26
27impl<A> MaxwellChangeEvent<A> {
28 pub fn new(accessor: A) -> Self {
29 Self(accessor)
30 }
31}
32
33impl<A> ChangeEvent for MaxwellChangeEvent<A>
34where
35 A: Access,
36{
37 fn op(&self) -> std::result::Result<super::ChangeEventOperation, super::AccessError> {
38 const OP: &str = "type";
39 if let Some(ScalarRefImpl::Utf8(op)) =
40 self.0.access(&[OP], &DataType::Varchar)?.to_datum_ref()
41 {
42 match op {
43 MAXWELL_INSERT_OP | MAXWELL_UPDATE_OP => return Ok(ChangeEventOperation::Upsert),
44 MAXWELL_DELETE_OP => return Ok(ChangeEventOperation::Delete),
45 _ => (),
46 }
47 }
48 Err(super::AccessError::Undefined {
49 name: "op".into(),
50 path: Default::default(),
51 })
52 }
53
54 fn access_field(&self, desc: &SourceColumnDesc) -> super::AccessResult<DatumCow<'_>> {
55 const DATA: &str = "data";
56 self.0.access(&[DATA, &desc.name], &desc.data_type)
57 }
58}