risingwave_connector/parser/avro/
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
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
// 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 std::fmt::Debug;
use std::sync::Arc;

use anyhow::Context;
use apache_avro::types::Value;
use apache_avro::{from_avro_datum, Reader, Schema};
use risingwave_common::{bail, try_match_expand};
use risingwave_connector_codec::decoder::avro::{
    avro_schema_to_column_descs, AvroAccess, AvroParseOptions, ResolvedAvroSchema,
};
use risingwave_pb::plan_common::ColumnDesc;

use super::{ConfluentSchemaCache, GlueSchemaCache as _, GlueSchemaCacheImpl};
use crate::error::ConnectorResult;
use crate::parser::unified::AccessImpl;
use crate::parser::util::bytes_from_url;
use crate::parser::{
    AccessBuilder, AvroProperties, EncodingProperties, MapHandling, SchemaLocation,
};
use crate::schema::schema_registry::{
    extract_schema_id, get_subject_by_strategy, handle_sr_list, Client,
};

// Default avro access builder
#[derive(Debug)]
pub struct AvroAccessBuilder {
    schema: Arc<ResolvedAvroSchema>,
    /// Refer to [`AvroParserConfig::writer_schema_cache`].
    writer_schema_cache: WriterSchemaCache,
    value: Option<Value>,
}

impl AccessBuilder for AvroAccessBuilder {
    async fn generate_accessor(&mut self, payload: Vec<u8>) -> ConnectorResult<AccessImpl<'_>> {
        self.value = self.parse_avro_value(&payload).await?;
        Ok(AccessImpl::Avro(AvroAccess::new(
            self.value.as_ref().unwrap(),
            AvroParseOptions::create(&self.schema.resolved_schema),
        )))
    }
}

impl AvroAccessBuilder {
    pub fn new(config: AvroParserConfig) -> ConnectorResult<Self> {
        let AvroParserConfig {
            schema,
            writer_schema_cache,
            ..
        } = config;
        Ok(Self {
            schema,
            writer_schema_cache,
            value: None,
        })
    }

    /// Note: we should use unresolved schema to parsing bytes into avro value.
    /// Otherwise it's an invalid schema and parsing will fail. (Avro error: Two named schema defined for same fullname)
    ///
    /// # Notes about how Avro data looks like
    ///
    /// First, it has two [serialization encodings: binary and JSON](https://avro.apache.org/docs/1.11.1/specification/#encodings).
    /// They don't have magic bytes and cannot be distinguished on their own.
    ///
    /// But in different cases, it starts with different headers, or magic bytes, which can be confusing.
    ///
    /// ## `apache_avro` API and headers
    ///
    /// - `apache_avro::Reader`: [Object Container Files](https://avro.apache.org/docs/1.11.1/specification/#object-container-files): contains file header, starting with 4 bytes `Obj1`. This is a batch file encoding. We don't use it.
    /// - `apache_avro::GenericSingleObjectReader`: [Single-object encoding](https://avro.apache.org/docs/1.11.1/specification/#single-object-encoding): starts with 2 bytes `0xC301`. This is designed to be used in places like Kafka, but Confluent schema registry doesn't use it.
    /// - `apache_avro::from_avro_datum`: no header, binary encoding. This is what we should use.
    ///
    /// ## Confluent schema registry
    ///
    /// - In Kafka ([Confluent schema registry wire format](https://docs.confluent.io/platform/7.6/schema-registry/fundamentals/serdes-develop/index.html#wire-format)):
    ///   starts with 5 bytes`0x00{schema_id:08x}` followed by Avro binary encoding.
    async fn parse_avro_value(&self, payload: &[u8]) -> ConnectorResult<Option<Value>> {
        // parse payload to avro value
        // if use confluent schema, get writer schema from confluent schema registry
        match &self.writer_schema_cache {
            WriterSchemaCache::Confluent(resolver) => {
                let (schema_id, mut raw_payload) = extract_schema_id(payload)?;
                let writer_schema = resolver.get_by_id(schema_id).await?;
                Ok(Some(from_avro_datum(
                    writer_schema.as_ref(),
                    &mut raw_payload,
                    Some(&self.schema.original_schema),
                )?))
            }
            WriterSchemaCache::File => {
                // FIXME: we should not use `Reader` (file header) here. See comment above and https://github.com/risingwavelabs/risingwave/issues/12871
                let mut reader = Reader::with_schema(&self.schema.original_schema, payload)?;
                match reader.next() {
                    Some(Ok(v)) => Ok(Some(v)),
                    Some(Err(e)) => Err(e)?,
                    None => bail!("avro parse unexpected eof"),
                }
            }
            WriterSchemaCache::Glue(resolver) => {
                // <https://github.com/awslabs/aws-glue-schema-registry/blob/v1.1.20/common/src/main/java/com/amazonaws/services/schemaregistry/utils/AWSSchemaRegistryConstants.java#L59-L61>
                // byte 0:      header version = 3
                // byte 1:      compression: 0 = no compression; 5 = zlib (unsupported)
                // byte 2..=17: 16-byte UUID as schema version id
                // byte 18..:   raw avro payload
                if payload.len() < 18 {
                    bail!("payload shorter than 18-byte glue header");
                }
                if payload[0] != 3 {
                    bail!(
                        "Only support glue header version 3 but found {}",
                        payload[0]
                    );
                }
                if payload[1] != 0 {
                    bail!("Non-zero compression {} not supported", payload[1]);
                }
                let schema_version_id = uuid::Uuid::from_slice(&payload[2..18]).unwrap();
                let writer_schema = resolver.get_by_id(schema_version_id).await?;
                let mut raw_payload = &payload[18..];
                Ok(Some(from_avro_datum(
                    writer_schema.as_ref(),
                    &mut raw_payload,
                    Some(&self.schema.original_schema),
                )?))
            }
        }
    }
}

