Skip to content

feat: HA for schema registry #11982

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 10 commits into from
Sep 5, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
18 changes: 17 additions & 1 deletion e2e_test/schema_registry/pb.slt
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,19 @@ FORMAT plain ENCODE protobuf(
message = 'test.User'
);

# for multiple schema registry nodes
statement ok
create table sr_pb_test_bk with (
connector = 'kafka',
topic = 'sr_pb_test',
properties.bootstrap.server = 'message_queue:29092',
scan.startup.mode = 'earliest',
message = 'test.User')
FORMAT plain ENCODE protobuf(
schema.registry = 'http://message_queue:8081,http://message_queue:8081',
message = 'test.User'
);

# Wait for source
sleep 10s

Expand All @@ -33,4 +46,7 @@ select min(id), max(id) from sr_pb_test;


statement ok
drop table sr_pb_test;
drop table sr_pb_test;

statement ok
drop table sr_pb_test_bk;
14 changes: 7 additions & 7 deletions src/connector/src/parser/avro/parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,12 @@ use risingwave_common::error::ErrorCode::{InternalError, ProtocolError};
use risingwave_common::error::{Result, RwError};
use risingwave_common::try_match_expand;
use risingwave_pb::plan_common::ColumnDesc;
use url::Url;

use super::schema_resolver::*;
use super::util::avro_schema_to_column_descs;
use crate::parser::schema_registry::{extract_schema_id, get_subject_by_strategy, Client};
use crate::parser::schema_registry::{
extract_schema_id, get_subject_by_strategy, handle_sr_list, Client,
};
use crate::parser::unified::avro::{AvroAccess, AvroParseOptions};
use crate::parser::unified::AccessImpl;
use crate::parser::util::{read_schema_from_http, read_schema_from_local, read_schema_from_s3};
Expand Down Expand Up @@ -115,9 +116,7 @@ impl AvroParserConfig {
let avro_config = try_match_expand!(encoding_properties, EncodingProperties::Avro)?;
let schema_location = &avro_config.row_schema_location;
let enable_upsert = avro_config.enable_upsert;
let url = Url::parse(schema_location).map_err(|e| {
InternalError(format!("failed to parse url ({}): {}", schema_location, e))
})?;
let url = handle_sr_list(schema_location.as_str())?;
if avro_config.use_schema_registry {
let client = Client::new(url, &avro_config.client_config)?;
let resolver = ConfluentSchemaResolver::new(client);
Expand Down Expand Up @@ -156,12 +155,13 @@ impl AvroParserConfig {
"avro upsert without schema registry is not supported".to_string(),
)));
}
let url = url.get(0).unwrap();
let schema_content = match url.scheme() {
"file" => read_schema_from_local(url.path()),
"s3" => {
read_schema_from_s3(&url, avro_config.aws_auth_props.as_ref().unwrap()).await
read_schema_from_s3(url, avro_config.aws_auth_props.as_ref().unwrap()).await
}
"https" | "http" => read_schema_from_http(&url).await,
"https" | "http" => read_schema_from_http(url).await,
scheme => Err(RwError::from(ProtocolError(format!(
"path scheme {} is not supported",
scheme
Expand Down
9 changes: 3 additions & 6 deletions src/connector/src/parser/debezium/avro_parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,14 @@ use std::sync::Arc;

use apache_avro::types::Value;
use apache_avro::{from_avro_datum, Schema};
use reqwest::Url;
use risingwave_common::error::ErrorCode::{InternalError, ProtocolError};
use risingwave_common::error::ErrorCode::ProtocolError;
use risingwave_common::error::{Result, RwError};
use risingwave_common::try_match_expand;
use risingwave_pb::plan_common::ColumnDesc;

use crate::parser::avro::schema_resolver::ConfluentSchemaResolver;
use crate::parser::avro::util::avro_schema_to_column_descs;
use crate::parser::schema_registry::{extract_schema_id, Client};
use crate::parser::schema_registry::{extract_schema_id, handle_sr_list, Client};
use crate::parser::unified::avro::{
avro_extract_field_schema, avro_schema_skip_union, AvroAccess, AvroParseOptions,
};
Expand Down Expand Up @@ -107,9 +106,7 @@ impl DebeziumAvroParserConfig {
let schema_location = &avro_config.row_schema_location;
let client_config = &avro_config.client_config;
let kafka_topic = &avro_config.topic;
let url = Url::parse(schema_location).map_err(|e| {
InternalError(format!("failed to parse url ({}): {}", schema_location, e))
})?;
let url = handle_sr_list(schema_location)?;
let client = Client::new(url, client_config)?;
let raw_schema = client
.get_schema_by_subject(format!("{}-key", &kafka_topic).as_str())
Expand Down
8 changes: 4 additions & 4 deletions src/connector/src/parser/json_parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,13 @@ use risingwave_common::error::ErrorCode::{self, InternalError, ProtocolError};
use risingwave_common::error::{Result, RwError};
use risingwave_common::try_match_expand;
use risingwave_pb::plan_common::ColumnDesc;
use url::Url;

use super::avro::schema_resolver::ConfluentSchemaResolver;
use super::schema_registry::Client;
use super::util::{get_kafka_topic, read_schema_from_http, read_schema_from_local};
use super::{EncodingProperties, SchemaRegistryAuth, SpecificParserConfig};
use crate::parser::avro::util::avro_schema_to_column_descs;
use crate::parser::schema_registry::handle_sr_list;
use crate::parser::unified::json::{JsonAccess, JsonParseOptions};
use crate::parser::unified::util::apply_row_accessor_on_stream_chunk_writer;
use crate::parser::unified::AccessImpl;
Expand Down Expand Up @@ -151,8 +151,7 @@ pub async fn schema_to_columns(
use_schema_registry: bool,
props: &HashMap<String, String>,
) -> anyhow::Result<Vec<ColumnDesc>> {
let url = Url::parse(schema_location)
.map_err(|e| InternalError(format!("failed to parse url ({}): {}", schema_location, e)))?;
let url = handle_sr_list(schema_location)?;
let schema_content = if use_schema_registry {
let schema_registry_auth = SchemaRegistryAuth::from(props);
let client = Client::new(url, &schema_registry_auth)?;
Expand All @@ -163,9 +162,10 @@ pub async fn schema_to_columns(
.await?
.content
} else {
let url = url.get(0).unwrap();
match url.scheme() {
"file" => read_schema_from_local(url.path()),
"https" | "http" => read_schema_from_http(&url).await,
"https" | "http" => read_schema_from_http(url).await,
scheme => Err(RwError::from(ProtocolError(format!(
"path scheme {} is not supported",
scheme
Expand Down
13 changes: 7 additions & 6 deletions src/connector/src/parser/protobuf/parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,12 @@ use risingwave_common::error::{Result, RwError};
use risingwave_common::try_match_expand;
use risingwave_common::types::{DataType, Datum, Decimal, ScalarImpl, F32, F64};
use risingwave_pb::plan_common::ColumnDesc;
use url::Url;

use super::schema_resolver::*;
use crate::aws_utils::load_file_descriptor_from_s3;
use crate::parser::schema_registry::{extract_schema_id, get_subject_by_strategy, Client};
use crate::parser::schema_registry::{
extract_schema_id, get_subject_by_strategy, handle_sr_list, Client,
};
use crate::parser::unified::protobuf::ProtobufAccess;
use crate::parser::unified::AccessImpl;
use crate::parser::{AccessBuilder, EncodingProperties};
Expand Down Expand Up @@ -80,8 +81,7 @@ impl ProtobufParserConfig {
let protobuf_config = try_match_expand!(encoding_properties, EncodingProperties::Protobuf)?;
let location = &protobuf_config.row_schema_location;
let message_name = &protobuf_config.message_name;
let url = Url::parse(location)
.map_err(|e| InternalError(format!("failed to parse url ({}): {}", location, e)))?;
let url = handle_sr_list(location.as_str())?;

let schema_bytes = if protobuf_config.use_schema_registry {
let (schema_key, schema_value) = get_subject_by_strategy(
Expand All @@ -99,6 +99,7 @@ impl ProtobufParserConfig {
let client = Client::new(url, &protobuf_config.client_config)?;
compile_file_descriptor_from_schema_registry(schema_value.as_str(), &client).await?
} else {
let url = url.get(0).unwrap();
match url.scheme() {
// TODO(Tao): support local file only when it's compiled in debug mode.
"file" => {
Expand All @@ -115,12 +116,12 @@ impl ProtobufParserConfig {
}
"s3" => {
load_file_descriptor_from_s3(
&url,
url,
protobuf_config.aws_auth_props.as_ref().unwrap(),
)
.await
}
"https" | "http" => load_file_descriptor_from_http(&url).await,
"https" | "http" => load_file_descriptor_from_http(url).await,
scheme => Err(RwError::from(ProtocolError(format!(
"path scheme {} is not supported",
scheme
Expand Down
Loading