risingwave_connector/parser/
bytes_parser.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use risingwave_common::try_match_expand;

use super::unified::bytes::BytesAccess;
use super::unified::AccessImpl;
use super::{AccessBuilder, EncodingProperties};
use crate::error::ConnectorResult;

#[derive(Debug)]
pub struct BytesAccessBuilder {
    column_name: Option<String>,
}

impl AccessBuilder for BytesAccessBuilder {
    #[allow(clippy::unused_async)]
    async fn generate_accessor(&mut self, payload: Vec<u8>) -> ConnectorResult<AccessImpl<'_>> {
        Ok(AccessImpl::Bytes(BytesAccess::new(
            &self.column_name,
            payload,
        )))
    }
}

impl BytesAccessBuilder {
    pub fn new(encoding_properties: EncodingProperties) -> ConnectorResult<Self> {
        let config = try_match_expand!(encoding_properties, EncodingProperties::Bytes)?;
        Ok(Self {
            column_name: config.column_name,
        })
    }
}

#[cfg(test)]
mod tests {
    use risingwave_common::array::Op;
    use risingwave_common::row::Row;
    use risingwave_common::types::{DataType, ScalarImpl, ToOwnedDatum};

    use crate::parser::plain_parser::PlainParser;
    use crate::parser::{
        BytesProperties, EncodingProperties, ProtocolProperties, SourceColumnDesc,
        SourceStreamChunkBuilder, SpecificParserConfig,
    };
    use crate::source::SourceContext;

    fn get_payload() -> Vec<Vec<u8>> {
        vec![br#"t"#.to_vec(), br#"random"#.to_vec()]
    }

    async fn test_bytes_parser(get_payload: fn() -> Vec<Vec<u8>>) {
        let descs = vec![SourceColumnDesc::simple("id", DataType::Bytea, 0.into())];
        let props = SpecificParserConfig {
            encoding_config: EncodingProperties::Bytes(BytesProperties { column_name: None }),
            protocol_config: ProtocolProperties::Plain,
        };
        let mut parser = PlainParser::new(props, descs.clone(), SourceContext::dummy().into())
            .await
            .unwrap();

        let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 2);

        for payload in get_payload() {
            let writer = builder.row_writer();
            parser
                .parse_inner(None, Some(payload), writer)
                .await
                .unwrap();
        }

        let chunk = builder.finish();
        let mut rows = chunk.rows();
        {
            let (op, row) = rows.next().unwrap();
            assert_eq!(op, Op::Insert);
            assert_eq!(
                row.datum_at(0).to_owned_datum(),
                Some(ScalarImpl::Bytea("t".as_bytes().into()))
            );
        }

        {
            let (op, row) = rows.next().unwrap();
            assert_eq!(op, Op::Insert);
            assert_eq!(
                row.datum_at(0).to_owned_datum(),
                Some(ScalarImpl::Bytea("random".as_bytes().into()))
            );
        }
    }

    #[tokio::test]
    async fn test_bytes_parse_object_top_level() {
        test_bytes_parser(get_payload).await;
    }
}