#[derive(Debug, Clone)]
pub struct AvroParserConfig {
    schema: Arc<ResolvedAvroSchema>,
    /// Writer schema is the schema used to write the data. When parsing Avro data, the exactly same schema
    /// must be used to decode the message, and then convert it with the reader schema.
    writer_schema_cache: WriterSchemaCache,

    map_handling: Option<MapHandling>,
}

#[derive(Debug, Clone)]
enum WriterSchemaCache {
    Confluent(Arc<ConfluentSchemaCache>),
    Glue(Arc<GlueSchemaCacheImpl>),
    File,
}

impl AvroParserConfig {
    pub async fn new(encoding_properties: EncodingProperties) -> ConnectorResult<Self> {
        let AvroProperties {
            schema_location,
            record_name,
            key_record_name,
            map_handling,
        } = try_match_expand!(encoding_properties, EncodingProperties::Avro)?;
        match schema_location {
            SchemaLocation::Confluent {
                urls: schema_location,
                client_config,
                name_strategy,
                topic,
            } => {
                let url = handle_sr_list(schema_location.as_str())?;
                let client = Client::new(url, &client_config)?;
                let resolver = ConfluentSchemaCache::new(client);

                if let Some(name) = &key_record_name {
                    bail!("unused FORMAT ENCODE option: key.message='{name}'");
                }
                let subject_value = get_subject_by_strategy(
                    &name_strategy,
                    topic.as_str(),
                    record_name.as_deref(),
                    false,
                )?;
                tracing::debug!("value subject {subject_value}");

                Ok(Self {
                    schema: Arc::new(ResolvedAvroSchema::create(
                        resolver.get_by_subject(&subject_value).await?,
                    )?),
                    writer_schema_cache: WriterSchemaCache::Confluent(Arc::new(resolver)),
                    map_handling,
                })
            }
            SchemaLocation::File {
                url: schema_location,
                aws_auth_props,
            } => {
                let url = handle_sr_list(schema_location.as_str())?;
                let url = url.first().unwrap();
                let schema_content = bytes_from_url(url, aws_auth_props.as_ref()).await?;
                let schema = Schema::parse_reader(&mut schema_content.as_slice())
                    .context("failed to parse avro schema")?;
                Ok(Self {
                    schema: Arc::new(ResolvedAvroSchema::create(Arc::new(schema))?),
                    writer_schema_cache: WriterSchemaCache::File,
                    map_handling,
                })
            }
            SchemaLocation::Glue {
                schema_arn,
                aws_auth_props,
                mock_config,
            } => {
                let resolver =
                    GlueSchemaCacheImpl::new(&aws_auth_props, mock_config.as_deref()).await?;
                let schema = resolver.get_by_name(&schema_arn).await?;
                Ok(Self {
                    schema: Arc::new(ResolvedAvroSchema::create(schema)?),
                    writer_schema_cache: WriterSchemaCache::Glue(Arc::new(resolver)),
                    map_handling,
                })
            }
        }
    }

    pub fn map_to_columns(&self) -> ConnectorResult<Vec<ColumnDesc>> {
        avro_schema_to_column_descs(&self.schema.resolved_schema, self.map_handling)
            .map_err(Into::into)
    }
}

#[cfg(test)]
mod test {
    use std::env;

    use url::Url;

    use super::*;
    use crate::connector_common::AwsAuthProps;

    fn test_data_path(file_name: &str) -> String {
        let curr_dir = env::current_dir().unwrap().into_os_string();
        curr_dir.into_string().unwrap() + "/src/test_data/" + file_name
    }

    #[tokio::test]
    #[ignore]
    async fn test_load_schema_from_s3() {
        let schema_location = "s3://mingchao-schemas/complex-schema.avsc".to_string();
        let url = Url::parse(&schema_location).unwrap();
        let aws_auth_config: AwsAuthProps =
            serde_json::from_str(r#"region":"ap-southeast-1"#).unwrap();
        let schema_content = bytes_from_url(&url, Some(&aws_auth_config)).await;
        assert!(schema_content.is_ok());
        let schema = Schema::parse_reader(&mut schema_content.unwrap().as_slice());
        assert!(schema.is_ok());
        println!("schema = {:?}", schema.unwrap());
    }

    #[tokio::test]
    async fn test_load_schema_from_local() {
        let schema_location = Url::from_file_path(test_data_path("complex-schema.avsc")).unwrap();
        let schema_content = bytes_from_url(&schema_location, None).await;
        assert!(schema_content.is_ok());
        let schema = Schema::parse_reader(&mut schema_content.unwrap().as_slice());
        assert!(schema.is_ok());
        println!("schema = {:?}", schema.unwrap());
    }

    #[tokio::test]
    #[ignore]
    async fn test_load_schema_from_https() {
        let schema_location =
            "https://mingchao-schemas.s3.ap-southeast-1.amazonaws.com/complex-schema.avsc";
        let url = Url::parse(schema_location).unwrap();
        let schema_content = bytes_from_url(&url, None).await;
        assert!(schema_content.is_ok());
        let schema = Schema::parse_reader(&mut schema_content.unwrap().as_slice());
        assert!(schema.is_ok());
        println!("schema = {:?}", schema.unwrap());
    }
}