-
Notifications
You must be signed in to change notification settings - Fork 591
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
refactor: additional column logic #18633
Closed
Closed
Changes from 2 commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -637,7 +637,14 @@ pub fn handle_addition_columns( | |
) -> Result<()> { | ||
let connector_name = with_properties.get_connector().unwrap(); // there must be a connector in source | ||
|
||
if get_supported_additional_columns(connector_name.as_str(), is_cdc_backfill_table).is_none() | ||
// CDC source will not pass the source_schema, and get the additional column set from | ||
// `CDC_BACKFILL_TABLE_ADDITIONAL_COLUMNS`, does not check FORMAT compatibility. | ||
// So set to FORMAT::Unspecified in this case. | ||
let format_type = &source_schema | ||
.and_then(|schema| format_to_prost(&schema.format).into()) | ||
.unwrap_or(FormatType::Unspecified); | ||
if get_supported_additional_columns(connector_name.as_str(), format_type, is_cdc_backfill_table) | ||
.is_none() | ||
&& !additional_columns.is_empty() | ||
{ | ||
return Err(RwError::from(ProtocolError(format!( | ||
|
@@ -667,6 +674,7 @@ pub fn handle_addition_columns( | |
data_type_name.as_deref(), | ||
true, | ||
is_cdc_backfill_table, | ||
format_type, | ||
)?; | ||
columns.push(ColumnCatalog::visible(col)); | ||
} | ||
|
@@ -899,12 +907,6 @@ pub(crate) async fn bind_source_pk( | |
} | ||
|
||
(Format::Debezium, Encode::Json) => { | ||
if !additional_column_names.is_empty() { | ||
return Err(RwError::from(ProtocolError(format!( | ||
"FORMAT DEBEZIUM forbids additional columns, but got {:?}", | ||
additional_column_names | ||
)))); | ||
} | ||
Comment on lines
-902
to
-907
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. There are other branches not covered: "(debezium, avro)", "(maxwell, json)", "(canal, json)", etc. |
||
if !sql_defined_pk { | ||
return Err(RwError::from(ProtocolError( | ||
"Primary key must be specified when creating source with FORMAT DEBEZIUM." | ||
|
@@ -990,7 +992,11 @@ pub(crate) async fn bind_source_pk( | |
} | ||
|
||
// Add a hidden column `_rw_kafka_timestamp` to each message from Kafka source. | ||
fn check_and_add_timestamp_column(with_properties: &WithOptions, columns: &mut Vec<ColumnCatalog>) { | ||
fn check_and_add_timestamp_column( | ||
with_properties: &WithOptions, | ||
columns: &mut Vec<ColumnCatalog>, | ||
format_type: &FormatType, | ||
) { | ||
if with_properties.is_kafka_connector() { | ||
if columns.iter().any(|col| { | ||
matches!( | ||
|
@@ -1012,6 +1018,7 @@ fn check_and_add_timestamp_column(with_properties: &WithOptions, columns: &mut V | |
None, | ||
true, | ||
false, | ||
format_type, | ||
) | ||
.unwrap(); | ||
columns.push(ColumnCatalog::hidden(col)); | ||
|
@@ -1542,7 +1549,11 @@ pub async fn bind_create_source_or_table_with_connector( | |
// compatible with the behavior that add a hidden column `_rw_kafka_timestamp` to each message from Kafka source | ||
if is_create_source { | ||
// must behind `handle_addition_columns` | ||
check_and_add_timestamp_column(&with_properties, &mut columns); | ||
check_and_add_timestamp_column( | ||
&with_properties, | ||
&mut columns, | ||
&format_to_prost(&source_schema.format), | ||
); | ||
} | ||
|
||
// resolve privatelink connection for Kafka | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think add
format
to the hashmap makes it harder to manage.may be better
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Because "connector metadata" and "format metadata" are relatively orthogonal.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
make sense.
But I am not sure whether we allow including
key
for debezium format. This is the only concern I use the nested structure.