risingwave_connector/schema/
protobuf.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
// 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::collections::BTreeMap;

use anyhow::Context as _;
use prost_reflect::{DescriptorPool, FileDescriptor, MessageDescriptor};
use prost_types::FileDescriptorSet;
use risingwave_connector_codec::common::protobuf::compile_pb;

use super::loader::{LoadedSchema, SchemaLoader};
use super::schema_registry::Subject;
use super::{
    invalid_option_error, InvalidOptionError, SchemaFetchError, MESSAGE_NAME_KEY,
    SCHEMA_LOCATION_KEY, SCHEMA_REGISTRY_KEY,
};
use crate::connector_common::AwsAuthProps;
use crate::parser::{EncodingProperties, ProtobufParserConfig, ProtobufProperties};

/// `aws_auth_props` is only required when reading `s3://` URL.
pub async fn fetch_descriptor(
    format_options: &BTreeMap<String, String>,
    topic: &str,
    aws_auth_props: Option<&AwsAuthProps>,
) -> Result<(MessageDescriptor, Option<i32>), SchemaFetchError> {
    let message_name = format_options
        .get(MESSAGE_NAME_KEY)
        .ok_or_else(|| invalid_option_error!("{MESSAGE_NAME_KEY} required"))?
        .clone();
    let schema_location = format_options.get(SCHEMA_LOCATION_KEY);
    let schema_registry = format_options.get(SCHEMA_REGISTRY_KEY);
    let row_schema_location = match (schema_location, schema_registry) {
        (Some(_), Some(_)) => {
            return Err(invalid_option_error!(
                "cannot use {SCHEMA_LOCATION_KEY} and {SCHEMA_REGISTRY_KEY} together"
            )
            .into())
        }
        (None, None) => {
            return Err(invalid_option_error!(
                "requires one of {SCHEMA_LOCATION_KEY} or {SCHEMA_REGISTRY_KEY}"
            )
            .into())
        }
        (None, Some(_)) => {
            let (md, sid) = fetch_from_registry(&message_name, format_options, topic).await?;
            return Ok((md, Some(sid)));
        }
        (Some(url), None) => url.clone(),
    };

    if row_schema_location.starts_with("s3") && aws_auth_props.is_none() {
        return Err(invalid_option_error!("s3 URL not supported yet").into());
    }

    let enc = EncodingProperties::Protobuf(ProtobufProperties {
        use_schema_registry: false,
        row_schema_location,
        message_name,
        aws_auth_props: aws_auth_props.cloned(),
        // name_strategy, topic, key_message_name, enable_upsert, client_config
        ..Default::default()
    });
    // Ideally, we should extract the schema loading logic from source parser to this place,
    // and call this in both source and sink.
    // But right now this function calls into source parser for its schema loading functionality.
    // This reversed dependency will be fixed when we support schema registry.
    let conf = ProtobufParserConfig::new(enc)
        .await
        .map_err(SchemaFetchError::YetToMigrate)?;
    Ok((conf.message_descriptor, None))
}

pub async fn fetch_from_registry(
    message_name: &str,
    format_options: &BTreeMap<String, String>,
    topic: &str,
) -> Result<(MessageDescriptor, i32), SchemaFetchError> {
    let loader = SchemaLoader::from_format_options(topic, format_options)?;

    let (vid, vpb) = loader.load_val_schema::<FileDescriptor>().await?;

    Ok((
        vpb.parent_pool().get_message_by_name(message_name).unwrap(),
        vid,
    ))
}

impl LoadedSchema for FileDescriptor {
    fn compile(primary: Subject, references: Vec<Subject>) -> Result<Self, SchemaFetchError> {
        let primary_name = primary.name.clone();

        match compile_pb_subject(primary, references)
            .context("failed to compile protobuf schema into fd set")
        {
            Err(e) => Err(SchemaFetchError::SchemaCompile(e.into())),
            Ok(fd_set) => DescriptorPool::from_file_descriptor_set(fd_set)
                .context("failed to convert fd set to descriptor pool")
                .and_then(|pool| {
                    pool.get_file_by_name(&primary_name)
                        .context("file lost after compilation")
                })
                .map_err(|e| SchemaFetchError::SchemaCompile(e.into())),
        }
    }
}

fn compile_pb_subject(
    primary_subject: Subject,
    dependency_subjects: Vec<Subject>,
) -> Result<FileDescriptorSet, SchemaFetchError> {
    compile_pb(
        (
            primary_subject.name.clone(),
            primary_subject.schema.content.clone(),
        ),
        dependency_subjects
            .into_iter()
            .map(|s| (s.name.clone(), s.schema.content.clone())),
    )
    .map_err(|e| SchemaFetchError::SchemaCompile(e.into()))
}