From 648dce1a4b8fe9aa70f5d9daa173874edddb514b Mon Sep 17 00:00:00 2001 From: tabVersion Date: Wed, 29 Nov 2023 10:31:14 +0800 Subject: [PATCH 01/50] stash --- .../src/handler/alter_table_column.rs | 1 + src/frontend/src/handler/create_source.rs | 25 ++++++++++++-- src/frontend/src/handler/create_table.rs | 33 ++++++++++++++++--- src/frontend/src/handler/explain.rs | 2 ++ src/frontend/src/handler/mod.rs | 2 ++ src/sqlparser/src/ast/mod.rs | 6 ++++ src/sqlparser/src/ast/statement.rs | 3 ++ src/sqlparser/src/parser.rs | 16 +++++++++ 8 files changed, 81 insertions(+), 7 deletions(-) diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index 649fade132e0d..cf68fabaa28f9 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -196,6 +196,7 @@ pub async fn handle_alter_table_column( source_watermarks, col_id_gen, append_only, + vec![], ) .await? } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 7c8e597bad3db..5064e04820ccf 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -52,7 +52,7 @@ use risingwave_pb::plan_common::{EncodeType, FormatType}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ get_delimiter, AstString, AvroSchema, ColumnDef, ConnectorSchema, CreateSourceStatement, - DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceWatermark, + DebeziumAvroSchema, Encode, Format, Ident, ProtobufSchema, SourceWatermark, }; use super::RwPgResponse; @@ -288,6 +288,7 @@ pub(crate) async fn bind_columns_from_source( session: &SessionImpl, source_schema: &ConnectorSchema, with_properties: &HashMap, + include_columns_options: Vec<(Ident, Option)>, create_cdc_source_job: bool, ) -> Result<(Option>, StreamSourceInfo)> { const MESSAGE_NAME_KEY: &str = "message"; @@ -317,7 +318,7 @@ pub(crate) async fn bind_columns_from_source( Ok(name_strategy) }; - let res = match (&source_schema.format, &source_schema.row_encode) { + let mut res = match (&source_schema.format, &source_schema.row_encode) { (Format::Native, Encode::Native) => ( None, StreamSourceInfo { @@ -862,6 +863,25 @@ pub(super) fn bind_source_watermark( Ok(watermark_descs) } +static CONNECTOR_COMPATOBLE_ADDITIONAL_COLUMNS: LazyLock< + HashMap< + String, + Vec<( + &'static str, + Box ColumnCatalog + Send + Sync + 'static>, + )>, + >, +> = LazyLock::new(|| { + convert_args!(hashmap!( + KAFKA_CONNECTOR => vec![("key", Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named(name, id, DataType::Bytea), + is_hidden: false, + } + }))], + )) +}); + // TODO: Better design if we want to support ENCODE KEY where we will have 4 dimensional array static CONNECTORS_COMPATIBLE_FORMATS: LazyLock>>> = LazyLock::new(|| { @@ -1117,6 +1137,7 @@ pub async fn handle_create_source( &session, &source_schema, &with_properties, + stmt.include_column_options, create_cdc_source_job, ) .await?; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 5bfa3240e7f6b..269e59a781f81 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -40,7 +40,7 @@ use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::{DefaultColumnDesc, GeneratedColumnDesc}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ - CdcTableInfo, ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, Format, + CdcTableInfo, ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, Format, Ident, ObjectName, SourceWatermark, TableConstraint, }; @@ -452,6 +452,7 @@ pub(crate) async fn gen_create_table_plan_with_source( source_watermarks: Vec, mut col_id_gen: ColumnIdGenerator, append_only: bool, + include_column_options: Vec<(Ident, Option)>, ) -> Result<(PlanRef, Option, PbTable)> { if append_only && source_schema.format != Format::Plain @@ -472,8 +473,14 @@ pub(crate) async fn gen_create_table_plan_with_source( let sql_pk_names = bind_sql_pk_names(&column_defs, &constraints)?; - let (columns_from_resolve_source, mut source_info) = - bind_columns_from_source(context.session_ctx(), &source_schema, &properties, false).await?; + let (columns_from_resolve_source, mut source_info) = bind_columns_from_source( + context.session_ctx(), + &source_schema, + &properties, + include_column_options, + false, + ) + .await?; let columns_from_sql = bind_sql_columns(&column_defs)?; let mut columns = bind_all_columns( @@ -939,8 +946,13 @@ pub(super) async fn handle_create_table_plan( constraints: Vec, source_watermarks: Vec, append_only: bool, + include_column_options: Vec<(Ident, Option)>, ) -> Result<(PlanRef, Option, PbTable, TableJobType)> { - let source_schema = check_create_table_with_source(context.with_options(), source_schema)?; + let source_schema = check_create_table_with_source( + context.with_options(), + source_schema, + &include_column_options, + )?; let ((plan, source, table), job_type) = match (source_schema, cdc_table_info.as_ref()) { @@ -954,6 +966,7 @@ pub(super) async fn handle_create_table_plan( source_watermarks, col_id_gen, append_only, + include_column_options, ) .await?, TableJobType::General, @@ -1005,6 +1018,7 @@ pub async fn handle_create_table( source_watermarks: Vec, append_only: bool, cdc_table_info: Option, + include_column_options: Vec<(Ident, Option)>, ) -> Result { let session = handler_args.session.clone(); @@ -1033,6 +1047,7 @@ pub async fn handle_create_table( constraints, source_watermarks, append_only, + include_column_options, ) .await?; @@ -1064,8 +1079,16 @@ pub async fn handle_create_table( pub fn check_create_table_with_source( with_options: &WithOptions, source_schema: Option, + include_column_options: &[(Ident, Option)], ) -> Result> { - if with_options.inner().contains_key(UPSTREAM_SOURCE_KEY) { + let defined_source = with_options.inner().contains_key(UPSTREAM_SOURCE_KEY); + if !include_column_options.is_empty() && !defined_source { + return Err(ErrorCode::InvalidInputSyntax( + "INCLUDE should be used with a connector".to_owned(), + ) + .into()); + } + if defined_source { source_schema.as_ref().ok_or_else(|| { ErrorCode::InvalidInputSyntax("Please specify a source schema using FORMAT".to_owned()) })?; diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index 6bec02b7bd4db..85dfea189b599 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -62,6 +62,7 @@ async fn do_handle_explain( source_watermarks, append_only, cdc_table_info, + include_column_options, .. } => { let col_id_gen = ColumnIdGenerator::new_initial(); @@ -78,6 +79,7 @@ async fn do_handle_explain( constraints, source_watermarks, append_only, + include_column_options, ) .await?; let context = plan.ctx(); diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 24cfb7c177e16..261f486df14d4 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -229,6 +229,7 @@ pub async fn handle( source_watermarks, append_only, cdc_table_info, + include_column_options, } => { if or_replace { bail_not_implemented!("CREATE OR REPLACE TABLE"); @@ -258,6 +259,7 @@ pub async fn handle( source_watermarks, append_only, cdc_table_info, + include_column_options, ) .await } diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index a57a6a9175ebd..4aa782b96245b 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1096,6 +1096,8 @@ pub enum Statement { query: Option>, /// `FROM cdc_source TABLE database_name.table_name` cdc_table_info: Option, + /// `INCLUDE a AS b INCLUDE c` + include_column_options: Vec<(Ident, Option)>, }, /// CREATE INDEX CreateIndex { @@ -1554,6 +1556,7 @@ impl fmt::Display for Statement { append_only, query, cdc_table_info, + include_column_options, } => { // We want to allow the following options // Empty column list, allowed by PostgreSQL: @@ -1579,6 +1582,9 @@ impl fmt::Display for Statement { if *append_only { write!(f, " APPEND ONLY")?; } + if !include_column_options.is_empty() { + todo!() + } if !with_options.is_empty() { write!(f, " WITH ({})", display_comma_separated(with_options))?; } diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index bb41db6b83e78..11e923a113bd6 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -82,6 +82,7 @@ pub struct CreateSourceStatement { pub with_properties: WithProperties, pub source_schema: CompatibleSourceSchema, pub source_watermarks: Vec, + pub include_column_options: Vec<(Ident, Option)>, } #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -724,6 +725,7 @@ impl ParseTo for CreateSourceStatement { // parse columns let (columns, constraints, source_watermarks) = p.parse_columns_with_watermark()?; + let include_options = p.parse_include_options()?; let with_options = p.parse_with_properties()?; let option = with_options @@ -745,6 +747,7 @@ impl ParseTo for CreateSourceStatement { with_properties: WithProperties(with_options), source_schema, source_watermarks, + include_column_options: include_options, }) } } diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 4db8b3ebdfff6..abb036725943b 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -2438,6 +2438,7 @@ impl Parser { } else { false }; + let include_options = self.parse_include_options()?; // PostgreSQL supports `WITH ( options )`, before `AS` let with_options = self.parse_with_properties()?; @@ -2495,9 +2496,24 @@ impl Parser { append_only, query, cdc_table_info, + include_column_options: include_options, }) } + pub fn parse_include_options(&mut self) -> Result)>, ParserError> { + let mut options = vec![]; + while self.parse_keyword(Keyword::INCLUDE) { + let add_column = self.parse_identifier()?; + if self.parse_keyword(Keyword::AS) { + let column_alias = self.parse_identifier()?; + options.push((add_column, Some(column_alias)).into()); + } else { + options.push((add_column, None).into()); + } + } + Ok(options) + } + pub fn parse_columns_with_watermark(&mut self) -> Result { let mut columns = vec![]; let mut constraints = vec![]; From 5b65a4ea89b990fc500d267bf1a068cf74574843 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 29 Nov 2023 16:51:28 +0800 Subject: [PATCH 02/50] fix compilation --- src/frontend/src/handler/create_source.rs | 38 +++++++++++++---------- 1 file changed, 22 insertions(+), 16 deletions(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 5064e04820ccf..e5c1ce39be947 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -863,23 +863,29 @@ pub(super) fn bind_source_watermark( Ok(watermark_descs) } -static CONNECTOR_COMPATOBLE_ADDITIONAL_COLUMNS: LazyLock< - HashMap< - String, - Vec<( - &'static str, - Box ColumnCatalog + Send + Sync + 'static>, - )>, - >, +type CompatibleAdditionalColumnsFn = + Box ColumnCatalog + Send + Sync + 'static>; + +static CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock< + HashMap>, > = LazyLock::new(|| { - convert_args!(hashmap!( - KAFKA_CONNECTOR => vec![("key", Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named(name, id, DataType::Bytea), - is_hidden: false, - } - }))], - )) + let mut res: HashMap> = + HashMap::new(); + + res.insert( + KAFKA_CONNECTOR.to_string(), + vec![( + "key", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named(name, id, DataType::Bytea), + is_hidden: false, + } + }), + )], + ); + + res }); // TODO: Better design if we want to support ENCODE KEY where we will have 4 dimensional array From 37d81925b2740b85ae243bf9aaff99fbf81a14bd Mon Sep 17 00:00:00 2001 From: tabVersion Date: Wed, 29 Nov 2023 16:54:39 +0800 Subject: [PATCH 03/50] stash --- src/frontend/src/handler/create_source.rs | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 5064e04820ccf..0c4942e06e072 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -564,6 +564,19 @@ pub(crate) async fn bind_columns_from_source( } }; + { + // add connector-spec additional columns + let connector_name = get_connector(with_properties).unwrap(); // there must be a connector in source + let addition_col_list = CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS + .get(connector_name.as_str()) + .ok_or_else(|| { + RwError::from(ProtocolError(format!( + "Connector {} accepts no additional column", + connector_name + ))) + })?; + } + { // fixme: remove this after correctly consuming the two options options.remove(SCHEMA_REGISTRY_USERNAME); @@ -863,7 +876,7 @@ pub(super) fn bind_source_watermark( Ok(watermark_descs) } -static CONNECTOR_COMPATOBLE_ADDITIONAL_COLUMNS: LazyLock< +static CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock< HashMap< String, Vec<( From d7c39c4c3f7ec035925a81a7fed1351dd5ba0efa Mon Sep 17 00:00:00 2001 From: tabVersion Date: Wed, 29 Nov 2023 22:48:00 +0800 Subject: [PATCH 04/50] feat: Refactor source and table creation handling - Modified the `bind_columns_from_source` function to remove the `include_columns_options` parameter. - Added the `handle_addition_columns` function to add connector-specific columns to the column catalog. - Modified the `handle_create_source` function to handle additional columns, primary key constraints, source watermarks, SQL column constraints, source schema checks, resolving private link connection for Kafka sources, and creating streaming jobs for CDC sources. - Added the `handle_create_source` function to create a catalog source. - Updated the `TestCase` struct to include methods for creating a table with a connector, creating a source, and running SQL queries. - Implemented various methods in the `TestCase` struct for creating schemas, explaining queries, and creating views and indexes. - Modified the `gen_create_table_plan_with_source` function and added the `handle_addition_columns` function to handle additional columns and primary key binding. - Updated the `TestCaseResult` struct to include the logical, optimized, batch, local batch, and stream plans, as well as any encountered errors during planning. Signed-off-by: tabVersion --- src/frontend/planner_test/src/lib.rs | 2 + src/frontend/src/handler/create_source.rs | 160 +++++++++++++++++++--- src/frontend/src/handler/create_table.rs | 15 +- src/sqlparser/src/parser.rs | 4 +- 4 files changed, 154 insertions(+), 27 deletions(-) diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index ea065af3746f6..ecf7a4ea9c079 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -424,6 +424,7 @@ impl TestCase { source_watermarks, append_only, cdc_table_info, + include_column_options, .. } => { let source_schema = source_schema.map(|schema| schema.into_v2_with_warning()); @@ -438,6 +439,7 @@ impl TestCase { source_watermarks, append_only, cdc_table_info, + include_column_options, ) .await?; } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index fffeae75b60a9..1b908607a6f66 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -288,7 +288,6 @@ pub(crate) async fn bind_columns_from_source( session: &SessionImpl, source_schema: &ConnectorSchema, with_properties: &HashMap, - include_columns_options: Vec<(Ident, Option)>, create_cdc_source_job: bool, ) -> Result<(Option>, StreamSourceInfo)> { const MESSAGE_NAME_KEY: &str = "message"; @@ -318,7 +317,7 @@ pub(crate) async fn bind_columns_from_source( Ok(name_strategy) }; - let mut res = match (&source_schema.format, &source_schema.row_encode) { + let res = match (&source_schema.format, &source_schema.row_encode) { (Format::Native, Encode::Native) => ( None, StreamSourceInfo { @@ -564,19 +563,6 @@ pub(crate) async fn bind_columns_from_source( } }; - { - // add connector-spec additional columns - let connector_name = get_connector(with_properties).unwrap(); // there must be a connector in source - let addition_col_list = CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS - .get(connector_name.as_str()) - .ok_or_else(|| { - RwError::from(ProtocolError(format!( - "Connector {} accepts no additional column", - connector_name - ))) - })?; - } - { // fixme: remove this after correctly consuming the two options options.remove(SCHEMA_REGISTRY_USERNAME); @@ -600,6 +586,57 @@ pub(crate) async fn bind_columns_from_source( Ok(res) } +/// add connector-spec columns to the end of column catalog +pub fn handle_addition_columns( + with_properties: &HashMap, + mut include_columns_options: Vec<(Ident, Option)>, + columns: &mut Vec, +) -> Result<()> { + let connector_name = get_connector(with_properties).unwrap(); // there must be a connector in source + let addition_col_list = CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS + .get(connector_name.as_str()) + .ok_or_else(|| { + RwError::from(ProtocolError(format!( + "Connector {} accepts no additional column", + connector_name + ))) + })?; + let gen_default_column_name = |connector_name: &str, addi_column_name: &str| { + format!("_rw_{}_{}", connector_name, addi_column_name) + }; + + let latest_col_id: ColumnId = columns + .iter() + .map(|col| col.column_desc.column_id) + .max() + .unwrap(); // there must be at least one column in the column catalog + + for (col_name, gen_column_catalog_fn) in addition_col_list { + // always insert in spec order + if let Some(idx) = include_columns_options + .iter() + .position(|(col, _)| col.real_value().eq_ignore_ascii_case(col_name)) + { + let (_, alias) = include_columns_options.remove(idx); + columns.push(gen_column_catalog_fn( + latest_col_id.next(), + alias + .map(|alias| alias.real_value()) + .unwrap_or_else(|| gen_default_column_name(connector_name.as_str(), col_name)) + .as_str(), + )) + } + } + if !include_columns_options.is_empty() { + return Err(RwError::from(ProtocolError(format!( + "Unknown additional columns {:?}", + include_columns_options + )))); + } + + Ok(()) +} + /// Bind columns from both source and sql defined. pub(crate) fn bind_all_columns( source_schema: &ConnectorSchema, @@ -887,6 +924,59 @@ static CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock< res.insert( KAFKA_CONNECTOR.to_string(), + vec![ + ( + "key", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named(name, id, DataType::Bytea), + is_hidden: false, + } + }), + ), + ( + "timestamp", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named(name, id, DataType::Timestamptz), + is_hidden: false, + } + }), + ), + ( + "partition", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named(name, id, DataType::Int64), + is_hidden: false, + } + }), + ), + ( + "offset", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named(name, id, DataType::Int64), + is_hidden: false, + } + }), + ), + // Todo(tabVersion): add header column desc + // ( + // "header", + // Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + // ColumnCatalog { + // column_desc: ColumnDesc::named(name, id, DataType::List( + // + // )), + // is_hidden: false, + // } + // }), + // ), + ], + ); + res.insert( + PULSAR_CONNECTOR.to_string(), vec![( "key", Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { @@ -897,6 +987,43 @@ static CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock< }), )], ); + res.insert( + KINESIS_CONNECTOR.to_string(), + vec![( + "key", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named(name, id, DataType::Varchar), + is_hidden: false, + } + }), + )], + ); + res.insert( + S3_CONNECTOR.to_string(), + vec![( + "file", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named(name, id, DataType::Varchar), + is_hidden: false, + } + }), + )], + ); + res.insert( + // TODO(tabVersion): change to Opendal S3 and GCS + S3_V2_CONNECTOR.to_string(), + vec![( + "file", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named(name, id, DataType::Varchar), + is_hidden: false, + } + }), + )], + ); res }); @@ -1156,7 +1283,6 @@ pub async fn handle_create_source( &session, &source_schema, &with_properties, - stmt.include_column_options, create_cdc_source_job, ) .await?; @@ -1168,6 +1294,8 @@ pub async fn handle_create_source( columns_from_sql, &stmt.columns, )?; + // add additional columns before bind pk, because `format upsert` requires the key column + handle_addition_columns(&with_properties, stmt.include_column_options, &mut columns)?; let pk_names = bind_source_pk( &source_schema, &source_info, diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 269e59a781f81..8fce4d6d2282b 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -53,7 +53,7 @@ use crate::catalog::{check_valid_column_name, ColumnId}; use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; use crate::handler::create_source::{ bind_all_columns, bind_columns_from_source, bind_source_pk, bind_source_watermark, - check_source_schema, validate_compatibility, UPSTREAM_SOURCE_KEY, + check_source_schema, handle_addition_columns, validate_compatibility, UPSTREAM_SOURCE_KEY, }; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::{LogicalCdcScan, LogicalSource}; @@ -473,14 +473,8 @@ pub(crate) async fn gen_create_table_plan_with_source( let sql_pk_names = bind_sql_pk_names(&column_defs, &constraints)?; - let (columns_from_resolve_source, mut source_info) = bind_columns_from_source( - context.session_ctx(), - &source_schema, - &properties, - include_column_options, - false, - ) - .await?; + let (columns_from_resolve_source, mut source_info) = + bind_columns_from_source(context.session_ctx(), &source_schema, &properties, false).await?; let columns_from_sql = bind_sql_columns(&column_defs)?; let mut columns = bind_all_columns( @@ -489,6 +483,9 @@ pub(crate) async fn gen_create_table_plan_with_source( columns_from_sql, &column_defs, )?; + + // add additional columns before bind pk, because `format upsert` requires the key column + handle_addition_columns(&properties, include_column_options, &mut columns)?; let pk_names = bind_source_pk( &source_schema, &source_info, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index abb036725943b..a7dc63c0d956e 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -2506,9 +2506,9 @@ impl Parser { let add_column = self.parse_identifier()?; if self.parse_keyword(Keyword::AS) { let column_alias = self.parse_identifier()?; - options.push((add_column, Some(column_alias)).into()); + options.push((add_column, Some(column_alias))); } else { - options.push((add_column, None).into()); + options.push((add_column, None)); } } Ok(options) From 922fa1c5e4ac539294269117b306612ea54299f8 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Thu, 30 Nov 2023 17:18:26 +0800 Subject: [PATCH 05/50] add new field in ColumnDesc --- proto/plan_common.proto | 14 ++ src/common/src/catalog/column.rs | 30 +++- src/common/src/catalog/test_utils.rs | 3 +- .../src/parser/additional_columns.rs | 165 ++++++++++++++++++ src/connector/src/parser/avro/util.rs | 3 +- .../src/parser/debezium/simd_json_parser.rs | 3 + src/connector/src/parser/mod.rs | 1 + src/connector/src/parser/protobuf/parser.rs | 3 +- src/connector/src/source/manager.rs | 5 + src/connector/src/source/mod.rs | 3 + src/frontend/src/binder/expr/mod.rs | 2 + src/frontend/src/catalog/table_catalog.rs | 3 +- src/frontend/src/handler/create_source.rs | 116 +----------- src/frontend/src/handler/create_table.rs | 3 +- 14 files changed, 233 insertions(+), 121 deletions(-) create mode 100644 src/connector/src/parser/additional_columns.rs diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 4f1eccfab8666..724f975454f58 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -15,6 +15,16 @@ message Field { string name = 2; } +enum AdditionalColumnType { + UNSPECIFIED = 0; + KEY = 1; + TIMESTAMP = 2; + PARTITION = 3; + OFFSET = 4; + HEADER = 5; + FILENAME = 6; +} + message ColumnDesc { data.DataType column_type = 1; int32 column_id = 2; @@ -40,6 +50,10 @@ message ColumnDesc { // This field is used to store the description set by the `comment on` clause. optional string description = 8; + + // This field is used to represent the connector-spec additional column type. + // UNSPECIFIED or unset for normal column. + AdditionalColumnType additional_column_type = 9; } message ColumnCatalog { diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index 68c1618073169..40229339427f6 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -17,7 +17,7 @@ use std::borrow::Cow; use itertools::Itertools; use risingwave_pb::expr::ExprNode; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; -use risingwave_pb::plan_common::{PbColumnCatalog, PbColumnDesc}; +use risingwave_pb::plan_common::{AdditionalColumnType, PbColumnCatalog, PbColumnDesc}; use super::row_id_column_desc; use crate::catalog::{cdc_table_name_column_desc, offset_column_desc, Field, ROW_ID_COLUMN_ID}; @@ -101,6 +101,7 @@ pub struct ColumnDesc { pub type_name: String, pub generated_or_default_column: Option, pub description: Option, + pub additional_column_type: AdditionalColumnType, } impl ColumnDesc { @@ -113,6 +114,7 @@ impl ColumnDesc { type_name: String::new(), generated_or_default_column: None, description: None, + additional_column_type: AdditionalColumnType::Unspecified, } } @@ -125,6 +127,25 @@ impl ColumnDesc { type_name: String::new(), generated_or_default_column: None, description: None, + additional_column_type: AdditionalColumnType::Unspecified, + } + } + + pub fn named_with_additional_column( + name: impl Into, + column_id: ColumnId, + data_type: DataType, + additional_column_type: AdditionalColumnType, + ) -> ColumnDesc { + ColumnDesc { + data_type, + column_id, + name: name.into(), + field_descs: vec![], + type_name: String::new(), + generated_or_default_column: None, + description: None, + additional_column_type, } } @@ -143,6 +164,7 @@ impl ColumnDesc { type_name: self.type_name.clone(), generated_or_default_column: self.generated_or_default_column.clone(), description: self.description.clone(), + additional_column_type: self.additional_column_type as i32, } } @@ -169,6 +191,7 @@ impl ColumnDesc { type_name: "".to_string(), generated_or_default_column: None, description: None, + additional_column_type: AdditionalColumnType::Unspecified, } } @@ -190,6 +213,7 @@ impl ColumnDesc { type_name: type_name.to_string(), generated_or_default_column: None, description: None, + additional_column_type: AdditionalColumnType::Unspecified, } } @@ -206,6 +230,7 @@ impl ColumnDesc { type_name: field.type_name.clone(), description: None, generated_or_default_column: None, + additional_column_type: AdditionalColumnType::Unspecified, } } @@ -230,6 +255,7 @@ impl ColumnDesc { impl From for ColumnDesc { fn from(prost: PbColumnDesc) -> Self { + let additional_column_type = prost.additional_column_type(); let field_descs: Vec = prost .field_descs .into_iter() @@ -243,6 +269,7 @@ impl From for ColumnDesc { field_descs, generated_or_default_column: prost.generated_or_default_column, description: prost.description.clone(), + additional_column_type, } } } @@ -263,6 +290,7 @@ impl From<&ColumnDesc> for PbColumnDesc { type_name: c.type_name.clone(), generated_or_default_column: c.generated_or_default_column.clone(), description: c.description.clone(), + additional_column_type: c.additional_column_type as i32, } } } diff --git a/src/common/src/catalog/test_utils.rs b/src/common/src/catalog/test_utils.rs index 6b524edb92430..d1d6123759e15 100644 --- a/src/common/src/catalog/test_utils.rs +++ b/src/common/src/catalog/test_utils.rs @@ -15,7 +15,7 @@ use itertools::Itertools; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; -use risingwave_pb::plan_common::ColumnDesc; +use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDesc}; pub trait ColumnDescTestExt { /// Create a [`ColumnDesc`] with the given name and type. @@ -58,6 +58,7 @@ impl ColumnDescTestExt for ColumnDesc { field_descs: fields, generated_or_default_column: None, description: None, + additional_column_type: AdditionalColumnType::Unspecified as i32, } } } diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs new file mode 100644 index 0000000000000..378c63b17ca3a --- /dev/null +++ b/src/connector/src/parser/additional_columns.rs @@ -0,0 +1,165 @@ +use std::collections::HashMap; +use std::sync::LazyLock; + +use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; +use risingwave_common::types::DataType; +use risingwave_pb::plan_common::AdditionalColumnType; + +use crate::source::{ + KAFKA_CONNECTOR, KINESIS_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, S3_V2_CONNECTOR, +}; + +pub type CompatibleAdditionalColumnsFn = + Box ColumnCatalog + Send + Sync + 'static>; + +pub static CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock< + HashMap>, +> = LazyLock::new(|| { + let mut res: HashMap> = + HashMap::new(); + + res.insert( + KAFKA_CONNECTOR.to_string(), + vec![ + ( + "key", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Bytea, + AdditionalColumnType::Key, + ), + is_hidden: false, + } + }), + ), + ( + "timestamp", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Timestamptz, + AdditionalColumnType::Timestamp, + ), + is_hidden: false, + } + }), + ), + ( + "partition", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Int64, + AdditionalColumnType::Partition, + ), + is_hidden: false, + } + }), + ), + ( + "offset", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Int64, + AdditionalColumnType::Offset, + ), + is_hidden: false, + } + }), + ), + // Todo(tabVersion): add header column desc + // ( + // "header", + // Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + // ColumnCatalog { + // column_desc: ColumnDesc::named(name, id, DataType::List( + // + // )), + // is_hidden: false, + // } + // }), + // ), + ], + ); + res.insert( + PULSAR_CONNECTOR.to_string(), + vec![( + "key", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Bytea, + AdditionalColumnType::Key, + ), + is_hidden: false, + } + }), + )], + ); + res.insert( + KINESIS_CONNECTOR.to_string(), + vec![( + "key", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Bytea, + AdditionalColumnType::Key, + ), + is_hidden: false, + } + }), + )], + ); + res.insert( + S3_CONNECTOR.to_string(), + vec![( + "file", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Varchar, + AdditionalColumnType::Filename, + ), + is_hidden: false, + } + }), + )], + ); + res.insert( + // TODO(tabVersion): change to Opendal S3 and GCS + S3_V2_CONNECTOR.to_string(), + vec![( + "file", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Varchar, + AdditionalColumnType::Filename, + ), + is_hidden: false, + } + }), + )], + ); + + res +}); diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 7f7ded02c4396..813107c1be93f 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -15,7 +15,7 @@ use apache_avro::schema::{DecimalSchema, RecordSchema, Schema}; use itertools::Itertools; use risingwave_common::types::{DataType, Decimal}; -use risingwave_pb::plan_common::ColumnDesc; +use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDesc}; pub fn avro_schema_to_column_descs(schema: &Schema) -> anyhow::Result> { if let Schema::Record(RecordSchema { fields, .. }) = schema { @@ -58,6 +58,7 @@ fn avro_field_to_column_desc( type_name: schema_name.to_string(), generated_or_default_column: None, description: None, + additional_column_type: AdditionalColumnType::Unspecified as i32, }) } _ => { diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index 9ee966456799b..c58fae5095bd4 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -501,6 +501,8 @@ mod tests { // postgres-specific data-type mapping tests mod test3_postgres { + use risingwave_pb::plan_common::AdditionalColumnType; + use super::*; use crate::source::SourceColumnType; @@ -564,6 +566,7 @@ mod tests { fields: vec![], column_type: SourceColumnType::Normal, is_pk: false, + additional_column_type: AdditionalColumnType::Unspecified, }, SourceColumnDesc::simple("o_enum", DataType::Varchar, ColumnId::from(8)), SourceColumnDesc::simple("o_char", DataType::Varchar, ColumnId::from(9)), diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index c256bb919cce0..e49feb434c75a 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -52,6 +52,7 @@ use crate::source::{ StreamChunkWithState, }; +pub mod additional_columns; mod avro; mod bytes_parser; mod canal; diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index 99ebafe46bd3c..756590a9ae140 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -25,7 +25,7 @@ use risingwave_common::error::ErrorCode::{InternalError, ProtocolError}; use risingwave_common::error::{Result, RwError}; use risingwave_common::try_match_expand; use risingwave_common::types::{DataType, Datum, Decimal, JsonbVal, ScalarImpl, F32, F64}; -use risingwave_pb::plan_common::ColumnDesc; +use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDesc}; use super::schema_resolver::*; use crate::aws_utils::load_file_descriptor_from_s3; @@ -213,6 +213,7 @@ impl ProtobufParserConfig { type_name: m.full_name().to_string(), generated_or_default_column: None, description: None, + additional_column_type: AdditionalColumnType::Unspecified as i32, }) } else { *index += 1; diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index c766f05099810..4d4b81086c1e7 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -19,6 +19,7 @@ use risingwave_common::catalog::{ TABLE_NAME_COLUMN_NAME, }; use risingwave_common::types::DataType; +use risingwave_pb::plan_common::AdditionalColumnType; /// `SourceColumnDesc` is used to describe a column in the Source and is used as the column /// counterpart in `StreamScan` @@ -32,6 +33,7 @@ pub struct SourceColumnDesc { // `is_pk` is used to indicate whether the column is part of the primary key columns. pub is_pk: bool, + pub additional_column_type: AdditionalColumnType, } /// `SourceColumnType` is used to indicate the type of a column emitted by the Source. @@ -81,6 +83,7 @@ impl SourceColumnDesc { fields: vec![], column_type: SourceColumnType::Normal, is_pk: false, + additional_column_type: AdditionalColumnType::Unspecified, } } @@ -112,6 +115,7 @@ impl From<&ColumnDesc> for SourceColumnDesc { fields: c.field_descs.clone(), column_type, is_pk: false, + additional_column_type: c.additional_column_type, } } } @@ -126,6 +130,7 @@ impl From<&SourceColumnDesc> for ColumnDesc { type_name: "".to_string(), generated_or_default_column: None, description: None, + additional_column_type: s.additional_column_type, } } } diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index c866ed6c3c223..c537f59991821 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -39,6 +39,9 @@ pub mod test_source; pub use manager::{SourceColumnDesc, SourceColumnType}; pub use mock_external_table::MockExternalTableReader; +pub use crate::parser::additional_columns::{ + CompatibleAdditionalColumnsFn, CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS, +}; pub use crate::source::filesystem::{S3_CONNECTOR, S3_V2_CONNECTOR}; pub use crate::source::nexmark::NEXMARK_CONNECTOR; pub use crate::source::pulsar::PULSAR_CONNECTOR; diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index c52c42dbc973d..4438ec9d6861d 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -18,6 +18,7 @@ use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::zip_eq_fast; use risingwave_common::{bail_not_implemented, not_implemented}; +use risingwave_pb::plan_common::AdditionalColumnType; use risingwave_sqlparser::ast::{ Array, BinaryOperator, DataType as AstDataType, Expr, Function, JsonPredicateType, ObjectName, Query, StructField, TrimWhereField, UnaryOperator, @@ -574,6 +575,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { type_name: "".to_string(), generated_or_default_column: None, description: None, + additional_column_type: AdditionalColumnType::Unspecified, }) } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 750a06da7d231..fa43101a2b0db 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -584,7 +584,7 @@ mod tests { use risingwave_common::types::*; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::{PbStreamJobStatus, PbTable}; - use risingwave_pb::plan_common::{PbColumnCatalog, PbColumnDesc}; + use risingwave_pb::plan_common::{AdditionalColumnType, PbColumnCatalog, PbColumnDesc}; use super::*; use crate::catalog::table_catalog::{TableCatalog, TableType}; @@ -676,6 +676,7 @@ mod tests { type_name: ".test.Country".to_string(), description: None, generated_or_default_column: None, + additional_column_type: AdditionalColumnType::Unspecified, }, is_hidden: false } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 1b908607a6f66..2e7e7e275a092 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -27,6 +27,7 @@ use risingwave_common::catalog::{ use risingwave_common::error::ErrorCode::{self, InvalidInputSyntax, ProtocolError}; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::DataType; +use risingwave_connector::parser::additional_columns::CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS; use risingwave_connector::parser::{ schema_to_columns, AvroParserConfig, DebeziumAvroParserConfig, ProtobufParserConfig, SpecificParserConfig, @@ -913,121 +914,6 @@ pub(super) fn bind_source_watermark( Ok(watermark_descs) } -type CompatibleAdditionalColumnsFn = - Box ColumnCatalog + Send + Sync + 'static>; - -static CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock< - HashMap>, -> = LazyLock::new(|| { - let mut res: HashMap> = - HashMap::new(); - - res.insert( - KAFKA_CONNECTOR.to_string(), - vec![ - ( - "key", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named(name, id, DataType::Bytea), - is_hidden: false, - } - }), - ), - ( - "timestamp", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named(name, id, DataType::Timestamptz), - is_hidden: false, - } - }), - ), - ( - "partition", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named(name, id, DataType::Int64), - is_hidden: false, - } - }), - ), - ( - "offset", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named(name, id, DataType::Int64), - is_hidden: false, - } - }), - ), - // Todo(tabVersion): add header column desc - // ( - // "header", - // Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - // ColumnCatalog { - // column_desc: ColumnDesc::named(name, id, DataType::List( - // - // )), - // is_hidden: false, - // } - // }), - // ), - ], - ); - res.insert( - PULSAR_CONNECTOR.to_string(), - vec![( - "key", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named(name, id, DataType::Bytea), - is_hidden: false, - } - }), - )], - ); - res.insert( - KINESIS_CONNECTOR.to_string(), - vec![( - "key", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named(name, id, DataType::Varchar), - is_hidden: false, - } - }), - )], - ); - res.insert( - S3_CONNECTOR.to_string(), - vec![( - "file", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named(name, id, DataType::Varchar), - is_hidden: false, - } - }), - )], - ); - res.insert( - // TODO(tabVersion): change to Opendal S3 and GCS - S3_V2_CONNECTOR.to_string(), - vec![( - "file", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named(name, id, DataType::Varchar), - is_hidden: false, - } - }), - )], - ); - - res -}); - // TODO: Better design if we want to support ENCODE KEY where we will have 4 dimensional array static CONNECTORS_COMPATIBLE_FORMATS: LazyLock>>> = LazyLock::new(|| { diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 8fce4d6d2282b..e048d8b47ffa5 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -37,7 +37,7 @@ use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::{PbSource, PbTable, StreamSourceInfo, WatermarkDesc}; use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; -use risingwave_pb::plan_common::{DefaultColumnDesc, GeneratedColumnDesc}; +use risingwave_pb::plan_common::{AdditionalColumnType, DefaultColumnDesc, GeneratedColumnDesc}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ CdcTableInfo, ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, Format, Ident, @@ -207,6 +207,7 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> type_name: "".to_string(), generated_or_default_column: None, description: None, + additional_column_type: AdditionalColumnType::Unspecified, }, is_hidden: false, }); From 7b268ccce198aa301098aaf5e38f064f00bfab68 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Thu, 30 Nov 2023 18:11:40 +0800 Subject: [PATCH 06/50] bind_source_pk --- src/frontend/src/handler/create_source.rs | 50 +++++++++++++++++++++-- 1 file changed, 46 insertions(+), 4 deletions(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 2e7e7e275a092..0705ae4932734 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -49,7 +49,7 @@ use risingwave_connector::source::{ use risingwave_pb::catalog::{ PbSchemaRegistryNameStrategy, PbSource, StreamSourceInfo, WatermarkDesc, }; -use risingwave_pb::plan_common::{EncodeType, FormatType}; +use risingwave_pb::plan_common::{AdditionalColumnType, EncodeType, FormatType}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ get_delimiter, AstString, AvroSchema, ColumnDef, ConnectorSchema, CreateSourceStatement, @@ -743,7 +743,20 @@ pub(crate) async fn bind_source_pk( sql_defined_pk_names: Vec, with_properties: &HashMap, ) -> Result> { + // todo: handle pk carefully, revisit logic later + let sql_defined_pk = !sql_defined_pk_names.is_empty(); + let key_column_name: Option = { + // iter columns to check if contains additional columns from key part + // return the key column names if exists + columns.iter().find_map(|catalog| { + if catalog.column_desc.additional_column_type == AdditionalColumnType::Key { + Some(catalog.name().to_string()) + } else { + None + } + }) + }; let res = match (&source_schema.format, &source_schema.row_encode) { (Format::Native, Encode::Native) | (Format::Plain, Encode::Json | Encode::Csv) => { @@ -756,14 +769,38 @@ pub(crate) async fn bind_source_pk( sql_defined_pk_names } (Format::Upsert, Encode::Json) => { - if sql_defined_pk { + if let Some(ref key_column_name) = key_column_name && sql_defined_pk { + if sql_defined_pk_names.len() != 1 { + return Err(RwError::from(ProtocolError( + format!("upsert json supports only one primary key column ({}).", key_column_name) + ))); + } + // the column name have been converted to real value in `handle_addition_columns` + // so we don't ignore ascii case here + if key_column_name.eq(sql_defined_pk_names[0].as_str()) { + return Err(RwError::from(ProtocolError(format!( + "upsert json's key column {} not match with sql defined primary key {}", + key_column_name, sql_defined_pk_names[0] + )))); + } sql_defined_pk_names } else { - add_default_key_column(columns); - vec![DEFAULT_KEY_COLUMN_NAME.into()] + return if key_column_name.is_none() { + Err( + RwError::from(ProtocolError("INCLUDE KEY clause must be set for FORMAT UPSERT ENCODE JSON".to_string())) + ) + } else { + Err(RwError::from(ProtocolError(format!( + "Primary key must be specified to {} when creating source with FORMAT UPSERT ENCODE JSON", + key_column_name.unwrap())))) + } } } (Format::Upsert, Encode::Avro) => { + // todo: check logic here: + // * if defined pk, it must be the same as key column + // * if not defined pk, extract pk from schema but put a mark on the columns + // * if no pk in schema, use the key column as primary key if sql_defined_pk { if sql_defined_pk_names.len() != 1 { return Err(RwError::from(ProtocolError( @@ -783,6 +820,11 @@ pub(crate) async fn bind_source_pk( } (Format::Debezium, Encode::Json) => { + if key_column_name.is_some() { + return Err(RwError::from(ProtocolError( + "INCLUDE KEY clause cannot be set for FORMAT DEBEZIUM ENCODE JSON".to_string(), + ))); + } if !sql_defined_pk { return Err(RwError::from(ProtocolError( "Primary key must be specified when creating source with FORMAT DEBEZIUM." From 842d72218fa3dc18f7765ae492ab3dc434c778cd Mon Sep 17 00:00:00 2001 From: tabVersion Date: Thu, 30 Nov 2023 18:13:52 +0800 Subject: [PATCH 07/50] add license --- src/connector/src/parser/additional_columns.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index 378c63b17ca3a..135b0b4af2359 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -1,3 +1,17 @@ +// Copyright 2023 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::HashMap; use std::sync::LazyLock; From c31f624e0ebc9cbbc85400cc10f4d453531796ae Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 4 Dec 2023 14:16:16 +0800 Subject: [PATCH 08/50] feat: Implement additional column validation for various formats - Added import for `format` from `std::fmt` module in `create_source.rs` - Updated the `additional_column_names` vector with additional column names - Implemented validation for additional columns in Debezium, DebeziumMongo, Maxwell, and Canal formats in `create_source.rs` Signed-off-by: tabVersion --- src/frontend/src/handler/create_source.rs | 44 ++++++++++++++++++++--- 1 file changed, 40 insertions(+), 4 deletions(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 0705ae4932734..478b0a362f5b4 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::{BTreeMap, HashMap}; +use std::fmt::format; use std::rc::Rc; use std::sync::LazyLock; @@ -757,6 +758,16 @@ pub(crate) async fn bind_source_pk( } }) }; + let additional_column_names = columns + .iter() + .filter_map(|col| { + if col.column_desc.additional_column_type != AdditionalColumnType::Unspecified { + Some(col.name().to_string()) + } else { + None + } + }) + .collect_vec(); let res = match (&source_schema.format, &source_schema.row_encode) { (Format::Native, Encode::Native) | (Format::Plain, Encode::Json | Encode::Csv) => { @@ -820,10 +831,11 @@ pub(crate) async fn bind_source_pk( } (Format::Debezium, Encode::Json) => { - if key_column_name.is_some() { - return Err(RwError::from(ProtocolError( - "INCLUDE KEY clause cannot be set for FORMAT DEBEZIUM ENCODE JSON".to_string(), - ))); + if !additional_column_names.is_empty() { + return Err(RwError::from(ProtocolError(format!( + "FORMAT DEBEZIUM forbids additional columns, but got {:?}", + additional_column_names + )))); } if !sql_defined_pk { return Err(RwError::from(ProtocolError( @@ -834,6 +846,12 @@ pub(crate) async fn bind_source_pk( sql_defined_pk_names } (Format::Debezium, Encode::Avro) => { + if !additional_column_names.is_empty() { + return Err(RwError::from(ProtocolError(format!( + "FORMAT DEBEZIUM forbids additional columns, but got {:?}", + additional_column_names + )))); + } if sql_defined_pk { sql_defined_pk_names } else { @@ -855,6 +873,12 @@ pub(crate) async fn bind_source_pk( } } (Format::DebeziumMongo, Encode::Json) => { + if !additional_column_names.is_empty() { + return Err(RwError::from(ProtocolError(format!( + "FORMAT DEBEZIUMMONGO forbids additional columns, but got {:?}", + additional_column_names + )))); + } if sql_defined_pk { sql_defined_pk_names } else { @@ -863,6 +887,12 @@ pub(crate) async fn bind_source_pk( } (Format::Maxwell, Encode::Json) => { + if !additional_column_names.is_empty() { + return Err(RwError::from(ProtocolError(format!( + "FORMAT MAXWELL forbids additional columns, but got {:?}", + additional_column_names + )))); + } if !sql_defined_pk { return Err(RwError::from(ProtocolError( "Primary key must be specified when creating source with FORMAT MAXWELL ENCODE JSON." @@ -873,6 +903,12 @@ pub(crate) async fn bind_source_pk( } (Format::Canal, Encode::Json) => { + if !additional_column_names.is_empty() { + return Err(RwError::from(ProtocolError(format!( + "FORMAT CANAL forbids additional columns, but got {:?}", + additional_column_names + )))); + } if !sql_defined_pk { return Err(RwError::from(ProtocolError( "Primary key must be specified when creating source with FORMAT CANAL ENCODE JSON." From 981a74c89ee6eaa5614bd158772511641a4fd91b Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 4 Dec 2023 14:27:34 +0800 Subject: [PATCH 09/50] stash --- src/frontend/src/handler/create_source.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 478b0a362f5b4..e14ca098abcc1 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -775,6 +775,7 @@ pub(crate) async fn bind_source_pk( } (Format::Plain, _) => { if is_key_mq_connector(with_properties) { + // todo: forbid later add_default_key_column(columns); } sql_defined_pk_names From ebfe20b2dabe8575a708a9a9c72ae66720b597a3 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 4 Dec 2023 20:29:33 +0800 Subject: [PATCH 10/50] format --- src/frontend/src/handler/create_source.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 2df10bc106293..bba8e1458a078 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::collections::{BTreeMap, HashMap}; -use std::fmt::format; use std::rc::Rc; use std::sync::LazyLock; From 30c5768ac70e5fab37433061bceeb3c5e07c4ed8 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 5 Dec 2023 14:34:26 +0800 Subject: [PATCH 11/50] stash --- src/frontend/src/handler/create_source.rs | 97 +++++++++++++++++++++-- src/sqlparser/src/ast/mod.rs | 12 ++- 2 files changed, 101 insertions(+), 8 deletions(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index bba8e1458a078..d1688d9e10fe3 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -778,7 +778,7 @@ pub(crate) async fn bind_source_pk( } // the column name have been converted to real value in `handle_addition_columns` // so we don't ignore ascii case here - if key_column_name.eq(sql_defined_pk_names[0].as_str()) { + if !key_column_name.eq(sql_defined_pk_names[0].as_str()) { return Err(RwError::from(ProtocolError(format!( "upsert json's key column {} not match with sql defined primary key {}", key_column_name, sql_defined_pk_names[0] @@ -1370,6 +1370,7 @@ pub async fn handle_create_source( #[cfg(test)] pub mod tests { use std::collections::HashMap; + use std::sync::Arc; use risingwave_common::catalog::{ CDC_SOURCE_COLUMN_NUM, DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, OFFSET_COLUMN_NAME, @@ -1378,9 +1379,19 @@ pub mod tests { use risingwave_common::types::DataType; use crate::catalog::root_catalog::SchemaPath; + use crate::catalog::source_catalog::SourceCatalog; use crate::handler::create_source::debezium_cdc_source_schema; use crate::test_utils::{create_proto_file, LocalFrontend, PROTO_FILE_DATA}; + const GET_COLUMN_FROM_CATALOG: fn(&Arc) -> HashMap<&str, DataType> = + |catalog: &Arc| -> HashMap<&str, DataType> { + catalog + .columns + .iter() + .map(|col| (col.name(), col.data_type().clone())) + .collect::>() + }; + #[tokio::test] async fn test_create_source_handler() { let proto_file = create_proto_file(PROTO_FILE_DATA); @@ -1403,11 +1414,7 @@ pub mod tests { .unwrap(); assert_eq!(source.name, "t"); - let columns = source - .columns - .iter() - .map(|col| (col.name(), col.data_type().clone())) - .collect::>(); + let columns = GET_COLUMN_FROM_CATALOG(source); let city_type = DataType::new_struct( vec![DataType::Varchar, DataType::Varchar], @@ -1470,4 +1477,82 @@ pub mod tests { // make sure it doesn't broken by future PRs assert_eq!(CDC_SOURCE_COLUMN_NUM, columns.len() as u32); } + + #[tokio::test] + async fn test_source_addition_columns() { + // test derive include column for format plain + let sql = + "CREATE SOURCE s (v1 int) include key as _rw_kafka_key with (connector = 'kafka') format plain encode json" + .to_string(); + let frontend = LocalFrontend::new(Default::default()).await; + frontend.run_sql(sql).await.unwrap(); + let session = frontend.session_ref(); + let catalog_reader = session.env().catalog_reader().read_guard(); + let (source, _) = catalog_reader + .get_source_by_name( + DEFAULT_DATABASE_NAME, + SchemaPath::Name(DEFAULT_SCHEMA_NAME), + "s", + ) + .unwrap(); + assert_eq!(source.name, "s"); + + let columns = GET_COLUMN_FROM_CATALOG(source); + let expect_columns = maplit::hashmap! { + ROWID_PREFIX => DataType::Serial, + "v1" => DataType::Int32, + "_rw_kafka_key" => DataType::Bytea, + // todo: kafka connector will automatically derive the column + // will change to a required field in the include clause + "_rw_kafka_timestamp" => DataType::Timestamptz, + }; + assert_eq!(columns, expect_columns); + + // test derive include column for format upsert + let sql = "CREATE SOURCE s1 (v1 int) with (connector = 'kafka') format upsert encode json" + .to_string(); + match frontend.run_sql(sql).await { + Err(e) => { + assert_eq!( + e.to_string(), + "Protocol error: INCLUDE KEY clause must be set for FORMAT UPSERT ENCODE JSON" + ) + } + _ => unreachable!(), + } + + let sql = "CREATE SOURCE s2 (v1 int) include key as _rw_kafka_key with (connector = 'kafka') format upsert encode json" + .to_string(); + match frontend.run_sql(sql).await { + Err(e) => { + assert_eq!(e.to_string(), "Protocol error: Primary key must be specified to _rw_kafka_key when creating source with FORMAT UPSERT ENCODE JSON") + } + _ => unreachable!(), + } + + let sql = "CREATE TABLE t3 (v1 int, primary key(v1)) with (connector = 'kafka') format upsert encode json" + .to_string(); + frontend.run_sql(sql).await.unwrap(); + let (table, _) = catalog_reader + .get_table_by_name( + DEFAULT_DATABASE_NAME, + SchemaPath::Name(DEFAULT_SCHEMA_NAME), + "t3", + ) + .unwrap(); + assert_eq!(table.name, "t3"); + // let columns = table + // .columns + // .iter() + // .map(|col| (col.name(), col.data_type().clone())) + // .collect::>(); + // let expect_columns = maplit::hashmap! { + // "v1" => DataType::Int32, + // "some_key" => DataType::Bytea, + // // todo: kafka connector will automatically derive the column + // // will change to a required field in the include clause + // "_rw_kafka_timestamp" => DataType::Timestamptz, + // }; + // assert_eq!(columns, expect_columns); + } } diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 4aa782b96245b..bf7815ddf4d18 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1582,8 +1582,16 @@ impl fmt::Display for Statement { if *append_only { write!(f, " APPEND ONLY")?; } - if !include_column_options.is_empty() { - todo!() + if !include_column_options.is_empty() { // (Ident, Option) + write!(f, " INCLUDE {}", display_comma_separated( + include_column_options.iter().map(|(a, b)| { + if let Some(b) = b { + format!("{} AS {}", a, b) + } else { + a.to_string() + } + }).collect_vec().as_slice() + ))?; } if !with_options.is_empty() { write!(f, " WITH ({})", display_comma_separated(with_options))?; From 0ce99ccb17b2bbe83182126c38c302a2212c357e Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 5 Dec 2023 14:47:55 +0800 Subject: [PATCH 12/50] fix --- src/frontend/src/handler/create_source.rs | 25 ----------------------- 1 file changed, 25 deletions(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index d1688d9e10fe3..3a2597c2e0cd2 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1529,30 +1529,5 @@ pub mod tests { } _ => unreachable!(), } - - let sql = "CREATE TABLE t3 (v1 int, primary key(v1)) with (connector = 'kafka') format upsert encode json" - .to_string(); - frontend.run_sql(sql).await.unwrap(); - let (table, _) = catalog_reader - .get_table_by_name( - DEFAULT_DATABASE_NAME, - SchemaPath::Name(DEFAULT_SCHEMA_NAME), - "t3", - ) - .unwrap(); - assert_eq!(table.name, "t3"); - // let columns = table - // .columns - // .iter() - // .map(|col| (col.name(), col.data_type().clone())) - // .collect::>(); - // let expect_columns = maplit::hashmap! { - // "v1" => DataType::Int32, - // "some_key" => DataType::Bytea, - // // todo: kafka connector will automatically derive the column - // // will change to a required field in the include clause - // "_rw_kafka_timestamp" => DataType::Timestamptz, - // }; - // assert_eq!(columns, expect_columns); } } From c67f13899fcdf383287773204421cd48985a095f Mon Sep 17 00:00:00 2001 From: tabVersion Date: Wed, 6 Dec 2023 15:37:31 +0800 Subject: [PATCH 13/50] compatible with prev version --- src/connector/src/source/manager.rs | 4 +++ .../src/from_proto/source/trad_source.rs | 27 ++++++++++++++++++- 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 4d4b81086c1e7..0446ba235bd31 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -33,6 +33,10 @@ pub struct SourceColumnDesc { // `is_pk` is used to indicate whether the column is part of the primary key columns. pub is_pk: bool, + + // `additional_column_type` and `column_type` are orthogonal + // `additional_column_type` is used to indicate the column is from which part of the message + // `column_type` is used to indicate the type of the column, only used in cdc scenario pub additional_column_type: AdditionalColumnType, } diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index da5e789522eaf..6fe12eb805275 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -12,10 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::{ColumnId, Schema, TableId}; +use risingwave_common::catalog::{ColumnId, Schema, TableId, DEFAULT_KEY_COLUMN_NAME}; use risingwave_common::util::sort_util::OrderType; use risingwave_connector::source::external::{CdcTableType, SchemaTableName}; use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; +use risingwave_pb::data::data_type::TypeName as PbTypeName; +use risingwave_pb::plan_common::{AdditionalColumnType, FormatType, PbEncodeType}; use risingwave_pb::stream_plan::SourceNode; use risingwave_source::source_desc::SourceDescBuilder; use risingwave_storage::panic_store::PanicStateStore; @@ -53,6 +55,29 @@ impl ExecutorBuilder for SourceExecutorBuilder { let source_name = source.source_name.clone(); let source_info = source.get_info()?; + let mut source_columns = source.columns.clone(); + + { + // compatible code: introduced in https://github.com/risingwavelabs/risingwave/pull/13707 + // for upsert and (avro | protobuf) overwrite the `_rw_key` column's ColumnDesc.additional_column_type to Key + if source_info.format() == FormatType::Upsert + && (source_info.row_encode() == PbEncodeType::Avro + || source_info.row_encode() == PbEncodeType::Protobuf) + { + let _ = source_columns.iter_mut().map(|c| { + let _ = c.column_desc.as_mut().map(|desc| { + let is_bytea = desc + .get_column_type() + .map(|col_type| col_type.type_name == PbTypeName::Bytea as i32) + .unwrap(); + if desc.name == DEFAULT_KEY_COLUMN_NAME && is_bytea { + desc.additional_column_type = AdditionalColumnType::Key as i32; + } + }); + }); + } + } + let source_desc_builder = SourceDescBuilder::new( source.columns.clone(), params.env.source_metrics(), From f4c4e447cc64baf225100c3970b6b9ca2925e305 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Wed, 6 Dec 2023 15:46:29 +0800 Subject: [PATCH 14/50] merge fix --- src/stream/src/from_proto/source/trad_source.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 65274d793a4d3..367e2e345cdc9 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -76,7 +76,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { } let source_desc_builder = SourceDescBuilder::new( - source.columns.clone(), + source_columns.clone(), params.env.source_metrics(), source.row_id_index.map(|x| x as _), source.properties.clone(), @@ -99,8 +99,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { chunk_size: params.env.config().developer.chunk_size, }; - let source_column_ids: Vec<_> = source - .columns + let source_column_ids: Vec<_> = source_columns .iter() .map(|column| ColumnId::from(column.get_column_desc().unwrap().column_id)) .collect(); From 4f6bca263eddeb1af5937797113040a55b9cc87b Mon Sep 17 00:00:00 2001 From: tabVersion Date: Thu, 7 Dec 2023 12:12:32 +0800 Subject: [PATCH 15/50] change parser trait, stash --- .../src/parser/canal/simd_json_parser.rs | 13 ++++++----- src/connector/src/parser/csv_parser.rs | 14 +++++++----- .../src/parser/debezium/debezium_parser.rs | 22 +++++++++---------- .../src/parser/debezium/mongo_json_parser.rs | 10 ++++----- src/connector/src/parser/json_parser.rs | 10 ++++----- .../src/parser/maxwell/maxwell_parser.rs | 11 +++++----- src/connector/src/parser/mod.rs | 13 +++++------ src/connector/src/parser/plain_parser.rs | 11 +++++----- src/connector/src/parser/upsert_parser.rs | 13 ++++++----- src/connector/src/parser/util.rs | 2 +- 10 files changed, 61 insertions(+), 58 deletions(-) diff --git a/src/connector/src/parser/canal/simd_json_parser.rs b/src/connector/src/parser/canal/simd_json_parser.rs index bea0561330774..e42581e99e85a 100644 --- a/src/connector/src/parser/canal/simd_json_parser.rs +++ b/src/connector/src/parser/canal/simd_json_parser.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::future::Future; + use itertools::Itertools; use risingwave_common::error::ErrorCode::{self, ProtocolError}; use risingwave_common::error::{Result, RwError}; @@ -26,7 +28,7 @@ use crate::parser::unified::ChangeEventOperation; use crate::parser::{ ByteStreamSourceParser, JsonProperties, ParserFormat, SourceStreamChunkRowWriter, }; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; const DATA: &str = "data"; const OP: &str = "type"; @@ -128,13 +130,12 @@ impl ByteStreamSourceParser for CanalJsonParser { ParserFormat::CanalJson } - async fn parse_one<'a>( + fn parse_one<'a>( &'a mut self, - _key: Option>, - payload: Option>, + message: SourceMessage, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { - only_parse_payload!(self, payload, writer) + ) -> impl Future> + Send + 'a { + only_parse_payload!(self, message.payload, writer) } } diff --git a/src/connector/src/parser/csv_parser.rs b/src/connector/src/parser/csv_parser.rs index 6513b293239b6..60b62158e619a 100644 --- a/src/connector/src/parser/csv_parser.rs +++ b/src/connector/src/parser/csv_parser.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::future::Future; + use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::{Date, Decimal, Time, Timestamp, Timestamptz}; @@ -20,7 +22,7 @@ use super::unified::{AccessError, AccessResult}; use super::{ByteStreamSourceParser, CsvProperties}; use crate::only_parse_payload; use crate::parser::{ParserFormat, SourceStreamChunkRowWriter}; -use crate::source::{DataType, SourceColumnDesc, SourceContext, SourceContextRef}; +use crate::source::{DataType, SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; macro_rules! parse { ($v:ident, $t:ty) => { @@ -156,13 +158,13 @@ impl ByteStreamSourceParser for CsvParser { ParserFormat::Csv } - async fn parse_one<'a>( + fn parse_one<'a>( &'a mut self, - _key: Option>, - payload: Option>, + message: SourceMessage, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { - only_parse_payload!(self, payload, writer) + ) -> impl Future> + Send + 'a { + // ignore other part becasue we only read payload from csv + only_parse_payload!(self, message.payload, writer) } } diff --git a/src/connector/src/parser/debezium/debezium_parser.rs b/src/connector/src/parser/debezium/debezium_parser.rs index 58ac1d85857c0..c254f1e11aa44 100644 --- a/src/connector/src/parser/debezium/debezium_parser.rs +++ b/src/connector/src/parser/debezium/debezium_parser.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::future::Future; + use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; @@ -25,7 +27,7 @@ use crate::parser::{ ParseResult, ParserFormat, ProtocolProperties, SourceStreamChunkRowWriter, SpecificParserConfig, }; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; #[derive(Debug)] pub struct DebeziumParser { @@ -133,21 +135,19 @@ impl ByteStreamSourceParser for DebeziumParser { } #[allow(clippy::unused_async)] // false positive for `async_trait` - async fn parse_one<'a>( + fn parse_one<'a>( &'a mut self, - _key: Option>, - _payload: Option>, - _writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + message: SourceMessage, + writer: SourceStreamChunkRowWriter<'a>, + ) -> impl Future> + Send + 'a { unreachable!("should call `parse_one_with_txn` instead") } - async fn parse_one_with_txn<'a>( + fn parse_one_with_txn<'a>( &'a mut self, - key: Option>, - payload: Option>, + message: SourceMessage, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result { - self.parse_inner(key, payload, writer).await + ) -> impl Future> + Send + 'a { + self.parse_inner(message, payload, writer).await } } diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index e4baacb895b62..9e3f38a182f16 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::fmt::Debug; +use std::future::Future; use risingwave_common::error::ErrorCode::{self, ProtocolError}; use risingwave_common::error::{Result, RwError}; @@ -25,7 +26,7 @@ use crate::parser::unified::debezium::{DebeziumChangeEvent, MongoProjection}; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer; use crate::parser::{ByteStreamSourceParser, ParserFormat, SourceStreamChunkRowWriter}; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; #[derive(Debug)] pub struct DebeziumMongoJsonParser { @@ -117,12 +118,11 @@ impl ByteStreamSourceParser for DebeziumMongoJsonParser { ParserFormat::DebeziumMongo } - async fn parse_one<'a>( + fn parse_one<'a>( &'a mut self, - _key: Option>, - payload: Option>, + message: SourceMessage, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> impl Future> + Send + 'a { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 86b2719e34f6b..6036f98fb186e 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::HashMap; +use std::future::Future; use apache_avro::Schema; use itertools::{Either, Itertools}; @@ -34,7 +35,7 @@ use crate::parser::{ AccessBuilder, ByteStreamSourceParser, ParserFormat, SourceStreamChunkRowWriter, }; use crate::schema::schema_registry::{handle_sr_list, Client}; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; #[derive(Debug)] pub struct JsonAccessBuilder { @@ -186,12 +187,11 @@ impl ByteStreamSourceParser for JsonParser { ParserFormat::Json } - async fn parse_one<'a>( + fn parse_one<'a>( &'a mut self, - _key: Option>, - payload: Option>, + message: SourceMessage, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> impl Future> + Send + 'a { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/maxwell/maxwell_parser.rs b/src/connector/src/parser/maxwell/maxwell_parser.rs index 6fa2dab9eaa7e..861064d19454d 100644 --- a/src/connector/src/parser/maxwell/maxwell_parser.rs +++ b/src/connector/src/parser/maxwell/maxwell_parser.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::future::Future; + use risingwave_common::error::{ErrorCode, Result, RwError}; use crate::only_parse_payload; @@ -21,7 +23,7 @@ use crate::parser::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, ParserFormat, SourceStreamChunkRowWriter, SpecificParserConfig, }; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; #[derive(Debug)] pub struct MaxwellParser { @@ -78,12 +80,11 @@ impl ByteStreamSourceParser for MaxwellParser { ParserFormat::Maxwell } - async fn parse_one<'a>( + fn parse_one<'a>( &'a mut self, - _key: Option>, - payload: Option>, + message: SourceMessage, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> impl Future> + Send + 'a { // restrict the behaviours since there is no corresponding // key/value test for maxwell yet. only_parse_payload!(self, payload, writer) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index e49feb434c75a..320fda9e553be 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -48,8 +48,8 @@ use crate::parser::maxwell::MaxwellParser; use crate::schema::schema_registry::SchemaRegistryAuth; use crate::source::{ extract_source_struct, BoxSourceStream, SourceColumnDesc, SourceColumnType, SourceContext, - SourceContextRef, SourceEncode, SourceFormat, SourceMeta, SourceWithStateStream, SplitId, - StreamChunkWithState, + SourceContextRef, SourceEncode, SourceFormat, SourceMessage, SourceMeta, SourceWithStateStream, + SplitId, StreamChunkWithState, }; pub mod additional_columns; @@ -446,8 +446,7 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { /// Returns error if **any** of the rows in the message failed to parse. fn parse_one<'a>( &'a mut self, - key: Option>, - payload: Option>, + message: SourceMessage, writer: SourceStreamChunkRowWriter<'a>, ) -> impl Future> + Send + 'a; @@ -460,11 +459,10 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { /// Returns error if **any** of the rows in the message failed to parse. fn parse_one_with_txn<'a>( &'a mut self, - key: Option>, - payload: Option>, + message: SourceMessage, writer: SourceStreamChunkRowWriter<'a>, ) -> impl Future> + Send + 'a { - self.parse_one(key, payload, writer) + self.parse_one(message, writer) .map_ok(|_| ParseResult::Rows) } } @@ -564,7 +562,6 @@ async fn into_chunk_stream(mut parser: P, data_stream match parser .parse_one_with_txn( msg.key, - msg.payload, builder.row_writer().with_meta(MessageMeta { meta: &msg.meta, split_id: &msg.split_id, diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index defb7ef54a1e6..47f73ac8210f2 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::future::Future; + use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{ErrorCode, Result, RwError}; @@ -22,7 +24,7 @@ use super::{ }; use crate::only_parse_payload; use crate::parser::ParserFormat; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; #[derive(Debug)] pub struct PlainParser { @@ -80,12 +82,11 @@ impl ByteStreamSourceParser for PlainParser { ParserFormat::Plain } - async fn parse_one<'a>( + fn parse_one<'a>( &'a mut self, - _key: Option>, - payload: Option>, + message: SourceMessage, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { + ) -> impl Future> + Send + 'a { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index 71210b9e4b8f8..13616690063f5 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::future::Future; + use risingwave_common::catalog::DEFAULT_KEY_COLUMN_NAME; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; @@ -26,7 +28,7 @@ use super::{ }; use crate::extract_key_config; use crate::parser::ParserFormat; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; #[derive(Debug)] pub struct UpsertParser { @@ -127,12 +129,11 @@ impl ByteStreamSourceParser for UpsertParser { ParserFormat::Upsert } - async fn parse_one<'a>( + fn parse_one<'a>( &'a mut self, - key: Option>, - payload: Option>, + message: SourceMessage, writer: SourceStreamChunkRowWriter<'a>, - ) -> Result<()> { - self.parse_inner(key, payload, writer).await + ) -> impl Future> + Send + 'a { + self.parse_inner(message, payload, writer).await } } diff --git a/src/connector/src/parser/util.rs b/src/connector/src/parser/util.rs index 1e07fe1032589..93a982b8cabca 100644 --- a/src/connector/src/parser/util.rs +++ b/src/connector/src/parser/util.rs @@ -98,7 +98,7 @@ pub(super) fn at_least_one_ok(mut results: Vec>) -> Result<()> { // For parser that doesn't support key currently #[macro_export] macro_rules! only_parse_payload { - ($self:ident, $payload:ident, $writer:ident) => { + ($self:ident, $payload:expr, $writer:ident) => { if let Some(payload) = $payload { $self.parse_inner(payload, $writer).await } else { From 2d0ad748725cc7a9e47508c775eb92a80d040b14 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Thu, 7 Dec 2023 16:09:16 +0800 Subject: [PATCH 16/50] rerun Signed-off-by: tabVersion From 6fb592207af1f1d7613784ce794ae88fbfddd089 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 11 Dec 2023 04:46:50 +0800 Subject: [PATCH 17/50] stash --- src/connector/src/parser/mod.rs | 102 ++++++++++++++++++++------------ 1 file changed, 65 insertions(+), 37 deletions(-) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 320fda9e553be..f1e1e1c41eada 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -33,6 +33,7 @@ use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::catalog::{ SchemaRegistryNameStrategy as PbSchemaRegistryNameStrategy, StreamSourceInfo, }; +use risingwave_pb::plan_common::AdditionalColumnType; use tracing_futures::Instrument; use self::avro::AvroAccessBuilder; @@ -310,32 +311,64 @@ impl SourceStreamChunkRowWriter<'_> { mut f: impl FnMut(&SourceColumnDesc) -> AccessResult, ) -> AccessResult<()> { let mut wrapped_f = |desc: &SourceColumnDesc| { - if let Some(meta_value) = - (self.row_meta.as_ref()).and_then(|row_meta| row_meta.value_for_column(desc)) - { - // For meta columns, fill in the meta data. - Ok(A::output_for(meta_value)) - } else { - // For normal columns, call the user provided closure. - match f(desc) { - Ok(output) => Ok(output), - - // Throw error for failed access to primary key columns. - Err(e) if desc.is_pk => Err(e), - // Ignore error for other columns and fill in `NULL` instead. - Err(error) => { - // TODO: figure out a way to fill in not-null default value if user specifies one - // TODO: decide whether the error should not be ignored (e.g., even not a valid Debezium message) - // TODO: not using tracing span to provide `split_id` and `offset` due to performance concern, - // see #13105 - tracing::warn!( - %error, - split_id = self.row_meta.as_ref().map(|m| m.split_id), - offset = self.row_meta.as_ref().map(|m| m.offset), - column = desc.name, - "failed to parse non-pk column, padding with `NULL`" - ); - Ok(A::output_for(Datum::None)) + match (&desc.column_type, &desc.additional_column_type) { + (&SourceColumnType::Offset | &SourceColumnType::RowId, _) => { + // SourceColumnType is for CDC source only. + Ok(A::output_for( + self.row_meta + .as_ref() + .and_then(|row_meta| row_meta.value_for_column(desc)) + .unwrap(), // handled all match cases in internal match, unwrap is safe + )) + } + (&SourceColumnType::Meta, _) + if matches!( + &self.row_meta, + &Some(SourceMeta::Kafka | SourceMeta::DebeziumCdc) + ) => + { + // SourceColumnType is for CDC source only. + return Ok(A::output_for( + self.row_meta + .as_ref() + .and_then(|row_meta| row_meta.value_for_column(desc)) + .unwrap(), // handled all match cases in internal match, unwrap is safe + )); + } + ( + _, + &AdditionalColumnType::Timestamp + | &AdditionalColumnType::Partition + | &AdditionalColumnType::Filename + | &AdditionalColumnType::Offset + | &AdditionalColumnType::Header, + // AdditionalColumnType::Unspecified is means it comes from message payload + // AdditionalColumnType::Key is processed in normal process, together with Unspecified ones + ) => { + todo!() + } + (_, _) => { + // For normal columns, call the user provided closure. + match f(desc) { + Ok(output) => Ok(output), + + // Throw error for failed access to primary key columns. + Err(e) if desc.is_pk => Err(e), + // Ignore error for other columns and fill in `NULL` instead. + Err(error) => { + // TODO: figure out a way to fill in not-null default value if user specifies one + // TODO: decide whether the error should not be ignored (e.g., even not a valid Debezium message) + // TODO: not using tracing span to provide `split_id` and `offset` due to performance concern, + // see #13105 + tracing::warn!( + %error, + split_id = self.row_meta.as_ref().map(|m| m.split_id), + offset = self.row_meta.as_ref().map(|m| m.offset), + column = desc.name, + "failed to parse non-pk column, padding with `NULL`" + ); + Ok(A::output_for(Datum::None)) + } } } } @@ -559,17 +592,12 @@ async fn into_chunk_stream(mut parser: P, data_stream split_offset_mapping.insert(msg.split_id.clone(), msg.offset.clone()); let old_op_num = builder.op_num(); - match parser - .parse_one_with_txn( - msg.key, - builder.row_writer().with_meta(MessageMeta { - meta: &msg.meta, - split_id: &msg.split_id, - offset: &msg.offset, - }), - ) - .await - { + let row_builder = builder.row_writer().with_meta(MessageMeta { + meta: &msg.meta, + split_id: &msg.split_id, + offset: &msg.offset, + }); + match parser.parse_one_with_txn(msg, row_builder).await { // It's possible that parsing multiple rows in a single message PARTIALLY failed. // We still have to maintain the row number in this case. res @ (Ok(ParseResult::Rows) | Err(_)) => { From 383e8449e89fb782e75d51a50420f92e7c4309b0 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 11 Dec 2023 04:48:30 +0800 Subject: [PATCH 18/50] Revert "change parser trait, stash" This reverts commit 4f6bca26 --- .../src/parser/canal/simd_json_parser.rs | 13 ++++---- src/connector/src/parser/csv_parser.rs | 14 ++++----- .../src/parser/debezium/debezium_parser.rs | 22 +++++++------- .../src/parser/debezium/mongo_json_parser.rs | 10 +++---- src/connector/src/parser/json_parser.rs | 10 +++---- .../src/parser/maxwell/maxwell_parser.rs | 11 ++++--- src/connector/src/parser/mod.rs | 30 ++++++++++++------- src/connector/src/parser/plain_parser.rs | 11 ++++--- src/connector/src/parser/upsert_parser.rs | 13 ++++---- src/connector/src/parser/util.rs | 2 +- 10 files changed, 69 insertions(+), 67 deletions(-) diff --git a/src/connector/src/parser/canal/simd_json_parser.rs b/src/connector/src/parser/canal/simd_json_parser.rs index e42581e99e85a..bea0561330774 100644 --- a/src/connector/src/parser/canal/simd_json_parser.rs +++ b/src/connector/src/parser/canal/simd_json_parser.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::future::Future; - use itertools::Itertools; use risingwave_common::error::ErrorCode::{self, ProtocolError}; use risingwave_common::error::{Result, RwError}; @@ -28,7 +26,7 @@ use crate::parser::unified::ChangeEventOperation; use crate::parser::{ ByteStreamSourceParser, JsonProperties, ParserFormat, SourceStreamChunkRowWriter, }; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; const DATA: &str = "data"; const OP: &str = "type"; @@ -130,12 +128,13 @@ impl ByteStreamSourceParser for CanalJsonParser { ParserFormat::CanalJson } - fn parse_one<'a>( + async fn parse_one<'a>( &'a mut self, - message: SourceMessage, + _key: Option>, + payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { - only_parse_payload!(self, message.payload, writer) + ) -> Result<()> { + only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/csv_parser.rs b/src/connector/src/parser/csv_parser.rs index 60b62158e619a..6513b293239b6 100644 --- a/src/connector/src/parser/csv_parser.rs +++ b/src/connector/src/parser/csv_parser.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::future::Future; - use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::{Date, Decimal, Time, Timestamp, Timestamptz}; @@ -22,7 +20,7 @@ use super::unified::{AccessError, AccessResult}; use super::{ByteStreamSourceParser, CsvProperties}; use crate::only_parse_payload; use crate::parser::{ParserFormat, SourceStreamChunkRowWriter}; -use crate::source::{DataType, SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; +use crate::source::{DataType, SourceColumnDesc, SourceContext, SourceContextRef}; macro_rules! parse { ($v:ident, $t:ty) => { @@ -158,13 +156,13 @@ impl ByteStreamSourceParser for CsvParser { ParserFormat::Csv } - fn parse_one<'a>( + async fn parse_one<'a>( &'a mut self, - message: SourceMessage, + _key: Option>, + payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { - // ignore other part becasue we only read payload from csv - only_parse_payload!(self, message.payload, writer) + ) -> Result<()> { + only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/debezium/debezium_parser.rs b/src/connector/src/parser/debezium/debezium_parser.rs index c254f1e11aa44..58ac1d85857c0 100644 --- a/src/connector/src/parser/debezium/debezium_parser.rs +++ b/src/connector/src/parser/debezium/debezium_parser.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::future::Future; - use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; @@ -27,7 +25,7 @@ use crate::parser::{ ParseResult, ParserFormat, ProtocolProperties, SourceStreamChunkRowWriter, SpecificParserConfig, }; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; #[derive(Debug)] pub struct DebeziumParser { @@ -135,19 +133,21 @@ impl ByteStreamSourceParser for DebeziumParser { } #[allow(clippy::unused_async)] // false positive for `async_trait` - fn parse_one<'a>( + async fn parse_one<'a>( &'a mut self, - message: SourceMessage, - writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { + _key: Option>, + _payload: Option>, + _writer: SourceStreamChunkRowWriter<'a>, + ) -> Result<()> { unreachable!("should call `parse_one_with_txn` instead") } - fn parse_one_with_txn<'a>( + async fn parse_one_with_txn<'a>( &'a mut self, - message: SourceMessage, + key: Option>, + payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { - self.parse_inner(message, payload, writer).await + ) -> Result { + self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 9e3f38a182f16..e4baacb895b62 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::fmt::Debug; -use std::future::Future; use risingwave_common::error::ErrorCode::{self, ProtocolError}; use risingwave_common::error::{Result, RwError}; @@ -26,7 +25,7 @@ use crate::parser::unified::debezium::{DebeziumChangeEvent, MongoProjection}; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer; use crate::parser::{ByteStreamSourceParser, ParserFormat, SourceStreamChunkRowWriter}; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; #[derive(Debug)] pub struct DebeziumMongoJsonParser { @@ -118,11 +117,12 @@ impl ByteStreamSourceParser for DebeziumMongoJsonParser { ParserFormat::DebeziumMongo } - fn parse_one<'a>( + async fn parse_one<'a>( &'a mut self, - message: SourceMessage, + _key: Option>, + payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { + ) -> Result<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 6036f98fb186e..86b2719e34f6b 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::collections::HashMap; -use std::future::Future; use apache_avro::Schema; use itertools::{Either, Itertools}; @@ -35,7 +34,7 @@ use crate::parser::{ AccessBuilder, ByteStreamSourceParser, ParserFormat, SourceStreamChunkRowWriter, }; use crate::schema::schema_registry::{handle_sr_list, Client}; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; #[derive(Debug)] pub struct JsonAccessBuilder { @@ -187,11 +186,12 @@ impl ByteStreamSourceParser for JsonParser { ParserFormat::Json } - fn parse_one<'a>( + async fn parse_one<'a>( &'a mut self, - message: SourceMessage, + _key: Option>, + payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { + ) -> Result<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/maxwell/maxwell_parser.rs b/src/connector/src/parser/maxwell/maxwell_parser.rs index 861064d19454d..6fa2dab9eaa7e 100644 --- a/src/connector/src/parser/maxwell/maxwell_parser.rs +++ b/src/connector/src/parser/maxwell/maxwell_parser.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::future::Future; - use risingwave_common::error::{ErrorCode, Result, RwError}; use crate::only_parse_payload; @@ -23,7 +21,7 @@ use crate::parser::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, ParserFormat, SourceStreamChunkRowWriter, SpecificParserConfig, }; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; #[derive(Debug)] pub struct MaxwellParser { @@ -80,11 +78,12 @@ impl ByteStreamSourceParser for MaxwellParser { ParserFormat::Maxwell } - fn parse_one<'a>( + async fn parse_one<'a>( &'a mut self, - message: SourceMessage, + _key: Option>, + payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { + ) -> Result<()> { // restrict the behaviours since there is no corresponding // key/value test for maxwell yet. only_parse_payload!(self, payload, writer) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index f1e1e1c41eada..7a71473f3487f 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -49,8 +49,8 @@ use crate::parser::maxwell::MaxwellParser; use crate::schema::schema_registry::SchemaRegistryAuth; use crate::source::{ extract_source_struct, BoxSourceStream, SourceColumnDesc, SourceColumnType, SourceContext, - SourceContextRef, SourceEncode, SourceFormat, SourceMessage, SourceMeta, SourceWithStateStream, - SplitId, StreamChunkWithState, + SourceContextRef, SourceEncode, SourceFormat, SourceMeta, SourceWithStateStream, SplitId, + StreamChunkWithState, }; pub mod additional_columns; @@ -479,7 +479,8 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { /// Returns error if **any** of the rows in the message failed to parse. fn parse_one<'a>( &'a mut self, - message: SourceMessage, + key: Option>, + payload: Option>, writer: SourceStreamChunkRowWriter<'a>, ) -> impl Future> + Send + 'a; @@ -492,10 +493,11 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { /// Returns error if **any** of the rows in the message failed to parse. fn parse_one_with_txn<'a>( &'a mut self, - message: SourceMessage, + key: Option>, + payload: Option>, writer: SourceStreamChunkRowWriter<'a>, ) -> impl Future> + Send + 'a { - self.parse_one(message, writer) + self.parse_one(key, payload, writer) .map_ok(|_| ParseResult::Rows) } } @@ -592,12 +594,18 @@ async fn into_chunk_stream(mut parser: P, data_stream split_offset_mapping.insert(msg.split_id.clone(), msg.offset.clone()); let old_op_num = builder.op_num(); - let row_builder = builder.row_writer().with_meta(MessageMeta { - meta: &msg.meta, - split_id: &msg.split_id, - offset: &msg.offset, - }); - match parser.parse_one_with_txn(msg, row_builder).await { + match parser + .parse_one_with_txn( + msg.key, + msg.payload, + builder.row_writer().with_meta(MessageMeta { + meta: &msg.meta, + split_id: &msg.split_id, + offset: &msg.offset, + }), + ) + .await + { // It's possible that parsing multiple rows in a single message PARTIALLY failed. // We still have to maintain the row number in this case. res @ (Ok(ParseResult::Rows) | Err(_)) => { diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 47f73ac8210f2..defb7ef54a1e6 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::future::Future; - use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{ErrorCode, Result, RwError}; @@ -24,7 +22,7 @@ use super::{ }; use crate::only_parse_payload; use crate::parser::ParserFormat; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; #[derive(Debug)] pub struct PlainParser { @@ -82,11 +80,12 @@ impl ByteStreamSourceParser for PlainParser { ParserFormat::Plain } - fn parse_one<'a>( + async fn parse_one<'a>( &'a mut self, - message: SourceMessage, + _key: Option>, + payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { + ) -> Result<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index 13616690063f5..71210b9e4b8f8 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::future::Future; - use risingwave_common::catalog::DEFAULT_KEY_COLUMN_NAME; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; @@ -28,7 +26,7 @@ use super::{ }; use crate::extract_key_config; use crate::parser::ParserFormat; -use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef, SourceMessage}; +use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; #[derive(Debug)] pub struct UpsertParser { @@ -129,11 +127,12 @@ impl ByteStreamSourceParser for UpsertParser { ParserFormat::Upsert } - fn parse_one<'a>( + async fn parse_one<'a>( &'a mut self, - message: SourceMessage, + key: Option>, + payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { - self.parse_inner(message, payload, writer).await + ) -> Result<()> { + self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/util.rs b/src/connector/src/parser/util.rs index 93a982b8cabca..1e07fe1032589 100644 --- a/src/connector/src/parser/util.rs +++ b/src/connector/src/parser/util.rs @@ -98,7 +98,7 @@ pub(super) fn at_least_one_ok(mut results: Vec>) -> Result<()> { // For parser that doesn't support key currently #[macro_export] macro_rules! only_parse_payload { - ($self:ident, $payload:expr, $writer:ident) => { + ($self:ident, $payload:ident, $writer:ident) => { if let Some(payload) = $payload { $self.parse_inner(payload, $writer).await } else { From 64e665667b30cb0f6cd6309699bd2c6870ed0b66 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 11 Dec 2023 05:14:50 +0800 Subject: [PATCH 19/50] fix --- src/connector/src/parser/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 7a71473f3487f..f48fda5ae3882 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -323,8 +323,8 @@ impl SourceStreamChunkRowWriter<'_> { } (&SourceColumnType::Meta, _) if matches!( - &self.row_meta, - &Some(SourceMeta::Kafka | SourceMeta::DebeziumCdc) + &self.row_meta.map(|ele| ele.meta), + &Some(SourceMeta::Kafka(_) | SourceMeta::DebeziumCdc(_)) ) => { // SourceColumnType is for CDC source only. From b2cafab93797db4c6ffe6125f81435353b83c960 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 11 Dec 2023 06:56:38 +0800 Subject: [PATCH 20/50] stash --- src/connector/src/parser/upsert_parser.rs | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index 71210b9e4b8f8..0927c8635c10c 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -15,6 +15,7 @@ use risingwave_common::catalog::DEFAULT_KEY_COLUMN_NAME; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; +use risingwave_pb::plan_common::AdditionalColumnType; use super::bytes_parser::BytesAccessBuilder; use super::unified::upsert::UpsertChangeEvent; @@ -52,13 +53,14 @@ async fn build_accessor_builder( } } -fn check_rw_default_key(columns: &Vec) -> bool { - for col in columns { - if col.name.starts_with(DEFAULT_KEY_COLUMN_NAME) { - return true; +fn get_key_column_name(columns: &Vec) -> Option { + columns.iter().find_map(|column| { + if column.additional_column_type == AdditionalColumnType::Key { + Some(column.name.clone()) + } else { + None } - } - false + }) } impl UpsertParser { @@ -67,12 +69,12 @@ impl UpsertParser { rw_columns: Vec, source_ctx: SourceContextRef, ) -> Result { - // check whether columns has `DEFAULT_KEY_COLUMN_NAME`, if so, the key accessor should be + // check whether columns has Key as AdditionalColumnType, if so, the key accessor should be // bytes - let key_builder = if check_rw_default_key(&rw_columns) { + let key_builder = if let Some(key_column_name) = get_key_column_name(&rw_columns) { AccessBuilderImpl::Bytes(BytesAccessBuilder::new(EncodingProperties::Bytes( BytesProperties { - column_name: Some(DEFAULT_KEY_COLUMN_NAME.into()), + column_name: Some(key_column_name), }, ))?) } else { From ac034368f3ddd294b313b451b3f5a372a76f33b9 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 11 Dec 2023 17:17:38 +0800 Subject: [PATCH 21/50] refactor: Refactor and standardize the `access_field` function and imports - Import `SourceColumnDesc` from `crate::source` in `upsert.rs`, `maxwell.rs`, and `debezium.rs` - Modify the `access_field` function to take a `desc: &SourceColumnDesc` parameter instead of separate `name` and `type_expected` parameters in `upsert.rs`, `maxwell.rs`, and `debezium.rs` - Remove unreachable code in `upsert_parser.rs` - Change the parameter signature of the `apply_row_operation_on_stream_chunk_writer_with_op` function in `util.rs` - Fix error messages and add error messages for various cases in `create_source.rs` Signed-off-by: tabVersion --- src/connector/src/parser/unified/debezium.rs | 13 +++---- src/connector/src/parser/unified/maxwell.rs | 5 ++- src/connector/src/parser/unified/mod.rs | 7 ++-- src/connector/src/parser/unified/upsert.rs | 12 +++--- src/connector/src/parser/unified/util.rs | 2 +- src/connector/src/parser/upsert_parser.rs | 5 ++- src/frontend/src/handler/create_source.rs | 40 ++++++++++---------- 7 files changed, 44 insertions(+), 40 deletions(-) diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index e392e31e3644d..8011bc0a42361 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -16,6 +16,7 @@ use risingwave_common::types::{DataType, Datum, ScalarImpl}; use super::{Access, AccessError, ChangeEvent, ChangeEventOperation}; use crate::parser::TransactionControl; +use crate::source::SourceColumnDesc; pub struct DebeziumChangeEvent { value_accessor: Option, @@ -89,20 +90,16 @@ impl ChangeEvent for DebeziumChangeEvent where A: Access, { - fn access_field( - &self, - name: &str, - type_expected: &risingwave_common::types::DataType, - ) -> super::AccessResult { + fn access_field(&self, desc: &SourceColumnDesc) -> super::AccessResult { match self.op()? { ChangeEventOperation::Delete => { if let Some(va) = self.value_accessor.as_ref() { - va.access(&[BEFORE, name], Some(type_expected)) + va.access(&[BEFORE, &desc.name], Some(&desc.data_type)) } else { self.key_accessor .as_ref() .unwrap() - .access(&[name], Some(type_expected)) + .access(&[&desc.name], Some(&desc.data_type)) } } @@ -111,7 +108,7 @@ where .value_accessor .as_ref() .unwrap() - .access(&[AFTER, name], Some(type_expected)), + .access(&[AFTER, &desc.name], Some(&desc.data_type)), } } diff --git a/src/connector/src/parser/unified/maxwell.rs b/src/connector/src/parser/unified/maxwell.rs index 1ccb83353f03d..7303d082d916c 100644 --- a/src/connector/src/parser/unified/maxwell.rs +++ b/src/connector/src/parser/unified/maxwell.rs @@ -16,6 +16,7 @@ use risingwave_common::types::{DataType, ScalarImpl}; use super::{Access, ChangeEvent}; use crate::parser::unified::ChangeEventOperation; +use crate::source::SourceColumnDesc; pub const MAXWELL_INSERT_OP: &str = "insert"; pub const MAXWELL_UPDATE_OP: &str = "update"; @@ -48,8 +49,8 @@ where }) } - fn access_field(&self, name: &str, type_expected: &DataType) -> super::AccessResult { + fn access_field(&self, desc: &SourceColumnDesc) -> super::AccessResult { const DATA: &str = "data"; - self.0.access(&[DATA, name], Some(type_expected)) + self.0.access(&[DATA, &desc.name], Some(&desc.data_type)) } } diff --git a/src/connector/src/parser/unified/mod.rs b/src/connector/src/parser/unified/mod.rs index f99fd0d2769f8..8c8abcdd0f30c 100644 --- a/src/connector/src/parser/unified/mod.rs +++ b/src/connector/src/parser/unified/mod.rs @@ -22,6 +22,7 @@ use self::avro::AvroAccess; use self::bytes::BytesAccess; use self::json::JsonAccess; use self::protobuf::ProtobufAccess; +use crate::source::SourceColumnDesc; pub mod avro; pub mod bytes; @@ -69,7 +70,7 @@ pub trait ChangeEvent { /// Access the operation type. fn op(&self) -> std::result::Result; /// Access the field after the operation. - fn access_field(&self, name: &str, type_expected: &DataType) -> AccessResult; + fn access_field(&self, desc: &SourceColumnDesc) -> AccessResult; } impl ChangeEvent for (ChangeEventOperation, A) @@ -80,8 +81,8 @@ where Ok(self.0) } - fn access_field(&self, name: &str, type_expected: &DataType) -> AccessResult { - self.1.access(&[name], Some(type_expected)) + fn access_field(&self, desc: &SourceColumnDesc) -> AccessResult { + self.1.access(&[desc.name.as_str()], Some(&desc.data_type)) } } diff --git a/src/connector/src/parser/unified/upsert.rs b/src/connector/src/parser/unified/upsert.rs index 2697d4bdf8151..f311f9a11c2b4 100644 --- a/src/connector/src/parser/unified/upsert.rs +++ b/src/connector/src/parser/unified/upsert.rs @@ -16,6 +16,7 @@ use risingwave_common::types::DataType; use super::{Access, ChangeEvent, ChangeEventOperation}; use crate::parser::unified::AccessError; +use crate::source::SourceColumnDesc; /// `UpsertAccess` wraps a key-value message format into an upsert source. /// A key accessor and a value accessor are required. @@ -102,22 +103,23 @@ where } } - fn access_field(&self, name: &str, type_expected: &DataType) -> super::AccessResult { + fn access_field(&self, desc: &SourceColumnDesc) -> super::AccessResult { // access value firstly - match self.access(&["value", name], Some(type_expected)) { + match self.access(&["value", &desc.name], Some(&desc.data_type)) { Err(AccessError::Undefined { .. }) => (), // fallthrough other => return other, }; - match self.access(&["key", name], Some(type_expected)) { + match self.access(&["key", &desc.name], Some(&desc.data_type)) { Err(AccessError::Undefined { .. }) => (), // fallthrough other => return other, }; if let Some(key_as_column_name) = &self.key_as_column_name - && name == key_as_column_name + && &desc.name == key_as_column_name { - return self.access(&["key"], Some(type_expected)); + // todo: check logic later + return self.access(&["key"], Some(&desc.data_type)); } Ok(None) diff --git a/src/connector/src/parser/unified/util.rs b/src/connector/src/parser/unified/util.rs index 92cf5da3ac81c..c25892835df70 100644 --- a/src/connector/src/parser/unified/util.rs +++ b/src/connector/src/parser/unified/util.rs @@ -24,7 +24,7 @@ pub fn apply_row_operation_on_stream_chunk_writer_with_op( writer: &mut SourceStreamChunkRowWriter<'_>, op: ChangeEventOperation, ) -> AccessResult<()> { - let f = |column: &SourceColumnDesc| row_op.access_field(&column.name, &column.data_type); + let f = |column: &SourceColumnDesc| row_op.access_field(column); match op { ChangeEventOperation::Upsert => writer.insert(f), ChangeEventOperation::Delete => writer.delete(f), diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index 0927c8635c10c..de5c463419bcc 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -72,14 +72,15 @@ impl UpsertParser { // check whether columns has Key as AdditionalColumnType, if so, the key accessor should be // bytes let key_builder = if let Some(key_column_name) = get_key_column_name(&rw_columns) { + // later: if key column has other type other than bytes, build other accessor. + // For now, all key columns are bytes AccessBuilderImpl::Bytes(BytesAccessBuilder::new(EncodingProperties::Bytes( BytesProperties { column_name: Some(key_column_name), }, ))?) } else { - let (key_config, key_type) = extract_key_config!(props); - build_accessor_builder(key_config, key_type).await? + unreachable!("format upsert must have key column") }; let payload_builder = build_accessor_builder(props.encoding_config, EncodingType::Value).await?; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 240d85a964f4b..375bf2975f6ac 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -743,8 +743,6 @@ pub(crate) async fn bind_source_pk( sql_defined_pk_names: Vec, with_properties: &HashMap, ) -> Result> { - // todo: handle pk carefully, revisit logic later - let sql_defined_pk = !sql_defined_pk_names.is_empty(); let key_column_name: Option = { // iter columns to check if contains additional columns from key part @@ -770,6 +768,9 @@ pub(crate) async fn bind_source_pk( let res = match (&source_schema.format, &source_schema.row_encode) { (Format::Native, Encode::Native) | (Format::Plain, _) => sql_defined_pk_names, + + // For all Upsert formats, we only accept one and only key column as primary key. + // Additional KEY columns must be set in this case and must be primary key. (Format::Upsert, Encode::Json) => { if let Some(ref key_column_name) = key_column_name && sql_defined_pk { if sql_defined_pk_names.len() != 1 { @@ -799,25 +800,26 @@ pub(crate) async fn bind_source_pk( } } (Format::Upsert, Encode::Avro) => { - // todo: check logic here: - // * if defined pk, it must be the same as key column - // * if not defined pk, extract pk from schema but put a mark on the columns - // * if no pk in schema, use the key column as primary key - if sql_defined_pk { - if sql_defined_pk_names.len() != 1 { - return Err(RwError::from(ProtocolError( - "upsert avro supports only one primary key column.".to_string(), - ))); + // prev deprecated logic: + // if key schema can be inferred from schema registry, then use it + if sql_defined_pk && sql_defined_pk_names.len() != 1 { + return Err(RwError::from(ProtocolError( + "upsert avro supports only one primary key column.".to_string(), + ))); + } + if let Some(ref key_column_name) = key_column_name { + if key_column_name.eq(sql_defined_pk_names[0].as_str()) { + sql_defined_pk_names + } else { + return Err(RwError::from(ProtocolError(format!( + "upsert avro's key column {} not match with sql defined primary key {}", + key_column_name, sql_defined_pk_names[0] + )))); } - sql_defined_pk_names - } else if let Some(extracted_pk_names) = - extract_upsert_avro_table_pk_columns(source_info, with_properties).await? - { - extracted_pk_names } else { - // For upsert avro, if we can't extract pk from schema, use message key as primary key - add_default_key_column(columns); - vec![DEFAULT_KEY_COLUMN_NAME.into()] + return Err(RwError::from(ProtocolError( + "INCLUDE KEY clause must be set for FORMAT UPSERT ENCODE AVRO".to_string(), + ))); } } From fece45b40cd1f8fff078377a723240491a9cf571 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 11 Dec 2023 18:26:29 +0800 Subject: [PATCH 22/50] Refactor parsing logic and imports in connector code - Make `get_key_column_name` public in `upsert_parser.rs` - Update `PlainParser` struct in `plain_parser.rs` to include a new `key_builder` field and initialize it appropriately - Modify `parse_inner` method in `plain_parser.rs` to handle key and payload parameters as options and use `key_builder` and `payload_builder` accessors - Import new dependencies and update logic for parsing key and payload in `plain_parser.rs` - Modify `access_field` function in `upsert.rs` to handle accessing key columns differently and remove unreachable code - Remove `key_builder` attribute from `PlainParser` struct in `parser.rs` and add new test functions and module Signed-off-by: tabVersion --- src/connector/src/parser/avro/parser.rs | 1 + src/connector/src/parser/plain_parser.rs | 46 +++++++++++++++++++--- src/connector/src/parser/unified/upsert.rs | 36 ++++++++++------- src/connector/src/parser/upsert_parser.rs | 2 +- 4 files changed, 63 insertions(+), 22 deletions(-) diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 10473a031e89c..05fd2d0e3babf 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -305,6 +305,7 @@ mod test { let conf = new_avro_conf_from_local(file_name).await?; Ok(PlainParser { + key_builder: None, payload_builder: AccessBuilderImpl::Avro(AvroAccessBuilder::new( conf, EncodingType::Value, diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index defb7ef54a1e6..8430e326222e5 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -21,11 +21,17 @@ use super::{ SourceStreamChunkRowWriter, SpecificParserConfig, }; use crate::only_parse_payload; -use crate::parser::ParserFormat; +use crate::parser::bytes_parser::BytesAccessBuilder; +use crate::parser::unified::upsert::UpsertChangeEvent; +use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer_with_op; +use crate::parser::unified::{AccessImpl, ChangeEventOperation}; +use crate::parser::upsert_parser::get_key_column_name; +use crate::parser::{BytesProperties, ParserFormat}; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; #[derive(Debug)] pub struct PlainParser { + pub key_builder: Option, pub payload_builder: AccessBuilderImpl, pub(crate) rw_columns: Vec, pub source_ctx: SourceContextRef, @@ -37,6 +43,16 @@ impl PlainParser { rw_columns: Vec, source_ctx: SourceContextRef, ) -> Result { + let key_builder = if let Some(key_column_name) = get_key_column_name(&rw_columns) { + Some(AccessBuilderImpl::Bytes(BytesAccessBuilder::new( + EncodingProperties::Bytes(BytesProperties { + column_name: Some(key_column_name), + }), + )?)) + } else { + None + }; + let payload_builder = match props.encoding_config { EncodingProperties::Protobuf(_) | EncodingProperties::Avro(_) @@ -50,6 +66,7 @@ impl PlainParser { } }; Ok(Self { + key_builder, payload_builder, rw_columns, source_ctx, @@ -58,12 +75,29 @@ impl PlainParser { pub async fn parse_inner( &mut self, - payload: Vec, + key: Option>, + payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, ) -> Result<()> { - let accessor = self.payload_builder.generate_accessor(payload).await?; + // reuse upsert component but always insert + let mut row_op: UpsertChangeEvent, AccessImpl<'_, '_>> = + UpsertChangeEvent::default(); + let change_event_op = ChangeEventOperation::Upsert; + if let Some(data) = key { + // key is optional in format plain + row_op = row_op.with_key(self.key_builder.generate_accessor(data).await?); + } + if let Some(data) = payload { + row_op = row_op.with_value(self.payload_builder.generate_accessor(data).await?); + } else { + return Err(RwError::from(ErrorCode::InternalError( + "payload is required in format plain, but got empty".to_string(), + ))); + } - apply_row_accessor_on_stream_chunk_writer(accessor, &mut writer).map_err(Into::into) + apply_row_operation_on_stream_chunk_writer_with_op(row_op, &mut writer, change_event_op) + .map_err(Into::into) + // apply_row_accessor_on_stream_chunk_writer(accessor, &mut writer).map_err(Into::into) } } @@ -82,10 +116,10 @@ impl ByteStreamSourceParser for PlainParser { async fn parse_one<'a>( &'a mut self, - _key: Option>, + key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, ) -> Result<()> { - only_parse_payload!(self, payload, writer) + self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/unified/upsert.rs b/src/connector/src/parser/unified/upsert.rs index f311f9a11c2b4..565ca51cf9b05 100644 --- a/src/connector/src/parser/unified/upsert.rs +++ b/src/connector/src/parser/unified/upsert.rs @@ -13,6 +13,7 @@ // limitations under the License. use risingwave_common::types::DataType; +use risingwave_pb::plan_common::AdditionalColumnType; use super::{Access, ChangeEvent, ChangeEventOperation}; use crate::parser::unified::AccessError; @@ -105,21 +106,26 @@ where fn access_field(&self, desc: &SourceColumnDesc) -> super::AccessResult { // access value firstly - match self.access(&["value", &desc.name], Some(&desc.data_type)) { - Err(AccessError::Undefined { .. }) => (), // fallthrough - other => return other, - }; - - match self.access(&["key", &desc.name], Some(&desc.data_type)) { - Err(AccessError::Undefined { .. }) => (), // fallthrough - other => return other, - }; - - if let Some(key_as_column_name) = &self.key_as_column_name - && &desc.name == key_as_column_name - { - // todo: check logic later - return self.access(&["key"], Some(&desc.data_type)); + match desc.additional_column_type { + AdditionalColumnType::Key => { + match self.access(&["key", &desc.name], Some(&desc.data_type)) { + Err(AccessError::Undefined { .. }) => (), // fallthrough + other => return other, + }; + if let Some(key_as_column_name) = &self.key_as_column_name + && &desc.name == key_as_column_name + { + // todo: check logic later + return self.access(&["key"], Some(&desc.data_type)); + } + } + AdditionalColumnType::Unspecified => { + match self.access(&["value", &desc.name], Some(&desc.data_type)) { + Err(AccessError::Undefined { .. }) => (), + other => return other, + }; + } + _ => unreachable!(), } Ok(None) diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index de5c463419bcc..e9f2d2b712982 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -53,7 +53,7 @@ async fn build_accessor_builder( } } -fn get_key_column_name(columns: &Vec) -> Option { +pub fn get_key_column_name(columns: &Vec) -> Option { columns.iter().find_map(|column| { if column.additional_column_type == AdditionalColumnType::Key { Some(column.name.clone()) From 74c3f868f13ecdb1d94ce55fff364fe019b214e3 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 11 Dec 2023 19:08:36 +0800 Subject: [PATCH 23/50] stash --- src/connector/src/parser/plain_parser.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 8430e326222e5..985eeb848c80e 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -83,9 +83,9 @@ impl PlainParser { let mut row_op: UpsertChangeEvent, AccessImpl<'_, '_>> = UpsertChangeEvent::default(); let change_event_op = ChangeEventOperation::Upsert; - if let Some(data) = key { + if let Some(data) = key && let Some(key_builder) = self.key_builder.as_mut() { // key is optional in format plain - row_op = row_op.with_key(self.key_builder.generate_accessor(data).await?); + row_op = row_op.with_key(key_builder.generate_accessor(data).await?); } if let Some(data) = payload { row_op = row_op.with_value(self.payload_builder.generate_accessor(data).await?); From 1a9653f76e980540caa56be5e7e8548eba3baefa Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 12 Dec 2023 14:15:13 +0800 Subject: [PATCH 24/50] fix --- src/frontend/src/handler/create_table.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index fd5f32b6498cc..22e74781fe247 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -1070,6 +1070,7 @@ pub async fn generate_stream_graph_for_table( source_watermarks, col_id_gen, append_only, + vec![], ) .await? } From f45a0a98b8b92e6292866303a85710e00e77f9d7 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 12 Dec 2023 17:47:47 +0800 Subject: [PATCH 25/50] fix --- src/connector/src/parser/avro/parser.rs | 5 ++- src/connector/src/parser/bytes_parser.rs | 5 ++- src/connector/src/parser/mod.rs | 6 +-- src/connector/src/parser/plain_parser.rs | 5 +-- src/connector/src/parser/unified/upsert.rs | 19 ++++----- src/connector/src/parser/upsert_parser.rs | 4 +- src/frontend/src/handler/create_source.rs | 49 ++-------------------- 7 files changed, 25 insertions(+), 68 deletions(-) diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 05fd2d0e3babf..9429ce157f1e8 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -333,7 +333,10 @@ mod test { let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 1); { let writer = builder.row_writer(); - parser.parse_inner(input_data, writer).await.unwrap(); + parser + .parse_inner(None, Some(input_data), writer) + .await + .unwrap(); } let chunk = builder.finish(); let (op, row) = chunk.rows().next().unwrap(); diff --git a/src/connector/src/parser/bytes_parser.rs b/src/connector/src/parser/bytes_parser.rs index 2a0b2f1b90f2a..b80836af1de82 100644 --- a/src/connector/src/parser/bytes_parser.rs +++ b/src/connector/src/parser/bytes_parser.rs @@ -74,7 +74,10 @@ mod tests { for payload in get_payload() { let writer = builder.row_writer(); - parser.parse_inner(payload, writer).await.unwrap(); + parser + .parse_inner(None, Some(payload), writer) + .await + .unwrap(); } let chunk = builder.finish(); diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index f20fa59cfa2d4..e0ade5a0898cb 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -776,9 +776,9 @@ impl ByteStreamSourceParserImpl { let protocol = &parser_config.specific.protocol_config; let encode = &parser_config.specific.encoding_config; match (protocol, encode) { - (ProtocolProperties::Plain, EncodingProperties::Json(_)) => { - JsonParser::new(parser_config.specific, rw_columns, source_ctx).map(Self::Json) - } + // (ProtocolProperties::Plain, EncodingProperties::Json(_)) => { + // JsonParser::new(parser_config.specific, rw_columns, source_ctx).map(Self::Json) + // } (ProtocolProperties::Plain, EncodingProperties::Csv(config)) => { CsvParser::new(rw_columns, *config, source_ctx).map(Self::Csv) } diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 985eeb848c80e..c414a5b8e59c4 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -15,12 +15,10 @@ use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{ErrorCode, Result, RwError}; -use super::unified::util::apply_row_accessor_on_stream_chunk_writer; use super::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, }; -use crate::only_parse_payload; use crate::parser::bytes_parser::BytesAccessBuilder; use crate::parser::unified::upsert::UpsertChangeEvent; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer_with_op; @@ -54,7 +52,8 @@ impl PlainParser { }; let payload_builder = match props.encoding_config { - EncodingProperties::Protobuf(_) + EncodingProperties::Json(_) + | EncodingProperties::Protobuf(_) | EncodingProperties::Avro(_) | EncodingProperties::Bytes(_) => { AccessBuilderImpl::new_default(props.encoding_config, EncodingType::Value).await? diff --git a/src/connector/src/parser/unified/upsert.rs b/src/connector/src/parser/unified/upsert.rs index 565ca51cf9b05..140b8f0404d97 100644 --- a/src/connector/src/parser/unified/upsert.rs +++ b/src/connector/src/parser/unified/upsert.rs @@ -24,7 +24,7 @@ use crate::source::SourceColumnDesc; pub struct UpsertChangeEvent { key_accessor: Option, value_accessor: Option, - key_as_column_name: Option, + key_column_name: Option, } impl Default for UpsertChangeEvent { @@ -32,7 +32,7 @@ impl Default for UpsertChangeEvent { Self { key_accessor: None, value_accessor: None, - key_as_column_name: None, + key_column_name: None, } } } @@ -54,8 +54,8 @@ impl UpsertChangeEvent { self } - pub fn with_key_as_column_name(mut self, name: impl ToString) -> Self { - self.key_as_column_name = Some(name.to_string()); + pub fn with_key_column_name(mut self, name: impl ToString) -> Self { + self.key_column_name = Some(name.to_string()); self } } @@ -108,15 +108,12 @@ where // access value firstly match desc.additional_column_type { AdditionalColumnType::Key => { - match self.access(&["key", &desc.name], Some(&desc.data_type)) { - Err(AccessError::Undefined { .. }) => (), // fallthrough - other => return other, - }; - if let Some(key_as_column_name) = &self.key_as_column_name + return if let Some(key_as_column_name) = &self.key_column_name && &desc.name == key_as_column_name { - // todo: check logic later - return self.access(&["key"], Some(&desc.data_type)); + self.access(&["key"], Some(&desc.data_type)) + } else { + self.access(&["key", &desc.name], Some(&desc.data_type)) } } AdditionalColumnType::Unspecified => { diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index e9f2d2b712982..10027bc10bbb6 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::DEFAULT_KEY_COLUMN_NAME; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; use risingwave_pb::plan_common::AdditionalColumnType; @@ -25,7 +24,6 @@ use super::{ AccessBuilderImpl, ByteStreamSourceParser, BytesProperties, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, }; -use crate::extract_key_config; use crate::parser::ParserFormat; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; @@ -53,7 +51,7 @@ async fn build_accessor_builder( } } -pub fn get_key_column_name(columns: &Vec) -> Option { +pub fn get_key_column_name(columns: &[SourceColumnDesc]) -> Option { columns.iter().find_map(|column| { if column.additional_column_type == AdditionalColumnType::Key { Some(column.name.clone()) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 60a8de30be134..d1b7b8bd3e69d 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -22,8 +22,8 @@ use itertools::Itertools; use maplit::{convert_args, hashmap}; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::catalog::{ - is_column_ids_dedup, ColumnCatalog, ColumnDesc, TableId, DEFAULT_KEY_COLUMN_NAME, - INITIAL_SOURCE_VERSION_ID, KAFKA_TIMESTAMP_COLUMN_NAME, + is_column_ids_dedup, ColumnCatalog, ColumnDesc, TableId, INITIAL_SOURCE_VERSION_ID, + KAFKA_TIMESTAMP_COLUMN_NAME, }; use risingwave_common::error::ErrorCode::{self, InvalidInputSyntax, ProtocolError}; use risingwave_common::error::{Result, RwError}; @@ -135,37 +135,6 @@ async fn extract_avro_table_schema( .collect_vec()) } -/// Extract Avro primary key columns. -async fn extract_upsert_avro_table_pk_columns( - info: &StreamSourceInfo, - with_properties: &HashMap, -) -> Result>> { - let parser_config = SpecificParserConfig::new(info, with_properties)?; - let conf = AvroParserConfig::new(parser_config.encoding_config).await?; - let vec_column_desc = conf.map_to_columns()?; - - conf.extract_pks() - .ok() - .map(|pk_desc| { - pk_desc - .into_iter() - .map(|desc| { - vec_column_desc - .iter() - .find(|x| x.name == desc.name) - .ok_or_else(|| { - RwError::from(ErrorCode::InternalError(format!( - "Can not found primary key column {} in value schema", - desc.name - ))) - }) - }) - .map_ok(|desc| desc.name.clone()) - .collect::>>() - }) - .transpose() -} - async fn extract_debezium_avro_table_pk_columns( info: &StreamSourceInfo, with_properties: &HashMap, @@ -739,7 +708,7 @@ pub(crate) fn bind_all_columns( pub(crate) async fn bind_source_pk( source_schema: &ConnectorSchema, source_info: &StreamSourceInfo, - columns: &mut Vec, + columns: &mut [ColumnCatalog], sql_defined_pk_names: Vec, with_properties: &HashMap, ) -> Result> { @@ -939,18 +908,6 @@ fn check_and_add_timestamp_column( } } -fn add_default_key_column(columns: &mut Vec) { - let column = ColumnCatalog { - column_desc: ColumnDesc::named( - DEFAULT_KEY_COLUMN_NAME, - (columns.len() as i32).into(), - DataType::Bytea, - ), - is_hidden: false, - }; - columns.push(column); -} - pub(super) fn bind_source_watermark( session: &SessionImpl, name: String, From 7650a9fa1a4da3f6559a516c05bd1f0a062b7d6b Mon Sep 17 00:00:00 2001 From: tabVersion Date: Wed, 13 Dec 2023 14:54:08 +0800 Subject: [PATCH 26/50] fix e2e --- e2e_test/source/basic/kafka.slt | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index b4782d36ced80..7ed49166c8773 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -415,8 +415,10 @@ CREATE TABLE upsert_students_default_key ( "lastName" VARCHAR, age INT, height REAL, - weight REAL + weight REAL, + primary key (rw_key) ) +INCLUDE KEY AS rw_key WITH ( connector = 'kafka', properties.bootstrap.server = 'message_queue:29092', @@ -430,8 +432,10 @@ CREATE TABLE upsert_students ( "lastName" VARCHAR, age INT, height REAL, - weight REAL + weight REAL, + primary key (rw_key) ) +INCLUDE KEY AS rw_key WITH ( connector = 'kafka', properties.bootstrap.server = 'message_queue:29092', From 104a47897327f1ca441d8b5946edd3101a90674d Mon Sep 17 00:00:00 2001 From: tabVersion Date: Wed, 13 Dec 2023 16:53:54 +0800 Subject: [PATCH 27/50] fix e2e --- src/connector/src/parser/json_parser.rs | 33 +++++++++++++++++------ src/frontend/src/handler/create_source.rs | 23 ++++++++++++++++ src/sqlparser/tests/testdata/create.yaml | 6 ++--- src/tests/sqlsmith/src/lib.rs | 7 +++++ 4 files changed, 58 insertions(+), 11 deletions(-) diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 86b2719e34f6b..ce2d009d0f04e 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -206,6 +206,7 @@ mod tests { use risingwave_common::row::Row; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::{DataType, ScalarImpl, ToOwnedDatum}; + use risingwave_pb::plan_common::AdditionalColumnType; use super::JsonParser; use crate::parser::upsert_parser::UpsertParser; @@ -213,6 +214,7 @@ mod tests { EncodingProperties, JsonProperties, ProtocolProperties, SourceColumnDesc, SourceStreamChunkBuilder, SpecificParserConfig, }; + use crate::source::SourceColumnType; fn get_payload() -> Vec> { vec![ @@ -574,9 +576,19 @@ mod tests { (r#"{"a":2}"#, r#""#), ] .to_vec(); + let key_column_desc = SourceColumnDesc { + name: "rw_key".into(), + data_type: DataType::Bytea, + column_id: 2.into(), + fields: vec![], + column_type: SourceColumnType::Normal, + is_pk: true, + additional_column_type: AdditionalColumnType::Key, + }; let descs = vec![ SourceColumnDesc::simple("a", DataType::Int32, 0.into()), SourceColumnDesc::simple("b", DataType::Int32, 1.into()), + key_column_desc, ]; let props = SpecificParserConfig { key_encoding_config: None, @@ -590,7 +602,7 @@ mod tests { .unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs, 4); for item in items { - let test = parser + parser .parse_inner( Some(item.0.as_bytes().to_vec()), if !item.1.is_empty() { @@ -600,13 +612,21 @@ mod tests { }, builder.row_writer(), ) - .await; - println!("{:?}", test); + .await + .unwrap(); } let chunk = builder.finish(); - let mut rows = chunk.rows(); + // expected chunk + // +---+---+---+------------------+ + // | + | 1 | 2 | \x7b2261223a317d | + // | + | 1 | 3 | \x7b2261223a317d | + // | + | 2 | 2 | \x7b2261223a327d | + // | - | | | \x7b2261223a327d | + // +---+---+---+------------------+ + + let mut rows = chunk.rows(); { let (op, row) = rows.next().unwrap(); assert_eq!(op, Op::Insert); @@ -635,10 +655,7 @@ mod tests { { let (op, row) = rows.next().unwrap(); assert_eq!(op, Op::Delete); - assert_eq!( - row.datum_at(0).to_owned_datum(), - (Some(ScalarImpl::Int32(2))) - ); + assert_eq!(row.datum_at(0).to_owned_datum(), (None)); } } } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index d1b7b8bd3e69d..fd1cddf47824a 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -562,7 +562,30 @@ pub fn handle_addition_columns( mut include_columns_options: Vec<(Ident, Option)>, columns: &mut Vec, ) -> Result<()> { + // refer the logic from `parse_source_schema_with_connector` + const EXEMPTED_CONNECTORS: [&str; 2] = ["nexmark", "datagen"]; + let is_connector_exempted_from_additional_column = |connector_name: &str| -> bool { + if connector_name.contains("-cdc") || EXEMPTED_CONNECTORS.contains(&connector_name) { + true + } else { + false + } + }; + let connector_name = get_connector(with_properties).unwrap(); // there must be a connector in source + + let is_exempted = is_connector_exempted_from_additional_column(connector_name.as_str()); + match (is_exempted, include_columns_options.is_empty()) { + (true, true) => return Ok(()), + (true, false) => { + return Err(RwError::from(ProtocolError(format!( + "connector {} does not work with additional columns, but got {:?}", + connector_name, include_columns_options + )))) + } + (false, _) => {} + } + let addition_col_list = CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS .get(connector_name.as_str()) .ok_or_else(|| { diff --git a/src/sqlparser/tests/testdata/create.yaml b/src/sqlparser/tests/testdata/create.yaml index dd189960e213a..57f3f9433dae0 100644 --- a/src/sqlparser/tests/testdata/create.yaml +++ b/src/sqlparser/tests/testdata/create.yaml @@ -25,13 +25,13 @@ formatted_sql: CREATE TABLE sbtest10 (id INT PRIMARY KEY, k INT, c CHARACTER VARYING, pad CHARACTER VARYING) FROM sbtest TABLE 'mydb.sbtest10' - input: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://') formatted_sql: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://') - formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(ConnectorSchema { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "location", quote_style: None }]), value: SingleQuotedString("file://") }] }), source_watermarks: [] } }' + formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(ConnectorSchema { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "location", quote_style: None }]), value: SingleQuotedString("file://") }] }), source_watermarks: [], include_column_options: [] } }' - input: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://') formatted_sql: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://') - formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(ConnectorSchema { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: SingleQuotedString("http://") }] }), source_watermarks: [] } }' + formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(ConnectorSchema { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: SingleQuotedString("http://") }] }), source_watermarks: [], include_column_options: [] } }' - input: CREATE SOURCE bid (auction INTEGER, bidder INTEGER, price INTEGER, WATERMARK FOR auction AS auction - 1, "date_time" TIMESTAMP) with (connector = 'nexmark', nexmark.table.type = 'Bid', nexmark.split.num = '12', nexmark.min.event.gap.in.ns = '0') formatted_sql: CREATE SOURCE bid (auction INT, bidder INT, price INT, "date_time" TIMESTAMP, WATERMARK FOR auction AS auction - 1) WITH (connector = 'nexmark', nexmark.table.type = 'Bid', nexmark.split.num = '12', nexmark.min.event.gap.in.ns = '0') FORMAT NATIVE ENCODE NATIVE - formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: false, columns: [ColumnDef { name: Ident { value: "auction", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "bidder", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "price", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "date_time", quote_style: Some(''"'') }, data_type: Some(Timestamp(false)), collation: None, options: [] }], constraints: [], source_name: ObjectName([Ident { value: "bid", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "connector", quote_style: None }]), value: SingleQuotedString("nexmark") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "table", quote_style: None }, Ident { value: "type", quote_style: None }]), value: SingleQuotedString("Bid") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "split", quote_style: None }, Ident { value: "num", quote_style: None }]), value: SingleQuotedString("12") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "min", quote_style: None }, Ident { value: "event", quote_style: None }, Ident { value: "gap", quote_style: None }, Ident { value: "in", quote_style: None }, Ident { value: "ns", quote_style: None }]), value: SingleQuotedString("0") }]), source_schema: V2(ConnectorSchema { format: Native, row_encode: Native, row_options: [] }), source_watermarks: [SourceWatermark { column: Ident { value: "auction", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "auction", quote_style: None }), op: Minus, right: Value(Number("1")) } }] } }' + formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: false, columns: [ColumnDef { name: Ident { value: "auction", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "bidder", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "price", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "date_time", quote_style: Some(''"'') }, data_type: Some(Timestamp(false)), collation: None, options: [] }], constraints: [], source_name: ObjectName([Ident { value: "bid", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "connector", quote_style: None }]), value: SingleQuotedString("nexmark") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "table", quote_style: None }, Ident { value: "type", quote_style: None }]), value: SingleQuotedString("Bid") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "split", quote_style: None }, Ident { value: "num", quote_style: None }]), value: SingleQuotedString("12") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "min", quote_style: None }, Ident { value: "event", quote_style: None }, Ident { value: "gap", quote_style: None }, Ident { value: "in", quote_style: None }, Ident { value: "ns", quote_style: None }]), value: SingleQuotedString("0") }]), source_schema: V2(ConnectorSchema { format: Native, row_encode: Native, row_options: [] }), source_watermarks: [SourceWatermark { column: Ident { value: "auction", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "auction", quote_style: None }), op: Minus, right: Value(Number("1")) } }], include_column_options: [] } }' - input: CREATE TABLE T (v1 INT, v2 STRUCT) formatted_sql: CREATE TABLE T (v1 INT, v2 STRUCT) - input: CREATE TABLE T (v1 INT, v2 STRUCT>) diff --git a/src/tests/sqlsmith/src/lib.rs b/src/tests/sqlsmith/src/lib.rs index 6b6f164d3cd73..c5a9c9b919aa4 100644 --- a/src/tests/sqlsmith/src/lib.rs +++ b/src/tests/sqlsmith/src/lib.rs @@ -281,6 +281,7 @@ CREATE TABLE t3(v1 int, v2 bool, v3 smallint); append_only: false, query: None, cdc_table_info: None, + include_column_options: [], }, CreateTable { or_replace: false, @@ -325,6 +326,7 @@ CREATE TABLE t3(v1 int, v2 bool, v3 smallint); append_only: false, query: None, cdc_table_info: None, + include_column_options: [], }, CreateTable { or_replace: false, @@ -380,6 +382,7 @@ CREATE TABLE t3(v1 int, v2 bool, v3 smallint); append_only: false, query: None, cdc_table_info: None, + include_column_options: [], }, ], )"#]], @@ -511,6 +514,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY append_only: false, query: None, cdc_table_info: None, + include_column_options: [], }, CreateTable { or_replace: false, @@ -562,6 +566,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY append_only: false, query: None, cdc_table_info: None, + include_column_options: [], }, CreateTable { or_replace: false, @@ -620,6 +625,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY append_only: false, query: None, cdc_table_info: None, + include_column_options: [], }, CreateTable { or_replace: false, @@ -696,6 +702,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY append_only: false, query: None, cdc_table_info: None, + include_column_options: [], }, ], )"#]], From 4926b844e65fe74739f9d5eccb2c400d56b9ad8f Mon Sep 17 00:00:00 2001 From: tabVersion Date: Wed, 13 Dec 2023 16:55:45 +0800 Subject: [PATCH 28/50] format --- src/frontend/src/handler/create_source.rs | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index fd1cddf47824a..fea9bb15760c1 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -565,11 +565,7 @@ pub fn handle_addition_columns( // refer the logic from `parse_source_schema_with_connector` const EXEMPTED_CONNECTORS: [&str; 2] = ["nexmark", "datagen"]; let is_connector_exempted_from_additional_column = |connector_name: &str| -> bool { - if connector_name.contains("-cdc") || EXEMPTED_CONNECTORS.contains(&connector_name) { - true - } else { - false - } + connector_name.contains("-cdc") || EXEMPTED_CONNECTORS.contains(&connector_name) }; let connector_name = get_connector(with_properties).unwrap(); // there must be a connector in source From 00b1ae85ff63b52f94facb6afd0ee902a2b9e4b3 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Wed, 13 Dec 2023 17:16:27 +0800 Subject: [PATCH 29/50] fix test --- e2e_test/sink/kafka/avro.slt | 4 +++- e2e_test/source/basic/kafka.slt | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/e2e_test/sink/kafka/avro.slt b/e2e_test/sink/kafka/avro.slt index a9040c93f793f..45ecf302f0ddd 100644 --- a/e2e_test/sink/kafka/avro.slt +++ b/e2e_test/sink/kafka/avro.slt @@ -1,5 +1,7 @@ statement ok -create table from_kafka with ( +create table from_kafka ( primary key (some_key) ) +include key as some_key +with ( connector = 'kafka', topic = 'test-rw-sink-upsert-avro', properties.bootstrap.server = 'message_queue:29092') diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index 7ed49166c8773..ab7616084db5f 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -427,7 +427,7 @@ FORMAT UPSERT ENCODE JSON statement ok CREATE TABLE upsert_students ( - "ID" INT PRIMARY KEY, + "ID" INT, "firstName" VARCHAR, "lastName" VARCHAR, age INT, From 33c13547d1597771504c106a4d4470cd065e7d81 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Thu, 14 Dec 2023 12:56:54 +0800 Subject: [PATCH 30/50] fix test --- e2e_test/source/basic/nosim_kafka.slt | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/e2e_test/source/basic/nosim_kafka.slt b/e2e_test/source/basic/nosim_kafka.slt index 333d9b5909ee3..e09d815c49155 100644 --- a/e2e_test/source/basic/nosim_kafka.slt +++ b/e2e_test/source/basic/nosim_kafka.slt @@ -3,7 +3,8 @@ # If we cannot extract key schema, use message key as varchar primary key statement ok -CREATE TABLE upsert_avro_json_default_key () +CREATE TABLE upsert_avro_json_default_key ( primary key (rw_key) ) +INCLUDE KEY AS rw_key WITH ( connector = 'kafka', properties.bootstrap.server = 'message_queue:29092', @@ -13,7 +14,8 @@ FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); # key schema should be a subset of value schema statement error -CREATE TABLE upsert_student_key_not_subset_of_value_avro_json () +CREATE TABLE upsert_student_key_not_subset_of_value_avro_json ( primary key (rw_key) ) +INCLUDE KEY AS rw_key WITH ( connector = 'kafka', properties.bootstrap.server = 'message_queue:29092', @@ -22,7 +24,8 @@ FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); statement ok -CREATE TABLE upsert_student_avro_json () +CREATE TABLE upsert_student_avro_json ( primary key (rw_key) ) +INCLUDE KEY AS rw_key WITH ( connector = 'kafka', properties.bootstrap.server = 'message_queue:29092', @@ -68,7 +71,8 @@ CREATE TABLE kafka_json_schema_plain with ( ) FORMAT PLAIN ENCODE JSON (schema.registry = 'http://schemaregistry:8082'); statement ok -CREATE TABLE kafka_json_schema_upsert (PRIMARY KEY(id)) +CREATE TABLE kafka_json_schema_upsert (PRIMARY KEY(rw_key)) +INCLUDE KEY AS rw_key with ( connector = 'kafka', kafka.topic = 'kafka_upsert_json_schema', From c71f49be70b2a4054fc2aa0e14967c20daa1e624 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Thu, 14 Dec 2023 22:31:26 +0800 Subject: [PATCH 31/50] pre delete json parser --- src/connector/src/parser/json_parser.rs | 2 ++ src/connector/src/parser/mod.rs | 2 +- src/source/benches/json_parser.rs | 23 +++++++++++++++++++---- 3 files changed, 22 insertions(+), 5 deletions(-) diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index ce2d009d0f04e..57de75ea83c22 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -36,6 +36,7 @@ use crate::parser::{ use crate::schema::schema_registry::{handle_sr_list, Client}; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; +#[deprecated(note = "Please use plain parser instead.")] #[derive(Debug)] pub struct JsonAccessBuilder { value: Option>, @@ -69,6 +70,7 @@ impl JsonAccessBuilder { } /// Parser for JSON format +#[deprecated(note = "Please use plain parser instead.")] #[derive(Debug)] pub struct JsonParser { rw_columns: Vec, diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index e0ade5a0898cb..e4c2291e61074 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -64,7 +64,7 @@ mod debezium; mod json_parser; mod maxwell; mod mysql; -mod plain_parser; +pub mod plain_parser; mod protobuf; mod unified; mod upsert_parser; diff --git a/src/source/benches/json_parser.rs b/src/source/benches/json_parser.rs index e54a51befa9f1..5b6ea94c5a7bb 100644 --- a/src/source/benches/json_parser.rs +++ b/src/source/benches/json_parser.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + use criterion::{criterion_group, criterion_main, BatchSize, Criterion}; use futures::executor::block_on; use paste::paste; @@ -19,8 +21,11 @@ use rand::distributions::Alphanumeric; use rand::prelude::*; use risingwave_common::catalog::ColumnId; use risingwave_common::types::{DataType, Date, Timestamp}; -use risingwave_connector::parser::{DebeziumParser, JsonParser, SourceStreamChunkBuilder}; -use risingwave_connector::source::SourceColumnDesc; +use risingwave_connector::parser::plain_parser::PlainParser; +use risingwave_connector::parser::{ + DebeziumParser, SourceStreamChunkBuilder, SpecificParserConfig, +}; +use risingwave_connector::source::{SourceColumnDesc, SourceContext}; macro_rules! create_debezium_bench_helpers { ($op:ident, $op_sym:expr, $bench_function:expr) => { @@ -122,12 +127,19 @@ fn get_descs() -> Vec { fn bench_json_parser(c: &mut Criterion) { let descs = get_descs(); - let parser = JsonParser::new_for_test(descs.clone()).unwrap(); let rt = tokio::runtime::Builder::new_multi_thread() .enable_all() .build() .unwrap(); let records = generate_json_rows(); + let ctx = Arc::new(SourceContext::default()); + let mut parser = rt + .block_on(PlainParser::new( + SpecificParserConfig::DEFAULT_PLAIN_JSON, + descs.clone(), + ctx.clone(), + )) + .unwrap(); c.bench_function("json_parser", |b| { b.to_async(&rt).iter_batched( || records.clone(), @@ -136,7 +148,10 @@ fn bench_json_parser(c: &mut Criterion) { SourceStreamChunkBuilder::with_capacity(descs.clone(), NUM_RECORDS); for record in records { let writer = builder.row_writer(); - parser.parse_inner(record, writer).await.unwrap(); + parser + .parse_inner(None, Some(record), writer) + .await + .unwrap(); } }, BatchSize::SmallInput, From 30f95165160479b39638fd226bb58086741efc38 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Fri, 15 Dec 2023 13:23:56 +0800 Subject: [PATCH 32/50] remove unwrap array for kafka --- .../source/test_data/kafka_1_partition_mv_topic.1 | 4 +++- scripts/source/test_data/kafka_1_partition_topic.1 | 3 ++- scripts/source/test_data/kafka_3_partition_topic.3 | 3 ++- scripts/source/test_data/kafka_4_partition_topic.4 | 3 ++- .../kafka_4_partition_topic_with_100_message.4 | 5 ++++- src/connector/src/parser/json_parser.rs | 2 -- src/source/benches/json_parser.rs | 14 +++++++------- 7 files changed, 20 insertions(+), 14 deletions(-) diff --git a/scripts/source/test_data/kafka_1_partition_mv_topic.1 b/scripts/source/test_data/kafka_1_partition_mv_topic.1 index 08bd5e7820f74..a11f9eb3c1c5a 100644 --- a/scripts/source/test_data/kafka_1_partition_mv_topic.1 +++ b/scripts/source/test_data/kafka_1_partition_mv_topic.1 @@ -15,4 +15,6 @@ {"v1":7,"v2":"name0"} {"v1":0,"v2":"name9"} {"v1":3,"v2":"name2"} -[{"v1":7,"v2":"name5"},{"v1":1,"v2":"name7"},{"v1":3,"v2":"name9"}] +{"v1":7,"v2":"name5"} +{"v1":1,"v2":"name7"} +{"v1":3,"v2":"name9"} diff --git a/scripts/source/test_data/kafka_1_partition_topic.1 b/scripts/source/test_data/kafka_1_partition_topic.1 index 0334f4d057aa7..3fb1b528d5fa2 100644 --- a/scripts/source/test_data/kafka_1_partition_topic.1 +++ b/scripts/source/test_data/kafka_1_partition_topic.1 @@ -1,3 +1,4 @@ {"v1": 1, "v2": "1"} {"v1": 2, "v2": "22"} -[{"v1": 3, "v2": "333"},{"v1": 4, "v2": "4444"}] \ No newline at end of file +{"v1": 3, "v2": "333"} +{"v1": 4, "v2": "4444"} \ No newline at end of file diff --git a/scripts/source/test_data/kafka_3_partition_topic.3 b/scripts/source/test_data/kafka_3_partition_topic.3 index 0334f4d057aa7..3fb1b528d5fa2 100644 --- a/scripts/source/test_data/kafka_3_partition_topic.3 +++ b/scripts/source/test_data/kafka_3_partition_topic.3 @@ -1,3 +1,4 @@ {"v1": 1, "v2": "1"} {"v1": 2, "v2": "22"} -[{"v1": 3, "v2": "333"},{"v1": 4, "v2": "4444"}] \ No newline at end of file +{"v1": 3, "v2": "333"} +{"v1": 4, "v2": "4444"} \ No newline at end of file diff --git a/scripts/source/test_data/kafka_4_partition_topic.4 b/scripts/source/test_data/kafka_4_partition_topic.4 index 0334f4d057aa7..3fb1b528d5fa2 100644 --- a/scripts/source/test_data/kafka_4_partition_topic.4 +++ b/scripts/source/test_data/kafka_4_partition_topic.4 @@ -1,3 +1,4 @@ {"v1": 1, "v2": "1"} {"v1": 2, "v2": "22"} -[{"v1": 3, "v2": "333"},{"v1": 4, "v2": "4444"}] \ No newline at end of file +{"v1": 3, "v2": "333"} +{"v1": 4, "v2": "4444"} \ No newline at end of file diff --git a/scripts/source/test_data/kafka_4_partition_topic_with_100_message.4 b/scripts/source/test_data/kafka_4_partition_topic_with_100_message.4 index cc1ad14663da2..aae7a4687ca55 100644 --- a/scripts/source/test_data/kafka_4_partition_topic_with_100_message.4 +++ b/scripts/source/test_data/kafka_4_partition_topic_with_100_message.4 @@ -94,4 +94,7 @@ {"v1": 93, "v2": "QE53BJ", "v3": [93, 93, 93], "v4": {"v5": 93, "v6": 94}} {"v1": 94, "v2": "9Q7W89", "v3": [94, 94, 94], "v4": {"v5": 94, "v6": 95}} {"v1": 95, "v2": "VGDBS1", "v3": [95, 95, 95], "v4": {"v5": 95, "v6": 96}} -[{"v1": 96, "v2": "KK6WEX", "v3": [96, 96, 96], "v4": {"v5": 96, "v6": 97}},{"v1": 97, "v2": "XRTK3Y", "v3": [97, 97, 97], "v4": {"v5": 97, "v6": 98}},{"v1": 98, "v2": "ZQ2TCL", "v3": [98, 98, 98], "v4": {"v5": 98, "v6": 99}},{"v1": 99, "v2": "15UCX7", "v3": [99, 99, 99], "v4": {"v5": 99, "v6": 100}}] \ No newline at end of file +{"v1": 96, "v2": "KK6WEX", "v3": [96, 96, 96], "v4": {"v5": 96, "v6": 97}} +{"v1": 97, "v2": "XRTK3Y", "v3": [97, 97, 97], "v4": {"v5": 97, "v6": 98}} +{"v1": 98, "v2": "ZQ2TCL", "v3": [98, 98, 98], "v4": {"v5": 98, "v6": 99}} +{"v1": 99, "v2": "15UCX7", "v3": [99, 99, 99], "v4": {"v5": 99, "v6": 100}} \ No newline at end of file diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 57de75ea83c22..ce2d009d0f04e 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -36,7 +36,6 @@ use crate::parser::{ use crate::schema::schema_registry::{handle_sr_list, Client}; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; -#[deprecated(note = "Please use plain parser instead.")] #[derive(Debug)] pub struct JsonAccessBuilder { value: Option>, @@ -70,7 +69,6 @@ impl JsonAccessBuilder { } /// Parser for JSON format -#[deprecated(note = "Please use plain parser instead.")] #[derive(Debug)] pub struct JsonParser { rw_columns: Vec, diff --git a/src/source/benches/json_parser.rs b/src/source/benches/json_parser.rs index 5b6ea94c5a7bb..961dd592cade1 100644 --- a/src/source/benches/json_parser.rs +++ b/src/source/benches/json_parser.rs @@ -133,17 +133,17 @@ fn bench_json_parser(c: &mut Criterion) { .unwrap(); let records = generate_json_rows(); let ctx = Arc::new(SourceContext::default()); - let mut parser = rt - .block_on(PlainParser::new( - SpecificParserConfig::DEFAULT_PLAIN_JSON, - descs.clone(), - ctx.clone(), - )) - .unwrap(); c.bench_function("json_parser", |b| { b.to_async(&rt).iter_batched( || records.clone(), |records| async { + let mut parser = rt + .block_on(PlainParser::new( + SpecificParserConfig::DEFAULT_PLAIN_JSON, + descs.clone(), + ctx.clone(), + )) + .unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(descs.clone(), NUM_RECORDS); for record in records { From 4c05e90a0a3566c98efff3094912d351400b305c Mon Sep 17 00:00:00 2001 From: tabVersion Date: Fri, 15 Dec 2023 14:32:54 +0800 Subject: [PATCH 33/50] feat: Refactor handling of exempted connectors and addition columns - Remove `upsert_student_key_not_subset_of_value_avro_json` table in `e2e_test/source/basic/nosim_kafka.slt` - Add `upsert_student_avro_json` table in `e2e_test/source/basic/nosim_kafka.slt` - Add new exempted connector "pubsub" in `src/frontend/src/handler/create_source.rs` - Modify logic for handling addition columns in `handle_addition_columns` function in `src/frontend/src/handler/create_source.rs` - Add check for exempted connector in `handle_addition_columns` function in `src/frontend/src/handler/create_source.rs` - Implement logic to generate default column names based on connector and additional column name in `src/frontend/src/handler/create_source.rs` - Insert additional columns in specific order based on `include_columns_options` in `src/frontend/src/handler/create_source.rs` - Remove unnecessary empty line in `src/frontend/src/handler/create_source.rs` - Add `primary key(rw_key)` and `INCLUDE KEY AS rw_key` to the `create table` statement for table `t1` in `e2e_test/source/basic/schema_registry.slt` - Remove `connector`, `topic`, and `properties.bootstrap.server` properties from the `create table` statement for table `t1` in `e2e_test/source/basic/schema_registry.slt` - Update the `format upsert encode avro` parameters for table `t1` in `e2e_test/source/basic/schema_registry.slt`, including changing `schema.registry` and `message` - Remove `key.message` parameter from the `format upsert encode avro` parameters for table `t1` in `e2e_test/source/basic/schema_registry.slt` Signed-off-by: tabVersion --- e2e_test/source/basic/nosim_kafka.slt | 12 ------------ e2e_test/source/basic/schema_registry.slt | 4 +++- src/frontend/src/handler/create_source.rs | 2 +- 3 files changed, 4 insertions(+), 14 deletions(-) diff --git a/e2e_test/source/basic/nosim_kafka.slt b/e2e_test/source/basic/nosim_kafka.slt index e09d815c49155..3e251009df1dd 100644 --- a/e2e_test/source/basic/nosim_kafka.slt +++ b/e2e_test/source/basic/nosim_kafka.slt @@ -11,18 +11,6 @@ WITH ( topic = 'upsert_avro_json') FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); - -# key schema should be a subset of value schema -statement error -CREATE TABLE upsert_student_key_not_subset_of_value_avro_json ( primary key (rw_key) ) -INCLUDE KEY AS rw_key -WITH ( - connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', - topic = 'upsert_student_key_not_subset_of_value_avro_json') -FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); - - statement ok CREATE TABLE upsert_student_avro_json ( primary key (rw_key) ) INCLUDE KEY AS rw_key diff --git a/e2e_test/source/basic/schema_registry.slt b/e2e_test/source/basic/schema_registry.slt index 650d493ef5003..76f867b2b1d0e 100644 --- a/e2e_test/source/basic/schema_registry.slt +++ b/e2e_test/source/basic/schema_registry.slt @@ -47,7 +47,9 @@ create table t1 () with ( ); statement ok -create table t1 () with ( +create table t1 (primary key(rw_key)) +INCLUDE KEY AS rw_key +with ( connector = 'kafka', topic = 'upsert_avro_json-topic-record', properties.bootstrap.server = 'message_queue:29092' diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index fea9bb15760c1..5d8219b33a6ea 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -563,7 +563,7 @@ pub fn handle_addition_columns( columns: &mut Vec, ) -> Result<()> { // refer the logic from `parse_source_schema_with_connector` - const EXEMPTED_CONNECTORS: [&str; 2] = ["nexmark", "datagen"]; + const EXEMPTED_CONNECTORS: [&str; 3] = ["nexmark", "datagen", "pubsub"]; let is_connector_exempted_from_additional_column = |connector_name: &str| -> bool { connector_name.contains("-cdc") || EXEMPTED_CONNECTORS.contains(&connector_name) }; From 8c2ca18b8372988ccaf742d2eb3662c548d9090f Mon Sep 17 00:00:00 2001 From: tabVersion Date: Fri, 15 Dec 2023 15:07:41 +0800 Subject: [PATCH 34/50] fix --- src/frontend/src/handler/create_source.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 5d8219b33a6ea..24bc7ad70704b 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -563,7 +563,7 @@ pub fn handle_addition_columns( columns: &mut Vec, ) -> Result<()> { // refer the logic from `parse_source_schema_with_connector` - const EXEMPTED_CONNECTORS: [&str; 3] = ["nexmark", "datagen", "pubsub"]; + const EXEMPTED_CONNECTORS: [&str; 3] = ["nexmark", "datagen", "google_pubsub"]; let is_connector_exempted_from_additional_column = |connector_name: &str| -> bool { connector_name.contains("-cdc") || EXEMPTED_CONNECTORS.contains(&connector_name) }; From 0377f84a7f1c6b77f2f629c985c457e7dfedeed0 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Fri, 15 Dec 2023 15:51:40 +0800 Subject: [PATCH 35/50] fix --- e2e_test/source/basic/inlcude_key_as.slt | 59 ++++++++++++++++++++++++ src/storage/src/row_serde/mod.rs | 56 +++++++++++----------- 2 files changed, 89 insertions(+), 26 deletions(-) create mode 100644 e2e_test/source/basic/inlcude_key_as.slt diff --git a/e2e_test/source/basic/inlcude_key_as.slt b/e2e_test/source/basic/inlcude_key_as.slt new file mode 100644 index 0000000000000..ac68f28896f8c --- /dev/null +++ b/e2e_test/source/basic/inlcude_key_as.slt @@ -0,0 +1,59 @@ +# upsert format must have a pk +statement error +CREATE TABLE upsert_students_default_key ( + "ID" INT, + "firstName" VARCHAR, + "lastName" VARCHAR, + age INT, + height REAL, + weight REAL +) +INCLUDE KEY AS rw_key +WITH ( + connector = 'kafka', + properties.bootstrap.server = 'message_queue:29092', + topic = 'upsert_json') +FORMAT UPSERT ENCODE JSON + +# upsert format pk must be the key column +statement error +CREATE TABLE upsert_students_default_key ( + "ID" INT primary key, + "firstName" VARCHAR, + "lastName" VARCHAR, + age INT, + height REAL, + weight REAL +) +INCLUDE KEY AS rw_key +WITH ( + connector = 'kafka', + properties.bootstrap.server = 'message_queue:29092', + topic = 'upsert_json') +FORMAT UPSERT ENCODE JSON + +statement ok +CREATE TABLE upsert_students_default_key ( + "ID" INT, + "firstName" VARCHAR, + "lastName" VARCHAR, + age INT, + height REAL, + weight REAL, +) +INCLUDE KEY AS rw_key +WITH ( + connector = 'kafka', + properties.bootstrap.server = 'message_queue:29092', + topic = 'upsert_json') +FORMAT PLAIN ENCODE JSON + +select * from upsert_students_default_key; + +# Wait enough time to ensure SourceExecutor consumes all Kafka data. +sleep 1s + +query I +select count(rw_key) from upsert_students_default_key +---- +15 diff --git a/src/storage/src/row_serde/mod.rs b/src/storage/src/row_serde/mod.rs index 5fc99b8b6945a..06c980cccf6d1 100644 --- a/src/storage/src/row_serde/mod.rs +++ b/src/storage/src/row_serde/mod.rs @@ -84,32 +84,34 @@ mod test { check( result, expect![[r#" - ( - [ - ColumnDesc { - data_type: Int64, - column_id: #2, - name: "", - field_descs: [], - type_name: "", - generated_or_default_column: None, - description: None, - }, - ColumnDesc { - data_type: Int16, - column_id: #3, - name: "", - field_descs: [], - type_name: "", - generated_or_default_column: None, - description: None, - }, - ], - [ - 1, - 2, - ], - )"#]], + ( + [ + ColumnDesc { + data_type: Int64, + column_id: #2, + name: "", + field_descs: [], + type_name: "", + generated_or_default_column: None, + description: None, + additional_column_type: Unspecified, + }, + ColumnDesc { + data_type: Int16, + column_id: #3, + name: "", + field_descs: [], + type_name: "", + generated_or_default_column: None, + description: None, + additional_column_type: Unspecified, + }, + ], + [ + 1, + 2, + ], + )"#]], ); let table_columns = vec![ @@ -132,6 +134,7 @@ mod test { type_name: "", generated_or_default_column: None, description: None, + additional_column_type: Unspecified, }, ColumnDesc { data_type: Varchar, @@ -141,6 +144,7 @@ mod test { type_name: "", generated_or_default_column: None, description: None, + additional_column_type: Unspecified, }, ], [ From 131faf5e3391e02d19c6f4c09826fce51b9399ce Mon Sep 17 00:00:00 2001 From: tabVersion Date: Fri, 15 Dec 2023 16:18:21 +0800 Subject: [PATCH 36/50] fix --- e2e_test/source/basic/inlcude_key_as.slt | 1 + 1 file changed, 1 insertion(+) diff --git a/e2e_test/source/basic/inlcude_key_as.slt b/e2e_test/source/basic/inlcude_key_as.slt index ac68f28896f8c..fbeac3afe0e73 100644 --- a/e2e_test/source/basic/inlcude_key_as.slt +++ b/e2e_test/source/basic/inlcude_key_as.slt @@ -48,6 +48,7 @@ WITH ( topic = 'upsert_json') FORMAT PLAIN ENCODE JSON +statement ok select * from upsert_students_default_key; # Wait enough time to ensure SourceExecutor consumes all Kafka data. From 285e3a23bf12f628f83935c1a353da66d19c6330 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Sat, 16 Dec 2023 12:43:16 +0800 Subject: [PATCH 37/50] fix --- src/frontend/src/handler/create_source.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 24bc7ad70704b..8475f63179b06 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -563,7 +563,7 @@ pub fn handle_addition_columns( columns: &mut Vec, ) -> Result<()> { // refer the logic from `parse_source_schema_with_connector` - const EXEMPTED_CONNECTORS: [&str; 3] = ["nexmark", "datagen", "google_pubsub"]; + const EXEMPTED_CONNECTORS: [&str; 4] = ["nexmark", "datagen", "google_pubsub", "test"]; let is_connector_exempted_from_additional_column = |connector_name: &str| -> bool { connector_name.contains("-cdc") || EXEMPTED_CONNECTORS.contains(&connector_name) }; From 6c797b740de6409c4fb6379fee42c84b1440f92e Mon Sep 17 00:00:00 2001 From: tabVersion Date: Sat, 16 Dec 2023 17:17:45 +0800 Subject: [PATCH 38/50] fix --- e2e_test/source/basic/inlcude_key_as.slt | 13 ++++++++----- src/connector/src/parser/json_parser.rs | 6 +++++- src/connector/src/parser/plain_parser.rs | 7 ++----- src/connector/src/parser/unified/upsert.rs | 10 ++-------- 4 files changed, 17 insertions(+), 19 deletions(-) diff --git a/e2e_test/source/basic/inlcude_key_as.slt b/e2e_test/source/basic/inlcude_key_as.slt index fbeac3afe0e73..83a7c2ce53cde 100644 --- a/e2e_test/source/basic/inlcude_key_as.slt +++ b/e2e_test/source/basic/inlcude_key_as.slt @@ -11,7 +11,7 @@ CREATE TABLE upsert_students_default_key ( INCLUDE KEY AS rw_key WITH ( connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'upsert_json') FORMAT UPSERT ENCODE JSON @@ -28,7 +28,7 @@ CREATE TABLE upsert_students_default_key ( INCLUDE KEY AS rw_key WITH ( connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'upsert_json') FORMAT UPSERT ENCODE JSON @@ -44,17 +44,20 @@ CREATE TABLE upsert_students_default_key ( INCLUDE KEY AS rw_key WITH ( connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', - topic = 'upsert_json') + properties.bootstrap.server = '127.0.0.1:29092', + topic = 'test') FORMAT PLAIN ENCODE JSON statement ok select * from upsert_students_default_key; # Wait enough time to ensure SourceExecutor consumes all Kafka data. -sleep 1s +sleep 3s query I select count(rw_key) from upsert_students_default_key ---- 15 + +statement ok +drop table upsert_students_default_key diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index ce2d009d0f04e..a3997f2296617 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -45,7 +45,11 @@ pub struct JsonAccessBuilder { impl AccessBuilder for JsonAccessBuilder { #[allow(clippy::unused_async)] async fn generate_accessor(&mut self, payload: Vec) -> Result> { - self.value = Some(payload); + if payload.is_empty() { + self.value = Some("{}".into()); + } else { + self.value = Some(payload); + } let value = simd_json::to_borrowed_value( &mut self.value.as_mut().unwrap()[self.payload_start_idx..], ) diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index c414a5b8e59c4..90f1ba212bfd4 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -82,21 +82,18 @@ impl PlainParser { let mut row_op: UpsertChangeEvent, AccessImpl<'_, '_>> = UpsertChangeEvent::default(); let change_event_op = ChangeEventOperation::Upsert; + if let Some(data) = key && let Some(key_builder) = self.key_builder.as_mut() { // key is optional in format plain row_op = row_op.with_key(key_builder.generate_accessor(data).await?); } if let Some(data) = payload { + // the data part also can be an empty vec row_op = row_op.with_value(self.payload_builder.generate_accessor(data).await?); - } else { - return Err(RwError::from(ErrorCode::InternalError( - "payload is required in format plain, but got empty".to_string(), - ))); } apply_row_operation_on_stream_chunk_writer_with_op(row_op, &mut writer, change_event_op) .map_err(Into::into) - // apply_row_accessor_on_stream_chunk_writer(accessor, &mut writer).map_err(Into::into) } } diff --git a/src/connector/src/parser/unified/upsert.rs b/src/connector/src/parser/unified/upsert.rs index 140b8f0404d97..d00e0574b8b48 100644 --- a/src/connector/src/parser/unified/upsert.rs +++ b/src/connector/src/parser/unified/upsert.rs @@ -105,10 +105,9 @@ where } fn access_field(&self, desc: &SourceColumnDesc) -> super::AccessResult { - // access value firstly match desc.additional_column_type { AdditionalColumnType::Key => { - return if let Some(key_as_column_name) = &self.key_column_name + if let Some(key_as_column_name) = &self.key_column_name && &desc.name == key_as_column_name { self.access(&["key"], Some(&desc.data_type)) @@ -117,14 +116,9 @@ where } } AdditionalColumnType::Unspecified => { - match self.access(&["value", &desc.name], Some(&desc.data_type)) { - Err(AccessError::Undefined { .. }) => (), - other => return other, - }; + self.access(&["value", &desc.name], Some(&desc.data_type)) } _ => unreachable!(), } - - Ok(None) } } From 4f8600c223cc2c7fca7a0eb7d053281319140f50 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Sat, 16 Dec 2023 17:31:54 +0800 Subject: [PATCH 39/50] format --- src/connector/src/parser/plain_parser.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 90f1ba212bfd4..c02a373a082eb 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -13,7 +13,7 @@ // limitations under the License. use risingwave_common::error::ErrorCode::ProtocolError; -use risingwave_common::error::{ErrorCode, Result, RwError}; +use risingwave_common::error::{Result, RwError}; use super::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, From ec06a13c91f226ced1b5e7903d4ff68e5d8c4878 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Sat, 16 Dec 2023 20:04:19 +0800 Subject: [PATCH 40/50] fix broker addr --- e2e_test/source/basic/inlcude_key_as.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e_test/source/basic/inlcude_key_as.slt b/e2e_test/source/basic/inlcude_key_as.slt index 83a7c2ce53cde..d7780d4916376 100644 --- a/e2e_test/source/basic/inlcude_key_as.slt +++ b/e2e_test/source/basic/inlcude_key_as.slt @@ -44,8 +44,8 @@ CREATE TABLE upsert_students_default_key ( INCLUDE KEY AS rw_key WITH ( connector = 'kafka', - properties.bootstrap.server = '127.0.0.1:29092', - topic = 'test') + properties.bootstrap.server = 'message_queue:29092', + topic = 'upsert_json') FORMAT PLAIN ENCODE JSON statement ok From 3c9b563fd4c745c500d7dbeff1a102931af6c062 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 18 Dec 2023 14:46:42 +0800 Subject: [PATCH 41/50] refactor: Refactor primary key definitions in test database - Revised table structure to improve performance and optimize key usage - Added primary key constraint and included key in two tables for efficient data manipulation - Simplified primary key definition in one table for better readability and maintainability Signed-off-by: tabVersion --- .../source/basic/old_row_format_syntax/kafka.slt | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/e2e_test/source/basic/old_row_format_syntax/kafka.slt b/e2e_test/source/basic/old_row_format_syntax/kafka.slt index 77b8b759b8624..0acfade58e713 100644 --- a/e2e_test/source/basic/old_row_format_syntax/kafka.slt +++ b/e2e_test/source/basic/old_row_format_syntax/kafka.slt @@ -375,26 +375,30 @@ CREATE TABLE upsert_students_default_key ( "lastName" VARCHAR, age INT, height REAL, - weight REAL + weight REAL, + primary key (rw_key) ) +INCLUDE KEY AS rw_key WITH ( connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'upsert_json') ROW FORMAT UPSERT_JSON statement ok CREATE TABLE upsert_students ( - "ID" INT PRIMARY KEY, + "ID" INT, "firstName" VARCHAR, "lastName" VARCHAR, age INT, height REAL, - weight REAL + weight REAL, + primary key (rw_key) ) +INCLUDE KEY AS rw_key WITH ( connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', + properties.bootstrap.server = 'message_queue:29092', topic = 'upsert_json') ROW FORMAT UPSERT_JSON From 4c70a6be495446e7d72e7e002f43b7597047184b Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 18 Dec 2023 14:55:52 +0800 Subject: [PATCH 42/50] fix --- src/frontend/src/handler/create_table.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index a8acb7c64b871..aaa5d8f3bdaa0 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -490,7 +490,7 @@ pub(crate) async fn gen_create_table_plan_with_source( )?; // add additional columns before bind pk, because `format upsert` requires the key column - handle_addition_columns(&properties, include_column_options, &mut columns)?; + handle_addition_columns(&with_properties, include_column_options, &mut columns)?; let pk_names = bind_source_pk( &source_schema, &source_info, From 9904da0677dcaeda30ccb99630b9a54a47ab7065 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Mon, 18 Dec 2023 15:07:38 +0800 Subject: [PATCH 43/50] remove legacy avro behavior --- .../basic/old_row_format_syntax/kafka.slt | 41 ------ .../old_row_format_syntax/nosim_kafka.slt | 138 ------------------ src/connector/src/parser/mod.rs | 3 - 3 files changed, 182 deletions(-) delete mode 100644 e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt diff --git a/e2e_test/source/basic/old_row_format_syntax/kafka.slt b/e2e_test/source/basic/old_row_format_syntax/kafka.slt index 0acfade58e713..8e8c42a6e41f3 100644 --- a/e2e_test/source/basic/old_row_format_syntax/kafka.slt +++ b/e2e_test/source/basic/old_row_format_syntax/kafka.slt @@ -368,23 +368,6 @@ WITH ( topic = 'debezium_mongo_json_customers_no_schema_field') ROW FORMAT DEBEZIUM_MONGO_JSON -statement ok -CREATE TABLE upsert_students_default_key ( - "ID" INT, - "firstName" VARCHAR, - "lastName" VARCHAR, - age INT, - height REAL, - weight REAL, - primary key (rw_key) -) -INCLUDE KEY AS rw_key -WITH ( - connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', - topic = 'upsert_json') -ROW FORMAT UPSERT_JSON - statement ok CREATE TABLE upsert_students ( "ID" INT, @@ -686,27 +669,6 @@ ORDER BY 6 Leah Davis 18 5.7 140 9 Jacob Anderson 20 5.8 155 -query II -SELECT - "ID", - "firstName", - "lastName", - "age", - "height", - "weight" -FROM - upsert_students_default_key -ORDER BY - "ID"; ----- -1 Ethan Martinez 18 6.1 180 -2 Emily Jackson 19 5.4 110 -3 Noah Thompson 21 6.3 195 -4 Emma Brown 20 5.3 130 -5 Michael Williams 22 6.2 190 -6 Leah Davis 18 5.7 140 -9 Jacob Anderson 20 5.8 155 - query II select L_ORDERKEY, @@ -795,8 +757,5 @@ DROP TABLE mongo_customers_no_schema_field; statement ok DROP TABLE upsert_students; -statement ok -DROP TABLE upsert_students_default_key; - statement ok drop table dbz_ignore_case_json; diff --git a/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt b/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt deleted file mode 100644 index eaeb70ae5bad9..0000000000000 --- a/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt +++ /dev/null @@ -1,138 +0,0 @@ -# Start with nosim to avoid running in deterministic test - - -# If we cannot extract key schema, use message key as varchar primary key -statement ok -CREATE TABLE upsert_avro_json_default_key () -WITH ( - connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', - topic = 'upsert_avro_json') -ROW FORMAT UPSERT_AVRO -row schema location confluent schema registry 'http://message_queue:8081' - - -# key schema should be a subset of value schema -statement error -CREATE TABLE upsert_student_key_not_subset_of_value_avro_json () -WITH ( - connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', - topic = 'upsert_student_key_not_subset_of_value_avro_json') -ROW FORMAT UPSERT_AVRO -row schema location confluent schema registry 'http://message_queue:8081' - - -statement ok -CREATE TABLE upsert_student_avro_json () -WITH ( - connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', - topic = 'upsert_student_avro_json') -ROW FORMAT UPSERT_AVRO -row schema location confluent schema registry 'http://message_queue:8081' - - -# TODO: Uncomment this when we add test data kafka key with format `"ID":id` -# statement ok -# CREATE TABLE upsert_avro_json ( -# PRIMARY KEY("ID") -# ) -# WITH ( -# connector = 'kafka', -# properties.bootstrap.server = 'message_queue:29092', -# topic = 'upsert_avro_json') -# ROW FORMAT UPSERT_AVRO -# row schema location confluent schema registry 'http://message_queue:8081' - -statement ok -CREATE TABLE debezium_non_compact (PRIMARY KEY(order_id)) with ( - connector = 'kafka', - kafka.topic = 'debezium_non_compact_avro_json', - kafka.brokers = 'message_queue:29092', - kafka.scan.startup.mode = 'earliest' -) ROW FORMAT DEBEZIUM_AVRO ROW SCHEMA LOCATION CONFLUENT SCHEMA REGISTRY 'http://message_queue:8081'; - - -statement ok -CREATE TABLE debezium_compact (PRIMARY KEY(order_id)) with ( - connector = 'kafka', - kafka.topic = 'debezium_compact_avro_json', - kafka.brokers = 'message_queue:29092', - kafka.scan.startup.mode = 'earliest' -) ROW FORMAT DEBEZIUM_AVRO ROW SCHEMA LOCATION CONFLUENT SCHEMA REGISTRY 'http://message_queue:8081'; - -statement ok -flush; - -# Wait enough time to ensure SourceExecutor consumes all Kafka data. -sleep 10s - -query II -SELECT - op_type, "ID", "CLASS_ID", "ITEM_ID", "ATTR_ID", "ATTR_VALUE", "ORG_ID", "UNIT_INFO", "UPD_TIME" -FROM - upsert_avro_json_default_key -ORDER BY - "ID"; ----- -update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z -delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z - -# query II -# SELECT -# * -# FROM -# upsert_avro_json -# ORDER BY -# "ID"; -# ---- -# update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z -# delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -# delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -# delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z - - -query II -SELECT - "ID", "firstName", "lastName", "age", "height", "weight" -FROM - upsert_student_avro_json -ORDER BY - "ID"; ----- -1 Ethan Martinez 18 6.1 180 -2 Emily Jackson 19 5.4 110 -3 Noah Thompson 21 6.3 195 -4 Emma Brown 20 5.3 130 -5 Michael Williams 22 6.2 190 -6 Leah Davis 18 5.7 140 -9 Jacob Anderson 20 5.8 155 - -query I -select count(*) from debezium_non_compact; ----- -2 - -query I -select count(*) from debezium_compact; ----- -2 - -statement ok -DROP TABLE upsert_avro_json_default_key; - -# statement ok -# DROP TABLE upsert_avro_json; - - -statement ok -DROP TABLE upsert_student_avro_json; - -statement ok -DROP TABLE debezium_non_compact; - -statement ok -DROP TABLE debezium_compact; \ No newline at end of file diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 61125c531fdb2..5e0e2003f4d93 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -788,9 +788,6 @@ impl ByteStreamSourceParserImpl { let protocol = &parser_config.specific.protocol_config; let encode = &parser_config.specific.encoding_config; match (protocol, encode) { - // (ProtocolProperties::Plain, EncodingProperties::Json(_)) => { - // JsonParser::new(parser_config.specific, rw_columns, source_ctx).map(Self::Json) - // } (ProtocolProperties::Plain, EncodingProperties::Csv(config)) => { CsvParser::new(rw_columns, *config, source_ctx).map(Self::Csv) } From a4c6ba632acbc11e662fc982edc6937936e19d0b Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 19 Dec 2023 14:56:09 +0800 Subject: [PATCH 44/50] refactor: Refactor additional columns handling and imports - Refactor handling of additional columns in `create_source.rs` - Remove unused imports and variables in `additional_columns.rs` - Create separate functions for each connector's compatible columns - Update import and remove unnecessary import in `mod.rs` Signed-off-by: tabVersion --- .../src/parser/additional_columns.rs | 208 ++++++++---------- src/connector/src/source/mod.rs | 2 +- src/frontend/src/handler/create_source.rs | 47 ++-- 3 files changed, 114 insertions(+), 143 deletions(-) diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index 135b0b4af2359..20451bf804dfd 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -12,9 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; -use std::sync::LazyLock; - use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; use risingwave_common::types::DataType; use risingwave_pb::plan_common::AdditionalColumnType; @@ -26,88 +23,22 @@ use crate::source::{ pub type CompatibleAdditionalColumnsFn = Box ColumnCatalog + Send + Sync + 'static>; -pub static CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock< - HashMap>, -> = LazyLock::new(|| { - let mut res: HashMap> = - HashMap::new(); +pub fn get_connector_compatible_additional_columns( + connector_name: &str, +) -> Option> { + let compatible_columns = match connector_name { + KAFKA_CONNECTOR => kafka_compatible_column_vec(), + PULSAR_CONNECTOR => pulsar_compatible_column_vec(), + KINESIS_CONNECTOR => kinesis_compatible_column_vec(), + S3_V2_CONNECTOR | S3_CONNECTOR => s3_compatible_column_column_vec(), + _ => return None, + }; + Some(compatible_columns) +} - res.insert( - KAFKA_CONNECTOR.to_string(), - vec![ - ( - "key", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named_with_additional_column( - name, - id, - DataType::Bytea, - AdditionalColumnType::Key, - ), - is_hidden: false, - } - }), - ), - ( - "timestamp", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named_with_additional_column( - name, - id, - DataType::Timestamptz, - AdditionalColumnType::Timestamp, - ), - is_hidden: false, - } - }), - ), - ( - "partition", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named_with_additional_column( - name, - id, - DataType::Int64, - AdditionalColumnType::Partition, - ), - is_hidden: false, - } - }), - ), - ( - "offset", - Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - ColumnCatalog { - column_desc: ColumnDesc::named_with_additional_column( - name, - id, - DataType::Int64, - AdditionalColumnType::Offset, - ), - is_hidden: false, - } - }), - ), - // Todo(tabVersion): add header column desc - // ( - // "header", - // Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { - // ColumnCatalog { - // column_desc: ColumnDesc::named(name, id, DataType::List( - // - // )), - // is_hidden: false, - // } - // }), - // ), - ], - ); - res.insert( - PULSAR_CONNECTOR.to_string(), - vec![( +fn kafka_compatible_column_vec() -> Vec<(&'static str, CompatibleAdditionalColumnsFn)> { + vec![ + ( "key", Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { ColumnCatalog { @@ -120,60 +51,111 @@ pub static CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock< is_hidden: false, } }), - )], - ); - res.insert( - KINESIS_CONNECTOR.to_string(), - vec![( - "key", + ), + ( + "timestamp", Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { ColumnCatalog { column_desc: ColumnDesc::named_with_additional_column( name, id, - DataType::Bytea, - AdditionalColumnType::Key, + DataType::Timestamptz, + AdditionalColumnType::Timestamp, ), is_hidden: false, } }), - )], - ); - res.insert( - S3_CONNECTOR.to_string(), - vec![( - "file", + ), + ( + "partition", Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { ColumnCatalog { column_desc: ColumnDesc::named_with_additional_column( name, id, - DataType::Varchar, - AdditionalColumnType::Filename, + DataType::Int64, + AdditionalColumnType::Partition, ), is_hidden: false, } }), - )], - ); - res.insert( - // TODO(tabVersion): change to Opendal S3 and GCS - S3_V2_CONNECTOR.to_string(), - vec![( - "file", + ), + ( + "offset", Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { ColumnCatalog { column_desc: ColumnDesc::named_with_additional_column( name, id, - DataType::Varchar, - AdditionalColumnType::Filename, + DataType::Int64, + AdditionalColumnType::Offset, ), is_hidden: false, } }), - )], - ); + ), + // Todo(tabVersion): add header column desc + // ( + // "header", + // Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + // ColumnCatalog { + // column_desc: ColumnDesc::named(name, id, DataType::List( + // + // )), + // is_hidden: false, + // } + // }), + // ), + ] +} + +fn pulsar_compatible_column_vec() -> Vec<(&'static str, CompatibleAdditionalColumnsFn)> { + vec![( + "key", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Bytea, + AdditionalColumnType::Key, + ), + is_hidden: false, + } + }), + )] +} + +fn kinesis_compatible_column_vec() -> Vec<(&'static str, CompatibleAdditionalColumnsFn)> { + vec![( + "key", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Bytea, + AdditionalColumnType::Key, + ), + is_hidden: false, + } + }), + )] +} - res -}); +fn s3_compatible_column_column_vec() -> Vec<(&'static str, CompatibleAdditionalColumnsFn)> { + vec![( + "file", + Box::new(|id: ColumnId, name: &str| -> ColumnCatalog { + ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + name, + id, + DataType::Varchar, + AdditionalColumnType::Filename, + ), + is_hidden: false, + } + }), + )] +} diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 670f50af4eca7..1c05571c4f632 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -40,7 +40,7 @@ pub use manager::{SourceColumnDesc, SourceColumnType}; pub use mock_external_table::MockExternalTableReader; pub use crate::parser::additional_columns::{ - CompatibleAdditionalColumnsFn, CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS, + get_connector_compatible_additional_columns, CompatibleAdditionalColumnsFn, }; pub use crate::source::filesystem::{S3_CONNECTOR, S3_V2_CONNECTOR}; pub use crate::source::nexmark::NEXMARK_CONNECTOR; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 0a51b2fc4728d..99d548732fa9f 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -27,7 +27,6 @@ use risingwave_common::catalog::{ use risingwave_common::error::ErrorCode::{self, InvalidInputSyntax, ProtocolError}; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::DataType; -use risingwave_connector::parser::additional_columns::CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS; use risingwave_connector::parser::{ schema_to_columns, AvroParserConfig, DebeziumAvroParserConfig, ProtobufParserConfig, SpecificParserConfig, @@ -44,8 +43,9 @@ use risingwave_connector::source::external::CdcTableType; use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType}; use risingwave_connector::source::test_source::TEST_CONNECTOR; use risingwave_connector::source::{ - GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, - PULSAR_CONNECTOR, S3_CONNECTOR, S3_V2_CONNECTOR, + get_connector_compatible_additional_columns, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, + KINESIS_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, + S3_V2_CONNECTOR, }; use risingwave_pb::catalog::{ PbSchemaRegistryNameStrategy, PbSource, StreamSourceInfo, WatermarkDesc, @@ -672,34 +672,23 @@ pub fn handle_addition_columns( mut include_columns_options: Vec<(Ident, Option)>, columns: &mut Vec, ) -> Result<()> { - // refer the logic from `parse_source_schema_with_connector` - const EXEMPTED_CONNECTORS: [&str; 4] = ["nexmark", "datagen", "google_pubsub", "test"]; - let is_connector_exempted_from_additional_column = |connector_name: &str| -> bool { - connector_name.contains("-cdc") || EXEMPTED_CONNECTORS.contains(&connector_name) - }; - let connector_name = get_connector(with_properties).unwrap(); // there must be a connector in source - let is_exempted = is_connector_exempted_from_additional_column(connector_name.as_str()); - match (is_exempted, include_columns_options.is_empty()) { - (true, true) => return Ok(()), - (true, false) => { - return Err(RwError::from(ProtocolError(format!( - "connector {} does not work with additional columns, but got {:?}", - connector_name, include_columns_options - )))) - } - (false, _) => {} - } - - let addition_col_list = CONNECTOR_COMPATIBLE_ADDITIONAL_COLUMNS - .get(connector_name.as_str()) - .ok_or_else(|| { - RwError::from(ProtocolError(format!( - "Connector {} accepts no additional column", - connector_name - ))) - })?; + let addition_col_list = + match get_connector_compatible_additional_columns(connector_name.as_str()) { + Some(cols) => cols, + // early return if there are no accepted additional columns for the connector + None => { + return if include_columns_options.is_empty() { + Ok(()) + } else { + Err(RwError::from(ProtocolError(format!( + "Connector {} accepts no additional column but got {:?}", + connector_name, include_columns_options + )))) + } + } + }; let gen_default_column_name = |connector_name: &str, addi_column_name: &str| { format!("_rw_{}_{}", connector_name, addi_column_name) }; From 4da8dbfd6f93c3d1b00aaed86307ac0d236aed02 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 19 Dec 2023 15:47:11 +0800 Subject: [PATCH 45/50] change additionalColumn_type to Normal rather than Unspecified --- proto/plan_common.proto | 1 + src/common/src/catalog/column.rs | 10 +++++----- src/common/src/catalog/test_utils.rs | 3 ++- src/connector/src/parser/avro/util.rs | 3 ++- .../src/parser/debezium/simd_json_parser.rs | 2 +- src/connector/src/parser/mod.rs | 2 +- src/connector/src/parser/protobuf/parser.rs | 2 +- src/connector/src/parser/unified/upsert.rs | 2 +- src/connector/src/source/manager.rs | 2 +- src/frontend/src/binder/expr/mod.rs | 2 +- src/frontend/src/catalog/table_catalog.rs | 2 +- src/frontend/src/handler/create_source.rs | 18 ++++++++++-------- src/frontend/src/handler/create_table.rs | 2 +- src/storage/src/row_serde/mod.rs | 8 ++++---- 14 files changed, 32 insertions(+), 27 deletions(-) diff --git a/proto/plan_common.proto b/proto/plan_common.proto index af3cda33c6641..74427d1b29798 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -23,6 +23,7 @@ enum AdditionalColumnType { OFFSET = 4; HEADER = 5; FILENAME = 6; + NORMAL = 7; } message ColumnDesc { diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index 40229339427f6..5a5baa83fcbda 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -114,7 +114,7 @@ impl ColumnDesc { type_name: String::new(), generated_or_default_column: None, description: None, - additional_column_type: AdditionalColumnType::Unspecified, + additional_column_type: AdditionalColumnType::Normal, } } @@ -127,7 +127,7 @@ impl ColumnDesc { type_name: String::new(), generated_or_default_column: None, description: None, - additional_column_type: AdditionalColumnType::Unspecified, + additional_column_type: AdditionalColumnType::Normal, } } @@ -191,7 +191,7 @@ impl ColumnDesc { type_name: "".to_string(), generated_or_default_column: None, description: None, - additional_column_type: AdditionalColumnType::Unspecified, + additional_column_type: AdditionalColumnType::Normal, } } @@ -213,7 +213,7 @@ impl ColumnDesc { type_name: type_name.to_string(), generated_or_default_column: None, description: None, - additional_column_type: AdditionalColumnType::Unspecified, + additional_column_type: AdditionalColumnType::Normal, } } @@ -230,7 +230,7 @@ impl ColumnDesc { type_name: field.type_name.clone(), description: None, generated_or_default_column: None, - additional_column_type: AdditionalColumnType::Unspecified, + additional_column_type: AdditionalColumnType::Normal, } } diff --git a/src/common/src/catalog/test_utils.rs b/src/common/src/catalog/test_utils.rs index d1d6123759e15..10534e774e4c6 100644 --- a/src/common/src/catalog/test_utils.rs +++ b/src/common/src/catalog/test_utils.rs @@ -35,6 +35,7 @@ impl ColumnDescTestExt for ColumnDesc { column_type: Some(data_type), column_id, name: name.to_string(), + additional_column_type: AdditionalColumnType::Normal as i32, ..Default::default() } } @@ -58,7 +59,7 @@ impl ColumnDescTestExt for ColumnDesc { field_descs: fields, generated_or_default_column: None, description: None, - additional_column_type: AdditionalColumnType::Unspecified as i32, + additional_column_type: AdditionalColumnType::Normal as i32, } } } diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 443fe495bd659..8ee799164051b 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -61,7 +61,7 @@ fn avro_field_to_column_desc( type_name: schema_name.to_string(), generated_or_default_column: None, description: None, - additional_column_type: AdditionalColumnType::Unspecified as i32, + additional_column_type: AdditionalColumnType::Normal as i32, }) } _ => { @@ -70,6 +70,7 @@ fn avro_field_to_column_desc( column_type: Some(data_type.to_protobuf()), column_id: *index, name: name.to_owned(), + additional_column_type: AdditionalColumnType::Normal as i32, ..Default::default() }) } diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index c58fae5095bd4..3de1c45bb7615 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -566,7 +566,7 @@ mod tests { fields: vec![], column_type: SourceColumnType::Normal, is_pk: false, - additional_column_type: AdditionalColumnType::Unspecified, + additional_column_type: AdditionalColumnType::Normal, }, SourceColumnDesc::simple("o_enum", DataType::Varchar, ColumnId::from(8)), SourceColumnDesc::simple("o_char", DataType::Varchar, ColumnId::from(9)), diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 5e0e2003f4d93..908c799c905e8 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -345,7 +345,7 @@ impl SourceStreamChunkRowWriter<'_> { | &AdditionalColumnType::Filename | &AdditionalColumnType::Offset | &AdditionalColumnType::Header, - // AdditionalColumnType::Unspecified is means it comes from message payload + // AdditionalColumnType::Unspecified and AdditionalColumnType::Normal is means it comes from message payload // AdditionalColumnType::Key is processed in normal process, together with Unspecified ones ) => { todo!() diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index 96aa22dec0201..feaf62e21b643 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -213,7 +213,7 @@ impl ProtobufParserConfig { type_name: m.full_name().to_string(), generated_or_default_column: None, description: None, - additional_column_type: AdditionalColumnType::Unspecified as i32, + additional_column_type: AdditionalColumnType::Normal as i32, }) } else { *index += 1; diff --git a/src/connector/src/parser/unified/upsert.rs b/src/connector/src/parser/unified/upsert.rs index d00e0574b8b48..20de7fed43b66 100644 --- a/src/connector/src/parser/unified/upsert.rs +++ b/src/connector/src/parser/unified/upsert.rs @@ -115,7 +115,7 @@ where self.access(&["key", &desc.name], Some(&desc.data_type)) } } - AdditionalColumnType::Unspecified => { + AdditionalColumnType::Unspecified | AdditionalColumnType::Normal => { self.access(&["value", &desc.name], Some(&desc.data_type)) } _ => unreachable!(), diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 0446ba235bd31..1c43d6f386c1f 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -87,7 +87,7 @@ impl SourceColumnDesc { fields: vec![], column_type: SourceColumnType::Normal, is_pk: false, - additional_column_type: AdditionalColumnType::Unspecified, + additional_column_type: AdditionalColumnType::Normal, } } diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 4438ec9d6861d..372c5e7623771 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -575,7 +575,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { type_name: "".to_string(), generated_or_default_column: None, description: None, - additional_column_type: AdditionalColumnType::Unspecified, + additional_column_type: AdditionalColumnType::Normal, }) } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 01913850e09c8..2868be7ad053b 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -681,7 +681,7 @@ mod tests { type_name: ".test.Country".to_string(), description: None, generated_or_default_column: None, - additional_column_type: AdditionalColumnType::Unspecified, + additional_column_type: AdditionalColumnType::Normal, }, is_hidden: false } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 99d548732fa9f..4343bfd73a615 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -669,7 +669,7 @@ pub(crate) async fn bind_columns_from_source( /// add connector-spec columns to the end of column catalog pub fn handle_addition_columns( with_properties: &HashMap, - mut include_columns_options: Vec<(Ident, Option)>, + mut additional_columns: Vec<(Ident, Option)>, columns: &mut Vec, ) -> Result<()> { let connector_name = get_connector(with_properties).unwrap(); // there must be a connector in source @@ -679,12 +679,12 @@ pub fn handle_addition_columns( Some(cols) => cols, // early return if there are no accepted additional columns for the connector None => { - return if include_columns_options.is_empty() { + return if additional_columns.is_empty() { Ok(()) } else { Err(RwError::from(ProtocolError(format!( "Connector {} accepts no additional column but got {:?}", - connector_name, include_columns_options + connector_name, additional_columns )))) } } @@ -701,11 +701,11 @@ pub fn handle_addition_columns( for (col_name, gen_column_catalog_fn) in addition_col_list { // always insert in spec order - if let Some(idx) = include_columns_options + if let Some(idx) = additional_columns .iter() .position(|(col, _)| col.real_value().eq_ignore_ascii_case(col_name)) { - let (_, alias) = include_columns_options.remove(idx); + let (_, alias) = additional_columns.remove(idx); columns.push(gen_column_catalog_fn( latest_col_id.next(), alias @@ -715,10 +715,10 @@ pub fn handle_addition_columns( )) } } - if !include_columns_options.is_empty() { + if !additional_columns.is_empty() { return Err(RwError::from(ProtocolError(format!( "Unknown additional columns {:?}", - include_columns_options + additional_columns )))); } @@ -845,7 +845,9 @@ pub(crate) async fn bind_source_pk( let additional_column_names = columns .iter() .filter_map(|col| { - if col.column_desc.additional_column_type != AdditionalColumnType::Unspecified { + if (col.column_desc.additional_column_type != AdditionalColumnType::Unspecified) + && (col.column_desc.additional_column_type != AdditionalColumnType::Normal) + { Some(col.name().to_string()) } else { None diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index aaa5d8f3bdaa0..44a9b6a8c20ff 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -206,7 +206,7 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> type_name: "".to_string(), generated_or_default_column: None, description: None, - additional_column_type: AdditionalColumnType::Unspecified, + additional_column_type: AdditionalColumnType::Normal, }, is_hidden: false, }); diff --git a/src/storage/src/row_serde/mod.rs b/src/storage/src/row_serde/mod.rs index 06c980cccf6d1..0674e34fb0d02 100644 --- a/src/storage/src/row_serde/mod.rs +++ b/src/storage/src/row_serde/mod.rs @@ -94,7 +94,7 @@ mod test { type_name: "", generated_or_default_column: None, description: None, - additional_column_type: Unspecified, + additional_column_type: Normal, }, ColumnDesc { data_type: Int16, @@ -104,7 +104,7 @@ mod test { type_name: "", generated_or_default_column: None, description: None, - additional_column_type: Unspecified, + additional_column_type: Normal, }, ], [ @@ -134,7 +134,7 @@ mod test { type_name: "", generated_or_default_column: None, description: None, - additional_column_type: Unspecified, + additional_column_type: Normal, }, ColumnDesc { data_type: Varchar, @@ -144,7 +144,7 @@ mod test { type_name: "", generated_or_default_column: None, description: None, - additional_column_type: Unspecified, + additional_column_type: Normal, }, ], [ From d38e03833ef22ea744b277858f9f1c8dc88f700a Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 19 Dec 2023 16:29:02 +0800 Subject: [PATCH 46/50] add version for column_desc --- proto/plan_common.proto | 12 ++++++++++++ src/common/src/catalog/column.rs | 15 ++++++++++++++- src/common/src/catalog/test_utils.rs | 4 +++- src/connector/src/parser/avro/util.rs | 4 +++- src/connector/src/parser/protobuf/parser.rs | 5 ++++- src/connector/src/source/manager.rs | 3 ++- src/frontend/src/binder/expr/mod.rs | 3 ++- src/frontend/src/catalog/table_catalog.rs | 5 ++++- src/frontend/src/handler/create_table.rs | 5 ++++- src/storage/src/row_serde/mod.rs | 4 ++++ 10 files changed, 52 insertions(+), 8 deletions(-) diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 74427d1b29798..932d813c87388 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -26,6 +26,16 @@ enum AdditionalColumnType { NORMAL = 7; } +enum ColumnDescVersion { + COLUMN_DESC_VERSION_UNSPECIFIED = 0; + // Introduced in https://github.com/risingwavelabs/risingwave/pull/13707#discussion_r1429947537, + // in case DEFAULT_KEY_COLUMN_NAME changes + COLUMN_DESC_VERSION_PR_13707 = 1; + + // for test only + COLUMN_DESC_VERSION_MAX = 2147483647; +} + message ColumnDesc { data.DataType column_type = 1; int32 column_id = 2; @@ -55,6 +65,8 @@ message ColumnDesc { // This field is used to represent the connector-spec additional column type. // UNSPECIFIED or unset for normal column. AdditionalColumnType additional_column_type = 9; + + ColumnDescVersion version = 10; } message ColumnCatalog { diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index 5a5baa83fcbda..3780aa1f83eb8 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -17,7 +17,9 @@ use std::borrow::Cow; use itertools::Itertools; use risingwave_pb::expr::ExprNode; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; -use risingwave_pb::plan_common::{AdditionalColumnType, PbColumnCatalog, PbColumnDesc}; +use risingwave_pb::plan_common::{ + AdditionalColumnType, ColumnDescVersion, PbColumnCatalog, PbColumnDesc, +}; use super::row_id_column_desc; use crate::catalog::{cdc_table_name_column_desc, offset_column_desc, Field, ROW_ID_COLUMN_ID}; @@ -102,6 +104,7 @@ pub struct ColumnDesc { pub generated_or_default_column: Option, pub description: Option, pub additional_column_type: AdditionalColumnType, + pub version: ColumnDescVersion, } impl ColumnDesc { @@ -115,6 +118,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column_type: AdditionalColumnType::Normal, + version: ColumnDescVersion::Pr13707, } } @@ -128,6 +132,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column_type: AdditionalColumnType::Normal, + version: ColumnDescVersion::Pr13707, } } @@ -146,6 +151,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column_type, + version: ColumnDescVersion::Pr13707, } } @@ -165,6 +171,7 @@ impl ColumnDesc { generated_or_default_column: self.generated_or_default_column.clone(), description: self.description.clone(), additional_column_type: self.additional_column_type as i32, + version: self.version as i32, } } @@ -192,6 +199,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column_type: AdditionalColumnType::Normal, + version: ColumnDescVersion::Pr13707, } } @@ -214,6 +222,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column_type: AdditionalColumnType::Normal, + version: ColumnDescVersion::Pr13707, } } @@ -231,6 +240,7 @@ impl ColumnDesc { description: None, generated_or_default_column: None, additional_column_type: AdditionalColumnType::Normal, + version: ColumnDescVersion::Pr13707, } } @@ -256,6 +266,7 @@ impl ColumnDesc { impl From for ColumnDesc { fn from(prost: PbColumnDesc) -> Self { let additional_column_type = prost.additional_column_type(); + let version = prost.version(); let field_descs: Vec = prost .field_descs .into_iter() @@ -270,6 +281,7 @@ impl From for ColumnDesc { generated_or_default_column: prost.generated_or_default_column, description: prost.description.clone(), additional_column_type, + version, } } } @@ -291,6 +303,7 @@ impl From<&ColumnDesc> for PbColumnDesc { generated_or_default_column: c.generated_or_default_column.clone(), description: c.description.clone(), additional_column_type: c.additional_column_type as i32, + version: c.version as i32, } } } diff --git a/src/common/src/catalog/test_utils.rs b/src/common/src/catalog/test_utils.rs index 10534e774e4c6..34af52530912f 100644 --- a/src/common/src/catalog/test_utils.rs +++ b/src/common/src/catalog/test_utils.rs @@ -15,7 +15,7 @@ use itertools::Itertools; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; -use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDesc}; +use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDesc, ColumnDescVersion}; pub trait ColumnDescTestExt { /// Create a [`ColumnDesc`] with the given name and type. @@ -36,6 +36,7 @@ impl ColumnDescTestExt for ColumnDesc { column_id, name: name.to_string(), additional_column_type: AdditionalColumnType::Normal as i32, + version: ColumnDescVersion::Pr13707 as i32, ..Default::default() } } @@ -60,6 +61,7 @@ impl ColumnDescTestExt for ColumnDesc { generated_or_default_column: None, description: None, additional_column_type: AdditionalColumnType::Normal as i32, + version: ColumnDescVersion::Pr13707 as i32, } } } diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 8ee799164051b..1e5873e051753 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -18,7 +18,7 @@ use apache_avro::schema::{DecimalSchema, RecordSchema, Schema}; use itertools::Itertools; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{DataType, Decimal}; -use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDesc}; +use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDesc, ColumnDescVersion}; pub fn avro_schema_to_column_descs(schema: &Schema) -> anyhow::Result> { if let Schema::Record(RecordSchema { fields, .. }) = schema { @@ -62,6 +62,7 @@ fn avro_field_to_column_desc( generated_or_default_column: None, description: None, additional_column_type: AdditionalColumnType::Normal as i32, + version: ColumnDescVersion::Pr13707 as i32, }) } _ => { @@ -71,6 +72,7 @@ fn avro_field_to_column_desc( column_id: *index, name: name.to_owned(), additional_column_type: AdditionalColumnType::Normal as i32, + version: ColumnDescVersion::Pr13707 as i32, ..Default::default() }) } diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index feaf62e21b643..4ab09ae19d79e 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -25,7 +25,7 @@ use risingwave_common::error::ErrorCode::{InternalError, ProtocolError}; use risingwave_common::error::{Result, RwError}; use risingwave_common::try_match_expand; use risingwave_common::types::{DataType, Datum, Decimal, JsonbVal, ScalarImpl, F32, F64}; -use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDesc}; +use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDesc, ColumnDescVersion}; use super::schema_resolver::*; use crate::aws_utils::load_file_descriptor_from_s3; @@ -214,6 +214,7 @@ impl ProtobufParserConfig { generated_or_default_column: None, description: None, additional_column_type: AdditionalColumnType::Normal as i32, + version: ColumnDescVersion::Pr13707 as i32, }) } else { *index += 1; @@ -221,6 +222,8 @@ impl ProtobufParserConfig { column_id: *index, name: field_descriptor.name().to_string(), column_type: Some(field_type.to_protobuf()), + additional_column_type: AdditionalColumnType::Normal as i32, + version: ColumnDescVersion::Pr13707 as i32, ..Default::default() }) } diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 1c43d6f386c1f..e818bca9b32c8 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -19,7 +19,7 @@ use risingwave_common::catalog::{ TABLE_NAME_COLUMN_NAME, }; use risingwave_common::types::DataType; -use risingwave_pb::plan_common::AdditionalColumnType; +use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDescVersion}; /// `SourceColumnDesc` is used to describe a column in the Source and is used as the column /// counterpart in `StreamScan` @@ -135,6 +135,7 @@ impl From<&SourceColumnDesc> for ColumnDesc { generated_or_default_column: None, description: None, additional_column_type: s.additional_column_type, + version: ColumnDescVersion::Pr13707, } } } diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 372c5e7623771..37b0464f0929c 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -18,7 +18,7 @@ use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::zip_eq_fast; use risingwave_common::{bail_not_implemented, not_implemented}; -use risingwave_pb::plan_common::AdditionalColumnType; +use risingwave_pb::plan_common::{AdditionalColumnType, ColumnDescVersion}; use risingwave_sqlparser::ast::{ Array, BinaryOperator, DataType as AstDataType, Expr, Function, JsonPredicateType, ObjectName, Query, StructField, TrimWhereField, UnaryOperator, @@ -576,6 +576,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { generated_or_default_column: None, description: None, additional_column_type: AdditionalColumnType::Normal, + version: ColumnDescVersion::Pr13707, }) } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 2868be7ad053b..9622ea3cfcedd 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -588,7 +588,9 @@ mod tests { use risingwave_common::types::*; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::{PbStreamJobStatus, PbTable}; - use risingwave_pb::plan_common::{AdditionalColumnType, PbColumnCatalog, PbColumnDesc}; + use risingwave_pb::plan_common::{ + AdditionalColumnType, ColumnDescVersion, PbColumnCatalog, PbColumnDesc, + }; use super::*; use crate::catalog::table_catalog::{TableCatalog, TableType}; @@ -682,6 +684,7 @@ mod tests { description: None, generated_or_default_column: None, additional_column_type: AdditionalColumnType::Normal, + version: ColumnDescVersion::Pr13707, }, is_hidden: false } diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 44a9b6a8c20ff..f7843310e482d 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -35,7 +35,9 @@ use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::{PbSource, PbTable, StreamSourceInfo, Table, WatermarkDesc}; use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; -use risingwave_pb::plan_common::{AdditionalColumnType, DefaultColumnDesc, GeneratedColumnDesc}; +use risingwave_pb::plan_common::{ + AdditionalColumnType, ColumnDescVersion, DefaultColumnDesc, GeneratedColumnDesc, +}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_sqlparser::ast::{ @@ -207,6 +209,7 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> generated_or_default_column: None, description: None, additional_column_type: AdditionalColumnType::Normal, + version: ColumnDescVersion::Pr13707, }, is_hidden: false, }); diff --git a/src/storage/src/row_serde/mod.rs b/src/storage/src/row_serde/mod.rs index 0674e34fb0d02..b94c8e663d9c4 100644 --- a/src/storage/src/row_serde/mod.rs +++ b/src/storage/src/row_serde/mod.rs @@ -95,6 +95,7 @@ mod test { generated_or_default_column: None, description: None, additional_column_type: Normal, + version: Pr13707, }, ColumnDesc { data_type: Int16, @@ -105,6 +106,7 @@ mod test { generated_or_default_column: None, description: None, additional_column_type: Normal, + version: Pr13707, }, ], [ @@ -135,6 +137,7 @@ mod test { generated_or_default_column: None, description: None, additional_column_type: Normal, + version: Pr13707, }, ColumnDesc { data_type: Varchar, @@ -145,6 +148,7 @@ mod test { generated_or_default_column: None, description: None, additional_column_type: Normal, + version: Pr13707, }, ], [ From be6d28e0438d95ac94a5e754dbbd28b80c3c8b29 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 19 Dec 2023 17:04:14 +0800 Subject: [PATCH 47/50] resolve comments --- src/connector/src/parser/additional_columns.rs | 4 ++-- src/connector/src/parser/mod.rs | 9 ++++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index 20451bf804dfd..767a656b63f44 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -17,7 +17,7 @@ use risingwave_common::types::DataType; use risingwave_pb::plan_common::AdditionalColumnType; use crate::source::{ - KAFKA_CONNECTOR, KINESIS_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, S3_V2_CONNECTOR, + KAFKA_CONNECTOR, KINESIS_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, }; pub type CompatibleAdditionalColumnsFn = @@ -30,7 +30,7 @@ pub fn get_connector_compatible_additional_columns( KAFKA_CONNECTOR => kafka_compatible_column_vec(), PULSAR_CONNECTOR => pulsar_compatible_column_vec(), KINESIS_CONNECTOR => kinesis_compatible_column_vec(), - S3_V2_CONNECTOR | S3_CONNECTOR => s3_compatible_column_column_vec(), + OPENDAL_S3_CONNECTOR | S3_CONNECTOR => s3_compatible_column_column_vec(), _ => return None, }; Some(compatible_columns) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 908c799c905e8..448c98ec571ae 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -16,6 +16,7 @@ use std::collections::HashMap; use std::fmt::Debug; use std::sync::LazyLock; +use anyhow::anyhow; use auto_enums::auto_enum; pub use avro::AvroParserConfig; pub use canal::*; @@ -48,6 +49,7 @@ use self::upsert_parser::UpsertParser; use self::util::get_kafka_topic; use crate::common::AwsAuthProps; use crate::parser::maxwell::MaxwellParser; +use crate::parser::unified::AccessError; use crate::schema::schema_registry::SchemaRegistryAuth; use crate::source::monitor::GLOBAL_SOURCE_METRICS; use crate::source::{ @@ -347,9 +349,10 @@ impl SourceStreamChunkRowWriter<'_> { | &AdditionalColumnType::Header, // AdditionalColumnType::Unspecified and AdditionalColumnType::Normal is means it comes from message payload // AdditionalColumnType::Key is processed in normal process, together with Unspecified ones - ) => { - todo!() - } + ) => Err(AccessError::Other(anyhow!( + "Column type {:?} not implemented yet", + &desc.additional_column_type + ))), (_, _) => { // For normal columns, call the user provided closure. match f(desc) { From c2c36424d5c69c54595e0643d4e7d2c7c848b7aa Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 19 Dec 2023 17:26:49 +0800 Subject: [PATCH 48/50] resolve comments --- src/common/src/catalog/mod.rs | 9 +++++++++ src/stream/src/from_proto/source/trad_source.rs | 14 +++++++++++--- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 54ad7b9d2fd6c..eac1f3350bdb4 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -29,6 +29,7 @@ pub use internal_table::*; use parse_display::Display; pub use physical_table::*; use risingwave_pb::catalog::HandleConflictBehavior as PbHandleConflictBehavior; +use risingwave_pb::plan_common::ColumnDescVersion; pub use schema::{test_utils as schema_test_utils, Field, FieldDisplay, Schema}; use thiserror_ext::AsReport; @@ -74,8 +75,16 @@ pub const RW_RESERVED_COLUMN_NAME_PREFIX: &str = "_rw_"; // When there is no primary key specified while creating source, will use the // the message key as primary key in `BYTEA` type with this name. +// Note: the field has version to track, please refer to `default_key_column_name_version_mapping` pub const DEFAULT_KEY_COLUMN_NAME: &str = "_rw_key"; +pub fn default_key_column_name_version_mapping(version: &ColumnDescVersion) -> &str { + match version { + ColumnDescVersion::Unspecified => DEFAULT_KEY_COLUMN_NAME, + _ => DEFAULT_KEY_COLUMN_NAME, + } +} + /// For kafka source, we attach a hidden column [`KAFKA_TIMESTAMP_COLUMN_NAME`] to it, so that we /// can limit the timestamp range when querying it directly with batch query. The column type is /// [`DataType::Timestamptz`]. For more details, please refer to diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 6a30d081de973..686d90f8e5f79 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -12,10 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::{ColumnId, TableId, DEFAULT_KEY_COLUMN_NAME}; +use risingwave_common::catalog::{default_key_column_name_version_mapping, ColumnId, TableId}; use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; use risingwave_pb::data::data_type::TypeName as PbTypeName; -use risingwave_pb::plan_common::{AdditionalColumnType, FormatType, PbEncodeType}; +use risingwave_pb::plan_common::{ + AdditionalColumnType, ColumnDescVersion, FormatType, PbEncodeType, +}; use risingwave_pb::stream_plan::SourceNode; use risingwave_source::source_desc::SourceDescBuilder; use risingwave_storage::panic_store::PanicStateStore; @@ -67,7 +69,13 @@ impl ExecutorBuilder for SourceExecutorBuilder { .get_column_type() .map(|col_type| col_type.type_name == PbTypeName::Bytea as i32) .unwrap(); - if desc.name == DEFAULT_KEY_COLUMN_NAME && is_bytea { + if desc.name == default_key_column_name_version_mapping( + &desc.version() + ) + && is_bytea + // the column is from a legacy version + && desc.version == ColumnDescVersion::Unspecified as i32 + { desc.additional_column_type = AdditionalColumnType::Key as i32; } }); From 412e4bcaba05f78125ae0c36596df0419d9e6a97 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 19 Dec 2023 17:45:48 +0800 Subject: [PATCH 49/50] refactor: Refactor error messages and handling for source creation with different formats - Handle various cases of error messages for different format and encoding combinations - Rename a variable and remove previously deprecated logic for avro format - Extract primary key columns from schema registry in certain cases - Add error messages for the presence or absence of additional columns and key specifications in different formats Signed-off-by: tabVersion --- src/frontend/src/handler/create_source.rs | 36 +++++------------------ 1 file changed, 7 insertions(+), 29 deletions(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 567bcbac36473..eea2bc32a7f77 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -860,18 +860,18 @@ pub(crate) async fn bind_source_pk( // For all Upsert formats, we only accept one and only key column as primary key. // Additional KEY columns must be set in this case and must be primary key. - (Format::Upsert, Encode::Json) => { + (Format::Upsert, encode @ Encode::Json | encode @ Encode::Avro) => { if let Some(ref key_column_name) = key_column_name && sql_defined_pk { if sql_defined_pk_names.len() != 1 { return Err(RwError::from(ProtocolError( - format!("upsert json supports only one primary key column ({}).", key_column_name) + format!("upsert {:?} supports only one primary key column ({}).", encode, key_column_name) ))); } // the column name have been converted to real value in `handle_addition_columns` // so we don't ignore ascii case here if !key_column_name.eq(sql_defined_pk_names[0].as_str()) { return Err(RwError::from(ProtocolError(format!( - "upsert json's key column {} not match with sql defined primary key {}", + "upsert {}'s key column {} not match with sql defined primary key {}", encode, key_column_name, sql_defined_pk_names[0] )))); } @@ -879,38 +879,16 @@ pub(crate) async fn bind_source_pk( } else { return if key_column_name.is_none() { Err( - RwError::from(ProtocolError("INCLUDE KEY clause must be set for FORMAT UPSERT ENCODE JSON".to_string())) + RwError::from(ProtocolError(format!("INCLUDE KEY clause must be set for FORMAT UPSERT ENCODE {:?}", encode) + )) ) } else { Err(RwError::from(ProtocolError(format!( - "Primary key must be specified to {} when creating source with FORMAT UPSERT ENCODE JSON", - key_column_name.unwrap())))) + "Primary key must be specified to {} when creating source with FORMAT UPSERT ENCODE {:?}", + key_column_name.unwrap(), encode)))) } } } - (Format::Upsert, Encode::Avro) => { - // prev deprecated logic: - // if key schema can be inferred from schema registry, then use it - if sql_defined_pk && sql_defined_pk_names.len() != 1 { - return Err(RwError::from(ProtocolError( - "upsert avro supports only one primary key column.".to_string(), - ))); - } - if let Some(ref key_column_name) = key_column_name { - if key_column_name.eq(sql_defined_pk_names[0].as_str()) { - sql_defined_pk_names - } else { - return Err(RwError::from(ProtocolError(format!( - "upsert avro's key column {} not match with sql defined primary key {}", - key_column_name, sql_defined_pk_names[0] - )))); - } - } else { - return Err(RwError::from(ProtocolError( - "INCLUDE KEY clause must be set for FORMAT UPSERT ENCODE AVRO".to_string(), - ))); - } - } (Format::Debezium, Encode::Json) => { if !additional_column_names.is_empty() { From 8e725173c6bcd394d22530d29cb94339a2c01427 Mon Sep 17 00:00:00 2001 From: tabVersion Date: Tue, 19 Dec 2023 19:51:51 +0800 Subject: [PATCH 50/50] fix --- src/frontend/src/handler/create_source.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index eea2bc32a7f77..8a9ef0edfbc38 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1647,7 +1647,7 @@ pub mod tests { Err(e) => { assert_eq!( e.to_string(), - "Protocol error: INCLUDE KEY clause must be set for FORMAT UPSERT ENCODE JSON" + "Protocol error: INCLUDE KEY clause must be set for FORMAT UPSERT ENCODE Json" ) } _ => unreachable!(), @@ -1657,7 +1657,7 @@ pub mod tests { .to_string(); match frontend.run_sql(sql).await { Err(e) => { - assert_eq!(e.to_string(), "Protocol error: Primary key must be specified to _rw_kafka_key when creating source with FORMAT UPSERT ENCODE JSON") + assert_eq!(e.to_string(), "Protocol error: Primary key must be specified to _rw_kafka_key when creating source with FORMAT UPSERT ENCODE Json") } _ => unreachable!(), }