Skip to content
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

fix(source): cleanup unused key schema logic & requirements for format upsert encode avro #17759

Merged
merged 4 commits into from
Jul 22, 2024
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
14 changes: 1 addition & 13 deletions e2e_test/source_inline/kafka/avro/name_strategy.slt
Original file line number Diff line number Diff line change
Expand Up @@ -84,17 +84,6 @@ python3 scripts/source/schema_registry_producer.py "${RISEDEV_KAFKA_BOOTSTRAP_S



statement error SCHEMA_REGISTRY_NAME_STRATEGY_TOPIC_RECORD_NAME_STRATEGY expect non-empty field key\.message
create table t_topic_record () with (
connector = 'kafka',
topic = 'upsert_avro_json-topic-record',
properties.bootstrap.server = '${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}'
) format upsert encode avro (
schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}',
schema.registry.name.strategy = 'topic_record_name_strategy',
message = 'CPLM.OBJ_ATTRIBUTE_VALUE'
);
Comment on lines -87 to -96
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it makes sense removing name strategy related things since we does not interpret key schema.


statement ok
create table t_topic_record (primary key(rw_key))
INCLUDE KEY AS rw_key
Expand All @@ -105,8 +94,7 @@ with (
) format upsert encode avro (
schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}',
schema.registry.name.strategy = 'topic_record_name_strategy',
message = 'CPLM.OBJ_ATTRIBUTE_VALUE',
key.message = 'string'
message = 'CPLM.OBJ_ATTRIBUTE_VALUE'
);


Expand Down
42 changes: 7 additions & 35 deletions src/connector/src/parser/avro/parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ use crate::error::ConnectorResult;
use crate::parser::unified::AccessImpl;
use crate::parser::util::bytes_from_url;
use crate::parser::{
AccessBuilder, AvroProperties, EncodingProperties, EncodingType, MapHandling, SchemaLocation,
AccessBuilder, AvroProperties, EncodingProperties, MapHandling, SchemaLocation,
};
use crate::schema::schema_registry::{
extract_schema_id, get_subject_by_strategy, handle_sr_list, Client,
Expand All @@ -55,18 +55,14 @@ impl AccessBuilder for AvroAccessBuilder {
}

impl AvroAccessBuilder {
pub fn new(config: AvroParserConfig, encoding_type: EncodingType) -> ConnectorResult<Self> {
pub fn new(config: AvroParserConfig) -> ConnectorResult<Self> {
let AvroParserConfig {
schema,
key_schema,
writer_schema_cache,
..
} = config;
Ok(Self {
schema: match encoding_type {
EncodingType::Key => key_schema.context("Avro with empty key schema")?,
EncodingType::Value => schema,
},
schema,
writer_schema_cache,
value: None,
})
Expand Down Expand Up @@ -148,7 +144,6 @@ impl AvroAccessBuilder {
#[derive(Debug, Clone)]
pub struct AvroParserConfig {
schema: Arc<ResolvedAvroSchema>,
key_schema: Option<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,
Expand All @@ -167,7 +162,6 @@ impl AvroParserConfig {
pub async fn new(encoding_properties: EncodingProperties) -> ConnectorResult<Self> {
let AvroProperties {
schema_location,
enable_upsert,
record_name,
key_record_name,
map_handling,
Expand All @@ -183,38 +177,21 @@ impl AvroParserConfig {
let client = Client::new(url, &client_config)?;
let resolver = ConfluentSchemaCache::new(client);

let subject_key = if enable_upsert {
Some(get_subject_by_strategy(
&name_strategy,
topic.as_str(),
key_record_name.as_deref(),
true,
)?)
} else {
if let Some(name) = &key_record_name {
bail!("unused FORMAT ENCODE option: key.message='{name}'");
}
None
};
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!("infer key subject {subject_key:?}, value subject {subject_value}");
tracing::debug!("value subject {subject_value}");

Ok(Self {
schema: Arc::new(ResolvedAvroSchema::create(
resolver.get_by_subject(&subject_value).await?,
)?),
key_schema: if let Some(subject_key) = subject_key {
Some(Arc::new(ResolvedAvroSchema::create(
resolver.get_by_subject(&subject_key).await?,
)?))
} else {
None
},
writer_schema_cache: WriterSchemaCache::Confluent(Arc::new(resolver)),
map_handling,
})
Expand All @@ -224,16 +201,12 @@ impl AvroParserConfig {
aws_auth_props,
} => {
let url = handle_sr_list(schema_location.as_str())?;
if enable_upsert {
bail!("avro upsert without schema registry is not supported");
}
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))?),
key_schema: None,
writer_schema_cache: WriterSchemaCache::File,
map_handling,
})
Expand All @@ -248,7 +221,6 @@ impl AvroParserConfig {
let schema = resolver.get_by_name(&schema_arn).await?;
Ok(Self {
schema: Arc::new(ResolvedAvroSchema::create(schema)?),
key_schema: None,
writer_schema_cache: WriterSchemaCache::Glue(Arc::new(resolver)),
map_handling,
})
Expand Down
3 changes: 0 additions & 3 deletions src/connector/src/parser/debezium/debezium_parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -75,9 +75,6 @@ async fn build_accessor_builder(
.unwrap_or(TimestamptzHandling::GuessNumberUnit),
)?,
)),
EncodingProperties::Protobuf(_) => {
Ok(AccessBuilderImpl::new_default(config, encoding_type).await?)
}
_ => bail!("unsupported encoding for Debezium"),
}
}
Expand Down
6 changes: 2 additions & 4 deletions src/connector/src/parser/maxwell/maxwell_parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ use crate::only_parse_payload;
use crate::parser::unified::maxwell::MaxwellChangeEvent;
use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer;
use crate::parser::{
AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, ParserFormat,
AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, ParserFormat,
SourceStreamChunkRowWriter, SpecificParserConfig,
};
use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef};
Expand All @@ -39,9 +39,7 @@ impl MaxwellParser {
) -> ConnectorResult<Self> {
match props.encoding_config {
EncodingProperties::Json(_) => {
let payload_builder =
AccessBuilderImpl::new_default(props.encoding_config, EncodingType::Value)
.await?;
let payload_builder = AccessBuilderImpl::new_default(props.encoding_config).await?;
Ok(Self {
payload_builder,
rw_columns,
Expand Down
11 changes: 2 additions & 9 deletions src/connector/src/parser/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -869,14 +869,11 @@ pub enum AccessBuilderImpl {
}

impl AccessBuilderImpl {
pub async fn new_default(
config: EncodingProperties,
kv: EncodingType,
) -> ConnectorResult<Self> {
pub async fn new_default(config: EncodingProperties) -> ConnectorResult<Self> {
let accessor = match config {
EncodingProperties::Avro(_) => {
let config = AvroParserConfig::new(config).await?;
AccessBuilderImpl::Avro(AvroAccessBuilder::new(config, kv)?)
AccessBuilderImpl::Avro(AvroAccessBuilder::new(config)?)
}
EncodingProperties::Protobuf(_) => {
let config = ProtobufParserConfig::new(config).await?;
Expand Down Expand Up @@ -1072,7 +1069,6 @@ impl SpecificParserConfig {
#[derive(Debug, Default, Clone)]
pub struct AvroProperties {
pub schema_location: SchemaLocation,
pub enable_upsert: bool,
pub record_name: Option<String>,
pub key_record_name: Option<String>,
pub map_handling: Option<MapHandling>,
Expand Down Expand Up @@ -1224,9 +1220,6 @@ impl SpecificParserConfig {
map_handling: MapHandling::from_options(&format_encode_options_with_secret)?,
..Default::default()
};
if format == SourceFormat::Upsert {
config.enable_upsert = true;
}
config.schema_location = if let Some(schema_arn) =
format_encode_options_with_secret.get(AWS_GLUE_SCHEMA_ARN_KEY)
{
Expand Down
6 changes: 3 additions & 3 deletions src/connector/src/parser/plain_parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,8 @@ use risingwave_common::bail;
use super::unified::json::TimestamptzHandling;
use super::unified::kv_event::KvEvent;
use super::{
AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType,
SourceStreamChunkRowWriter, SpecificParserConfig,
AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, SourceStreamChunkRowWriter,
SpecificParserConfig,
};
use crate::error::ConnectorResult;
use crate::parser::bytes_parser::BytesAccessBuilder;
Expand Down Expand Up @@ -61,7 +61,7 @@ impl PlainParser {
| EncodingProperties::Protobuf(_)
| EncodingProperties::Avro(_)
| EncodingProperties::Bytes(_) => {
AccessBuilderImpl::new_default(props.encoding_config, EncodingType::Value).await?
AccessBuilderImpl::new_default(props.encoding_config).await?
}
_ => bail!("Unsupported encoding for Plain"),
};
Expand Down
14 changes: 4 additions & 10 deletions src/connector/src/parser/upsert_parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColum
use super::bytes_parser::BytesAccessBuilder;
use super::unified::{AccessImpl, ChangeEventOperation};
use super::{
AccessBuilderImpl, ByteStreamSourceParser, BytesProperties, EncodingProperties, EncodingType,
AccessBuilderImpl, ByteStreamSourceParser, BytesProperties, EncodingProperties,
SourceStreamChunkRowWriter, SpecificParserConfig,
};
use crate::error::ConnectorResult;
Expand All @@ -34,16 +34,11 @@ pub struct UpsertParser {
source_ctx: SourceContextRef,
}

async fn build_accessor_builder(
config: EncodingProperties,
encoding_type: EncodingType,
) -> ConnectorResult<AccessBuilderImpl> {
async fn build_accessor_builder(config: EncodingProperties) -> ConnectorResult<AccessBuilderImpl> {
match config {
EncodingProperties::Json(_)
| EncodingProperties::Protobuf(_)
| EncodingProperties::Avro(_) => {
Ok(AccessBuilderImpl::new_default(config, encoding_type).await?)
}
| EncodingProperties::Avro(_) => Ok(AccessBuilderImpl::new_default(config).await?),
_ => bail!("unsupported encoding for Upsert"),
}
}
Expand Down Expand Up @@ -80,8 +75,7 @@ impl UpsertParser {
} else {
unreachable!("format upsert must have key column")
};
let payload_builder =
build_accessor_builder(props.encoding_config, EncodingType::Value).await?;
let payload_builder = build_accessor_builder(props.encoding_config).await?;
Ok(Self {
key_builder,
payload_builder,
Expand Down
Loading