From e697aae2e8a239e82e67a52ec299ba24bdfff012 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 16 May 2024 19:41:00 +0800 Subject: [PATCH 01/36] WIP: cdc meta columns --- proto/plan_common.proto | 6 +++ .../src/parser/additional_columns.rs | 43 +++++++++++++++++-- src/frontend/src/handler/create_table.rs | 8 +++- src/prost/build.rs | 8 ++++ 4 files changed, 61 insertions(+), 4 deletions(-) diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 8c73374a4d7d..c9e01be71ffe 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -198,6 +198,10 @@ message AdditionalColumnOffset {} message AdditionalColumnFilename {} +message AdditionalColumnTableName {} + +message AdditionalColumnDatabaseName {} + message AdditionalColumnHeader { string inner_field = 1; data.DataType data_type = 2; @@ -215,6 +219,8 @@ message AdditionalColumn { AdditionalColumnHeader header_inner = 5; AdditionalColumnFilename filename = 6; AdditionalColumnHeaders headers = 7; + AdditionalColumnTableName table_name = 8; + AdditionalColumnDatabaseName database_name = 9; } } diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index 59c47b06ee8b..ca2082b96f25 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -22,12 +22,13 @@ use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType as PbDataType; use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType; use risingwave_pb::plan_common::{ - AdditionalColumn, AdditionalColumnFilename, AdditionalColumnHeader, AdditionalColumnHeaders, - AdditionalColumnKey, AdditionalColumnOffset, AdditionalColumnPartition, - AdditionalColumnTimestamp, + AdditionalColumn, AdditionalColumnDatabaseName, AdditionalColumnFilename, + AdditionalColumnHeader, AdditionalColumnHeaders, AdditionalColumnKey, AdditionalColumnOffset, + AdditionalColumnPartition, AdditionalColumnTableName, AdditionalColumnTimestamp, }; use crate::error::ConnectorResult; +use crate::source::cdc::{MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR}; use crate::source::{ GCS_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, @@ -55,6 +56,14 @@ pub static COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock build_header_catalog(column_id, &column_name, inner_field_name, data_type), + + "table_name" => ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + column_name, + column_id, + DataType::Varchar, + AdditionalColumn { + column_type: Some(AdditionalColumnType::TableName( + AdditionalColumnTableName {}, + )), + }, + ), + is_hidden: false, + }, + "database_name" => ColumnCatalog { + column_desc: ColumnDesc::named_with_additional_column( + column_name, + column_id, + DataType::Varchar, + AdditionalColumn { + column_type: Some(AdditionalColumnType::DatabaseName( + AdditionalColumnDatabaseName {}, + )), + }, + ), + is_hidden: false, + }, + _ => unreachable!(), }; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 41415d598767..ddd53b724d5d 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -56,7 +56,7 @@ use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; use crate::handler::create_source::{ bind_columns_from_source, bind_connector_props, bind_create_source, bind_source_watermark, - UPSTREAM_SOURCE_KEY, + handle_addition_columns, UPSTREAM_SOURCE_KEY, }; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::generic::{CdcScanOptions, SourceNodeKind}; @@ -741,6 +741,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( mut col_id_gen: ColumnIdGenerator, on_conflict: Option, with_version_column: Option, + include_column_options: IncludeOption, ) -> Result<(PlanRef, PbTable)> { let session = context.session_ctx().clone(); let db_name = session.database(); @@ -766,6 +767,10 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( }; let mut columns = bind_sql_columns(&column_defs)?; + let with_properties = source.with_properties.clone().into_iter().collect(); + // add additional columns before bind pk, because `format upsert` requires the key column + handle_addition_columns(&with_properties, include_column_options, &mut columns)?; + tracing::debug!(?columns, "columns w/ added cols"); for c in &mut columns { c.column_desc.column_id = col_id_gen.generate(c.name()) @@ -998,6 +1003,7 @@ pub(super) async fn handle_create_table_plan( col_id_gen, on_conflict, with_version_column, + include_column_options, )?; ((plan, None, table), TableJobType::SharedCdcSource) diff --git a/src/prost/build.rs b/src/prost/build.rs index e031e5cfb01a..03d5801633ae 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -138,6 +138,14 @@ fn main() -> Result<(), Box> { .type_attribute("plan_common.AdditionalColumnHeader", "#[derive(Eq, Hash)]") .type_attribute("plan_common.AdditionalColumnHeaders", "#[derive(Eq, Hash)]") .type_attribute("plan_common.AdditionalColumnOffset", "#[derive(Eq, Hash)]") + .type_attribute( + "plan_common.AdditionalColumnTableName", + "#[derive(Eq, Hash)]", + ) + .type_attribute( + "plan_common.AdditionalColumnDatabaseName", + "#[derive(Eq, Hash)]", + ) .type_attribute("common.ColumnOrder", "#[derive(Eq, Hash)]") .type_attribute("common.OrderType", "#[derive(Eq, Hash)]") .type_attribute("common.Buffer", "#[derive(Eq)]") From 2c67c655cdc32f690a13591c5957f96a6966e74a Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 19 May 2024 23:56:37 +0800 Subject: [PATCH 02/36] cdc timestamp column --- .../risingwave/connector/api/TableSchema.java | 11 ++- proto/plan_common.proto | 6 ++ src/common/src/catalog/external_table.rs | 8 +- src/common/src/catalog/schema.rs | 2 +- .../src/parser/additional_columns.rs | 21 +++++ src/connector/src/parser/unified/debezium.rs | 66 ++++++++++++-- src/connector/src/parser/unified/mod.rs | 3 + src/frontend/src/handler/create_source.rs | 2 + src/frontend/src/handler/create_table.rs | 21 ++++- src/prost/build.rs | 8 ++ .../src/executor/backfill/cdc/cdc_backfill.rs | 89 +++++++++---------- .../backfill/cdc/upstream_table/external.rs | 14 --- .../backfill/cdc/upstream_table/snapshot.rs | 8 +- src/stream/src/from_proto/stream_cdc_scan.rs | 22 ++++- 14 files changed, 198 insertions(+), 83 deletions(-) diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java index ded49e003d19..324fd05de595 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java @@ -80,11 +80,14 @@ public Object getFromRow(String columnName, SinkRow row) { } public static TableSchema fromProto(ConnectorServiceProto.TableSchema tableSchema) { - return new TableSchema( - tableSchema.getColumnsList().stream() - .map(PlanCommon.ColumnDesc::getName) - .collect(Collectors.toList()), + // filter out additional columns + var columns = tableSchema.getColumnsList().stream() + .filter(col -> !col.hasAdditionalColumn()) + .collect(Collectors.toList()); + return new TableSchema( + columns.stream().map(PlanCommon.ColumnDesc::getName).collect(Collectors.toList()), + columns.stream() .map(PlanCommon.ColumnDesc::getColumnType) .collect(Collectors.toList()), tableSchema.getPkIndicesList().stream() diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 8c73374a4d7d..27cfa89498b4 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -108,6 +108,8 @@ message ExternalTableDesc { map connect_properties = 6; // upstream cdc source job id uint32 source_id = 7; + // Indices to the additional columns in the columns list. + repeated uint32 additional_column_indices = 8; } enum JoinType { @@ -198,6 +200,10 @@ message AdditionalColumnOffset {} message AdditionalColumnFilename {} +message AdditionalColumnTableName {} + +message AdditionalColumnDatabaseName {} + message AdditionalColumnHeader { string inner_field = 1; data.DataType data_type = 2; diff --git a/src/common/src/catalog/external_table.rs b/src/common/src/catalog/external_table.rs index 3de38a29a499..1c0a914f6820 100644 --- a/src/common/src/catalog/external_table.rs +++ b/src/common/src/catalog/external_table.rs @@ -37,11 +37,10 @@ pub struct CdcTableDesc { /// All columns in the table, noticed it is NOT sorted by columnId in the vec. pub columns: Vec, + pub additional_column_indices: Vec, /// Column indices for primary keys. pub stream_key: Vec, - pub value_indices: Vec, - /// properties will be passed into the `StreamScanNode` pub connect_properties: BTreeMap, } @@ -67,6 +66,11 @@ impl CdcTableDesc { table_name: self.external_table_name.clone(), stream_key: self.stream_key.iter().map(|k| *k as _).collect(), connect_properties: self.connect_properties.clone(), + additional_column_indices: self + .additional_column_indices + .iter() + .map(|k| *k as _) + .collect(), } } diff --git a/src/common/src/catalog/schema.rs b/src/common/src/catalog/schema.rs index 113d9f804b3d..a4fe8e00d1be 100644 --- a/src/common/src/catalog/schema.rs +++ b/src/common/src/catalog/schema.rs @@ -15,7 +15,7 @@ use std::ops::Index; use itertools::Itertools; -use risingwave_pb::plan_common::{PbColumnDesc, PbField}; +use risingwave_pb::plan_common::{AdditionalColumn, PbColumnDesc, PbField}; use super::ColumnDesc; use crate::array::ArrayBuilderImpl; diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index 59c47b06ee8b..e2a069e0955e 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -28,6 +28,7 @@ use risingwave_pb::plan_common::{ }; use crate::error::ConnectorResult; +use crate::source::cdc::{MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR}; use crate::source::{ GCS_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, @@ -55,6 +56,26 @@ pub static COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock { value_accessor: Option, key_accessor: Option, @@ -26,6 +60,8 @@ pub struct DebeziumChangeEvent { const BEFORE: &str = "before"; const AFTER: &str = "after"; +const SOURCE: &str = "source"; +const SOURCE_TS_MS: &str = "ts_ms"; const OP: &str = "op"; pub const TRANSACTION_STATUS: &str = "status"; pub const TRANSACTION_ID: &str = "id"; @@ -150,11 +186,31 @@ where } // value should not be None. - ChangeEventOperation::Upsert => self - .value_accessor - .as_ref() - .unwrap() - .access(&[AFTER, &desc.name], Some(&desc.data_type)), + ChangeEventOperation::Upsert => { + // For upsert operation, if desc is an additional column, access field in the `SOURCE` field. + desc.additional_column.column_type.as_ref().map_or_else( + || { + self.value_accessor + .as_ref() + .expect("value_accessor must be provided for upsert operation") + .access(&[AFTER, &desc.name], Some(&desc.data_type)) + }, + |additional_column_type| { + match additional_column_type { + &ColumnType::Timestamp(_) => { + // access payload.source.ts_ms + self.value_accessor + .as_ref() + .expect("value_accessor must be provided for upsert operation") + .access(&[SOURCE, &SOURCE_TS_MS], Some(&desc.data_type)) + } + _ => Err(AccessError::UnsupportedAdditionalColumn { + name: desc.name.clone(), + }), + } + }, + ) + } } } diff --git a/src/connector/src/parser/unified/mod.rs b/src/connector/src/parser/unified/mod.rs index 31667cd42e47..f54d48e4e0b9 100644 --- a/src/connector/src/parser/unified/mod.rs +++ b/src/connector/src/parser/unified/mod.rs @@ -104,6 +104,9 @@ pub enum AccessError { #[error("Unsupported data type `{ty}`")] UnsupportedType { ty: String }, + #[error("Unsupported additional column `{name}`")] + UnsupportedAdditionalColumn { name: String }, + /// Errors that are not categorized into variants above. #[error("{message}")] Uncategorized { message: String }, diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 49f3126f53b9..8c8f2a97414d 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -571,6 +571,7 @@ pub fn handle_addition_columns( .max() .unwrap(); // there must be at least one column in the column catalog + let mut additional_column_indices = Vec::new(); while let Some(item) = additional_columns.pop() { { // only allow header column have inner field @@ -587,6 +588,7 @@ pub fn handle_addition_columns( let data_type_name: Option = item .header_inner_expect_type .map(|dt| format!("{:?}", dt).to_lowercase()); + additional_column_indices.push(columns.len()); columns.push(build_additional_column_catalog( latest_col_id.next(), connector_name.as_str(), diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 41415d598767..5bc6b8930e53 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -56,7 +56,7 @@ use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; use crate::handler::create_source::{ bind_columns_from_source, bind_connector_props, bind_create_source, bind_source_watermark, - UPSTREAM_SOURCE_KEY, + handle_addition_columns, UPSTREAM_SOURCE_KEY, }; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::generic::{CdcScanOptions, SourceNodeKind}; @@ -741,6 +741,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( mut col_id_gen: ColumnIdGenerator, on_conflict: Option, with_version_column: Option, + include_column_options: IncludeOption, ) -> Result<(PlanRef, PbTable)> { let session = context.session_ctx().clone(); let db_name = session.database(); @@ -766,13 +767,21 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( }; let mut columns = bind_sql_columns(&column_defs)?; + let table_columns_len = columns.len(); + let with_properties = source.with_properties.clone().into_iter().collect(); + // append additional columns to the end + handle_addition_columns(&with_properties, include_column_options, &mut columns)?; for c in &mut columns { c.column_desc.column_id = col_id_gen.generate(c.name()) } let pk_names = bind_sql_pk_names(&column_defs, &constraints)?; - let (columns, pk_column_ids, _) = bind_pk_on_relation(columns, pk_names, true)?; + let (columns, pk_column_ids, row_id_index) = bind_pk_on_relation(columns, pk_names, true)?; + assert_eq!( + None, row_id_index, + "cdc table should not have row_id column" + ); let definition = context.normalized_sql().to_owned(); @@ -801,8 +810,8 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( external_table_name: external_table_name.clone(), pk: table_pk, columns: columns.iter().map(|c| c.column_desc.clone()).collect(), + additional_column_indices: (table_columns_len..columns.len()).collect(), stream_key: pk_column_indices, - value_indices: (0..columns.len()).collect_vec(), connect_properties, }; @@ -912,6 +921,7 @@ pub(super) async fn handle_create_table_plan( &handler_args.with_options, source_schema, &include_column_options, + &cdc_table_info, )?; let ((plan, source, table), job_type) = @@ -998,6 +1008,7 @@ pub(super) async fn handle_create_table_plan( col_id_gen, on_conflict, with_version_column, + include_column_options, )?; ((plan, None, table), TableJobType::SharedCdcSource) @@ -1091,7 +1102,11 @@ pub fn check_create_table_with_source( with_options: &WithOptions, source_schema: Option, include_column_options: &IncludeOption, + cdc_table_info: &Option, ) -> Result> { + if cdc_table_info.is_some() { + return Ok(source_schema); + } let defined_source = with_options.inner().contains_key(UPSTREAM_SOURCE_KEY); if !include_column_options.is_empty() && !defined_source { return Err(ErrorCode::InvalidInputSyntax( diff --git a/src/prost/build.rs b/src/prost/build.rs index e031e5cfb01a..03d5801633ae 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -138,6 +138,14 @@ fn main() -> Result<(), Box> { .type_attribute("plan_common.AdditionalColumnHeader", "#[derive(Eq, Hash)]") .type_attribute("plan_common.AdditionalColumnHeaders", "#[derive(Eq, Hash)]") .type_attribute("plan_common.AdditionalColumnOffset", "#[derive(Eq, Hash)]") + .type_attribute( + "plan_common.AdditionalColumnTableName", + "#[derive(Eq, Hash)]", + ) + .type_attribute( + "plan_common.AdditionalColumnDatabaseName", + "#[derive(Eq, Hash)]", + ) .type_attribute("common.ColumnOrder", "#[derive(Eq, Hash)]") .type_attribute("common.OrderType", "#[derive(Eq, Hash)]") .type_attribute("common.Buffer", "#[derive(Eq)]") diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index 6ccff4874e75..cbaa57740d2d 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -24,8 +24,8 @@ use risingwave_common::catalog::{ColumnDesc, ColumnId}; use risingwave_common::row::RowExt; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_connector::parser::{ - DebeziumParser, DebeziumProps, EncodingProperties, JsonProperties, ProtocolProperties, - SourceStreamChunkBuilder, SpecificParserConfig, + ByteStreamSourceParser, DebeziumParser, DebeziumProps, EncodingProperties, JsonProperties, + ProtocolProperties, SourceStreamChunkBuilder, SpecificParserConfig, }; use risingwave_connector::source::cdc::external::CdcOffset; use risingwave_connector::source::{SourceColumnDesc, SourceContext}; @@ -57,8 +57,12 @@ pub struct CdcBackfillExecutor { /// The column indices need to be forwarded to the downstream from the upstream and table scan. /// User may select a subset of columns from the upstream table. + /// TODO: the output_indices always be all columns of the output schema output_indices: Vec, + /// The schema of output chunk, including additional columns if any + output_columns: Vec, + /// State table of the `CdcBackfill` executor state_impl: CdcBackfillState, @@ -81,13 +85,14 @@ impl CdcBackfillExecutor { external_table: ExternalStorageTable, upstream: Executor, output_indices: Vec, + output_columns: Vec, progress: Option, metrics: Arc, state_table: StateTable, rate_limit_rps: Option, options: CdcScanOptions, ) -> Self { - let pk_in_output_indices = external_table.pk_in_output_indices().clone().unwrap(); + let pk_in_output_indices = external_table.pk_indices(); let upstream_table_id = external_table.table_id().table_id; let state_impl = CdcBackfillState::new( upstream_table_id, @@ -100,6 +105,7 @@ impl CdcBackfillExecutor { external_table, upstream, output_indices, + output_columns, state_impl, progress, metrics, @@ -134,13 +140,13 @@ impl CdcBackfillExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { - // The primary key columns, in the output columns of the upstream_table scan. - let pk_in_output_indices = self.external_table.pk_in_output_indices().unwrap(); + // The indices to primary key columns + let pk_indices = self.external_table.pk_indices().to_vec(); let pk_order = self.external_table.pk_order_types().to_vec(); let upstream_table_id = self.external_table.table_id().table_id; let upstream_table_name = self.external_table.qualified_table_name(); - let upstream_table_schema = self.external_table.schema().clone(); + // let upstream_table_schema = self.external_table.schema().clone(); let upstream_table_reader = UpstreamTableReader::new(self.external_table); let mut upstream = self.upstream.execute(); @@ -158,7 +164,7 @@ impl CdcBackfillExecutor { // if not, we should bypass the backfill directly. let mut state_impl = self.state_impl; - let mut upstream = transform_upstream(upstream, &upstream_table_schema) + let mut upstream = transform_upstream(upstream, &self.output_columns) .boxed() .peekable(); @@ -259,7 +265,7 @@ impl CdcBackfillExecutor { let read_args = SnapshotReadArgs::new( current_pk_pos.clone(), self.rate_limit_rps, - pk_in_output_indices.clone(), + pk_indices.clone(), ); let right_snapshot = pin!(upstream_table_reader @@ -423,8 +429,7 @@ impl CdcBackfillExecutor { // Raise the current position. // As snapshot read streams are ordered by pk, so we can // just use the last row to update `current_pos`. - current_pk_pos = - Some(get_new_pos(&chunk, &pk_in_output_indices)); + current_pk_pos = Some(get_new_pos(&chunk, &pk_indices)); tracing::trace!( "got a snapshot chunk: len {}, current_pk_pos {:?}", @@ -488,7 +493,7 @@ impl CdcBackfillExecutor { } Some(chunk) => { // Raise the current pk position. - current_pk_pos = Some(get_new_pos(&chunk, &pk_in_output_indices)); + current_pk_pos = Some(get_new_pos(&chunk, &pk_indices)); let row_count = chunk.cardinality() as u64; cur_barrier_snapshot_processed_rows += row_count; @@ -522,7 +527,7 @@ impl CdcBackfillExecutor { &offset_parse_func, chunk, current_pos, - &pk_in_output_indices, + &pk_indices, &pk_order, last_binlog_offset.clone(), )?, @@ -636,7 +641,7 @@ impl CdcBackfillExecutor { } #[try_stream(ok = Message, error = StreamExecutorError)] -pub async fn transform_upstream(upstream: BoxedMessageStream, schema: &Schema) { +pub async fn transform_upstream(upstream: BoxedMessageStream, output_columns: &[ColumnDesc]) { let props = SpecificParserConfig { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { @@ -646,9 +651,16 @@ pub async fn transform_upstream(upstream: BoxedMessageStream, schema: &Schema) { // the cdc message is generated internally so the key must exist. protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()), }; + + // convert to source column desc to feed into parser + let columns_with_meta = output_columns + .iter() + .map(|col| SourceColumnDesc::from(col)) + .collect_vec(); + let mut parser = DebeziumParser::new( props, - get_rw_columns(schema), + columns_with_meta.clone(), Arc::new(SourceContext::dummy()), ) .await @@ -659,7 +671,7 @@ pub async fn transform_upstream(upstream: BoxedMessageStream, schema: &Schema) { for msg in upstream { let mut msg = msg?; if let Message::Chunk(chunk) = &mut msg { - let parsed_chunk = parse_debezium_chunk(&mut parser, chunk, schema).await?; + let parsed_chunk = parse_debezium_chunk(&mut parser, chunk).await?; let _ = std::mem::replace(chunk, parsed_chunk); } yield msg; @@ -669,14 +681,13 @@ pub async fn transform_upstream(upstream: BoxedMessageStream, schema: &Schema) { async fn parse_debezium_chunk( parser: &mut DebeziumParser, chunk: &StreamChunk, - schema: &Schema, ) -> StreamExecutorResult { // here we transform the input chunk in (payload varchar, _rw_offset varchar, _rw_table_name varchar) schema // to chunk with downstream table schema `info.schema` of MergeNode contains the schema of the // table job with `_rw_offset` in the end // see `gen_create_table_plan_for_cdc_source` for details - let column_descs = get_rw_columns(schema); - let mut builder = SourceStreamChunkBuilder::with_capacity(column_descs, chunk.capacity()); + let mut builder = + SourceStreamChunkBuilder::with_capacity(parser.columns().to_vec(), chunk.capacity()); // The schema of input chunk (payload varchar, _rw_offset varchar, _rw_table_name varchar, _row_id) // We should use the debezium parser to parse the first column, @@ -715,10 +726,10 @@ async fn parse_debezium_chunk( new_rows.push(combined); } - let data_types = schema - .fields + let data_types = parser + .columns() .iter() - .map(|field| field.data_type.clone()) + .map(|col| col.data_type.clone()) .chain(std::iter::once(DataType::Varchar)) // _rw_offset column .collect_vec(); @@ -728,21 +739,6 @@ async fn parse_debezium_chunk( )) } -fn get_rw_columns(schema: &Schema) -> Vec { - schema - .fields - .iter() - .map(|field| { - let column_desc = ColumnDesc::named( - field.name.clone(), - ColumnId::placeholder(), - field.data_type.clone(), - ); - SourceColumnDesc::from(&column_desc) - }) - .collect_vec() -} - impl Execute for CdcBackfillExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() @@ -755,7 +751,7 @@ mod tests { use futures::{pin_mut, StreamExt}; use risingwave_common::array::{DataChunk, Op, StreamChunk}; - use risingwave_common::catalog::{Field, Schema}; + use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema}; use risingwave_common::types::{DataType, Datum, JsonbVal}; use risingwave_common::util::iter_util::ZipEqFast; @@ -798,16 +794,17 @@ mod tests { tx.push_chunk(chunk); let upstream = Box::new(source).execute(); - // schema of the CDC table - let rw_schema = Schema::new(vec![ - Field::with_name(DataType::Int64, "O_ORDERKEY"), // orderkey - Field::with_name(DataType::Int64, "O_CUSTKEY"), // custkey - Field::with_name(DataType::Varchar, "O_ORDERSTATUS"), // orderstatus - Field::with_name(DataType::Decimal, "O_TOTALPRICE"), // totalprice - Field::with_name(DataType::Date, "O_ORDERDATE"), // orderdate - ]); + // schema to the debezium parser + let columns = vec![ + ColumnDesc::named("O_ORDERKEY", ColumnId::new(1), DataType::Int64), + ColumnDesc::named("O_CUSTKEY", ColumnId::new(2), DataType::Int64), + ColumnDesc::named("O_ORDERSTATUS", ColumnId::new(3), DataType::Varchar), + ColumnDesc::named("O_TOTALPRICE", ColumnId::new(4), DataType::Decimal), + ColumnDesc::named("O_ORDERDATE", ColumnId::new(5), DataType::Date), + ColumnDesc::named("commit_ts", ColumnId::new(6), DataType::Timestamptz), + ]; - let parsed_stream = transform_upstream(upstream, &rw_schema); + let parsed_stream = transform_upstream(upstream, &columns); pin_mut!(parsed_stream); // the output chunk must contain the offset column if let Some(message) = parsed_stream.next().await { diff --git a/src/stream/src/executor/backfill/cdc/upstream_table/external.rs b/src/stream/src/executor/backfill/cdc/upstream_table/external.rs index a09d8e2954d9..e486f8f9bfc4 100644 --- a/src/stream/src/executor/backfill/cdc/upstream_table/external.rs +++ b/src/stream/src/executor/backfill/cdc/upstream_table/external.rs @@ -38,8 +38,6 @@ pub struct ExternalStorageTable { /// Indices of primary key. /// Note that the index is based on the all columns of the table. pk_indices: Vec, - - output_indices: Vec, } impl ExternalStorageTable { @@ -53,7 +51,6 @@ impl ExternalStorageTable { schema: Schema, pk_order_types: Vec, pk_indices: Vec, - output_indices: Vec, ) -> Self { Self { table_id, @@ -63,7 +60,6 @@ impl ExternalStorageTable { schema, pk_order_types, pk_indices, - output_indices, } } @@ -83,16 +79,6 @@ impl ExternalStorageTable { &self.pk_indices } - /// Get the indices of the primary key columns in the output columns. - /// - /// Returns `None` if any of the primary key columns is not in the output columns. - pub fn pk_in_output_indices(&self) -> Option> { - self.pk_indices - .iter() - .map(|&i| self.output_indices.iter().position(|&j| i == j)) - .collect() - } - pub fn schema_table_name(&self) -> SchemaTableName { SchemaTableName { schema_name: self.schema_name.clone(), diff --git a/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs b/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs index d81e9e46f3bb..5fe38ef15fed 100644 --- a/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs +++ b/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs @@ -46,19 +46,19 @@ pub trait UpstreamTableRead { pub struct SnapshotReadArgs { pub current_pos: Option, pub rate_limit_rps: Option, - pub pk_in_output_indices: Vec, + pub pk_indices: Vec, } impl SnapshotReadArgs { pub fn new( current_pos: Option, rate_limit_rps: Option, - pk_in_output_indices: Vec, + pk_indices: Vec, ) -> Self { Self { current_pos, rate_limit_rps, - pk_in_output_indices, + pk_indices, } } } @@ -139,7 +139,7 @@ impl UpstreamTableRead for UpstreamTableReader { let chunk = chunk?; let chunk_size = chunk.capacity(); read_count += chunk.cardinality(); - current_pk_pos = get_new_pos(&chunk, &read_args.pk_in_output_indices); + current_pk_pos = get_new_pos(&chunk, &read_args.pk_indices); if read_args.rate_limit_rps.is_none() || chunk_size == 0 { // no limit, or empty chunk diff --git a/src/stream/src/from_proto/stream_cdc_scan.rs b/src/stream/src/from_proto/stream_cdc_scan.rs index 43ddffda0060..0f3cfdedaa6a 100644 --- a/src/stream/src/from_proto/stream_cdc_scan.rs +++ b/src/stream/src/from_proto/stream_cdc_scan.rs @@ -45,9 +45,9 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { let table_desc: &ExternalTableDesc = node.get_cdc_table_desc()?; - let table_schema: Schema = table_desc.columns.iter().map(Into::into).collect(); - assert_eq!(output_indices, (0..table_schema.len()).collect_vec()); - assert_eq!(table_schema.data_types(), params.info.schema.data_types()); + let output_schema: Schema = table_desc.columns.iter().map(Into::into).collect(); + assert_eq!(output_indices, (0..output_schema.len()).collect_vec()); + assert_eq!(output_schema.data_types(), params.info.schema.data_types()); let properties: HashMap = table_desc .connect_properties @@ -75,6 +75,19 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { ..Default::default() }); let table_type = CdcTableType::from_properties(&properties); + + // filter out additional columns + let table_schema: Schema = table_desc + .columns + .iter() + .filter(|col| { + !col.additional_column + .as_ref() + .is_some_and(|a_col| a_col.column_type.is_some()) + }) + .map(Into::into) + .collect(); + let table_reader = table_type .create_table_reader( properties.clone(), @@ -92,7 +105,6 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { table_schema, table_pk_order_types, table_pk_indices, - output_indices.clone(), ); let vnodes = params.vnode_bitmap.map(Arc::new); @@ -101,11 +113,13 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { let state_table = StateTable::from_table_catalog(node.get_state_table()?, state_store, vnodes).await; + let output_columns = table_desc.columns.iter().map(Into::into).collect_vec(); let exec = CdcBackfillExecutor::new( params.actor_context.clone(), external_table, upstream, output_indices, + output_columns, None, params.executor_stats, state_table, From e3013359a18108074cf9526b2055dd8891168b61 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 20 May 2024 16:29:56 +0800 Subject: [PATCH 03/36] support include timestamp --- src/common/src/catalog/schema.rs | 2 +- .../src/parser/debezium/debezium_parser.rs | 63 ++++++++++++++++ src/connector/src/parser/mod.rs | 71 ++++++++++--------- src/connector/src/parser/unified/debezium.rs | 9 ++- src/frontend/src/handler/create_table.rs | 2 +- .../src/executor/backfill/cdc/cdc_backfill.rs | 14 ++-- .../backfill/cdc/upstream_table/snapshot.rs | 28 +++++++- 7 files changed, 146 insertions(+), 43 deletions(-) diff --git a/src/common/src/catalog/schema.rs b/src/common/src/catalog/schema.rs index a4fe8e00d1be..113d9f804b3d 100644 --- a/src/common/src/catalog/schema.rs +++ b/src/common/src/catalog/schema.rs @@ -15,7 +15,7 @@ use std::ops::Index; use itertools::Itertools; -use risingwave_pb::plan_common::{AdditionalColumn, PbColumnDesc, PbField}; +use risingwave_pb::plan_common::{PbColumnDesc, PbField}; use super::ColumnDesc; use crate::array::ArrayBuilderImpl; diff --git a/src/connector/src/parser/debezium/debezium_parser.rs b/src/connector/src/parser/debezium/debezium_parser.rs index 52d1e4e4a15a..382f42723c62 100644 --- a/src/connector/src/parser/debezium/debezium_parser.rs +++ b/src/connector/src/parser/debezium/debezium_parser.rs @@ -198,6 +198,9 @@ mod tests { use std::sync::Arc; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; + use risingwave_pb::plan_common::{ + additional_column, AdditionalColumn, AdditionalColumnTimestamp, + }; use super::*; use crate::parser::{JsonProperties, SourceStreamChunkBuilder, TransactionControl}; @@ -266,4 +269,64 @@ mod tests { _ => panic!("unexpected parse result: {:?}", res), } } + + #[tokio::test] + async fn test_parse_additional_columns() { + let columns = vec![ + ColumnDesc::named("O_ORDERKEY", ColumnId::new(1), DataType::Int64), + ColumnDesc::named("O_CUSTKEY", ColumnId::new(2), DataType::Int64), + ColumnDesc::named("O_ORDERSTATUS", ColumnId::new(3), DataType::Varchar), + ColumnDesc::named("O_TOTALPRICE", ColumnId::new(4), DataType::Decimal), + ColumnDesc::named("O_ORDERDATE", ColumnId::new(5), DataType::Date), + ColumnDesc::named_with_additional_column( + "commit_ts", + ColumnId::new(6), + DataType::Timestamptz, + AdditionalColumn { + column_type: Some(additional_column::ColumnType::Timestamp( + AdditionalColumnTimestamp {}, + )), + }, + ), + ]; + + let columns = columns + .iter() + .map(|c| SourceColumnDesc::from(c)) + .collect::>(); + + let props = SpecificParserConfig { + key_encoding_config: None, + encoding_config: EncodingProperties::Json(JsonProperties { + use_schema_registry: false, + timestamptz_handling: None, + }), + protocol_config: ProtocolProperties::Debezium(DebeziumProps::default()), + }; + let source_ctx = SourceContext { + connector_props: ConnectorProperties::PostgresCdc(Box::default()), + ..SourceContext::dummy() + }; + let mut parser = DebeziumParser::new(props, columns.clone(), Arc::new(source_ctx)) + .await + .unwrap(); + let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 0); + + let payload = r#"{ "payload": { "before": null, "after": { "O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" }, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null }, "op": "c", "ts_ms": 1695277757017, "transaction": null } }"#; + + let res = parser + .parse_one_with_txn( + None, + Some(payload.as_bytes().to_vec()), + builder.row_writer(), + ) + .await; + match res { + Ok(ParseResult::Rows) => { + let chunk = builder.finish(); + println!("chunk: {:#?}", chunk) + } + _ => panic!("unexpected parse result: {:?}", res), + } + } } diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 08e8bd3dd46e..16e5b5425ea7 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -345,6 +345,35 @@ impl SourceStreamChunkRowWriter<'_> { &mut self, mut f: impl FnMut(&SourceColumnDesc) -> AccessResult, ) -> AccessResult<()> { + let mut parse_field = |desc: &SourceColumnDesc| { + 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 + static LOG_SUPPERSSER: LazyLock = + LazyLock::new(LogSuppresser::default); + if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { + tracing::warn!( + error = %error.as_report(), + 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, + suppressed_count, + "failed to parse non-pk column, padding with `NULL`" + ); + } + Ok(A::output_for(Datum::None)) + } + } + }; + let mut wrapped_f = |desc: &SourceColumnDesc| { match (&desc.column_type, &desc.additional_column.column_type) { (&SourceColumnType::Offset | &SourceColumnType::RowId, _) => { @@ -370,14 +399,12 @@ impl SourceStreamChunkRowWriter<'_> { .unwrap(), // handled all match cases in internal match, unwrap is safe )); } - (_, &Some(AdditionalColumnType::Timestamp(_))) => { - return Ok(A::output_for( - self.row_meta - .as_ref() - .and_then(|ele| extreact_timestamp_from_meta(ele.meta)) - .unwrap_or(None), - )) - } + (_, &Some(AdditionalColumnType::Timestamp(_))) => match self.row_meta { + Some(row_meta) => Ok(A::output_for( + extreact_timestamp_from_meta(row_meta.meta).unwrap_or(None), + )), + None => parse_field(desc), // parse from payload + }, (_, &Some(AdditionalColumnType::Partition(_))) => { // the meta info does not involve spec connector return Ok(A::output_for( @@ -426,32 +453,7 @@ impl SourceStreamChunkRowWriter<'_> { } (_, _) => { // 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 - static LOG_SUPPERSSER: LazyLock = - LazyLock::new(LogSuppresser::default); - if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { - tracing::warn!( - error = %error.as_report(), - 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, - suppressed_count, - "failed to parse non-pk column, padding with `NULL`" - ); - } - Ok(A::output_for(Datum::None)) - } - } + parse_field(desc) } } }; @@ -462,6 +464,7 @@ impl SourceStreamChunkRowWriter<'_> { let result = (self.descs.iter()) .zip_eq_fast(self.builders.iter_mut()) .try_for_each(|(desc, builder)| { + println!("desc: name = {}", desc.name); wrapped_f(desc).map(|output| { A::apply(builder, output); applied_columns.push(builder); diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index a70d629f8172..83d253e9cf64 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -198,11 +198,18 @@ where |additional_column_type| { match additional_column_type { &ColumnType::Timestamp(_) => { + tracing::info!( + "access timestamp field: `{}, type: {:?}, column_type: {:?}`", + desc.name, + desc.data_type, + desc.additional_column.column_type + ); + // access payload.source.ts_ms self.value_accessor .as_ref() .expect("value_accessor must be provided for upsert operation") - .access(&[SOURCE, &SOURCE_TS_MS], Some(&desc.data_type)) + .access(&[SOURCE, SOURCE_TS_MS], Some(&desc.data_type)) } _ => Err(AccessError::UnsupportedAdditionalColumn { name: desc.name.clone(), diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index b978ced72a5b..5bc6b8930e53 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -56,7 +56,7 @@ use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; use crate::handler::create_source::{ bind_columns_from_source, bind_connector_props, bind_create_source, bind_source_watermark, - UPSTREAM_SOURCE_KEY, + handle_addition_columns, UPSTREAM_SOURCE_KEY, }; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::generic::{CdcScanOptions, SourceNodeKind}; diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index cbaa57740d2d..2d54ad6c6b48 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -20,7 +20,7 @@ use futures::stream::select_with_strategy; use itertools::Itertools; use risingwave_common::array::DataChunk; use risingwave_common::bail; -use risingwave_common::catalog::{ColumnDesc, ColumnId}; +use risingwave_common::catalog::ColumnDesc; use risingwave_common::row::RowExt; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_connector::parser::{ @@ -56,8 +56,6 @@ pub struct CdcBackfillExecutor { upstream: Executor, /// The column indices need to be forwarded to the downstream from the upstream and table scan. - /// User may select a subset of columns from the upstream table. - /// TODO: the output_indices always be all columns of the output schema output_indices: Vec, /// The schema of output chunk, including additional columns if any @@ -149,6 +147,13 @@ impl CdcBackfillExecutor { // let upstream_table_schema = self.external_table.schema().clone(); let upstream_table_reader = UpstreamTableReader::new(self.external_table); + let additional_columns = self + .output_columns + .iter() + .filter(|col| col.additional_column.column_type.is_some()) + .cloned() + .collect_vec(); + let mut upstream = self.upstream.execute(); // Current position of the upstream_table storage primary key. @@ -266,6 +271,7 @@ impl CdcBackfillExecutor { current_pk_pos.clone(), self.rate_limit_rps, pk_indices.clone(), + additional_columns.clone(), ); let right_snapshot = pin!(upstream_table_reader @@ -655,7 +661,7 @@ pub async fn transform_upstream(upstream: BoxedMessageStream, output_columns: &[ // convert to source column desc to feed into parser let columns_with_meta = output_columns .iter() - .map(|col| SourceColumnDesc::from(col)) + .map(SourceColumnDesc::from) .collect_vec(); let mut parser = DebeziumParser::new( diff --git a/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs b/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs index 5fe38ef15fed..ecca34bdfae7 100644 --- a/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs +++ b/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs @@ -21,6 +21,7 @@ use governor::clock::MonotonicClock; use governor::{Quota, RateLimiter}; use itertools::Itertools; use risingwave_common::array::StreamChunk; +use risingwave_common::catalog::ColumnDesc; use risingwave_common::row::OwnedRow; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_connector::source::cdc::external::{CdcOffset, ExternalTableReader}; @@ -47,6 +48,7 @@ pub struct SnapshotReadArgs { pub current_pos: Option, pub rate_limit_rps: Option, pub pk_indices: Vec, + pub additional_columns: Vec, } impl SnapshotReadArgs { @@ -54,11 +56,13 @@ impl SnapshotReadArgs { current_pos: Option, rate_limit_rps: Option, pk_indices: Vec, + additional_columns: Vec, ) -> Self { Self { current_pos, rate_limit_rps, pk_indices, + additional_columns, } } } @@ -80,6 +84,20 @@ impl UpstreamTableReader { } } +/// Append additional columns with value as null to the snapshot chunk +fn with_additional_columns( + snapshot_chunk: StreamChunk, + additional_columns: &[ColumnDesc], +) -> StreamChunk { + let (ops, mut columns, visibility) = snapshot_chunk.into_inner(); + for desc in additional_columns { + let mut builder = desc.data_type.create_array_builder(visibility.len()); + builder.append_n_null(visibility.len()); + columns.push(builder.finish().into()); + } + StreamChunk::with_visibility(ops, columns, visibility) +} + impl UpstreamTableRead for UpstreamTableReader { #[try_stream(ok = Option, error = StreamExecutorError)] async fn snapshot_read_full_table(&self, args: SnapshotReadArgs, batch_size: u32) { @@ -143,7 +161,10 @@ impl UpstreamTableRead for UpstreamTableReader { if read_args.rate_limit_rps.is_none() || chunk_size == 0 { // no limit, or empty chunk - yield Some(chunk); + yield Some(with_additional_columns( + chunk, + &read_args.additional_columns, + )); continue; } else { // Apply rate limit, see `risingwave_stream::executor::source::apply_rate_limit` for more. @@ -160,7 +181,10 @@ impl UpstreamTableRead for UpstreamTableReader { .until_n_ready(NonZeroU32::new(chunk_size as u32).unwrap()) .await .unwrap(); - yield Some(chunk); + yield Some(with_additional_columns( + chunk, + &read_args.additional_columns, + )); } } From 1a4f87146461f5e8d8a733922cc616290f81c90d Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 20 May 2024 16:40:18 +0800 Subject: [PATCH 04/36] clean code --- proto/plan_common.proto | 8 -------- src/common/src/catalog/external_table.rs | 6 ------ src/frontend/src/handler/create_source.rs | 2 -- src/frontend/src/handler/create_table.rs | 2 -- src/prost/build.rs | 8 -------- src/stream/src/executor/backfill/cdc/cdc_backfill.rs | 1 - 6 files changed, 27 deletions(-) diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 91c71ffa467f..8c73374a4d7d 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -108,8 +108,6 @@ message ExternalTableDesc { map connect_properties = 6; // upstream cdc source job id uint32 source_id = 7; - // Indices to the additional columns in the columns list. - repeated uint32 additional_column_indices = 8; } enum JoinType { @@ -200,10 +198,6 @@ message AdditionalColumnOffset {} message AdditionalColumnFilename {} -message AdditionalColumnTableName {} - -message AdditionalColumnDatabaseName {} - message AdditionalColumnHeader { string inner_field = 1; data.DataType data_type = 2; @@ -221,8 +215,6 @@ message AdditionalColumn { AdditionalColumnHeader header_inner = 5; AdditionalColumnFilename filename = 6; AdditionalColumnHeaders headers = 7; - AdditionalColumnTableName table_name = 8; - AdditionalColumnDatabaseName database_name = 9; } } diff --git a/src/common/src/catalog/external_table.rs b/src/common/src/catalog/external_table.rs index 1c0a914f6820..f9c644835806 100644 --- a/src/common/src/catalog/external_table.rs +++ b/src/common/src/catalog/external_table.rs @@ -37,7 +37,6 @@ pub struct CdcTableDesc { /// All columns in the table, noticed it is NOT sorted by columnId in the vec. pub columns: Vec, - pub additional_column_indices: Vec, /// Column indices for primary keys. pub stream_key: Vec, @@ -66,11 +65,6 @@ impl CdcTableDesc { table_name: self.external_table_name.clone(), stream_key: self.stream_key.iter().map(|k| *k as _).collect(), connect_properties: self.connect_properties.clone(), - additional_column_indices: self - .additional_column_indices - .iter() - .map(|k| *k as _) - .collect(), } } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 8c8f2a97414d..49f3126f53b9 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -571,7 +571,6 @@ pub fn handle_addition_columns( .max() .unwrap(); // there must be at least one column in the column catalog - let mut additional_column_indices = Vec::new(); while let Some(item) = additional_columns.pop() { { // only allow header column have inner field @@ -588,7 +587,6 @@ pub fn handle_addition_columns( let data_type_name: Option = item .header_inner_expect_type .map(|dt| format!("{:?}", dt).to_lowercase()); - additional_column_indices.push(columns.len()); columns.push(build_additional_column_catalog( latest_col_id.next(), connector_name.as_str(), diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 5bc6b8930e53..528fc24768fc 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -767,7 +767,6 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( }; let mut columns = bind_sql_columns(&column_defs)?; - let table_columns_len = columns.len(); let with_properties = source.with_properties.clone().into_iter().collect(); // append additional columns to the end handle_addition_columns(&with_properties, include_column_options, &mut columns)?; @@ -810,7 +809,6 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( external_table_name: external_table_name.clone(), pk: table_pk, columns: columns.iter().map(|c| c.column_desc.clone()).collect(), - additional_column_indices: (table_columns_len..columns.len()).collect(), stream_key: pk_column_indices, connect_properties, }; diff --git a/src/prost/build.rs b/src/prost/build.rs index 03d5801633ae..e031e5cfb01a 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -138,14 +138,6 @@ fn main() -> Result<(), Box> { .type_attribute("plan_common.AdditionalColumnHeader", "#[derive(Eq, Hash)]") .type_attribute("plan_common.AdditionalColumnHeaders", "#[derive(Eq, Hash)]") .type_attribute("plan_common.AdditionalColumnOffset", "#[derive(Eq, Hash)]") - .type_attribute( - "plan_common.AdditionalColumnTableName", - "#[derive(Eq, Hash)]", - ) - .type_attribute( - "plan_common.AdditionalColumnDatabaseName", - "#[derive(Eq, Hash)]", - ) .type_attribute("common.ColumnOrder", "#[derive(Eq, Hash)]") .type_attribute("common.OrderType", "#[derive(Eq, Hash)]") .type_attribute("common.Buffer", "#[derive(Eq)]") diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index 2d54ad6c6b48..1db978e684ec 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -144,7 +144,6 @@ impl CdcBackfillExecutor { let upstream_table_id = self.external_table.table_id().table_id; let upstream_table_name = self.external_table.qualified_table_name(); - // let upstream_table_schema = self.external_table.schema().clone(); let upstream_table_reader = UpstreamTableReader::new(self.external_table); let additional_columns = self From 05520e8df2a2d406c8aba52eb31ed43463f0d272 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 20 May 2024 16:47:23 +0800 Subject: [PATCH 05/36] minor --- src/connector/src/parser/mod.rs | 1 - src/frontend/src/handler/create_table.rs | 1 + src/stream/src/executor/backfill/cdc/cdc_backfill.rs | 4 ++-- src/stream/src/from_proto/stream_cdc_scan.rs | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 16e5b5425ea7..801930de3cb7 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -464,7 +464,6 @@ impl SourceStreamChunkRowWriter<'_> { let result = (self.descs.iter()) .zip_eq_fast(self.builders.iter_mut()) .try_for_each(|(desc, builder)| { - println!("desc: name = {}", desc.name); wrapped_f(desc).map(|output| { A::apply(builder, output); applied_columns.push(builder); diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 528fc24768fc..870b761348de 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -1102,6 +1102,7 @@ pub fn check_create_table_with_source( include_column_options: &IncludeOption, cdc_table_info: &Option, ) -> Result> { + // skip check for cdc table if cdc_table_info.is_some() { return Ok(source_schema); } diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index 1db978e684ec..584a74a0c03b 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -90,12 +90,12 @@ impl CdcBackfillExecutor { rate_limit_rps: Option, options: CdcScanOptions, ) -> Self { - let pk_in_output_indices = external_table.pk_indices(); + let pk_indices = external_table.pk_indices(); let upstream_table_id = external_table.table_id().table_id; let state_impl = CdcBackfillState::new( upstream_table_id, state_table, - pk_in_output_indices.len() + METADATA_STATE_LEN, + pk_indices.len() + METADATA_STATE_LEN, ); Self { diff --git a/src/stream/src/from_proto/stream_cdc_scan.rs b/src/stream/src/from_proto/stream_cdc_scan.rs index 0f3cfdedaa6a..55698c44e5e7 100644 --- a/src/stream/src/from_proto/stream_cdc_scan.rs +++ b/src/stream/src/from_proto/stream_cdc_scan.rs @@ -76,7 +76,7 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { }); let table_type = CdcTableType::from_properties(&properties); - // filter out additional columns + // Filter out additional columns to construct the external table schema let table_schema: Schema = table_desc .columns .iter() From 35a35d0cb3e40334ebd30274d7f4df067ee87ff3 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 20 May 2024 17:53:22 +0800 Subject: [PATCH 06/36] fix --- src/compute/tests/cdc_tests.rs | 7 ++++++- src/connector/src/parser/debezium/debezium_parser.rs | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index 76b2047cd492..628c2ca22cf4 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -183,7 +183,6 @@ async fn test_cdc_backfill() -> StreamResult<()> { table_schema.clone(), table_pk_order_types, table_pk_indices.clone(), - vec![0, 1], ); let actor_id = 0x1a; @@ -214,6 +213,11 @@ async fn test_cdc_backfill() -> StreamResult<()> { ) .await; + let output_columns = vec![ + ColumnDesc::named("id", ColumnId::new(1), DataType::Int64), // primary key + ColumnDesc::named("price", ColumnId::new(2), DataType::Float64), + ]; + let cdc_backfill = StreamExecutor::new( ExecutorInfo { schema: table_schema.clone(), @@ -225,6 +229,7 @@ async fn test_cdc_backfill() -> StreamResult<()> { external_table, mock_offset_executor, vec![0, 1], + output_columns, None, Arc::new(StreamingMetrics::unused()), state_table, diff --git a/src/connector/src/parser/debezium/debezium_parser.rs b/src/connector/src/parser/debezium/debezium_parser.rs index 382f42723c62..479cd46bd86e 100644 --- a/src/connector/src/parser/debezium/debezium_parser.rs +++ b/src/connector/src/parser/debezium/debezium_parser.rs @@ -292,7 +292,7 @@ mod tests { let columns = columns .iter() - .map(|c| SourceColumnDesc::from(c)) + .map(SourceColumnDesc::from) .collect::>(); let props = SpecificParserConfig { From f3e2b152bc4781d17f8c0bfd96966465d2bbcbbd Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 21 May 2024 14:24:25 +0800 Subject: [PATCH 07/36] e2e test for pg and mysql --- e2e_test/source/cdc/cdc.share_stream.slt | 36 ++++++++++++++++--- .../src/parser/debezium/debezium_parser.rs | 7 +++- .../backfill/cdc/upstream_table/snapshot.rs | 14 +++++++- 3 files changed, 51 insertions(+), 6 deletions(-) diff --git a/e2e_test/source/cdc/cdc.share_stream.slt b/e2e_test/source/cdc/cdc.share_stream.slt index 480c707fb6f4..43be0d2dc68e 100644 --- a/e2e_test/source/cdc/cdc.share_stream.slt +++ b/e2e_test/source/cdc/cdc.share_stream.slt @@ -41,7 +41,7 @@ create table rw.products_test ( id INT, name STRING, description STRING, PRIMARY KEY (id) -) from mysql_mytest table 'mytest.products'; +) include timestamp as commit_ts from mysql_mytest table 'mytest.products'; system ok mysql --protocol=tcp -u root mytest -e "INSERT INTO products VALUES (default,'Milk','Milk is a white liquid food'); @@ -145,12 +145,24 @@ SELECT * from orders_test_cnt 4 query ITT -SELECT * FROM rw.products_test order by id limit 3 +SELECT id,name,description FROM rw.products_test order by id limit 3 ---- 101 scooter Small 2-wheel scooter 102 car battery 12V car battery 103 12-pack drill bits 12-pack of drill bits with sizes ranging from #40 to #3 +# commit_ts of historical records should be '1970-01-01 00:00:00+00:00' +query I +SELECT count(*) as cnt from rw.products_test where commit_ts = '1970-01-01 00:00:00+00:00' +---- +9 + +# commit_ts of new records should greater than '1970-01-01 00:00:00+00:00' +query TTT +SELECT name,description FROM rw.products_test order by id where commit_ts > '1970-01-01 00:00:00+00:00' +---- +'Juice', '100ml Juice' + query ITTT SELECT order_id,order_date,customer_name,product_id FROM orders_test order by order_id limit 3 ---- @@ -230,7 +242,7 @@ CREATE TABLE person_new ( credit_card varchar, city varchar, PRIMARY KEY (id) -) FROM pg_source TABLE 'public.person'; +) INCLUDE TIMESTAMP AS commit_ts FROM pg_source TABLE 'public.person'; statement ok CREATE MATERIALIZED VIEW person_new_cnt AS SELECT COUNT(*) AS cnt FROM person_new; @@ -260,7 +272,7 @@ SELECT * from person_new_cnt 6 query ITTTT -SELECT * from person_new order by id; +SELECT id,name,email_address,credit_card,city from person_new order by id; ---- 1000 vicky noris yplkvgz@qbxfg.com 7878 5821 1864 2539 cheyenne 1001 peter white myckhsp@xpmpe.com 1781 2313 8157 6974 boise @@ -269,6 +281,22 @@ SELECT * from person_new order by id; 1101 white myc@xpmpe.com 8157 6974 se 1102 spencer wip@dkaap.com 9481 6270 angeles +# historical data +query ITTTT +SELECT id,name,email_address,credit_card,city from person_new order by id where commit_ts = '1970-01-01 00:00:00+00:00'; +---- +1000 vicky noris yplkvgz@qbxfg.com 7878 5821 1864 2539 cheyenne +1001 peter white myckhsp@xpmpe.com 1781 2313 8157 6974 boise +1002 sarah spencer wipvdbm@dkaap.com 3453 4987 9481 6270 los angeles + +# incremental data +query ITTTT +SELECT id,name,email_address,credit_card,city from person_new order by id where commit_ts > '1970-01-01 00:00:00+00:00'; +---- +1100 noris ypl@qbxfg.com 1864 2539 enne +1101 white myc@xpmpe.com 8157 6974 se +1102 spencer wip@dkaap.com 9481 6270 angeles + statement ok CREATE TABLE numeric_to_rw_int256_shared ( id int, diff --git a/src/connector/src/parser/debezium/debezium_parser.rs b/src/connector/src/parser/debezium/debezium_parser.rs index 479cd46bd86e..817c2a788f2b 100644 --- a/src/connector/src/parser/debezium/debezium_parser.rs +++ b/src/connector/src/parser/debezium/debezium_parser.rs @@ -198,6 +198,8 @@ mod tests { use std::sync::Arc; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId}; + use risingwave_common::row::Row; + use risingwave_common::types::Timestamptz; use risingwave_pb::plan_common::{ additional_column, AdditionalColumn, AdditionalColumnTimestamp, }; @@ -324,7 +326,10 @@ mod tests { match res { Ok(ParseResult::Rows) => { let chunk = builder.finish(); - println!("chunk: {:#?}", chunk) + for (_, row) in chunk.rows() { + let commit_ts = row.datum_at(5).unwrap().into_timestamptz(); + assert_eq!(commit_ts, Timestamptz::from_millis(1695277757000).unwrap()); + } } _ => panic!("unexpected parse result: {:?}", res), } diff --git a/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs b/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs index ecca34bdfae7..548b6a5feec7 100644 --- a/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs +++ b/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs @@ -23,8 +23,10 @@ use itertools::Itertools; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::ColumnDesc; use risingwave_common::row::OwnedRow; +use risingwave_common::types::{Scalar, Timestamptz}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_connector::source::cdc::external::{CdcOffset, ExternalTableReader}; +use risingwave_pb::plan_common::additional_column::ColumnType; use super::external::ExternalStorageTable; use crate::common::rate_limit::limited_chunk_size; @@ -92,7 +94,17 @@ fn with_additional_columns( let (ops, mut columns, visibility) = snapshot_chunk.into_inner(); for desc in additional_columns { let mut builder = desc.data_type.create_array_builder(visibility.len()); - builder.append_n_null(visibility.len()); + match desc.additional_column.column_type.as_ref().unwrap() { + // set default value for timestamp + &ColumnType::Timestamp(_) => builder.append_n( + visibility.len(), + Some(Timestamptz::default().to_scalar_value()), + ), + // set null for other additional columns + _ => { + builder.append_n_null(visibility.len()); + } + } columns.push(builder.finish().into()); } StreamChunk::with_visibility(ops, columns, visibility) From e27684ee9885fb79c071751852f3dd387e49c84d Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 21 May 2024 16:24:29 +0800 Subject: [PATCH 08/36] include timestamp for mongo --- e2e_test/source/cdc/mongodb/mongodb_basic.slt | 8 +++++++- .../src/parser/additional_columns.rs | 6 +++++- .../src/parser/debezium/mongo_json_parser.rs | 19 +++++++++++-------- src/connector/src/parser/mod.rs | 3 ++- src/connector/src/parser/unified/json.rs | 1 + src/connector/src/parser/util.rs | 1 + .../src/source/cdc/source/message.rs | 12 ++++++++++++ src/frontend/src/handler/create_source.rs | 6 ------ 8 files changed, 39 insertions(+), 17 deletions(-) diff --git a/e2e_test/source/cdc/mongodb/mongodb_basic.slt b/e2e_test/source/cdc/mongodb/mongodb_basic.slt index f3a815df0572..9eaad3cca41a 100644 --- a/e2e_test/source/cdc/mongodb/mongodb_basic.slt +++ b/e2e_test/source/cdc/mongodb/mongodb_basic.slt @@ -2,7 +2,7 @@ control substitution on statement ok -CREATE TABLE users (_id JSONB PRIMARY KEY, payload JSONB) WITH ( +CREATE TABLE users (_id JSONB PRIMARY KEY, payload JSONB) INCLUDE TIMESTAMP as commit_ts WITH ( connector = 'mongodb-cdc', mongodb.url = 'mongodb://mongodb:27017/?replicaSet=rs0', collection.name = 'random_data.*' @@ -24,5 +24,11 @@ select count(*) from normalized_users; ---- 55 +# historical data +query I +select count(*) from users where commit_ts = '1970-01-01 00:00:00+00:00'; +---- +55 + statement ok DROP TABLE users cascade diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index d30698155c9f..a7be80128bad 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -28,7 +28,7 @@ use risingwave_pb::plan_common::{ }; use crate::error::ConnectorResult; -use crate::source::cdc::{MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR}; +use crate::source::cdc::{MONGODB_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR}; use crate::source::{ GCS_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, @@ -64,6 +64,10 @@ pub static COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock anyhow::Result { match config { - EncodingProperties::MongoJson(_) => Ok(AccessBuilderImpl::DebeziumMongoJson( + EncodingProperties::MongoJson => Ok(AccessBuilderImpl::DebeziumMongoJson( DebeziumMongoJsonAccessBuilder::new()?, )), _ => bail!("unsupported encoding for DEBEZIUM_MONGO format"), @@ -72,15 +72,18 @@ impl DebeziumMongoJsonParser { .clone(); // _rw_{connector}_file/partition & _rw_{connector}_offset are created automatically. - if rw_columns.iter().filter(|desc| desc.is_visible()).count() != 2 { - bail!("Debezium Mongo needs no more columns except `_id` and `payload` in table"); + if rw_columns + .iter() + .filter(|desc| desc.is_visible() && desc.additional_column.column_type.is_none()) + .count() + != 2 + { + bail!("Debezium Mongo needs no more data columns except `_id` and `payload` in table"); } // encodings are fixed to MongoJson - let key_builder = - build_accessor_builder(EncodingProperties::MongoJson(JsonProperties::default()))?; - let payload_builder = - build_accessor_builder(EncodingProperties::MongoJson(JsonProperties::default()))?; + let key_builder = build_accessor_builder(EncodingProperties::MongoJson)?; + let payload_builder = build_accessor_builder(EncodingProperties::MongoJson)?; Ok(Self { rw_columns, diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 801930de3cb7..13ddcca59272 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -643,6 +643,7 @@ const MAX_ROWS_FOR_TRANSACTION: usize = 4096; #[try_stream(ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_chunk_stream(mut parser: P, data_stream: BoxSourceStream) { let columns = parser.columns().to_vec(); + tracing::info!("parser columns: {:?}", columns); let mut heartbeat_builder = SourceStreamChunkBuilder::with_capacity(columns.clone(), 0); let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 0); @@ -1018,7 +1019,7 @@ pub enum EncodingProperties { Protobuf(ProtobufProperties), Csv(CsvProperties), Json(JsonProperties), - MongoJson(JsonProperties), + MongoJson, Bytes(BytesProperties), Native, /// Encoding can't be specified because the source will determines it. Now only used in Iceberg. diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index 11c569832268..6dd406df7546 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -644,6 +644,7 @@ where 'a: 'b, { fn access(&self, path: &[&str], type_expected: Option<&DataType>) -> AccessResult { + tracing::info!("mongo access path: {:?}", path); let mut value = &self.value; for (idx, &key) in path.iter().enumerate() { if let Some(sub_value) = if self.options.ignoring_keycase { diff --git a/src/connector/src/parser/util.rs b/src/connector/src/parser/util.rs index 27eb7a8144bd..3b416ef1309e 100644 --- a/src/connector/src/parser/util.rs +++ b/src/connector/src/parser/util.rs @@ -130,6 +130,7 @@ pub(super) async fn bytes_from_url( pub fn extreact_timestamp_from_meta(meta: &SourceMeta) -> Option { match meta { SourceMeta::Kafka(kafka_meta) => kafka_meta.extract_timestamp(), + SourceMeta::DebeziumCdc(debezium_meta) => debezium_meta.extract_timestamp(), _ => None, } } diff --git a/src/connector/src/source/cdc/source/message.rs b/src/connector/src/source/cdc/source/message.rs index 5df937a83fbe..f45a4e37e8ca 100644 --- a/src/connector/src/source/cdc/source/message.rs +++ b/src/connector/src/source/cdc/source/message.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::types::{Datum, Scalar, Timestamptz}; use risingwave_pb::connector_service::CdcMessage; use crate::source::base::SourceMessage; @@ -26,6 +27,17 @@ pub struct DebeziumCdcMeta { pub is_transaction_meta: bool, } +impl DebeziumCdcMeta { + pub fn extract_timestamp(&self) -> Option { + Some( + Timestamptz::from_millis(self.source_ts_ms) + .unwrap() + .to_scalar_value(), + ) + .into() + } +} + impl From for SourceMessage { fn from(message: CdcMessage) -> Self { SourceMessage { diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 49f3126f53b9..5ae33c195d41 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -838,12 +838,6 @@ 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 { From b75c56b5742619f3169e5684f2085357f84cc28b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 22 May 2024 15:58:47 +0800 Subject: [PATCH 09/36] fix comment --- src/connector/src/parser/unified/debezium.rs | 12 +++++++++--- src/connector/src/parser/unified/json.rs | 1 - 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 83d253e9cf64..e772220f9f0c 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::types::{DataType, Datum, ScalarImpl}; +use risingwave_common::types::{DataType, Datum, Scalar, ScalarImpl, Timestamptz}; use risingwave_pb::plan_common::additional_column::ColumnType; use super::{Access, AccessError, AccessResult, ChangeEvent, ChangeEventOperation}; @@ -206,10 +206,16 @@ where ); // access payload.source.ts_ms - self.value_accessor + let ts_ms = self + .value_accessor .as_ref() .expect("value_accessor must be provided for upsert operation") - .access(&[SOURCE, SOURCE_TS_MS], Some(&desc.data_type)) + .access(&[SOURCE, SOURCE_TS_MS], Some(&DataType::Int64))?; + Ok(ts_ms.map(|scalar| { + Timestamptz::from_millis(scalar.into_int64()) + .expect("source.ts_ms must in millisecond") + .to_scalar_value() + })) } _ => Err(AccessError::UnsupportedAdditionalColumn { name: desc.name.clone(), diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index 6dd406df7546..11c569832268 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -644,7 +644,6 @@ where 'a: 'b, { fn access(&self, path: &[&str], type_expected: Option<&DataType>) -> AccessResult { - tracing::info!("mongo access path: {:?}", path); let mut value = &self.value; for (idx, &key) in path.iter().enumerate() { if let Some(sub_value) = if self.options.ignoring_keycase { From ccf2f8ced2b264398d61a29677154c2f802698de Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 23 May 2024 11:45:51 +0800 Subject: [PATCH 10/36] fix --- e2e_test/source/cdc_inline/postgres_create_drop.slt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/e2e_test/source/cdc_inline/postgres_create_drop.slt b/e2e_test/source/cdc_inline/postgres_create_drop.slt index 334f1eb2c9cc..471a19192e6f 100644 --- a/e2e_test/source/cdc_inline/postgres_create_drop.slt +++ b/e2e_test/source/cdc_inline/postgres_create_drop.slt @@ -26,6 +26,11 @@ create table tt1 (v1 int, statement ok drop table tt1; +statement ok +FLUSH; + +sleep 3s + statement ok create table tt1 (v1 int, v2 timestamptz, From 41c3a49bea1028321ef96b4969e447536507775f Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 23 May 2024 14:27:29 +0800 Subject: [PATCH 11/36] fix ci --- ci/scripts/e2e-source-test.sh | 3 ++- e2e_test/source/cdc_inline/postgres_create_drop.slt | 12 +++++++----- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 76e0900c32fe..5127731256c6 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -32,6 +32,7 @@ mkdir ./connector-node tar xf ./risingwave-connector.tar.gz -C ./connector-node echo "--- e2e, inline test" +RUST_LOG="debug,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ risedev ci-start ci-inline-source-test risedev slt './e2e_test/source_inline/**/*.slt' echo "--- Kill cluster" @@ -55,7 +56,7 @@ createdb psql < ./e2e_test/source/cdc/postgres_cdc.sql echo "--- starting risingwave cluster" -RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ +RUST_LOG="debug,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ risedev ci-start ci-1cn-1fe-with-recovery echo "--- mongodb cdc test" diff --git a/e2e_test/source/cdc_inline/postgres_create_drop.slt b/e2e_test/source/cdc_inline/postgres_create_drop.slt index 471a19192e6f..ad7425bab3df 100644 --- a/e2e_test/source/cdc_inline/postgres_create_drop.slt +++ b/e2e_test/source/cdc_inline/postgres_create_drop.slt @@ -23,13 +23,15 @@ create table tt1 (v1 int, slot.name = 'tt1_slot', ); -statement ok -drop table tt1; +sleep 3s -statement ok -FLUSH; +query IT +SELECT * FROM tt1; +---- +1 2023-10-23 10:00:00+00:00 -sleep 3s +statement ok +drop table tt1; statement ok create table tt1 (v1 int, From d98c1f7de38a28e569a857187a87b76413858554 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 23 May 2024 17:14:27 +0800 Subject: [PATCH 12/36] fix validate --- .../risingwave/connector/api/TableSchema.java | 31 ++++++++++++------- .../src/source/cdc/enumerator/mod.rs | 5 +-- src/connector/src/source/cdc/mod.rs | 16 ++++++++++ 3 files changed, 38 insertions(+), 14 deletions(-) diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java index 324fd05de595..c7e9f7035f1e 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/TableSchema.java @@ -20,12 +20,18 @@ import com.risingwave.proto.Data.DataType.TypeName; import com.risingwave.proto.PlanCommon; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class TableSchema { + + static final Logger LOG = LoggerFactory.getLogger(TableSchema.class); + private final List columnNames; private final Map columns; private final Map columnIndices; @@ -81,18 +87,19 @@ public Object getFromRow(String columnName, SinkRow row) { public static TableSchema fromProto(ConnectorServiceProto.TableSchema tableSchema) { // filter out additional columns - var columns = - tableSchema.getColumnsList().stream() - .filter(col -> !col.hasAdditionalColumn()) - .collect(Collectors.toList()); - return new TableSchema( - columns.stream().map(PlanCommon.ColumnDesc::getName).collect(Collectors.toList()), - columns.stream() - .map(PlanCommon.ColumnDesc::getColumnType) - .collect(Collectors.toList()), - tableSchema.getPkIndicesList().stream() - .map(i -> tableSchema.getColumns(i).getName()) - .collect(Collectors.toList())); + var instance = + new TableSchema( + tableSchema.getColumnsList().stream() + .map(PlanCommon.ColumnDesc::getName) + .collect(Collectors.toList()), + tableSchema.getColumnsList().stream() + .map(PlanCommon.ColumnDesc::getColumnType) + .collect(Collectors.toList()), + tableSchema.getPkIndicesList().stream() + .map(i -> tableSchema.getColumns(i).getName()) + .collect(Collectors.toList())); + LOG.info("table column names: {}", Arrays.toString(instance.getColumnNames())); + return instance; } public List getPrimaryKeys() { diff --git a/src/connector/src/source/cdc/enumerator/mod.rs b/src/connector/src/source/cdc/enumerator/mod.rs index 98018b6b6a11..bfe26c57a834 100644 --- a/src/connector/src/source/cdc/enumerator/mod.rs +++ b/src/connector/src/source/cdc/enumerator/mod.rs @@ -27,7 +27,8 @@ use risingwave_pb::connector_service::{SourceType, ValidateSourceRequest, Valida use crate::error::ConnectorResult; use crate::source::cdc::{ - CdcProperties, CdcSourceTypeTrait, Citus, DebeziumCdcSplit, Mongodb, Mysql, Postgres, + table_schema_exclude_additional_columns, CdcProperties, CdcSourceTypeTrait, Citus, + DebeziumCdcSplit, Mongodb, Mysql, Postgres, }; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; @@ -77,7 +78,7 @@ where source_id: source_id as u64, source_type: props.get_source_type_pb() as _, properties: props.properties, - table_schema: Some(props.table_schema), + table_schema: Some(table_schema_exclude_additional_columns(&props.table_schema)), is_source_job: props.is_cdc_source_job, is_backfill_table: props.is_backfill_table, }; diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index 4a1222a343e5..0ff35161e9ea 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -99,6 +99,22 @@ pub struct CdcProperties { pub _phantom: PhantomData, } +pub fn table_schema_exclude_additional_columns(table_schema: &TableSchema) -> TableSchema { + TableSchema { + columns: table_schema + .columns + .iter() + .filter(|col| { + col.additional_column + .as_ref() + .is_some_and(|val| val.column_type.is_none()) + }) + .cloned() + .collect(), + pk_indices: table_schema.pk_indices.clone(), + } +} + impl TryFromHashmap for CdcProperties { fn try_from_hashmap( properties: HashMap, From d4c15d3d5e48b808cc61678750e1ece22d484a00 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 23 May 2024 17:39:39 +0800 Subject: [PATCH 13/36] fix e2e --- e2e_test/source/cdc/cdc.share_stream.slt | 6 +++--- .../risingwave/connector/source/common/DbzSourceUtils.java | 5 +++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/e2e_test/source/cdc/cdc.share_stream.slt b/e2e_test/source/cdc/cdc.share_stream.slt index 43be0d2dc68e..471c94c6a868 100644 --- a/e2e_test/source/cdc/cdc.share_stream.slt +++ b/e2e_test/source/cdc/cdc.share_stream.slt @@ -159,7 +159,7 @@ SELECT count(*) as cnt from rw.products_test where commit_ts = '1970-01-01 00:00 # commit_ts of new records should greater than '1970-01-01 00:00:00+00:00' query TTT -SELECT name,description FROM rw.products_test order by id where commit_ts > '1970-01-01 00:00:00+00:00' +SELECT name,description FROM rw.products_test where commit_ts > '1970-01-01 00:00:00+00:00' order by id ---- 'Juice', '100ml Juice' @@ -283,7 +283,7 @@ SELECT id,name,email_address,credit_card,city from person_new order by id; # historical data query ITTTT -SELECT id,name,email_address,credit_card,city from person_new order by id where commit_ts = '1970-01-01 00:00:00+00:00'; +SELECT id,name,email_address,credit_card,city from person_new where commit_ts = '1970-01-01 00:00:00+00:00' order by id; ---- 1000 vicky noris yplkvgz@qbxfg.com 7878 5821 1864 2539 cheyenne 1001 peter white myckhsp@xpmpe.com 1781 2313 8157 6974 boise @@ -291,7 +291,7 @@ SELECT id,name,email_address,credit_card,city from person_new order by id where # incremental data query ITTTT -SELECT id,name,email_address,credit_card,city from person_new order by id where commit_ts > '1970-01-01 00:00:00+00:00'; +SELECT id,name,email_address,credit_card,city from person_new where commit_ts > '1970-01-01 00:00:00+00:00' order by id; ---- 1100 noris ypl@qbxfg.com 1864 2539 enne 1101 white myc@xpmpe.com 8157 6974 se diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzSourceUtils.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzSourceUtils.java index bd6720656811..83c6d59fac92 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzSourceUtils.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzSourceUtils.java @@ -153,8 +153,9 @@ private static boolean isStreamingRunning(String connector, String server, Strin mbeanServer.getAttribute( getStreamingMetricsObjectName(connector, server, contextName), "Connected"); - } catch (JMException ex) { - LOG.warn("Failed to get streaming metrics", ex); + } catch (JMException _ex) { + // ignore the exception, as it is expected when the streaming source + // (aka. binlog client) is not ready } return false; } From bc7d9ba57335a4c04f61038ac390e765d1f7ce0f Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 23 May 2024 18:38:27 +0800 Subject: [PATCH 14/36] fix e2e --- e2e_test/source/cdc/cdc.share_stream.slt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/e2e_test/source/cdc/cdc.share_stream.slt b/e2e_test/source/cdc/cdc.share_stream.slt index 471c94c6a868..3563ee6ed2a4 100644 --- a/e2e_test/source/cdc/cdc.share_stream.slt +++ b/e2e_test/source/cdc/cdc.share_stream.slt @@ -161,7 +161,8 @@ SELECT count(*) as cnt from rw.products_test where commit_ts = '1970-01-01 00:00 query TTT SELECT name,description FROM rw.products_test where commit_ts > '1970-01-01 00:00:00+00:00' order by id ---- -'Juice', '100ml Juice' +Milk Milk is a white liquid food +Juice 100ml Juice query ITTT SELECT order_id,order_date,customer_name,product_id FROM orders_test order by order_id limit 3 From 0089cc0fbccb4485f3a4dd418b42a103d3b194a0 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 24 May 2024 10:26:22 +0800 Subject: [PATCH 15/36] fix e2e --- e2e_test/source/cdc/cdc.check_new_rows.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e_test/source/cdc/cdc.check_new_rows.slt b/e2e_test/source/cdc/cdc.check_new_rows.slt index 77c8b6b5448c..25189eff09e6 100644 --- a/e2e_test/source/cdc/cdc.check_new_rows.slt +++ b/e2e_test/source/cdc/cdc.check_new_rows.slt @@ -15,7 +15,7 @@ select cnt from shipments_cnt; 4 query ITTTT -select * from person_new order by id; +SELECT id,name,email_address,credit_card,city from person_new order by id; ---- 1000 vicky noris yplkvgz@qbxfg.com 7878 5821 1864 2539 cheyenne 1001 peter white myckhsp@xpmpe.com 1781 2313 8157 6974 boise @@ -68,7 +68,7 @@ SELECT * from orders_test_cnt 5 query ITT -SELECT * FROM rw.products_test order by id limit 3 +SELECT id,name,description FROM rw.products_test order by id limit 3 ---- 101 RW Small 2-wheel scooter 102 RW 12V car battery From 4cb7cb612eecab7ba6f2ca64cfdbdcc88f73c7a5 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 24 May 2024 14:45:38 +0800 Subject: [PATCH 16/36] decouple cdc backfill table addi cols --- .../src/parser/additional_columns.rs | 32 ++++++++++++------- src/connector/src/source/reader/desc.rs | 4 +-- src/frontend/src/handler/create_source.rs | 16 +++++++--- src/frontend/src/handler/create_table.rs | 2 +- .../src/optimizer/plan_node/stream_source.rs | 8 +++-- .../optimizer/plan_node/stream_source_scan.rs | 8 +++-- 6 files changed, 45 insertions(+), 25 deletions(-) diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index a7be80128bad..012b7214c7e7 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -28,7 +28,7 @@ use risingwave_pb::plan_common::{ }; use crate::error::ConnectorResult; -use crate::source::cdc::{MONGODB_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR}; +use crate::source::cdc::MONGODB_CDC_CONNECTOR; use crate::source::{ GCS_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, @@ -56,14 +56,7 @@ pub static COMPATIBLE_ADDITIONAL_COLUMNS: LazyLock>> = + LazyLock::new(|| Some(HashSet::from(["timestamp"]))); + +pub fn get_supported_additional_columns( + connector_name: &str, + is_cdc_backfill: bool, +) -> Option<&HashSet<&'static str>> { + if is_cdc_backfill { + CDC_BACKFILL_TABLE_ADDITIONAL_COLUMNS.as_ref() + } else { + COMPATIBLE_ADDITIONAL_COLUMNS.get(connector_name) + } +} + pub fn gen_default_addition_col_name( connector_name: &str, additional_col_type: &str, @@ -100,9 +108,10 @@ pub fn build_additional_column_catalog( inner_field_name: Option<&str>, data_type: Option<&str>, reject_unknown_connector: bool, + is_cdc_backfill_table: bool, ) -> ConnectorResult { let compatible_columns = match ( - COMPATIBLE_ADDITIONAL_COLUMNS.get(connector_name), + get_supported_additional_columns(connector_name, is_cdc_backfill_table), reject_unknown_connector, ) { (Some(compat_cols), _) => compat_cols, @@ -203,7 +212,7 @@ pub fn build_additional_column_catalog( /// ## Returns /// - `columns_exist`: whether 1. `partition`/`file` and 2. `offset` columns are included in `columns`. /// - `additional_columns`: The `ColumnCatalog` for `partition`/`file` and `offset` columns. -pub fn add_partition_offset_cols( +pub fn source_add_partition_offset_cols( columns: &[ColumnCatalog], connector_name: &str, ) -> ([bool; 2], [ColumnCatalog; 2]) { @@ -232,6 +241,7 @@ pub fn add_partition_offset_cols( None, None, false, + false, ) .unwrap(), ) diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index cbd63a2a4906..a652c4054db6 100644 --- a/src/connector/src/source/reader/desc.rs +++ b/src/connector/src/source/reader/desc.rs @@ -26,7 +26,7 @@ use risingwave_pb::plan_common::PbColumnCatalog; use super::fs_reader::FsSourceReader; use super::reader::SourceReader; use crate::error::ConnectorResult; -use crate::parser::additional_columns::add_partition_offset_cols; +use crate::parser::additional_columns::source_add_partition_offset_cols; use crate::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use crate::source::monitor::SourceMetrics; use crate::source::{SourceColumnDesc, SourceColumnType, UPSTREAM_SOURCE_KEY}; @@ -98,7 +98,7 @@ impl SourceDescBuilder { .map(|c| ColumnCatalog::from(c.clone())) .collect_vec(); let (columns_exist, additional_columns) = - add_partition_offset_cols(&columns, &connector_name); + source_add_partition_offset_cols(&columns, &connector_name); let mut columns: Vec<_> = self .columns diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 5ae33c195d41..ffacf05faf90 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -29,7 +29,7 @@ use risingwave_common::catalog::{ }; use risingwave_common::types::DataType; use risingwave_connector::parser::additional_columns::{ - build_additional_column_catalog, COMPATIBLE_ADDITIONAL_COLUMNS, + build_additional_column_catalog, get_supported_additional_columns, }; use risingwave_connector::parser::{ schema_to_columns, AvroParserConfig, DebeziumAvroParserConfig, ProtobufParserConfig, @@ -551,12 +551,11 @@ pub fn handle_addition_columns( with_properties: &HashMap, mut additional_columns: IncludeOption, columns: &mut Vec, + is_cdc_backfill_table: bool, ) -> Result<()> { let connector_name = with_properties.get_connector().unwrap(); // there must be a connector in source - if COMPATIBLE_ADDITIONAL_COLUMNS - .get(connector_name.as_str()) - .is_none() + if get_supported_additional_columns(connector_name.as_str(), is_cdc_backfill_table).is_none() && !additional_columns.is_empty() { return Err(RwError::from(ProtocolError(format!( @@ -595,6 +594,7 @@ pub fn handle_addition_columns( item.inner_field.as_deref(), data_type_name.as_deref(), true, + is_cdc_backfill_table, )?); } @@ -911,6 +911,7 @@ fn check_and_add_timestamp_column( None, None, true, + false, ) .unwrap(); catalog.is_hidden = true; @@ -1355,7 +1356,12 @@ pub async fn bind_create_source( )?; // add additional columns before bind pk, because `format upsert` requires the key column - handle_addition_columns(&with_properties, include_column_options, &mut columns)?; + handle_addition_columns( + &with_properties, + include_column_options, + &mut columns, + false, + )?; // compatible with the behavior that add a hidden column `_rw_kafka_timestamp` to each message from Kafka source if is_create_source { // must behind `handle_addition_columns` diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 870b761348de..7f0f63db8d88 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -769,7 +769,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( let mut columns = bind_sql_columns(&column_defs)?; let with_properties = source.with_properties.clone().into_iter().collect(); // append additional columns to the end - handle_addition_columns(&with_properties, include_column_options, &mut columns)?; + handle_addition_columns(&with_properties, include_column_options, &mut columns, true)?; for c in &mut columns { c.column_desc.column_id = col_id_gen.generate(c.name()) diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index 7a5e9effbac9..865d444f8c15 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -18,7 +18,7 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_connector::parser::additional_columns::add_partition_offset_cols; +use risingwave_connector::parser::additional_columns::source_add_partition_offset_cols; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::{PbStreamSource, SourceNode}; @@ -46,8 +46,10 @@ impl StreamSource { if let Some(source_catalog) = &core.catalog && source_catalog.info.is_shared() { - let (columns_exist, additional_columns) = - add_partition_offset_cols(&core.column_catalog, &source_catalog.connector_name()); + let (columns_exist, additional_columns) = source_add_partition_offset_cols( + &core.column_catalog, + &source_catalog.connector_name(), + ); for (existed, mut c) in columns_exist.into_iter().zip_eq_fast(additional_columns) { c.is_hidden = true; if !existed { diff --git a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs index 12fbbfd7a4b8..02f794fe55a9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs @@ -21,7 +21,7 @@ use risingwave_common::catalog::Field; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::OrderType; -use risingwave_connector::parser::additional_columns::add_partition_offset_cols; +use risingwave_connector::parser::additional_columns::source_add_partition_offset_cols; use risingwave_pb::stream_plan::stream_node::{NodeBody, PbNodeBody}; use risingwave_pb::stream_plan::PbStreamNode; @@ -58,8 +58,10 @@ impl StreamSourceScan { if let Some(source_catalog) = &core.catalog && source_catalog.info.is_shared() { - let (columns_exist, additional_columns) = - add_partition_offset_cols(&core.column_catalog, &source_catalog.connector_name()); + let (columns_exist, additional_columns) = source_add_partition_offset_cols( + &core.column_catalog, + &source_catalog.connector_name(), + ); for (existed, mut c) in columns_exist.into_iter().zip_eq_fast(additional_columns) { c.is_hidden = true; if !existed { From 360a5ff1208c5bdb1dc000829b7e28d17e90be89 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 27 May 2024 11:30:34 +0800 Subject: [PATCH 17/36] clean code --- src/connector/src/parser/mod.rs | 1 - src/connector/src/parser/unified/debezium.rs | 7 ------- src/frontend/src/handler/create_table.rs | 22 +++++++++++++++----- 3 files changed, 17 insertions(+), 13 deletions(-) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 13ddcca59272..c8ce4bfdf53d 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -643,7 +643,6 @@ const MAX_ROWS_FOR_TRANSACTION: usize = 4096; #[try_stream(ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_chunk_stream(mut parser: P, data_stream: BoxSourceStream) { let columns = parser.columns().to_vec(); - tracing::info!("parser columns: {:?}", columns); let mut heartbeat_builder = SourceStreamChunkBuilder::with_capacity(columns.clone(), 0); let mut builder = SourceStreamChunkBuilder::with_capacity(columns, 0); diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index e772220f9f0c..966c5f167474 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -198,13 +198,6 @@ where |additional_column_type| { match additional_column_type { &ColumnType::Timestamp(_) => { - tracing::info!( - "access timestamp field: `{}, type: {:?}, column_type: {:?}`", - desc.name, - desc.data_type, - desc.additional_column.column_type - ); - // access payload.source.ts_ms let ts_ms = self .value_accessor diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 7f0f63db8d88..d4f8f27b9b5d 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -743,6 +743,22 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( with_version_column: Option, include_column_options: IncludeOption, ) -> Result<(PlanRef, PbTable)> { + if constraints.iter().any(|c| { + matches!( + c, + TableConstraint::Unique { + is_primary: true, + .. + } + ) + }) { + return Err(ErrorCode::NotSupported( + "CDC table without primary key constraint is not supported".to_owned(), + "Please define a primary key".to_owned(), + ) + .into()); + } + let session = context.session_ctx().clone(); let db_name = session.database(); let (schema_name, name) = Binder::resolve_schema_qualified_name(db_name, table_name)?; @@ -776,11 +792,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( } let pk_names = bind_sql_pk_names(&column_defs, &constraints)?; - let (columns, pk_column_ids, row_id_index) = bind_pk_on_relation(columns, pk_names, true)?; - assert_eq!( - None, row_id_index, - "cdc table should not have row_id column" - ); + let (columns, pk_column_ids, _row_id_index) = bind_pk_on_relation(columns, pk_names, true)?; let definition = context.normalized_sql().to_owned(); From d7ce5d0277502062c757be75647d1216a7c90475 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 27 May 2024 11:54:03 +0800 Subject: [PATCH 18/36] 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 d4f8f27b9b5d..4f3b81a20e63 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -743,7 +743,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( with_version_column: Option, include_column_options: IncludeOption, ) -> Result<(PlanRef, PbTable)> { - if constraints.iter().any(|c| { + if !constraints.iter().any(|c| { matches!( c, TableConstraint::Unique { From 57edad0f829c865e2a1aa30a91dc62df1bfab6da Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 28 May 2024 12:40:31 +0800 Subject: [PATCH 19/36] WIP: pg external table --- src/connector/src/source/cdc/external/mod.rs | 295 ++--------------- .../src/source/cdc/external/mysql.rs | 307 ++++++++++++++++++ .../src/source/cdc/external/postgres.rs | 39 ++- src/frontend/src/handler/create_table.rs | 191 ++++++----- src/meta/src/rpc/ddl_controller.rs | 49 +++ 5 files changed, 534 insertions(+), 347 deletions(-) create mode 100644 src/connector/src/source/cdc/external/mysql.rs diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index 32a1b1ccfa80..284b71642d01 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -13,10 +13,11 @@ // limitations under the License. pub mod mock_external_table; -mod postgres; +pub mod postgres; #[cfg(not(madsim))] mod maybe_tls_connector; +pub mod mysql; use std::collections::HashMap; use std::fmt; @@ -26,9 +27,6 @@ use futures::stream::BoxStream; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; -use mysql_async::prelude::*; -use mysql_common::params::Params; -use mysql_common::value::Value; use risingwave_common::bail; use risingwave_common::catalog::{Schema, OFFSET_COLUMN_NAME}; use risingwave_common::row::OwnedRow; @@ -39,7 +37,13 @@ use serde_derive::{Deserialize, Serialize}; use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::mysql_row_to_owned_row; use crate::source::cdc::external::mock_external_table::MockExternalTableReader; -use crate::source::cdc::external::postgres::{PostgresExternalTableReader, PostgresOffset}; +use crate::source::cdc::external::mysql::{ + MySqlExternalTable, MySqlExternalTableReader, MySqlOffset, +}; +use crate::source::cdc::external::postgres::{ + PostgresExternalTable, PostgresExternalTableReader, PostgresOffset, +}; +use crate::source::UnknownFields; use crate::WithPropertiesExt; #[derive(Debug)] @@ -128,18 +132,6 @@ impl SchemaTableName { } } -#[derive(Debug, Clone, Default, PartialEq, PartialOrd, Serialize, Deserialize)] -pub struct MySqlOffset { - pub filename: String, - pub position: u64, -} - -impl MySqlOffset { - pub fn new(filename: String, position: u64) -> Self { - Self { filename, position } - } -} - #[derive(Debug, Clone, PartialEq, PartialOrd, Serialize, Deserialize)] pub enum CdcOffset { MySql(MySqlOffset), @@ -189,24 +181,6 @@ pub struct DebeziumSourceOffset { pub tx_usec: Option, } -impl MySqlOffset { - pub fn parse_debezium_offset(offset: &str) -> ConnectorResult { - let dbz_offset: DebeziumOffset = serde_json::from_str(offset) - .with_context(|| format!("invalid upstream offset: {}", offset))?; - - Ok(Self { - filename: dbz_offset - .source_offset - .file - .context("binlog file not found in offset")?, - position: dbz_offset - .source_offset - .pos - .context("binlog position not found in offset")?, - }) - } -} - pub type CdcOffsetParseFunc = Box ConnectorResult + Send>; pub trait ExternalTableReader { @@ -227,13 +201,6 @@ pub enum ExternalTableReaderImpl { Mock(MockExternalTableReader), } -pub struct MySqlExternalTableReader { - rw_schema: Schema, - field_names: String, - // use mutex to provide shared mutable access to the connection - conn: tokio::sync::Mutex, -} - #[derive(Debug, Clone, Deserialize)] pub struct ExternalTableConfig { #[serde(rename = "hostname")] @@ -252,6 +219,15 @@ pub struct ExternalTableConfig { /// This field is optional. #[serde(rename = "ssl.mode", default = "Default::default")] pub sslmode: SslMode, + + #[serde(flatten)] + pub unknown_fields: HashMap, +} + +impl UnknownFields for ExternalTableConfig { + fn unknown_fields(&self) -> HashMap { + self.unknown_fields.clone() + } } #[derive(Debug, Clone, Deserialize)] @@ -279,236 +255,6 @@ impl fmt::Display for SslMode { } } -impl ExternalTableReader for MySqlExternalTableReader { - async fn current_cdc_offset(&self) -> ConnectorResult { - let mut conn = self.conn.lock().await; - - let sql = "SHOW MASTER STATUS".to_string(); - let mut rs = conn.query::(sql).await?; - let row = rs - .iter_mut() - .exactly_one() - .ok() - .context("expect exactly one row when reading binlog offset")?; - - Ok(CdcOffset::MySql(MySqlOffset { - filename: row.take("File").unwrap(), - position: row.take("Position").unwrap(), - })) - } - - fn snapshot_read( - &self, - table_name: SchemaTableName, - start_pk: Option, - primary_keys: Vec, - limit: u32, - ) -> BoxStream<'_, ConnectorResult> { - self.snapshot_read_inner(table_name, start_pk, primary_keys, limit) - } -} - -impl MySqlExternalTableReader { - pub async fn new( - with_properties: HashMap, - rw_schema: Schema, - ) -> ConnectorResult { - let config = serde_json::from_value::( - serde_json::to_value(with_properties).unwrap(), - ) - .context("failed to extract mysql connector properties")?; - - let mut opts_builder = mysql_async::OptsBuilder::default() - .user(Some(config.username)) - .pass(Some(config.password)) - .ip_or_hostname(config.host) - .tcp_port(config.port.parse::().unwrap()) - .db_name(Some(config.database)); - - opts_builder = match config.sslmode { - SslMode::Disabled | SslMode::Preferred => opts_builder.ssl_opts(None), - SslMode::Required => { - let ssl_without_verify = mysql_async::SslOpts::default() - .with_danger_accept_invalid_certs(true) - .with_danger_skip_domain_validation(true); - opts_builder.ssl_opts(Some(ssl_without_verify)) - } - }; - - let conn = mysql_async::Conn::new(mysql_async::Opts::from(opts_builder)).await?; - - let field_names = rw_schema - .fields - .iter() - .filter(|f| f.name != OFFSET_COLUMN_NAME) - .map(|f| Self::quote_column(f.name.as_str())) - .join(","); - - Ok(Self { - rw_schema, - field_names, - conn: tokio::sync::Mutex::new(conn), - }) - } - - pub fn get_normalized_table_name(table_name: &SchemaTableName) -> String { - // schema name is the database name in mysql - format!("`{}`.`{}`", table_name.schema_name, table_name.table_name) - } - - pub fn get_cdc_offset_parser() -> CdcOffsetParseFunc { - Box::new(move |offset| { - Ok(CdcOffset::MySql(MySqlOffset::parse_debezium_offset( - offset, - )?)) - }) - } - - #[try_stream(boxed, ok = OwnedRow, error = ConnectorError)] - async fn snapshot_read_inner( - &self, - table_name: SchemaTableName, - start_pk_row: Option, - primary_keys: Vec, - limit: u32, - ) { - let order_key = primary_keys - .iter() - .map(|col| Self::quote_column(col)) - .join(","); - let sql = if start_pk_row.is_none() { - format!( - "SELECT {} FROM {} ORDER BY {} LIMIT {limit}", - self.field_names, - Self::get_normalized_table_name(&table_name), - order_key, - ) - } else { - let filter_expr = Self::filter_expression(&primary_keys); - format!( - "SELECT {} FROM {} WHERE {} ORDER BY {} LIMIT {limit}", - self.field_names, - Self::get_normalized_table_name(&table_name), - filter_expr, - order_key, - ) - }; - - let mut conn = self.conn.lock().await; - - // Set session timezone to UTC - conn.exec_drop("SET time_zone = \"+00:00\"", ()).await?; - - if start_pk_row.is_none() { - let rs_stream = sql.stream::(&mut *conn).await?; - let row_stream = rs_stream.map(|row| { - // convert mysql row into OwnedRow - let mut row = row?; - Ok::<_, ConnectorError>(mysql_row_to_owned_row(&mut row, &self.rw_schema)) - }); - - pin_mut!(row_stream); - #[for_await] - for row in row_stream { - let row = row?; - yield row; - } - } else { - let field_map = self - .rw_schema - .fields - .iter() - .map(|f| (f.name.as_str(), f.data_type.clone())) - .collect::>(); - - // fill in start primary key params - let params: Vec<_> = primary_keys - .iter() - .zip_eq_fast(start_pk_row.unwrap().into_iter()) - .map(|(pk, datum)| { - if let Some(value) = datum { - let ty = field_map.get(pk.as_str()).unwrap(); - let val = match ty { - DataType::Boolean => Value::from(value.into_bool()), - DataType::Int16 => Value::from(value.into_int16()), - DataType::Int32 => Value::from(value.into_int32()), - DataType::Int64 => Value::from(value.into_int64()), - DataType::Float32 => Value::from(value.into_float32().into_inner()), - DataType::Float64 => Value::from(value.into_float64().into_inner()), - DataType::Varchar => Value::from(String::from(value.into_utf8())), - DataType::Date => Value::from(value.into_date().0), - DataType::Time => Value::from(value.into_time().0), - DataType::Timestamp => Value::from(value.into_timestamp().0), - _ => bail!("unsupported primary key data type: {}", ty), - }; - ConnectorResult::Ok((pk.clone(), val)) - } else { - bail!("primary key {} cannot be null", pk); - } - }) - .try_collect::<_, _, ConnectorError>()?; - - let rs_stream = sql - .with(Params::from(params)) - .stream::(&mut *conn) - .await?; - - let row_stream = rs_stream.map(|row| { - // convert mysql row into OwnedRow - let mut row = row?; - Ok::<_, ConnectorError>(mysql_row_to_owned_row(&mut row, &self.rw_schema)) - }); - - pin_mut!(row_stream); - #[for_await] - for row in row_stream { - let row = row?; - yield row; - } - }; - } - - // mysql cannot leverage the given key to narrow down the range of scan, - // we need to rewrite the comparison conditions by our own. - // (a, b) > (x, y) => (`a` > x) OR ((`a` = x) AND (`b` > y)) - fn filter_expression(columns: &[String]) -> String { - let mut conditions = vec![]; - // push the first condition - conditions.push(format!( - "({} > :{})", - Self::quote_column(&columns[0]), - columns[0] - )); - for i in 2..=columns.len() { - // '=' condition - let mut condition = String::new(); - for (j, col) in columns.iter().enumerate().take(i - 1) { - if j == 0 { - condition.push_str(&format!("({} = :{})", Self::quote_column(col), col)); - } else { - condition.push_str(&format!(" AND ({} = :{})", Self::quote_column(col), col)); - } - } - // '>' condition - condition.push_str(&format!( - " AND ({} > :{})", - Self::quote_column(&columns[i - 1]), - columns[i - 1] - )); - conditions.push(format!("({})", condition)); - } - if columns.len() > 1 { - conditions.join(" OR ") - } else { - conditions.join("") - } - } - - fn quote_column(column: &str) -> String { - format!("`{}`", column) - } -} - impl ExternalTableReader for ExternalTableReaderImpl { async fn current_cdc_offset(&self) -> ConnectorResult { match self { @@ -569,6 +315,11 @@ impl ExternalTableReaderImpl { } } +pub enum ExternalTableImpl { + MySql(MySqlExternalTable), + Postgres(PostgresExternalTable), +} + #[cfg(test)] mod tests { diff --git a/src/connector/src/source/cdc/external/mysql.rs b/src/connector/src/source/cdc/external/mysql.rs new file mode 100644 index 000000000000..1cf167bdad3f --- /dev/null +++ b/src/connector/src/source/cdc/external/mysql.rs @@ -0,0 +1,307 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; + +use anyhow::Context; +use futures::stream::BoxStream; +use futures::{pin_mut, StreamExt}; +use futures_async_stream::try_stream; +use itertools::Itertools; +use mysql_async::prelude::*; +use mysql_common::params::Params; +use mysql_common::value::Value; +use risingwave_common::bail; +use risingwave_common::catalog::{Schema, OFFSET_COLUMN_NAME}; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::DataType; +use risingwave_common::util::iter_util::ZipEqFast; +use serde_derive::{Deserialize, Serialize}; + +use crate::error::{ConnectorError, ConnectorResult}; +use crate::source::cdc::external::{ + mysql_row_to_owned_row, CdcOffset, CdcOffsetParseFunc, DebeziumOffset, ExternalTableConfig, + ExternalTableReader, SchemaTableName, SslMode, +}; + +#[derive(Debug, Clone, Default, PartialEq, PartialOrd, Serialize, Deserialize)] +pub struct MySqlOffset { + pub filename: String, + pub position: u64, +} + +impl MySqlOffset { + pub fn new(filename: String, position: u64) -> Self { + Self { filename, position } + } +} + +impl MySqlOffset { + pub fn parse_debezium_offset(offset: &str) -> ConnectorResult { + let dbz_offset: DebeziumOffset = serde_json::from_str(offset) + .with_context(|| format!("invalid upstream offset: {}", offset))?; + + Ok(Self { + filename: dbz_offset + .source_offset + .file + .context("binlog file not found in offset")?, + position: dbz_offset + .source_offset + .pos + .context("binlog position not found in offset")?, + }) + } +} + +pub struct MySqlExternalTable { + config: ExternalTableConfig, +} + +pub struct MySqlExternalTableReader { + rw_schema: Schema, + field_names: String, + // use mutex to provide shared mutable access to the connection + conn: tokio::sync::Mutex, +} + +impl ExternalTableReader for MySqlExternalTableReader { + async fn current_cdc_offset(&self) -> ConnectorResult { + let mut conn = self.conn.lock().await; + + let sql = "SHOW MASTER STATUS".to_string(); + let mut rs = conn.query::(sql).await?; + let row = rs + .iter_mut() + .exactly_one() + .ok() + .context("expect exactly one row when reading binlog offset")?; + + Ok(CdcOffset::MySql(MySqlOffset { + filename: row.take("File").unwrap(), + position: row.take("Position").unwrap(), + })) + } + + fn snapshot_read( + &self, + table_name: SchemaTableName, + start_pk: Option, + primary_keys: Vec, + limit: u32, + ) -> BoxStream<'_, ConnectorResult> { + self.snapshot_read_inner(table_name, start_pk, primary_keys, limit) + } +} + +impl MySqlExternalTableReader { + pub async fn new( + with_properties: HashMap, + rw_schema: Schema, + ) -> ConnectorResult { + let config = serde_json::from_value::( + serde_json::to_value(with_properties).unwrap(), + ) + .context("failed to extract mysql connector properties")?; + + let mut opts_builder = mysql_async::OptsBuilder::default() + .user(Some(config.username)) + .pass(Some(config.password)) + .ip_or_hostname(config.host) + .tcp_port(config.port.parse::().unwrap()) + .db_name(Some(config.database)); + + opts_builder = match config.sslmode { + SslMode::Disabled | SslMode::Preferred => opts_builder.ssl_opts(None), + SslMode::Required => { + let ssl_without_verify = mysql_async::SslOpts::default() + .with_danger_accept_invalid_certs(true) + .with_danger_skip_domain_validation(true); + opts_builder.ssl_opts(Some(ssl_without_verify)) + } + }; + + let conn = mysql_async::Conn::new(mysql_async::Opts::from(opts_builder)).await?; + + let field_names = rw_schema + .fields + .iter() + .filter(|f| f.name != OFFSET_COLUMN_NAME) + .map(|f| Self::quote_column(f.name.as_str())) + .join(","); + + Ok(Self { + rw_schema, + field_names, + conn: tokio::sync::Mutex::new(conn), + }) + } + + pub fn get_normalized_table_name(table_name: &SchemaTableName) -> String { + // schema name is the database name in mysql + format!("`{}`.`{}`", table_name.schema_name, table_name.table_name) + } + + pub fn get_cdc_offset_parser() -> CdcOffsetParseFunc { + Box::new(move |offset| { + Ok(CdcOffset::MySql(MySqlOffset::parse_debezium_offset( + offset, + )?)) + }) + } + + #[try_stream(boxed, ok = OwnedRow, error = ConnectorError)] + async fn snapshot_read_inner( + &self, + table_name: SchemaTableName, + start_pk_row: Option, + primary_keys: Vec, + limit: u32, + ) { + let order_key = primary_keys + .iter() + .map(|col| Self::quote_column(col)) + .join(","); + let sql = if start_pk_row.is_none() { + format!( + "SELECT {} FROM {} ORDER BY {} LIMIT {limit}", + self.field_names, + Self::get_normalized_table_name(&table_name), + order_key, + ) + } else { + let filter_expr = Self::filter_expression(&primary_keys); + format!( + "SELECT {} FROM {} WHERE {} ORDER BY {} LIMIT {limit}", + self.field_names, + Self::get_normalized_table_name(&table_name), + filter_expr, + order_key, + ) + }; + + let mut conn = self.conn.lock().await; + + // Set session timezone to UTC + conn.exec_drop("SET time_zone = \"+00:00\"", ()).await?; + + if start_pk_row.is_none() { + let rs_stream = sql.stream::(&mut *conn).await?; + let row_stream = rs_stream.map(|row| { + // convert mysql row into OwnedRow + let mut row = row?; + Ok::<_, ConnectorError>(mysql_row_to_owned_row(&mut row, &self.rw_schema)) + }); + + pin_mut!(row_stream); + #[for_await] + for row in row_stream { + let row = row?; + yield row; + } + } else { + let field_map = self + .rw_schema + .fields + .iter() + .map(|f| (f.name.as_str(), f.data_type.clone())) + .collect::>(); + + // fill in start primary key params + let params: Vec<_> = primary_keys + .iter() + .zip_eq_fast(start_pk_row.unwrap().into_iter()) + .map(|(pk, datum)| { + if let Some(value) = datum { + let ty = field_map.get(pk.as_str()).unwrap(); + let val = match ty { + DataType::Boolean => Value::from(value.into_bool()), + DataType::Int16 => Value::from(value.into_int16()), + DataType::Int32 => Value::from(value.into_int32()), + DataType::Int64 => Value::from(value.into_int64()), + DataType::Float32 => Value::from(value.into_float32().into_inner()), + DataType::Float64 => Value::from(value.into_float64().into_inner()), + DataType::Varchar => Value::from(String::from(value.into_utf8())), + DataType::Date => Value::from(value.into_date().0), + DataType::Time => Value::from(value.into_time().0), + DataType::Timestamp => Value::from(value.into_timestamp().0), + _ => bail!("unsupported primary key data type: {}", ty), + }; + ConnectorResult::Ok((pk.clone(), val)) + } else { + bail!("primary key {} cannot be null", pk); + } + }) + .try_collect::<_, _, ConnectorError>()?; + + let rs_stream = sql + .with(Params::from(params)) + .stream::(&mut *conn) + .await?; + + let row_stream = rs_stream.map(|row| { + // convert mysql row into OwnedRow + let mut row = row?; + Ok::<_, ConnectorError>(mysql_row_to_owned_row(&mut row, &self.rw_schema)) + }); + + pin_mut!(row_stream); + #[for_await] + for row in row_stream { + let row = row?; + yield row; + } + }; + } + + // mysql cannot leverage the given key to narrow down the range of scan, + // we need to rewrite the comparison conditions by our own. + // (a, b) > (x, y) => (`a` > x) OR ((`a` = x) AND (`b` > y)) + pub(crate) fn filter_expression(columns: &[String]) -> String { + let mut conditions = vec![]; + // push the first condition + conditions.push(format!( + "({} > :{})", + Self::quote_column(&columns[0]), + columns[0] + )); + for i in 2..=columns.len() { + // '=' condition + let mut condition = String::new(); + for (j, col) in columns.iter().enumerate().take(i - 1) { + if j == 0 { + condition.push_str(&format!("({} = :{})", Self::quote_column(col), col)); + } else { + condition.push_str(&format!(" AND ({} = :{})", Self::quote_column(col), col)); + } + } + // '>' condition + condition.push_str(&format!( + " AND ({} > :{})", + Self::quote_column(&columns[i - 1]), + columns[i - 1] + )); + conditions.push(format!("({})", condition)); + } + if columns.len() > 1 { + conditions.join(" OR ") + } else { + conditions.join("") + } + } + + fn quote_column(column: &str) -> String { + format!("`{}`", column) + } +} diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index a4e3117b4951..0f1a34913e56 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -22,8 +22,9 @@ use futures_async_stream::try_stream; use itertools::Itertools; use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; use postgres_openssl::MakeTlsConnector; -use risingwave_common::catalog::Schema; +use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId, Schema}; use risingwave_common::row::{OwnedRow, Row}; +use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; use serde_derive::{Deserialize, Serialize}; use thiserror_ext::AsReport; @@ -73,6 +74,42 @@ impl PostgresOffset { } } +pub struct PostgresExternalTable { + columns: Vec, + pk_indices: Vec, +} + +impl PostgresExternalTable { + pub fn new() -> Self { + Self { + columns: vec![], + pk_indices: vec![], + } + } + + pub async fn connect(&self) -> ConnectorResult<()> { + todo!("connect to postgres"); + // connect to external db and read the schema + } + + pub fn column_descs(&self) -> Vec { + vec![ + ColumnDesc::named("v1", ColumnId::new(1), DataType::Int32), + ColumnDesc::named("v2", ColumnId::new(2), DataType::Varchar), + ColumnDesc::named("v3", ColumnId::new(3), DataType::Decimal), + ColumnDesc::named("v4", ColumnId::new(4), DataType::Date), + ] + } + + pub fn pk_names(&self) -> Vec { + vec!["v1".to_string(), "v2".to_string()] + } + + pub fn pk_indices(&self) -> Vec { + vec![0, 1] + } +} + pub struct PostgresExternalTableReader { config: ExternalTableConfig, rw_schema: Schema, diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 4f3b81a20e63..fb7135bd1f95 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -16,7 +16,7 @@ use std::collections::{BTreeMap, HashMap}; use std::rc::Rc; use std::sync::Arc; -use anyhow::anyhow; +use anyhow::{anyhow, Context}; use either::Either; use fixedbitset::FixedBitSet; use itertools::Itertools; @@ -29,9 +29,11 @@ use risingwave_common::catalog::{ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::util::value_encoding::DatumToProtoExt; use risingwave_connector::source; +use risingwave_connector::source::cdc::external::postgres::PostgresExternalTable; use risingwave_connector::source::cdc::external::{ - DATABASE_NAME_KEY, SCHEMA_NAME_KEY, TABLE_NAME_KEY, + ExternalTableConfig, DATABASE_NAME_KEY, SCHEMA_NAME_KEY, TABLE_NAME_KEY, }; +use risingwave_connector::source::TryFromHashmap; use risingwave_pb::catalog::{PbSource, PbTable, Table, WatermarkDesc}; use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; @@ -731,67 +733,37 @@ fn gen_table_plan_inner( } #[allow(clippy::too_many_arguments)] -pub(crate) fn gen_create_table_plan_for_cdc_source( - context: OptimizerContextRef, - source_name: ObjectName, - table_name: ObjectName, +pub(crate) async fn gen_create_table_plan_for_cdc_source( + handler_args: HandlerArgs, + explain_options: ExplainOptions, + source: Arc, external_table_name: String, - column_defs: Vec, - constraints: Vec, + mut columns: Vec, + pk_names: Vec, + connect_properties: HashMap, mut col_id_gen: ColumnIdGenerator, on_conflict: Option, with_version_column: Option, include_column_options: IncludeOption, + resolved_table_name: String, + database_id: DatabaseId, + schema_id: SchemaId, ) -> Result<(PlanRef, PbTable)> { - if !constraints.iter().any(|c| { - matches!( - c, - TableConstraint::Unique { - is_primary: true, - .. - } - ) - }) { - return Err(ErrorCode::NotSupported( - "CDC table without primary key constraint is not supported".to_owned(), - "Please define a primary key".to_owned(), - ) - .into()); - } - + let context: OptimizerContextRef = OptimizerContext::new(handler_args, explain_options).into(); let session = context.session_ctx().clone(); - let db_name = session.database(); - let (schema_name, name) = Binder::resolve_schema_qualified_name(db_name, table_name)?; - let (database_id, schema_id) = - session.get_database_and_schema_id_for_create(schema_name.clone())?; - - // cdc table cannot be append-only - let append_only = false; - let (source_schema, source_name) = Binder::resolve_schema_qualified_name(db_name, source_name)?; - let source = { - let catalog_reader = session.env().catalog_reader().read_guard(); - let schema_name = source_schema - .clone() - .unwrap_or(DEFAULT_SCHEMA_NAME.to_string()); - let (source, _) = catalog_reader.get_source_by_name( - db_name, - SchemaPath::Name(schema_name.as_str()), - source_name.as_str(), - )?; - source.clone() - }; - - let mut columns = bind_sql_columns(&column_defs)?; - let with_properties = source.with_properties.clone().into_iter().collect(); // append additional columns to the end - handle_addition_columns(&with_properties, include_column_options, &mut columns, true)?; + handle_addition_columns( + &connect_properties, + include_column_options, + &mut columns, + true, + )?; for c in &mut columns { c.column_desc.column_id = col_id_gen.generate(c.name()) } - let pk_names = bind_sql_pk_names(&column_defs, &constraints)?; let (columns, pk_column_ids, _row_id_index) = bind_pk_on_relation(columns, pk_names, true)?; let definition = context.normalized_sql().to_owned(); @@ -812,9 +784,6 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( .map(|idx| ColumnOrder::new(*idx, OrderType::ascending())) .collect(); - let connect_properties = - derive_connect_properties(source.as_ref(), external_table_name.clone())?; - let cdc_table_desc = CdcTableDesc { table_id: TableId::placeholder(), // will be filled in meta node source_id: source.id.into(), // id of cdc source streaming job @@ -822,7 +791,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( pk: table_pk, columns: columns.iter().map(|c| c.column_desc.clone()).collect(), stream_key: pk_column_indices, - connect_properties, + connect_properties: connect_properties.into_iter().collect(), }; tracing::debug!(?cdc_table_desc, "create cdc table"); @@ -848,12 +817,12 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( let materialize = plan_root.gen_table_plan( context, - name, + resolved_table_name, columns, definition, pk_column_ids, None, - append_only, + false, on_conflict, with_version_column, vec![], @@ -870,13 +839,13 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( } fn derive_connect_properties( - source: &SourceCatalog, + source_with_properties: &HashMap, external_table_name: String, -) -> Result> { +) -> Result> { use source::cdc::{MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR}; // we should remove the prefix from `full_table_name` - let mut connect_properties = source.with_properties.clone(); - if let Some(connector) = source.with_properties.get(UPSTREAM_SOURCE_KEY) { + let mut connect_properties = source_with_properties.clone(); + if let Some(connector) = source_with_properties.get(UPSTREAM_SOURCE_KEY) { let table_name = match connector.as_str() { MYSQL_CDC_CONNECTOR => { let db_name = connect_properties.get(DATABASE_NAME_KEY).ok_or_else(|| { @@ -907,7 +876,7 @@ fn derive_connect_properties( }; connect_properties.insert(TABLE_NAME_KEY.into(), table_name.into()); } - Ok(connect_properties.into_iter().collect()) + Ok(connect_properties) } #[allow(clippy::too_many_arguments)] @@ -973,7 +942,6 @@ pub(super) async fn handle_create_table_plan( } (None, Some(cdc_table)) => { - let context = OptimizerContext::new(handler_args, explain_options); if append_only { return Err(ErrorCode::NotSupported( "append only modifier on the table created from a CDC source".into(), @@ -989,13 +957,6 @@ pub(super) async fn handle_create_table_plan( ) .into()); } - if wildcard_idx.is_some() { - return Err(ErrorCode::NotSupported( - "star(\"*\") defined on the table created from a CDC source".into(), - "Remove the star(\"*\") in the column list".into(), - ) - .into()); - } for c in &column_defs { for op in &c.options { if let ColumnOption::GeneratedColumns(_) = op.option { @@ -1008,18 +969,100 @@ pub(super) async fn handle_create_table_plan( } } } + if !wildcard_idx.is_some() + && !constraints.iter().any(|c| { + matches!( + c, + TableConstraint::Unique { + is_primary: true, + .. + } + ) + }) + { + return Err(ErrorCode::NotSupported( + "CDC table without primary key constraint is not supported".to_owned(), + "Please define a primary key".to_owned(), + ) + .into()); + } + + let session = handler_args.session.clone(); + let db_name = session.database(); + let (schema_name, resolved_table_name) = + Binder::resolve_schema_qualified_name(db_name, table_name)?; + let (database_id, schema_id) = + session.get_database_and_schema_id_for_create(schema_name.clone())?; + + // cdc table cannot be append-only + let need_auto_schema_map = wildcard_idx.is_some(); + let (source_schema, source_name) = + Binder::resolve_schema_qualified_name(db_name, cdc_table.source_name.clone())?; + + let source = { + let catalog_reader = session.env().catalog_reader().read_guard(); + let schema_name = source_schema + .clone() + .unwrap_or(DEFAULT_SCHEMA_NAME.to_string()); + let (source, _) = catalog_reader.get_source_by_name( + db_name, + SchemaPath::Name(schema_name.as_str()), + source_name.as_str(), + )?; + source.clone() + }; + + let source_with_properties = source.with_properties.clone().into_iter().collect(); + + let connect_properties = derive_connect_properties( + &source_with_properties, + cdc_table.external_table_name.clone(), + )?; + + // read cdc table schema from external db or parsing the schema from SQL definitions + let (columns, pk_names) = if need_auto_schema_map { + let _connector = connect_properties.get(UPSTREAM_SOURCE_KEY).unwrap(); + let _config = + ExternalTableConfig::try_from_hashmap(connect_properties.clone(), false) + .context("failed to extract external table config")?; + // TODO: create external table according to connector + let pg_table = PostgresExternalTable::new(); + pg_table.connect().await?; + ( + pg_table + .column_descs() + .into_iter() + .map(|column_desc| ColumnCatalog { + column_desc, + is_hidden: false, + }) + .collect(), + pg_table.pk_names(), + ) + } else { + ( + bind_sql_columns(&column_defs)?, + bind_sql_pk_names(&column_defs, &constraints)?, + ) + }; + let (plan, table) = gen_create_table_plan_for_cdc_source( - context.into(), - cdc_table.source_name.clone(), - table_name.clone(), + handler_args, + explain_options, + source.clone(), cdc_table.external_table_name.clone(), - column_defs, - constraints, + columns, + pk_names, + connect_properties, col_id_gen, on_conflict, with_version_column, include_column_options, - )?; + resolved_table_name, + database_id, + schema_id, + ) + .await?; ((plan, None, table), TableJobType::SharedCdcSource) } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index c8080498e176..ba7f3f8b80bd 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -26,12 +26,15 @@ use risingwave_common::hash::{ParallelUnitMapping, VirtualNode}; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::epoch::Epoch; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::util::stream_graph_visitor::{ visit_fragment, visit_stream_node, visit_stream_node_cont, }; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::dispatch_source_prop; use risingwave_connector::error::ConnectorError; +use risingwave_connector::source::cdc::external::postgres::PostgresExternalTable; +use risingwave_connector::source::cdc::external::ExternalTableConfig; use risingwave_connector::source::cdc::CdcSourceType; use risingwave_connector::source::{ ConnectorProperties, SourceEnumeratorContext, SourceProperties, SplitEnumerator, @@ -53,6 +56,7 @@ use risingwave_pb::ddl_service::{ }; use risingwave_pb::meta::table_fragments::PbFragment; use risingwave_pb::meta::PbTableParallelism; +use risingwave_pb::plan_common::ColumnCatalog; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ Dispatcher, DispatcherType, FragmentTypeFlag, MergeNode, PbStreamFragmentGraph, @@ -770,6 +774,51 @@ impl DdlController { src.id = self.gen_unique_id::<{ IdCategory::Table }>().await?; } fill_table_stream_graph_info(src, table, *job_type, &mut fragment_graph); + + // // TODO: read the table schema from upstream + // if matches!(job_type, TableJobType::SharedCdcSource) { + // let mut table_desc = None; + // + // for fragment in fragment_graph.fragments.values_mut() { + // visit_fragment(fragment, |node_body| { + // if let NodeBody::StreamCdcScan(cdc_scan_node) = node_body + // && let Some(external_table_desc) = + // cdc_scan_node.cdc_table_desc.as_mut() + // { + // table_desc = Some(external_table_desc.clone()); + // } + // }); + // } + // + // if let Some(table_desc) = table_desc { + // let properties: HashMap = + // table_desc.connect_properties.clone().into_iter().collect(); + // let _connector = properties.get(UPSTREAM_SOURCE_KEY).unwrap(); + // let _config = serde_json::from_value::( + // serde_json::to_value(properties).unwrap(), + // ) + // .context("failed to extract external table config")?; + // + // // connect + // let pg_table = PostgresExternalTable::new(); + // pg_table.connect().await?; + // table.columns = pg_table + // .column_descs() + // .into_iter() + // .map(|col| ColumnCatalog { + // column_desc: Some(col.to_protobuf()), + // is_hidden: false, + // }) + // .collect(); + // table.pk = pg_table + // .pk_indices() + // .into_iter() + // .map(|index| { + // ColumnOrder::new(index, OrderType::ascending()).to_protobuf() + // }) + // .collect(); + // } + // } } StreamingJob::Source(_) => { // set the inner source id of source node. From 0de0f159f89efdf4986cee1549aa6fbf5c21056e Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 28 May 2024 13:17:34 +0800 Subject: [PATCH 20/36] refactor --- .../src/source/cdc/external/postgres.rs | 11 +- src/frontend/src/handler/create_table.rs | 177 ++++++++++-------- 2 files changed, 110 insertions(+), 78 deletions(-) diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 0f1a34913e56..583a116d695a 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -87,9 +87,14 @@ impl PostgresExternalTable { } } - pub async fn connect(&self) -> ConnectorResult<()> { - todo!("connect to postgres"); - // connect to external db and read the schema + pub async fn connect(config: ExternalTableConfig) -> ConnectorResult { + // TODO: connect to external db and read the schema + tracing::debug!("connect to postgres"); + + Ok(Self { + columns: vec![], + pk_indices: vec![], + }) } pub fn column_descs(&self) -> Vec { diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index fb7135bd1f95..68248bd630ec 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -942,52 +942,15 @@ pub(super) async fn handle_create_table_plan( } (None, Some(cdc_table)) => { - if append_only { - return Err(ErrorCode::NotSupported( - "append only modifier on the table created from a CDC source".into(), - "Remove the APPEND ONLY clause".into(), - ) - .into()); - } - - if !source_watermarks.is_empty() { - return Err(ErrorCode::NotSupported( - "watermark defined on the table created from a CDC source".into(), - "Remove the Watermark definitions".into(), - ) - .into()); - } - for c in &column_defs { - for op in &c.options { - if let ColumnOption::GeneratedColumns(_) = op.option { - return Err(ErrorCode::NotSupported( - "generated column defined on the table created from a CDC source" - .into(), - "Remove the generated column in the column list".into(), - ) - .into()); - } - } - } - if !wildcard_idx.is_some() - && !constraints.iter().any(|c| { - matches!( - c, - TableConstraint::Unique { - is_primary: true, - .. - } - ) - }) - { - return Err(ErrorCode::NotSupported( - "CDC table without primary key constraint is not supported".to_owned(), - "Please define a primary key".to_owned(), - ) - .into()); - } + sanity_check_for_cdc_table( + append_only, + &column_defs, + &wildcard_idx, + &constraints, + &source_watermarks, + )?; - let session = handler_args.session.clone(); + let session = &handler_args.session; let db_name = session.database(); let (schema_name, resolved_table_name) = Binder::resolve_schema_qualified_name(db_name, table_name)?; @@ -995,7 +958,6 @@ pub(super) async fn handle_create_table_plan( session.get_database_and_schema_id_for_create(schema_name.clone())?; // cdc table cannot be append-only - let need_auto_schema_map = wildcard_idx.is_some(); let (source_schema, source_name) = Binder::resolve_schema_qualified_name(db_name, cdc_table.source_name.clone())?; @@ -1011,45 +973,24 @@ pub(super) async fn handle_create_table_plan( )?; source.clone() }; - let source_with_properties = source.with_properties.clone().into_iter().collect(); - let connect_properties = derive_connect_properties( &source_with_properties, cdc_table.external_table_name.clone(), )?; - // read cdc table schema from external db or parsing the schema from SQL definitions - let (columns, pk_names) = if need_auto_schema_map { - let _connector = connect_properties.get(UPSTREAM_SOURCE_KEY).unwrap(); - let _config = - ExternalTableConfig::try_from_hashmap(connect_properties.clone(), false) - .context("failed to extract external table config")?; - // TODO: create external table according to connector - let pg_table = PostgresExternalTable::new(); - pg_table.connect().await?; - ( - pg_table - .column_descs() - .into_iter() - .map(|column_desc| ColumnCatalog { - column_desc, - is_hidden: false, - }) - .collect(), - pg_table.pk_names(), - ) - } else { - ( - bind_sql_columns(&column_defs)?, - bind_sql_pk_names(&column_defs, &constraints)?, - ) - }; + let (columns, pk_names) = derive_schema_for_cdc_table( + &column_defs, + &constraints, + connect_properties.clone(), + wildcard_idx.is_some(), + ) + .await?; let (plan, table) = gen_create_table_plan_for_cdc_source( handler_args, explain_options, - source.clone(), + source, cdc_table.external_table_name.clone(), columns, pk_names, @@ -1076,6 +1017,92 @@ pub(super) async fn handle_create_table_plan( Ok((plan, source, table, job_type)) } +fn sanity_check_for_cdc_table( + append_only: bool, + column_defs: &Vec, + wildcard_idx: &Option, + constraints: &Vec, + source_watermarks: &Vec, +) -> Result<()> { + if append_only { + return Err(ErrorCode::NotSupported( + "append only modifier on the table created from a CDC source".into(), + "Remove the APPEND ONLY clause".into(), + ) + .into()); + } + + if !source_watermarks.is_empty() { + return Err(ErrorCode::NotSupported( + "watermark defined on the table created from a CDC source".into(), + "Remove the Watermark definitions".into(), + ) + .into()); + } + for c in column_defs { + for op in &c.options { + if let ColumnOption::GeneratedColumns(_) = op.option { + return Err(ErrorCode::NotSupported( + "generated column defined on the table created from a CDC source".into(), + "Remove the generated column in the column list".into(), + ) + .into()); + } + } + } + if !wildcard_idx.is_some() + && !constraints.iter().any(|c| { + matches!( + c, + TableConstraint::Unique { + is_primary: true, + .. + } + ) + }) + { + return Err(ErrorCode::NotSupported( + "CDC table without primary key constraint is not supported".to_owned(), + "Please define a primary key".to_owned(), + ) + .into()); + } + + Ok(()) +} + +async fn derive_schema_for_cdc_table( + column_defs: &Vec, + constraints: &Vec, + connect_properties: HashMap, + need_auto_schema_map: bool, +) -> Result<(Vec, Vec)> { + // read cdc table schema from external db or parsing the schema from SQL definitions + if need_auto_schema_map { + let _connector = connect_properties.get(UPSTREAM_SOURCE_KEY).unwrap(); + let config = ExternalTableConfig::try_from_hashmap(connect_properties.clone(), false) + .context("failed to extract external table config")?; + // TODO: create external table according to connector + let pg_table = PostgresExternalTable::connect(config).await?; + Ok(( + pg_table + .column_descs() + .into_iter() + .map(|column_desc| ColumnCatalog { + column_desc, + is_hidden: false, + }) + .collect(), + pg_table.pk_names(), + )) + } else { + Ok(( + bind_sql_columns(column_defs)?, + bind_sql_pk_names(column_defs, constraints)?, + )) + } +} + #[allow(clippy::too_many_arguments)] pub async fn handle_create_table( handler_args: HandlerArgs, From 173c91fb7f7b601d645b2a30a50d37c39427fc9b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 29 May 2024 10:00:41 +0800 Subject: [PATCH 21/36] PG: auto map schema --- Cargo.lock | 4 + Cargo.toml | 1 + src/connector/Cargo.toml | 2 + src/connector/src/error.rs | 1 + src/connector/src/macros.rs | 1 + src/connector/src/source/cdc/external/mod.rs | 50 ++++- .../src/source/cdc/external/mysql.rs | 25 ++- .../src/source/cdc/external/postgres.rs | 199 +++++++++++++++--- src/frontend/src/handler/create_table.rs | 21 +- 9 files changed, 252 insertions(+), 52 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 485d2435e147..27521c691258 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10566,12 +10566,14 @@ dependencies = [ "rustls-pemfile 2.1.1", "rustls-pki-types", "rw_futures_util", + "sea-schema", "serde", "serde_derive", "serde_json", "serde_with", "serde_yaml", "simd-json", + "sqlx", "strum 0.26.1", "strum_macros 0.26.1", "syn 1.0.109", @@ -12361,7 +12363,9 @@ checksum = "30d148608012d25222442d1ebbfafd1228dbc5221baf4ec35596494e27a2394e" dependencies = [ "futures", "sea-query", + "sea-query-binder", "sea-schema-derive", + "sqlx", ] [[package]] diff --git a/Cargo.toml b/Cargo.toml index ffeff47afb9a..bbe920091570 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -177,6 +177,7 @@ sea-orm = { version = "0.12.14", features = [ "sqlx-sqlite", "runtime-tokio-native-tls", ] } +sqlx = "0.7" tokio-util = "0.7" tracing-opentelemetry = "0.22" rand = { version = "0.8", features = ["small_rng"] } diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 1f5bf445da0f..68f5b3b5c1bf 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -128,6 +128,8 @@ serde_with = { version = "3", features = ["json"] } simd-json = "0.13.3" strum = "0.26" strum_macros = "0.26" +sea-schema = {version = "0.14", features = ["default", "sqlx-postgres", "sqlx-mysql"]} +sqlx = { workspace = true } tempfile = "3" thiserror = "1" thiserror-ext = { workspace = true } diff --git a/src/connector/src/error.rs b/src/connector/src/error.rs index c3a438c16838..ab4b3e7bc37b 100644 --- a/src/connector/src/error.rs +++ b/src/connector/src/error.rs @@ -46,6 +46,7 @@ def_anyhow_newtype! { // Connector errors opendal::Error => transparent, // believed to be self-explanatory + sqlx::Error => transparent, // believed to be self-explanatory mysql_async::Error => "MySQL error", tokio_postgres::Error => "Postgres error", apache_avro::Error => "Avro error", diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index b369e6d8a11e..3b97367a6430 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -238,6 +238,7 @@ macro_rules! impl_cdc_source_type { } )* + #[derive(Clone)] pub enum CdcSourceType { $( $cdc_source_type, diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index 284b71642d01..0764aed54656 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -22,13 +22,14 @@ pub mod mysql; use std::collections::HashMap; use std::fmt; -use anyhow::Context; +use anyhow::{anyhow, Context}; use futures::stream::BoxStream; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; +use mysql_async::prelude::AsQuery; use risingwave_common::bail; -use risingwave_common::catalog::{Schema, OFFSET_COLUMN_NAME}; +use risingwave_common::catalog::{ColumnDesc, Schema, OFFSET_COLUMN_NAME}; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; @@ -43,7 +44,8 @@ use crate::source::cdc::external::mysql::{ use crate::source::cdc::external::postgres::{ PostgresExternalTable, PostgresExternalTableReader, PostgresOffset, }; -use crate::source::UnknownFields; +use crate::source::cdc::CdcSourceType; +use crate::source::{UnknownFields, UPSTREAM_SOURCE_KEY}; use crate::WithPropertiesExt; #[derive(Debug)] @@ -203,6 +205,8 @@ pub enum ExternalTableReaderImpl { #[derive(Debug, Clone, Deserialize)] pub struct ExternalTableConfig { + pub connector: String, + #[serde(rename = "hostname")] pub host: String, pub port: String, @@ -219,14 +223,13 @@ pub struct ExternalTableConfig { /// This field is optional. #[serde(rename = "ssl.mode", default = "Default::default")] pub sslmode: SslMode, - - #[serde(flatten)] - pub unknown_fields: HashMap, } -impl UnknownFields for ExternalTableConfig { - fn unknown_fields(&self) -> HashMap { - self.unknown_fields.clone() +impl ExternalTableConfig { + pub fn try_from_hashmap(connect_properties: HashMap) -> ConnectorResult { + let json_value = serde_json::to_value(connect_properties)?; + let config = serde_json::from_value::(json_value)?; + Ok(config) } } @@ -320,6 +323,35 @@ pub enum ExternalTableImpl { Postgres(PostgresExternalTable), } +impl ExternalTableImpl { + pub async fn connect(config: ExternalTableConfig) -> ConnectorResult { + let cdc_source_type = CdcSourceType::from(config.connector.as_str()); + match cdc_source_type { + CdcSourceType::Mysql => Ok(ExternalTableImpl::MySql( + MySqlExternalTable::connect(config).await?, + )), + CdcSourceType::Postgres => Ok(ExternalTableImpl::Postgres( + PostgresExternalTable::connect(config).await?, + )), + _ => Err(anyhow!("Unsupported cdc connector type: {}", config.connector).into()), + } + } + + pub fn column_descs(&self) -> &Vec { + match self { + ExternalTableImpl::MySql(mysql) => mysql.column_descs(), + ExternalTableImpl::Postgres(postgres) => postgres.column_descs(), + } + } + + pub fn pk_names(&self) -> &Vec { + match self { + ExternalTableImpl::MySql(mysql) => mysql.pk_names(), + ExternalTableImpl::Postgres(postgres) => postgres.pk_names(), + } + } +} + #[cfg(test)] mod tests { diff --git a/src/connector/src/source/cdc/external/mysql.rs b/src/connector/src/source/cdc/external/mysql.rs index 1cf167bdad3f..de9b34b5db5e 100644 --- a/src/connector/src/source/cdc/external/mysql.rs +++ b/src/connector/src/source/cdc/external/mysql.rs @@ -23,7 +23,7 @@ use mysql_async::prelude::*; use mysql_common::params::Params; use mysql_common::value::Value; use risingwave_common::bail; -use risingwave_common::catalog::{Schema, OFFSET_COLUMN_NAME}; +use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema, OFFSET_COLUMN_NAME}; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; @@ -66,7 +66,28 @@ impl MySqlOffset { } pub struct MySqlExternalTable { - config: ExternalTableConfig, + columns: Vec, + pk_names: Vec, +} + +impl MySqlExternalTable { + pub async fn connect(config: ExternalTableConfig) -> ConnectorResult { + // TODO: connect to external db and read the schema + tracing::debug!("connect to mysql"); + + Ok(Self { + columns: vec![], + pk_names: vec![], + }) + } + + pub fn column_descs(&self) -> &Vec { + &self.columns + } + + pub fn pk_names(&self) -> &Vec { + &self.pk_names + } } pub struct MySqlExternalTableReader { diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 583a116d695a..cb972ea000a2 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -14,8 +14,9 @@ use std::cmp::Ordering; use std::collections::HashMap; +use std::sync::Arc; -use anyhow::Context; +use anyhow::{anyhow, Context}; use futures::stream::BoxStream; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; @@ -24,9 +25,13 @@ use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; use postgres_openssl::MakeTlsConnector; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId, Schema}; use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::DataType; +use risingwave_common::types::{DataType, StructType}; use risingwave_common::util::iter_util::ZipEqFast; +use sea_schema::postgres::def::{ColumnType, TableInfo, Type}; +use sea_schema::postgres::discovery::SchemaDiscovery; use serde_derive::{Deserialize, Serialize}; +use sqlx::postgres::{PgConnectOptions, PgSslMode}; +use sqlx::PgPool; use thiserror_ext::AsReport; use tokio_postgres::types::PgLsn; use tokio_postgres::{NoTls, Statement}; @@ -75,48 +80,155 @@ impl PostgresOffset { } pub struct PostgresExternalTable { - columns: Vec, - pk_indices: Vec, + column_descs: Vec, + pk_names: Vec, } impl PostgresExternalTable { - pub fn new() -> Self { - Self { - columns: vec![], - pk_indices: vec![], + pub async fn connect(config: ExternalTableConfig) -> ConnectorResult { + tracing::debug!("connect to postgres external table"); + let options = PgConnectOptions::new() + .username(&config.username) + .password(&config.password) + .host(&config.host) + .port(config.port.parse::().unwrap()) + .database(&config.database) + .ssl_mode(match config.sslmode { + SslMode::Disabled => PgSslMode::Disable, + SslMode::Preferred => PgSslMode::Prefer, + SslMode::Required => PgSslMode::Require, + }); + + let connection = PgPool::connect_with(options).await?; + let schema_discovery = SchemaDiscovery::new(connection, config.schema.as_str()); + // fetch column schema and primary key + let empty_map = HashMap::new(); + let table_schema = schema_discovery + .discover_table( + TableInfo { + name: config.table.clone(), + of_type: None, + }, + &empty_map, + ) + .await?; + + let mut column_descs = vec![]; + for col in table_schema.columns.iter() { + let data_type = type_to_rw_type(&col.col_type)?; + column_descs.push(ColumnDesc::named( + col.name.clone(), + ColumnId::placeholder(), + data_type, + )); } - } - pub async fn connect(config: ExternalTableConfig) -> ConnectorResult { - // TODO: connect to external db and read the schema - tracing::debug!("connect to postgres"); + if table_schema.primary_key_constraints.is_empty() { + return Err(anyhow!("Postgres table doesn't define the primary key").into()); + } + let mut pk_names = vec![]; + table_schema.primary_key_constraints.iter().for_each(|pk| { + pk_names.extend(pk.columns.clone()); + }); Ok(Self { - columns: vec![], - pk_indices: vec![], + column_descs, + pk_names, }) } - pub fn column_descs(&self) -> Vec { - vec![ - ColumnDesc::named("v1", ColumnId::new(1), DataType::Int32), - ColumnDesc::named("v2", ColumnId::new(2), DataType::Varchar), - ColumnDesc::named("v3", ColumnId::new(3), DataType::Decimal), - ColumnDesc::named("v4", ColumnId::new(4), DataType::Date), - ] + pub fn column_descs(&self) -> &Vec { + &self.column_descs } - pub fn pk_names(&self) -> Vec { - vec!["v1".to_string(), "v2".to_string()] + pub fn pk_names(&self) -> &Vec { + &self.pk_names } +} - pub fn pk_indices(&self) -> Vec { - vec![0, 1] - } +fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { + let dtype = match col_type { + ColumnType::SmallInt | ColumnType::SmallSerial => DataType::Int16, + ColumnType::Integer | ColumnType::Serial => DataType::Int32, + ColumnType::BigInt | ColumnType::BigSerial => DataType::Int64, + ColumnType::Money | ColumnType::Decimal(_) | ColumnType::Numeric(_) => DataType::Decimal, + ColumnType::Real => DataType::Float32, + ColumnType::DoublePrecision => DataType::Float64, + ColumnType::Varchar(_) | ColumnType::Char(_) | ColumnType::Text => DataType::Varchar, + ColumnType::Bytea => DataType::Bytea, + ColumnType::Timestamp(_) => DataType::Timestamp, + ColumnType::TimestampWithTimeZone(_) => DataType::Timestamptz, + ColumnType::Date => DataType::Date, + ColumnType::Time(_) | ColumnType::TimeWithTimeZone(_) => DataType::Time, + ColumnType::Interval(_) => DataType::Interval, + ColumnType::Boolean => DataType::Boolean, + ColumnType::Point => DataType::Struct(StructType::new(vec![ + ("x", DataType::Float32), + ("y", DataType::Float32), + ])), + ColumnType::Uuid => DataType::Varchar, + ColumnType::Xml => DataType::Varchar, + ColumnType::Json => DataType::Jsonb, + ColumnType::JsonBinary => DataType::Jsonb, + ColumnType::Array(def) => { + let item_type = match def.col_type.as_ref() { + Some(ty) => type_to_rw_type(ty.as_ref())?, + None => { + return Err(anyhow!("array type missing element type").into()); + } + }; + + DataType::List(Box::new(item_type)) + } + ColumnType::PgLsn => DataType::Int64, + ColumnType::Cidr + | ColumnType::Inet + | ColumnType::MacAddr + | ColumnType::MacAddr8 + | ColumnType::Int4Range + | ColumnType::Int8Range + | ColumnType::NumRange + | ColumnType::TsRange + | ColumnType::TsTzRange + | ColumnType::DateRange + | ColumnType::Enum(_) => DataType::Varchar, + + ColumnType::Line => { + return Err(anyhow!("line type not supported").into()); + } + ColumnType::Lseg => { + return Err(anyhow!("lseg type not supported").into()); + } + ColumnType::Box => { + return Err(anyhow!("box type not supported").into()); + } + ColumnType::Path => { + return Err(anyhow!("path type not supported").into()); + } + ColumnType::Polygon => { + return Err(anyhow!("polygon type not supported").into()); + } + ColumnType::Circle => { + return Err(anyhow!("circle type not supported").into()); + } + ColumnType::Bit(_) => { + return Err(anyhow!("bit type not supported").into()); + } + ColumnType::TsVector => { + return Err(anyhow!("tsvector type not supported").into()); + } + ColumnType::TsQuery => { + return Err(anyhow!("tsquery type not supported").into()); + } + ColumnType::Unknown(_) => { + return Err(anyhow!("unknown column type").into()); + } + }; + + Ok(dtype) } pub struct PostgresExternalTableReader { - config: ExternalTableConfig, rw_schema: Schema, field_names: String, prepared_scan_stmt: Statement, @@ -246,7 +358,6 @@ impl PostgresExternalTableReader { }; Ok(Self { - config, rw_schema, field_names, prepared_scan_stmt, @@ -342,15 +453,43 @@ impl PostgresExternalTableReader { #[cfg(test)] mod tests { + use std::collections::HashMap; + use futures::pin_mut; use futures_async_stream::for_await; use maplit::{convert_args, hashmap}; use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema}; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; + use sea_schema::postgres::def::TableInfo; + use sea_schema::postgres::discovery::SchemaDiscovery; + use sqlx::PgPool; + + use crate::source::cdc::external::postgres::{ + PostgresExternalTable, PostgresExternalTableReader, PostgresOffset, + }; + use crate::source::cdc::external::{ExternalTableConfig, ExternalTableReader, SchemaTableName}; - use crate::source::cdc::external::postgres::{PostgresExternalTableReader, PostgresOffset}; - use crate::source::cdc::external::{ExternalTableReader, SchemaTableName}; + #[ignore] + #[tokio::test] + async fn test_postgres_schema() { + let config = ExternalTableConfig { + connector: "postgres-cdc".to_string(), + host: "localhost".to_string(), + port: "8432".to_string(), + username: "myuser".to_string(), + password: "123456".to_string(), + database: "mydb".to_string(), + schema: "public".to_string(), + table: "mytest".to_string(), + sslmode: Default::default(), + }; + + let table = PostgresExternalTable::connect(config).await.unwrap(); + + println!("columns: {:?}", &table.column_descs); + println!("primary keys: {:?}", &table.pk_names); + } #[test] fn test_postgres_offset() { diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 68248bd630ec..5dc6e847eadd 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -31,9 +31,8 @@ use risingwave_common::util::value_encoding::DatumToProtoExt; use risingwave_connector::source; use risingwave_connector::source::cdc::external::postgres::PostgresExternalTable; use risingwave_connector::source::cdc::external::{ - ExternalTableConfig, DATABASE_NAME_KEY, SCHEMA_NAME_KEY, TABLE_NAME_KEY, + ExternalTableConfig, ExternalTableImpl, DATABASE_NAME_KEY, SCHEMA_NAME_KEY, TABLE_NAME_KEY, }; -use risingwave_connector::source::TryFromHashmap; use risingwave_pb::catalog::{PbSource, PbTable, Table, WatermarkDesc}; use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; @@ -733,7 +732,7 @@ fn gen_table_plan_inner( } #[allow(clippy::too_many_arguments)] -pub(crate) async fn gen_create_table_plan_for_cdc_source( +pub(crate) async fn gen_create_table_plan_for_cdc_table( handler_args: HandlerArgs, explain_options: ExplainOptions, source: Arc, @@ -987,7 +986,7 @@ pub(super) async fn handle_create_table_plan( ) .await?; - let (plan, table) = gen_create_table_plan_for_cdc_source( + let (plan, table) = gen_create_table_plan_for_cdc_table( handler_args, explain_options, source, @@ -1079,21 +1078,21 @@ async fn derive_schema_for_cdc_table( ) -> Result<(Vec, Vec)> { // read cdc table schema from external db or parsing the schema from SQL definitions if need_auto_schema_map { - let _connector = connect_properties.get(UPSTREAM_SOURCE_KEY).unwrap(); - let config = ExternalTableConfig::try_from_hashmap(connect_properties.clone(), false) + let config = ExternalTableConfig::try_from_hashmap(connect_properties) .context("failed to extract external table config")?; - // TODO: create external table according to connector - let pg_table = PostgresExternalTable::connect(config).await?; + + let table = ExternalTableImpl::connect(config).await?; Ok(( - pg_table + table .column_descs() - .into_iter() + .iter() + .cloned() .map(|column_desc| ColumnCatalog { column_desc, is_hidden: false, }) .collect(), - pg_table.pk_names(), + table.pk_names().clone(), )) } else { Ok(( From 31a84854245d5de54a15a372646b0cdecf2e268f Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 29 May 2024 10:52:49 +0800 Subject: [PATCH 22/36] WIP: mysql auto map schema --- .../src/source/cdc/external/mysql.rs | 149 +++++++++++++++++- 1 file changed, 143 insertions(+), 6 deletions(-) diff --git a/src/connector/src/source/cdc/external/mysql.rs b/src/connector/src/source/cdc/external/mysql.rs index de9b34b5db5e..d1009f24ebd1 100644 --- a/src/connector/src/source/cdc/external/mysql.rs +++ b/src/connector/src/source/cdc/external/mysql.rs @@ -14,7 +14,7 @@ use std::collections::HashMap; -use anyhow::Context; +use anyhow::{anyhow, Context}; use futures::stream::BoxStream; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; @@ -27,7 +27,11 @@ use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema, OFFSET_COLUMN_NAM use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqFast; +use sea_schema::mysql::def::{CharSet, Collation, ColumnKey, ColumnType, StorageEngine, TableInfo}; +use sea_schema::mysql::discovery::SchemaDiscovery; use serde_derive::{Deserialize, Serialize}; +use sqlx::mysql::MySqlConnectOptions; +use sqlx::MySqlPool; use crate::error::{ConnectorError, ConnectorResult}; use crate::source::cdc::external::{ @@ -66,23 +70,60 @@ impl MySqlOffset { } pub struct MySqlExternalTable { - columns: Vec, + column_descs: Vec, pk_names: Vec, } impl MySqlExternalTable { pub async fn connect(config: ExternalTableConfig) -> ConnectorResult { - // TODO: connect to external db and read the schema tracing::debug!("connect to mysql"); + let options = MySqlConnectOptions::new() + .username(&config.username) + .password(&config.password) + .host(&config.host) + .port(config.port.parse::().unwrap()) + .database(&config.database) + .ssl_mode(match config.sslmode { + SslMode::Disabled | SslMode::Preferred => sqlx::mysql::MySqlSslMode::Disabled, + SslMode::Required => sqlx::mysql::MySqlSslMode::Required, + }); + + let connection = MySqlPool::connect_with(options).await?; + let schema_discovery = SchemaDiscovery::new(connection, config.database.as_str()); + + let table_schema = schema_discovery + .discover_table(TableInfo { + name: config.table.clone(), + engine: StorageEngine::InnoDb, + auto_increment: None, + char_set: CharSet::Utf8Mb4, + collation: Collation::Utf8Mb40900AiCi, + comment: "".to_string(), + }) + .await?; + + let mut column_descs = vec![]; + let mut pk_names = vec![]; + for col in table_schema.columns.iter() { + let data_type = type_to_rw_type(&col.col_type)?; + column_descs.push(ColumnDesc::named( + col.name.clone(), + ColumnId::placeholder(), + data_type, + )); + if matches!(col.key, ColumnKey::Primary) { + pk_names.push(col.name.clone()); + } + } Ok(Self { - columns: vec![], - pk_names: vec![], + column_descs, + pk_names, }) } pub fn column_descs(&self) -> &Vec { - &self.columns + &self.column_descs } pub fn pk_names(&self) -> &Vec { @@ -90,6 +131,77 @@ impl MySqlExternalTable { } } +fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { + let dtype = match col_type { + ColumnType::Serial => DataType::Int32, + ColumnType::Bit(attr) => { + // return Err(anyhow!("line type not supported").into()); + tracing::info!("bit attr: {:?}", attr); + DataType::Boolean + } + ColumnType::TinyInt(_) | ColumnType::SmallInt(_) => DataType::Int16, + ColumnType::Bool => DataType::Boolean, + ColumnType::MediumInt(_) => DataType::Int32, + ColumnType::Int(_) => DataType::Int32, + ColumnType::BigInt(_) => DataType::Int64, + ColumnType::Decimal(_) => DataType::Decimal, + ColumnType::Float(_) => DataType::Float32, + ColumnType::Double(_) => DataType::Float64, + ColumnType::Date => DataType::Date, + ColumnType::Time(_) => DataType::Time, + ColumnType::DateTime(_) => DataType::Timestamp, + ColumnType::Timestamp(_) => DataType::Timestamptz, + ColumnType::Year => DataType::Int32, + ColumnType::Char(_) + | ColumnType::NChar(_) + | ColumnType::Varchar(_) + | ColumnType::NVarchar(_) => DataType::Varchar, + ColumnType::Binary(_) | ColumnType::Varbinary(_) => DataType::Bytea, + ColumnType::Text(_) + | ColumnType::TinyText(_) + | ColumnType::MediumText(_) + | ColumnType::LongText(_) => DataType::Varchar, + ColumnType::Blob(_) + | ColumnType::TinyBlob + | ColumnType::MediumBlob + | ColumnType::LongBlob => DataType::Bytea, + ColumnType::Enum(_) => DataType::Varchar, + ColumnType::Json => DataType::Jsonb, + ColumnType::Set(_) => { + return Err(anyhow!("set type not supported").into()); + } + ColumnType::Geometry(_) => { + return Err(anyhow!("geometry type not supported").into()); + } + ColumnType::Point(_) => { + return Err(anyhow!("point type not supported").into()); + } + ColumnType::LineString(_) => { + return Err(anyhow!("line string type not supported").into()); + } + ColumnType::Polygon(_) => { + return Err(anyhow!("polygon type not supported").into()); + } + ColumnType::MultiPoint(_) => { + return Err(anyhow!("multi point type not supported").into()); + } + ColumnType::MultiLineString(_) => { + return Err(anyhow!("multi line string type not supported").into()); + } + ColumnType::MultiPolygon(_) => { + return Err(anyhow!("multi polygon type not supported").into()); + } + ColumnType::GeometryCollection(_) => { + return Err(anyhow!("geometry collection type not supported").into()); + } + ColumnType::Unknown(_) => { + return Err(anyhow!("unknown column type").into()); + } + }; + + Ok(dtype) +} + pub struct MySqlExternalTableReader { rw_schema: Schema, field_names: String, @@ -326,3 +438,28 @@ impl MySqlExternalTableReader { format!("`{}`", column) } } +#[cfg(test)] +mod tests { + use crate::source::cdc::external::mysql::MySqlExternalTable; + use crate::source::cdc::external::ExternalTableConfig; + + #[ignore] + #[tokio::test] + async fn test_mysql_schema() { + let config = ExternalTableConfig { + connector: "mysql-cdc".to_string(), + host: "localhost".to_string(), + port: "8306".to_string(), + username: "root".to_string(), + password: "123456".to_string(), + database: "mydb".to_string(), + schema: "".to_string(), + table: "part".to_string(), + sslmode: Default::default(), + }; + + let table = MySqlExternalTable::connect(config).await.unwrap(); + println!("columns: {:?}", &table.column_descs); + println!("primary keys: {:?}", &table.pk_names); + } +} From a02542071accdd582e4a2a93acc5c12e8996e209 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Wed, 29 May 2024 10:55:15 +0800 Subject: [PATCH 23/36] clean code --- src/meta/src/rpc/ddl_controller.rs | 45 ------------------------------ 1 file changed, 45 deletions(-) diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 31776137be77..03769abd8efc 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -775,51 +775,6 @@ impl DdlController { src.id = self.gen_unique_id::<{ IdCategory::Table }>().await?; } fill_table_stream_graph_info(src, table, *job_type, &mut fragment_graph); - - // // TODO: read the table schema from upstream - // if matches!(job_type, TableJobType::SharedCdcSource) { - // let mut table_desc = None; - // - // for fragment in fragment_graph.fragments.values_mut() { - // visit_fragment(fragment, |node_body| { - // if let NodeBody::StreamCdcScan(cdc_scan_node) = node_body - // && let Some(external_table_desc) = - // cdc_scan_node.cdc_table_desc.as_mut() - // { - // table_desc = Some(external_table_desc.clone()); - // } - // }); - // } - // - // if let Some(table_desc) = table_desc { - // let properties: HashMap = - // table_desc.connect_properties.clone().into_iter().collect(); - // let _connector = properties.get(UPSTREAM_SOURCE_KEY).unwrap(); - // let _config = serde_json::from_value::( - // serde_json::to_value(properties).unwrap(), - // ) - // .context("failed to extract external table config")?; - // - // // connect - // let pg_table = PostgresExternalTable::new(); - // pg_table.connect().await?; - // table.columns = pg_table - // .column_descs() - // .into_iter() - // .map(|col| ColumnCatalog { - // column_desc: Some(col.to_protobuf()), - // is_hidden: false, - // }) - // .collect(); - // table.pk = pg_table - // .pk_indices() - // .into_iter() - // .map(|index| { - // ColumnOrder::new(index, OrderType::ascending()).to_protobuf() - // }) - // .collect(); - // } - // } } StreamingJob::Source(_) => { // set the inner source id of source node. From 3117fac5dbe7c3c4907e0e7c7fa4def16bea74b0 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 30 May 2024 19:03:14 +0800 Subject: [PATCH 24/36] fix mysql named param --- .../source/common/MySqlValidator.java | 7 +++---- .../src/source/cdc/external/mysql.rs | 21 +++++++++++++------ 2 files changed, 18 insertions(+), 10 deletions(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MySqlValidator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MySqlValidator.java index a130c7107499..a2f63a28bbd7 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MySqlValidator.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MySqlValidator.java @@ -195,10 +195,9 @@ private void validateTableSchema() throws SQLException { var field = res.getString(1); var dataType = res.getString(2); var key = res.getString(3); - schema.put(field.toLowerCase(), dataType); + schema.put(field, dataType); if (key.equalsIgnoreCase("PRI")) { - // RisingWave always use lower case for column name - pkFields.add(field.toLowerCase()); + pkFields.add(field); } } @@ -208,7 +207,7 @@ private void validateTableSchema() throws SQLException { if (e.getKey().startsWith(ValidatorUtils.INTERNAL_COLUMN_PREFIX)) { continue; } - var dataType = schema.get(e.getKey().toLowerCase()); + var dataType = schema.get(e.getKey()); if (dataType == null) { throw ValidatorUtils.invalidArgument( "Column '" + e.getKey() + "' not found in the upstream database"); diff --git a/src/connector/src/source/cdc/external/mysql.rs b/src/connector/src/source/cdc/external/mysql.rs index d1009f24ebd1..cd4fc7dcb54f 100644 --- a/src/connector/src/source/cdc/external/mysql.rs +++ b/src/connector/src/source/cdc/external/mysql.rs @@ -371,13 +371,14 @@ impl MySqlExternalTableReader { DataType::Timestamp => Value::from(value.into_timestamp().0), _ => bail!("unsupported primary key data type: {}", ty), }; - ConnectorResult::Ok((pk.clone(), val)) + ConnectorResult::Ok((pk.to_lowercase(), val)) } else { bail!("primary key {} cannot be null", pk); } }) .try_collect::<_, _, ConnectorError>()?; + tracing::debug!("snapshot read params: {:?}", ¶ms); let rs_stream = sql .with(Params::from(params)) .stream::(&mut *conn) @@ -401,29 +402,37 @@ impl MySqlExternalTableReader { // mysql cannot leverage the given key to narrow down the range of scan, // we need to rewrite the comparison conditions by our own. // (a, b) > (x, y) => (`a` > x) OR ((`a` = x) AND (`b` > y)) - pub(crate) fn filter_expression(columns: &[String]) -> String { + fn filter_expression(columns: &[String]) -> String { let mut conditions = vec![]; // push the first condition conditions.push(format!( "({} > :{})", Self::quote_column(&columns[0]), - columns[0] + columns[0].to_lowercase() )); for i in 2..=columns.len() { // '=' condition let mut condition = String::new(); for (j, col) in columns.iter().enumerate().take(i - 1) { if j == 0 { - condition.push_str(&format!("({} = :{})", Self::quote_column(col), col)); + condition.push_str(&format!( + "({} = :{})", + Self::quote_column(col), + col.to_lowercase() + )); } else { - condition.push_str(&format!(" AND ({} = :{})", Self::quote_column(col), col)); + condition.push_str(&format!( + " AND ({} = :{})", + Self::quote_column(col), + col.to_lowercase() + )); } } // '>' condition condition.push_str(&format!( " AND ({} > :{})", Self::quote_column(&columns[i - 1]), - columns[i - 1] + columns[i - 1].to_lowercase() )); conditions.push(format!("({})", condition)); } From 240302c89d9d190d8f5c4405adb77791c93627aa Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 3 Jun 2024 19:25:57 +0800 Subject: [PATCH 25/36] add e2e test --- .../cdc_inline/auto_schema_map_mysql.slt | 83 +++++++++ .../source/cdc_inline/auto_schema_map_pg.slt | 168 ++++++++++++++++++ .../src/source/cdc/external/mysql.rs | 30 ++-- .../src/source/cdc/external/postgres.rs | 25 +-- src/frontend/src/handler/create_table.rs | 46 ++--- 5 files changed, 297 insertions(+), 55 deletions(-) create mode 100644 e2e_test/source/cdc_inline/auto_schema_map_mysql.slt create mode 100644 e2e_test/source/cdc_inline/auto_schema_map_pg.slt diff --git a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt new file mode 100644 index 000000000000..0fc00f4e6383 --- /dev/null +++ b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt @@ -0,0 +1,83 @@ +control substitution on + +# test case need to cover all data types +system ok +mysql --protocol=tcp -u root mytest -e " + DROP TABLE IF EXISTS mysql_types_test; + CREATE TABLE IF NOT EXISTS mysql_types_test( + c_boolean boolean, + c_bit bit, + c_tinyint tinyint, + c_smallint smallint, + c_mediumint mediumint, + c_integer integer, + c_bigint bigint, + c_decimal decimal, + c_float float, + c_double double, + c_char_255 char(255), + c_varchar_10000 varchar(10000), + c_binary_255 binary(255), + c_varbinary_10000 varbinary(10000), + c_date date, + c_time time, + c_datetime datetime, + c_timestamp timestamp, + c_enum ENUM('happy','sad','ok'), + c_json JSON, + PRIMARY KEY (c_boolean,c_bigint,c_date) + ); + INSERT INTO mysql_types_test VALUES ( False, 0, null, null, -8388608, -2147483647, 9223372036854775806, -10.0, -9999.999999, -10000.0, 'c', 'd', '', '', '1001-01-01', '-838:59:59.000000', '2000-01-01 00:00:00.000000', null, 'happy', '[1,2]'); + INSERT INTO mysql_types_test VALUES ( True, 1, -128, -32767, -8388608, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, 'a', 'b', '', '', '1001-01-01', '00:00:00', '1998-01-01 00:00:00.000000', '1970-01-01 00:00:01', 'sad', '[3,4]'); + " + + +statement ok +create source mysql_source with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'root', + password = '${MYSQL_PWD:}', + database.name = 'mytest', + server.id = '5601' +); + + +statement ok +create table rw_mysql_types_test (*) from mysql_source table 'mytest.mysql_types_test'; + +sleep 3s + +# Name, Type, Is Hidden, Description +query TTTT +describe rw_mysql_types_test; +---- +c_boolean smallint false NULL +c_bit boolean false NULL +c_tinyint smallint false NULL +c_smallint smallint false NULL +c_mediumint integer false NULL +c_integer integer false NULL +c_bigint bigint false NULL +c_decimal numeric false NULL +c_float real false NULL +c_double double precision false NULL +c_char_255 character varying false NULL +c_varchar_10000 character varying false NULL +c_binary_255 bytea false NULL +c_varbinary_10000 bytea false NULL +c_date date false NULL +c_time time without time zone false NULL +c_datetime timestamp without time zone false NULL +c_timestamp timestamp with time zone false NULL +c_enum character varying false NULL +c_json jsonb false NULL +primary key c_boolean, c_bigint, c_date NULL NULL +distribution key c_boolean, c_bigint, c_date NULL NULL +table description rw_mysql_types_test NULL NULL + + + +#statement ok +#drop source mysql_source cascade; diff --git a/e2e_test/source/cdc_inline/auto_schema_map_pg.slt b/e2e_test/source/cdc_inline/auto_schema_map_pg.slt new file mode 100644 index 000000000000..8183a617293b --- /dev/null +++ b/e2e_test/source/cdc_inline/auto_schema_map_pg.slt @@ -0,0 +1,168 @@ +control substitution on + +# test case need to cover all data types +system ok +psql -c " + DROP TABLE IF EXISTS postgres_types_test; + CREATE TABLE IF NOT EXISTS postgres_types_test( + c_boolean boolean, + c_smallint smallint, + c_integer integer, + c_bigint bigint, + c_decimal decimal, + c_real real, + c_double_precision double precision, + c_varchar varchar, + c_bytea bytea, + c_date date, + c_time time, + c_timestamp timestamp, + c_timestamptz timestamptz, + c_interval interval, + c_jsonb jsonb, + c_uuid uuid, + c_enum mood, + c_boolean_array boolean[], + c_smallint_array smallint[], + c_integer_array integer[], + c_bigint_array bigint[], + c_decimal_array decimal[], + c_real_array real[], + c_double_precision_array double precision[], + c_varchar_array varchar[], + c_bytea_array bytea[], + c_date_array date[], + c_time_array time[], + c_timestamp_array timestamp[], + c_timestamptz_array timestamptz[], + c_interval_array interval[], + c_jsonb_array jsonb[], + c_uuid_array uuid[], + c_enum_array mood[], + PRIMARY KEY (c_boolean,c_bigint,c_date) + ); + INSERT INTO postgres_types_test VALUES ( False, 0, 0, 0, 0, 0, 0, '', '00'::bytea, '0001-01-01', '00:00:00', '2001-01-01 00:00:00'::timestamp, '2001-01-01 00:00:00-8'::timestamptz, interval '0 second', '{}', null, 'sad', array[]::boolean[], array[]::smallint[], array[]::integer[], array[]::bigint[], array[]::decimal[], array[]::real[], array[]::double precision[], array[]::varchar[], array[]::bytea[], array[]::date[], array[]::time[], array[]::timestamp[], array[]::timestamptz[], array[]::interval[], array[]::jsonb[], array[]::uuid[], array[]::mood[]); + INSERT INTO postgres_types_test VALUES ( False, -32767, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, 'd', '00'::bytea, '0001-01-01', '00:00:00', '2001-01-01 00:00:00'::timestamp, '2001-01-01 00:00:00-8'::timestamptz, interval '0 second', '{}', 'bb488f9b-330d-4012-b849-12adeb49e57e', 'happy', array[False::boolean]::boolean[], array[-32767::smallint]::smallint[], array[-2147483647::integer]::integer[], array[-9223372036854775807::bigint]::bigint[], array[-10.0::decimal]::decimal[], array[-9999.999999::real]::real[], array[-10000.0::double precision]::double precision[], array[''::varchar]::varchar[], array['00'::bytea]::bytea[], array['0001-01-01'::date]::date[], array['00:00:00'::time]::time[], array['2001-01-01 00:00:00'::timestamp::timestamp]::timestamp[], array['2001-01-01 00:00:00-8'::timestamptz::timestamptz]::timestamptz[], array[interval '0 second'::interval]::interval[], array['{}'::jsonb]::jsonb[], '{bb488f9b-330d-4012-b849-12adeb49e57e}', '{happy,ok,sad}'); + INSERT INTO postgres_types_test VALUES ( False, 1, 123, 1234567890, 123.45, 123.45, 123.456, 'a_varchar', 'DEADBEEF'::bytea, '0024-01-01', '12:34:56', '2024-05-19 12:34:56', '2024-05-19 12:34:56+00', INTERVAL '1 day', to_jsonb('hello'::text), '123e4567-e89b-12d3-a456-426614174000', 'happy', ARRAY[NULL, TRUE]::boolean[], ARRAY[NULL, 1::smallint], ARRAY[NULL, 123], ARRAY[NULL, 1234567890], ARRAY[NULL, 123.45::numeric], ARRAY[NULL, 123.45::real], ARRAY[NULL, 123.456], ARRAY[NULL, 'a_varchar'], ARRAY[NULL, 'DEADBEEF'::bytea], ARRAY[NULL, '2024-05-19'::date], ARRAY[NULL, '12:34:56'::time], ARRAY[NULL, '2024-05-19 12:34:56'::timestamp], ARRAY[NULL, '2024-05-19 12:34:56+00'::timestamptz], ARRAY[NULL, INTERVAL '1 day'], ARRAY[NULL, to_jsonb('hello'::text)], ARRAY[NULL, '123e4567-e89b-12d3-a456-426614174000'::uuid], ARRAY[NULL, 'happy'::mood]); + INSERT INTO postgres_types_test VALUES ( False, NULL, NULL, 1, NULL, NULL, NULL, NULL, NULL, '0024-05-19', NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL); + " + +statement ok +create source pg_source with ( + connector = 'postgres-cdc', + hostname = '${PGHOST:localhost}', + port = '${PGPORT:5432}', + username = '${PGUSER:$USER}', + password = '${PGPASSWORD:}', + database.name = '${PGDATABASE:postgres}', + slot.name = 'pg_slot' +); + + +statement ok +create table rw_postgres_types_test (*) from pg_source table 'public.postgres_types_test'; + +sleep 3s + +# Name, Type, Is Hidden, Description +query TTTT +describe rw_postgres_types_test; +---- +c_boolean boolean false NULL +c_smallint smallint false NULL +c_integer integer false NULL +c_bigint bigint false NULL +c_decimal numeric false NULL +c_real real false NULL +c_double_precision double precision false NULL +c_varchar character varying false NULL +c_bytea bytea false NULL +c_date date false NULL +c_time time without time zone false NULL +c_timestamp timestamp without time zone false NULL +c_timestamptz timestamp with time zone false NULL +c_interval interval false NULL +c_jsonb jsonb false NULL +c_uuid character varying false NULL +c_enum character varying false NULL +c_boolean_array boolean[] false NULL +c_smallint_array smallint[] false NULL +c_integer_array integer[] false NULL +c_bigint_array bigint[] false NULL +c_decimal_array numeric[] false NULL +c_real_array real[] false NULL +c_double_precision_array double precision[] false NULL +c_varchar_array character varying[] false NULL +c_bytea_array bytea[] false NULL +c_date_array date[] false NULL +c_time_array time without time zone[] false NULL +c_timestamp_array timestamp without time zone[] false NULL +c_timestamptz_array timestamp with time zone[] false NULL +c_interval_array interval[] false NULL +c_jsonb_array jsonb[] false NULL +c_uuid_array character varying[] false NULL +c_enum_array character varying[] false NULL +primary key c_boolean, c_bigint, c_date NULL NULL +distribution key c_boolean, c_bigint, c_date NULL NULL +table description rw_postgres_types_test NULL NULL + +query TTTTTTT +SELECT + c_boolean, + c_smallint, + c_integer, + c_bigint, + c_decimal, + c_real, + c_double_precision, + c_varchar, + c_bytea from rw_postgres_types_test where c_enum = 'happy' order by c_integer; +---- +f -32767 -2147483647 -9223372036854775807 -10.0 -10000 -10000 d \x3030 +f 1 123 1234567890 123.45 123.45 123.456 a_varchar \x4445414442454546 + +query TTTTT +SELECT + c_date, + c_time, + c_timestamp, + c_timestamptz, + c_interval from rw_postgres_types_test where c_enum = 'happy' order by c_integer; +---- +0001-01-01 00:00:00 2001-01-01 00:00:00 2001-01-01 08:00:00+00:00 00:00:00 +0024-01-01 12:34:56 2024-05-19 12:34:56 2024-05-19 12:34:56+00:00 1 day + +query TTTTTTT +SELECT + c_jsonb, + c_uuid, + c_enum, + c_boolean_array, + c_smallint_array, + c_integer_array, + c_bigint_array from rw_postgres_types_test where c_enum = 'happy' order by c_integer; +---- +{} bb488f9b-330d-4012-b849-12adeb49e57e happy {f} {-32767} {-2147483647} {-9223372036854775807} +"hello" 123e4567-e89b-12d3-a456-426614174000 happy {NULL,t} {NULL,1} {NULL,123} {NULL,1234567890} + +query TTTTTTTTTTTTT +SELECT + c_decimal_array, + c_real_array, + c_double_precision_array, + c_varchar_array, + c_bytea_array, + c_date_array, + c_time_array, + c_timestamp_array, + c_timestamptz_array, + c_interval_array, + c_jsonb_array, + c_uuid_array, + c_enum_array from rw_postgres_types_test where c_enum = 'happy' order by c_integer; +---- +{-10.0} {-10000} {-10000} {""} {"\\x3030"} {0001-01-01} {00:00:00} {"2001-01-01 00:00:00"} {"2001-01-01 08:00:00+00:00"} NULL {"{}"} {bb488f9b-330d-4012-b849-12adeb49e57e} {happy,ok,sad} +{NULL,123.45} {NULL,123.45} {NULL,123.456} {NULL,a_varchar} {NULL,"\\x4445414442454546"} {NULL,2024-05-19} {NULL,12:34:56} {NULL,"2024-05-19 12:34:56"} {NULL,"2024-05-19 12:34:56+00:00"} NULL {NULL,"\"hello\""} {NULL,123e4567-e89b-12d3-a456-426614174000} NULL + +statement ok +drop source pg_source cascade; diff --git a/src/connector/src/source/cdc/external/mysql.rs b/src/connector/src/source/cdc/external/mysql.rs index cd4fc7dcb54f..56b16ae29005 100644 --- a/src/connector/src/source/cdc/external/mysql.rs +++ b/src/connector/src/source/cdc/external/mysql.rs @@ -135,9 +135,13 @@ fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { let dtype = match col_type { ColumnType::Serial => DataType::Int32, ColumnType::Bit(attr) => { - // return Err(anyhow!("line type not supported").into()); - tracing::info!("bit attr: {:?}", attr); - DataType::Boolean + if let Some(1) = attr.maximum { + DataType::Boolean + } else { + return Err( + anyhow!("BIT({}) type not supported", attr.maximum.unwrap_or(0)).into(), + ); + } } ColumnType::TinyInt(_) | ColumnType::SmallInt(_) => DataType::Int16, ColumnType::Bool => DataType::Boolean, @@ -168,34 +172,34 @@ fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { ColumnType::Enum(_) => DataType::Varchar, ColumnType::Json => DataType::Jsonb, ColumnType::Set(_) => { - return Err(anyhow!("set type not supported").into()); + return Err(anyhow!("SET type not supported").into()); } ColumnType::Geometry(_) => { - return Err(anyhow!("geometry type not supported").into()); + return Err(anyhow!("GEOMETRY type not supported").into()); } ColumnType::Point(_) => { - return Err(anyhow!("point type not supported").into()); + return Err(anyhow!("POINT type not supported").into()); } ColumnType::LineString(_) => { - return Err(anyhow!("line string type not supported").into()); + return Err(anyhow!("LINE string type not supported").into()); } ColumnType::Polygon(_) => { - return Err(anyhow!("polygon type not supported").into()); + return Err(anyhow!("POLYGON type not supported").into()); } ColumnType::MultiPoint(_) => { - return Err(anyhow!("multi point type not supported").into()); + return Err(anyhow!("MULTI POINT type not supported").into()); } ColumnType::MultiLineString(_) => { - return Err(anyhow!("multi line string type not supported").into()); + return Err(anyhow!("MULTI LINE STRING type not supported").into()); } ColumnType::MultiPolygon(_) => { - return Err(anyhow!("multi polygon type not supported").into()); + return Err(anyhow!("MULTI POLYGON type not supported").into()); } ColumnType::GeometryCollection(_) => { - return Err(anyhow!("geometry collection type not supported").into()); + return Err(anyhow!("GEOMETRY COLLECTION type not supported").into()); } ColumnType::Unknown(_) => { - return Err(anyhow!("unknown column type").into()); + return Err(anyhow!("Unknown MySQL data type").into()); } }; diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 5acc80505102..50e0150c4200 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -174,7 +174,7 @@ fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { let item_type = match def.col_type.as_ref() { Some(ty) => type_to_rw_type(ty.as_ref())?, None => { - return Err(anyhow!("array type missing element type").into()); + return Err(anyhow!("ARRAY type missing element type").into()); } }; @@ -194,34 +194,35 @@ fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { | ColumnType::Enum(_) => DataType::Varchar, ColumnType::Line => { - return Err(anyhow!("line type not supported").into()); + return Err(anyhow!("LINE type not supported").into()); } ColumnType::Lseg => { - return Err(anyhow!("lseg type not supported").into()); + return Err(anyhow!("LSEG type not supported").into()); } ColumnType::Box => { - return Err(anyhow!("box type not supported").into()); + return Err(anyhow!("BOX type not supported").into()); } ColumnType::Path => { - return Err(anyhow!("path type not supported").into()); + return Err(anyhow!("PATH type not supported").into()); } ColumnType::Polygon => { - return Err(anyhow!("polygon type not supported").into()); + return Err(anyhow!("POLYGON type not supported").into()); } ColumnType::Circle => { - return Err(anyhow!("circle type not supported").into()); + return Err(anyhow!("CIRCLE type not supported").into()); } ColumnType::Bit(_) => { - return Err(anyhow!("bit type not supported").into()); + return Err(anyhow!("BIT type not supported").into()); } ColumnType::TsVector => { - return Err(anyhow!("tsvector type not supported").into()); + return Err(anyhow!("TSVECTOR type not supported").into()); } ColumnType::TsQuery => { - return Err(anyhow!("tsquery type not supported").into()); + return Err(anyhow!("TSQUERY type not supported").into()); } - ColumnType::Unknown(_) => { - return Err(anyhow!("unknown column type").into()); + ColumnType::Unknown(name) => { + tracing::warn!("Unknown Postgres data type: {name}, map to varchar"); + DataType::Varchar } }; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index b75a75ddb487..ea5b6df343ba 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -1024,19 +1024,22 @@ fn sanity_check_for_cdc_table( source_watermarks: &Vec, ) -> Result<()> { // cdc table must have primary key constraint or primary key column - if !constraints.iter().any(|c| { - matches!( - c, - TableConstraint::Unique { - is_primary: true, - .. - } - ) - }) && !column_defs.iter().any(|col| { - col.options - .iter() - .any(|opt| matches!(opt.option, ColumnOption::Unique { is_primary: true })) - }) { + if !wildcard_idx.is_some() + && !constraints.iter().any(|c| { + matches!( + c, + TableConstraint::Unique { + is_primary: true, + .. + } + ) + }) + && !column_defs.iter().any(|col| { + col.options + .iter() + .any(|opt| matches!(opt.option, ColumnOption::Unique { is_primary: true })) + }) + { return Err(ErrorCode::NotSupported( "CDC table without primary key constraint is not supported".to_owned(), "Please define a primary key".to_owned(), @@ -1069,23 +1072,6 @@ fn sanity_check_for_cdc_table( } } } - if !wildcard_idx.is_some() - && !constraints.iter().any(|c| { - matches!( - c, - TableConstraint::Unique { - is_primary: true, - .. - } - ) - }) - { - return Err(ErrorCode::NotSupported( - "CDC table without primary key constraint is not supported".to_owned(), - "Please define a primary key".to_owned(), - ) - .into()); - } Ok(()) } From 6b45cc3d0a6a36d5467372aa050cf8e028c633b4 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 4 Jun 2024 11:37:04 +0800 Subject: [PATCH 26/36] mysql e2e --- .../cdc_inline/auto_schema_map_mysql.slt | 36 +++++++++++++++++-- 1 file changed, 34 insertions(+), 2 deletions(-) diff --git a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt index 0fc00f4e6383..cb3539fd51d6 100644 --- a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt +++ b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt @@ -77,7 +77,39 @@ primary key c_boolean, c_bigint, c_date NULL NULL distribution key c_boolean, c_bigint, c_date NULL NULL table description rw_mysql_types_test NULL NULL +query TTTTTTTTTTTTT +SELECT + c_boolean, + c_bit, + c_tinyint, + c_smallint, + c_mediumint, + c_integer, + c_bigint, + c_decimal, + c_float, + c_double, + c_char_255, + c_varchar_10000, +FROM rw_mysql_types_test order by c_integer; +---- +0 NULL NULL NULL -8388608 -2147483647 9223372036854775806 -10 -10000 -10000 c d +1 NULL -128 -32767 -8388608 -2147483647 -9223372036854775807 -10 -10000 -10000 a b +query TTTTTTTT +SELECT + c_binary_255, + c_varbinary_10000, + c_date, + c_time, + c_datetime, + c_timestamp, + c_enum, + c_json +FROM rw_mysql_types_test order by c_integer; +---- +\x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 \x 1001-01-01 NULL 2000-01-01 00:00:00 NULL happy [1, 2] +\x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 \x 1001-01-01 00:00:00 1998-01-01 00:00:00 1970-01-01 00:00:01+00:00 sad [3, 4] -#statement ok -#drop source mysql_source cascade; +statement ok +drop source mysql_source cascade; From 2a5ae542c6ae8139486287ae4fb3db2654f7700d Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 4 Jun 2024 12:11:00 +0800 Subject: [PATCH 27/36] fix --- e2e_test/source/cdc_inline/auto_schema_map_mysql.slt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt index cb3539fd51d6..5994f50e8bc1 100644 --- a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt +++ b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt @@ -1,6 +1,9 @@ control substitution on # test case need to cover all data types +system ok +mysql --protocol=tcp -u root -e "DROP DATABASE IF EXISTS mytest; CREATE DATABASE mytest;" + system ok mysql --protocol=tcp -u root mytest -e " DROP TABLE IF EXISTS mysql_types_test; From a73b3d9584d81f0f727999f2d48de043e6a366d6 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 4 Jun 2024 13:48:11 +0800 Subject: [PATCH 28/36] enhance e2e --- e2e_test/source/cdc_inline/auto_schema_map_mysql.slt | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt index 5994f50e8bc1..7d277775aa2b 100644 --- a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt +++ b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt @@ -14,7 +14,7 @@ mysql --protocol=tcp -u root mytest -e " c_smallint smallint, c_mediumint mediumint, c_integer integer, - c_bigint bigint, + `c_Bigint` bigint, c_decimal decimal, c_float float, c_double double, @@ -28,7 +28,7 @@ mysql --protocol=tcp -u root mytest -e " c_timestamp timestamp, c_enum ENUM('happy','sad','ok'), c_json JSON, - PRIMARY KEY (c_boolean,c_bigint,c_date) + PRIMARY KEY (c_boolean,`c_Bigint`,c_date) ); INSERT INTO mysql_types_test VALUES ( False, 0, null, null, -8388608, -2147483647, 9223372036854775806, -10.0, -9999.999999, -10000.0, 'c', 'd', '', '', '1001-01-01', '-838:59:59.000000', '2000-01-01 00:00:00.000000', null, 'happy', '[1,2]'); INSERT INTO mysql_types_test VALUES ( True, 1, -128, -32767, -8388608, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, 'a', 'b', '', '', '1001-01-01', '00:00:00', '1998-01-01 00:00:00.000000', '1970-01-01 00:00:01', 'sad', '[3,4]'); @@ -62,7 +62,7 @@ c_tinyint smallint false NULL c_smallint smallint false NULL c_mediumint integer false NULL c_integer integer false NULL -c_bigint bigint false NULL +c_Bigint bigint false NULL c_decimal numeric false NULL c_float real false NULL c_double double precision false NULL @@ -76,8 +76,8 @@ c_datetime timestamp without time zone false NULL c_timestamp timestamp with time zone false NULL c_enum character varying false NULL c_json jsonb false NULL -primary key c_boolean, c_bigint, c_date NULL NULL -distribution key c_boolean, c_bigint, c_date NULL NULL +primary key c_boolean, c_Bigint, c_date NULL NULL +distribution key c_boolean, c_Bigint, c_date NULL NULL table description rw_mysql_types_test NULL NULL query TTTTTTTTTTTTT @@ -88,7 +88,7 @@ SELECT c_smallint, c_mediumint, c_integer, - c_bigint, + "c_Bigint", c_decimal, c_float, c_double, From 24cef4c2cd856a2407ec854e8c0a9e8089b6bd67 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 4 Jun 2024 13:56:17 +0800 Subject: [PATCH 29/36] * and columns cannot used together --- src/frontend/src/handler/create_table.rs | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index ea5b6df343ba..9af7b5ec876c 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -1023,6 +1023,15 @@ fn sanity_check_for_cdc_table( constraints: &Vec, source_watermarks: &Vec, ) -> Result<()> { + // wildcard cannot be used with column definitions + if wildcard_idx.is_some() && !column_defs.is_empty() { + return Err(ErrorCode::NotSupported( + "wildcard(*) and column definitions cannot be used together".to_owned(), + "Remove the wildcard or column definitions".to_owned(), + ) + .into()); + } + // cdc table must have primary key constraint or primary key column if !wildcard_idx.is_some() && !constraints.iter().any(|c| { From 0d5b582be99863d0e2b5a9f0d946ad98f5ed1bed Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 4 Jun 2024 13:59:44 +0800 Subject: [PATCH 30/36] minor --- src/connector/src/source/cdc/external/postgres.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 50e0150c4200..e8f5d279aee8 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -221,6 +221,7 @@ fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { return Err(anyhow!("TSQUERY type not supported").into()); } ColumnType::Unknown(name) => { + // NOTES: user-defined enum type is classified as `Unknown` tracing::warn!("Unknown Postgres data type: {name}, map to varchar"); DataType::Varchar } From c4302839a89eb7e96ac902dcaea31ea6d6d5adff Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 4 Jun 2024 15:31:17 +0800 Subject: [PATCH 31/36] fix mysql e2e --- .../cdc_inline/auto_schema_map_mysql.slt | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt index 7d277775aa2b..87e59736e037 100644 --- a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt +++ b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt @@ -14,7 +14,7 @@ mysql --protocol=tcp -u root mytest -e " c_smallint smallint, c_mediumint mediumint, c_integer integer, - `c_Bigint` bigint, + c_Bigint bigint, c_decimal decimal, c_float float, c_double double, @@ -28,7 +28,7 @@ mysql --protocol=tcp -u root mytest -e " c_timestamp timestamp, c_enum ENUM('happy','sad','ok'), c_json JSON, - PRIMARY KEY (c_boolean,`c_Bigint`,c_date) + PRIMARY KEY (c_boolean,c_Bigint,c_date) ); INSERT INTO mysql_types_test VALUES ( False, 0, null, null, -8388608, -2147483647, 9223372036854775806, -10.0, -9999.999999, -10000.0, 'c', 'd', '', '', '1001-01-01', '-838:59:59.000000', '2000-01-01 00:00:00.000000', null, 'happy', '[1,2]'); INSERT INTO mysql_types_test VALUES ( True, 1, -128, -32767, -8388608, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, 'a', 'b', '', '', '1001-01-01', '00:00:00', '1998-01-01 00:00:00.000000', '1970-01-01 00:00:01', 'sad', '[3,4]'); @@ -94,14 +94,14 @@ SELECT c_double, c_char_255, c_varchar_10000, -FROM rw_mysql_types_test order by c_integer; + c_binary_255 +FROM rw_mysql_types_test order by c_boolean; ---- -0 NULL NULL NULL -8388608 -2147483647 9223372036854775806 -10 -10000 -10000 c d -1 NULL -128 -32767 -8388608 -2147483647 -9223372036854775807 -10 -10000 -10000 a b +0 NULL NULL NULL -8388608 -2147483647 9223372036854775806 -10 -10000 -10000 c d \x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 +1 NULL -128 -32767 -8388608 -2147483647 -9223372036854775807 -10 -10000 -10000 a b \x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 query TTTTTTTT SELECT - c_binary_255, c_varbinary_10000, c_date, c_time, @@ -109,10 +109,10 @@ SELECT c_timestamp, c_enum, c_json -FROM rw_mysql_types_test order by c_integer; +FROM rw_mysql_types_test order by c_boolean; ---- -\x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 \x 1001-01-01 NULL 2000-01-01 00:00:00 NULL happy [1, 2] -\x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 \x 1001-01-01 00:00:00 1998-01-01 00:00:00 1970-01-01 00:00:01+00:00 sad [3, 4] +\x 1001-01-01 NULL 2000-01-01 00:00:00 NULL happy [1, 2] +\x 1001-01-01 00:00:00 1998-01-01 00:00:00 1970-01-01 00:00:01+00:00 sad [3, 4] statement ok drop source mysql_source cascade; From b131e30b88fab59344ce4bf24b68e516d750660d Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 4 Jun 2024 16:21:42 +0800 Subject: [PATCH 32/36] clean code --- src/connector/src/source/cdc/external/mod.rs | 102 +----------------- .../src/source/cdc/external/mysql.rs | 92 +++++++++++++++- .../src/source/cdc/external/postgres.rs | 11 +- src/frontend/src/handler/create_table.rs | 8 +- src/meta/src/rpc/ddl_controller.rs | 4 - 5 files changed, 99 insertions(+), 118 deletions(-) diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index 0764aed54656..01c285b39072 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -22,17 +22,13 @@ pub mod mysql; use std::collections::HashMap; use std::fmt; -use anyhow::{anyhow, Context}; +use anyhow::anyhow; +use futures::pin_mut; use futures::stream::BoxStream; -use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; -use itertools::Itertools; -use mysql_async::prelude::AsQuery; use risingwave_common::bail; -use risingwave_common::catalog::{ColumnDesc, Schema, OFFSET_COLUMN_NAME}; +use risingwave_common::catalog::{ColumnDesc, Schema}; use risingwave_common::row::OwnedRow; -use risingwave_common::types::DataType; -use risingwave_common::util::iter_util::ZipEqFast; use serde_derive::{Deserialize, Serialize}; use crate::error::{ConnectorError, ConnectorResult}; @@ -45,7 +41,6 @@ use crate::source::cdc::external::postgres::{ PostgresExternalTable, PostgresExternalTableReader, PostgresOffset, }; use crate::source::cdc::CdcSourceType; -use crate::source::{UnknownFields, UPSTREAM_SOURCE_KEY}; use crate::WithPropertiesExt; #[derive(Debug)] @@ -351,94 +346,3 @@ impl ExternalTableImpl { } } } - -#[cfg(test)] -mod tests { - - use futures::pin_mut; - use futures_async_stream::for_await; - use maplit::{convert_args, hashmap}; - use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema}; - use risingwave_common::types::DataType; - - use crate::source::cdc::external::{ - CdcOffset, ExternalTableReader, MySqlExternalTableReader, MySqlOffset, SchemaTableName, - }; - - #[test] - fn test_mysql_filter_expr() { - let cols = vec!["id".to_string()]; - let expr = MySqlExternalTableReader::filter_expression(&cols); - assert_eq!(expr, "(`id` > :id)"); - - let cols = vec!["aa".to_string(), "bb".to_string(), "cc".to_string()]; - let expr = MySqlExternalTableReader::filter_expression(&cols); - assert_eq!( - expr, - "(`aa` > :aa) OR ((`aa` = :aa) AND (`bb` > :bb)) OR ((`aa` = :aa) AND (`bb` = :bb) AND (`cc` > :cc))" - ); - } - - #[test] - fn test_mysql_binlog_offset() { - let off0_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000001", "pos": 105622, "snapshot": true }, "isHeartbeat": false }"#; - let off1_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000007", "pos": 1062363217, "snapshot": true }, "isHeartbeat": false }"#; - let off2_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000007", "pos": 659687560, "snapshot": true }, "isHeartbeat": false }"#; - let off3_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000008", "pos": 7665875, "snapshot": true }, "isHeartbeat": false }"#; - let off4_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000008", "pos": 7665875, "snapshot": true }, "isHeartbeat": false }"#; - - let off0 = CdcOffset::MySql(MySqlOffset::parse_debezium_offset(off0_str).unwrap()); - let off1 = CdcOffset::MySql(MySqlOffset::parse_debezium_offset(off1_str).unwrap()); - let off2 = CdcOffset::MySql(MySqlOffset::parse_debezium_offset(off2_str).unwrap()); - let off3 = CdcOffset::MySql(MySqlOffset::parse_debezium_offset(off3_str).unwrap()); - let off4 = CdcOffset::MySql(MySqlOffset::parse_debezium_offset(off4_str).unwrap()); - - assert!(off0 <= off1); - assert!(off1 > off2); - assert!(off2 < off3); - assert_eq!(off3, off4); - } - - // manual test case - #[ignore] - #[tokio::test] - async fn test_mysql_table_reader() { - let columns = vec![ - ColumnDesc::named("v1", ColumnId::new(1), DataType::Int32), - ColumnDesc::named("v2", ColumnId::new(2), DataType::Decimal), - ColumnDesc::named("v3", ColumnId::new(3), DataType::Varchar), - ColumnDesc::named("v4", ColumnId::new(4), DataType::Date), - ]; - let rw_schema = Schema { - fields: columns.iter().map(Field::from).collect(), - }; - let props = convert_args!(hashmap!( - "hostname" => "localhost", - "port" => "8306", - "username" => "root", - "password" => "123456", - "database.name" => "mytest", - "table.name" => "t1")); - - let reader = MySqlExternalTableReader::new(props, rw_schema) - .await - .unwrap(); - let offset = reader.current_cdc_offset().await.unwrap(); - println!("BinlogOffset: {:?}", offset); - - let off0_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000001", "pos": 105622, "snapshot": true }, "isHeartbeat": false }"#; - let parser = MySqlExternalTableReader::get_cdc_offset_parser(); - println!("parsed offset: {:?}", parser(off0_str).unwrap()); - let table_name = SchemaTableName { - schema_name: "mytest".to_string(), - table_name: "t1".to_string(), - }; - - let stream = reader.snapshot_read(table_name, None, vec!["v1".to_string()], 1000); - pin_mut!(stream); - #[for_await] - for row in stream { - println!("OwnedRow: {:?}", row); - } - } -} diff --git a/src/connector/src/source/cdc/external/mysql.rs b/src/connector/src/source/cdc/external/mysql.rs index 56b16ae29005..f82fff2d1139 100644 --- a/src/connector/src/source/cdc/external/mysql.rs +++ b/src/connector/src/source/cdc/external/mysql.rs @@ -104,7 +104,7 @@ impl MySqlExternalTable { let mut column_descs = vec![]; let mut pk_names = vec![]; - for col in table_schema.columns.iter() { + for col in &table_schema.columns { let data_type = type_to_rw_type(&col.col_type)?; column_descs.push(ColumnDesc::named( col.name.clone(), @@ -451,10 +451,21 @@ impl MySqlExternalTableReader { format!("`{}`", column) } } + #[cfg(test)] mod tests { + + use futures::pin_mut; + use futures_async_stream::for_await; + use maplit::{convert_args, hashmap}; + use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema}; + use risingwave_common::types::DataType; + use crate::source::cdc::external::mysql::MySqlExternalTable; - use crate::source::cdc::external::ExternalTableConfig; + use crate::source::cdc::external::{ + CdcOffset, ExternalTableConfig, ExternalTableReader, MySqlExternalTableReader, MySqlOffset, + SchemaTableName, + }; #[ignore] #[tokio::test] @@ -475,4 +486,81 @@ mod tests { println!("columns: {:?}", &table.column_descs); println!("primary keys: {:?}", &table.pk_names); } + + #[test] + fn test_mysql_filter_expr() { + let cols = vec!["id".to_string()]; + let expr = MySqlExternalTableReader::filter_expression(&cols); + assert_eq!(expr, "(`id` > :id)"); + + let cols = vec!["aa".to_string(), "bb".to_string(), "cc".to_string()]; + let expr = MySqlExternalTableReader::filter_expression(&cols); + assert_eq!( + expr, + "(`aa` > :aa) OR ((`aa` = :aa) AND (`bb` > :bb)) OR ((`aa` = :aa) AND (`bb` = :bb) AND (`cc` > :cc))" + ); + } + + #[test] + fn test_mysql_binlog_offset() { + let off0_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000001", "pos": 105622, "snapshot": true }, "isHeartbeat": false }"#; + let off1_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000007", "pos": 1062363217, "snapshot": true }, "isHeartbeat": false }"#; + let off2_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000007", "pos": 659687560, "snapshot": true }, "isHeartbeat": false }"#; + let off3_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000008", "pos": 7665875, "snapshot": true }, "isHeartbeat": false }"#; + let off4_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000008", "pos": 7665875, "snapshot": true }, "isHeartbeat": false }"#; + + let off0 = CdcOffset::MySql(MySqlOffset::parse_debezium_offset(off0_str).unwrap()); + let off1 = CdcOffset::MySql(MySqlOffset::parse_debezium_offset(off1_str).unwrap()); + let off2 = CdcOffset::MySql(MySqlOffset::parse_debezium_offset(off2_str).unwrap()); + let off3 = CdcOffset::MySql(MySqlOffset::parse_debezium_offset(off3_str).unwrap()); + let off4 = CdcOffset::MySql(MySqlOffset::parse_debezium_offset(off4_str).unwrap()); + + assert!(off0 <= off1); + assert!(off1 > off2); + assert!(off2 < off3); + assert_eq!(off3, off4); + } + + // manual test case + #[ignore] + #[tokio::test] + async fn test_mysql_table_reader() { + let columns = vec![ + ColumnDesc::named("v1", ColumnId::new(1), DataType::Int32), + ColumnDesc::named("v2", ColumnId::new(2), DataType::Decimal), + ColumnDesc::named("v3", ColumnId::new(3), DataType::Varchar), + ColumnDesc::named("v4", ColumnId::new(4), DataType::Date), + ]; + let rw_schema = Schema { + fields: columns.iter().map(Field::from).collect(), + }; + let props = convert_args!(hashmap!( + "hostname" => "localhost", + "port" => "8306", + "username" => "root", + "password" => "123456", + "database.name" => "mytest", + "table.name" => "t1")); + + let reader = MySqlExternalTableReader::new(props, rw_schema) + .await + .unwrap(); + let offset = reader.current_cdc_offset().await.unwrap(); + println!("BinlogOffset: {:?}", offset); + + let off0_str = r#"{ "sourcePartition": { "server": "test" }, "sourceOffset": { "ts_sec": 1670876905, "file": "binlog.000001", "pos": 105622, "snapshot": true }, "isHeartbeat": false }"#; + let parser = MySqlExternalTableReader::get_cdc_offset_parser(); + println!("parsed offset: {:?}", parser(off0_str).unwrap()); + let table_name = SchemaTableName { + schema_name: "mytest".to_string(), + table_name: "t1".to_string(), + }; + + let stream = reader.snapshot_read(table_name, None, vec!["v1".to_string()], 1000); + pin_mut!(stream); + #[for_await] + for row in stream { + println!("OwnedRow: {:?}", row); + } + } } diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index e8f5d279aee8..4ae9a3beee75 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -14,7 +14,6 @@ use std::cmp::Ordering; use std::collections::HashMap; -use std::sync::Arc; use anyhow::{anyhow, Context}; use futures::stream::BoxStream; @@ -23,11 +22,11 @@ use futures_async_stream::try_stream; use itertools::Itertools; use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; use postgres_openssl::MakeTlsConnector; -use risingwave_common::catalog::{ColumnCatalog, ColumnDesc, ColumnId, Schema}; +use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema}; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, StructType}; use risingwave_common::util::iter_util::ZipEqFast; -use sea_schema::postgres::def::{ColumnType, TableInfo, Type}; +use sea_schema::postgres::def::{ColumnType, TableInfo}; use sea_schema::postgres::discovery::SchemaDiscovery; use serde_derive::{Deserialize, Serialize}; use sqlx::postgres::{PgConnectOptions, PgSslMode}; @@ -114,7 +113,7 @@ impl PostgresExternalTable { .await?; let mut column_descs = vec![]; - for col in table_schema.columns.iter() { + for col in &table_schema.columns { let data_type = type_to_rw_type(&col.col_type)?; column_descs.push(ColumnDesc::named( col.name.clone(), @@ -455,7 +454,6 @@ impl PostgresExternalTableReader { #[cfg(test)] mod tests { - use std::collections::HashMap; use futures::pin_mut; use futures_async_stream::for_await; @@ -463,9 +461,6 @@ mod tests { use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema}; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; - use sea_schema::postgres::def::TableInfo; - use sea_schema::postgres::discovery::SchemaDiscovery; - use sqlx::PgPool; use crate::source::cdc::external::postgres::{ PostgresExternalTable, PostgresExternalTableReader, PostgresOffset, diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 9af7b5ec876c..ee5649ca5b74 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, HashMap}; +use std::collections::HashMap; use std::rc::Rc; use std::sync::Arc; @@ -29,7 +29,6 @@ use risingwave_common::catalog::{ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::util::value_encoding::DatumToProtoExt; use risingwave_connector::source; -use risingwave_connector::source::cdc::external::postgres::PostgresExternalTable; use risingwave_connector::source::cdc::external::{ ExternalTableConfig, ExternalTableImpl, DATABASE_NAME_KEY, SCHEMA_NAME_KEY, TABLE_NAME_KEY, }; @@ -732,7 +731,7 @@ fn gen_table_plan_inner( } #[allow(clippy::too_many_arguments)] -pub(crate) async fn gen_create_table_plan_for_cdc_table( +pub(crate) fn gen_create_table_plan_for_cdc_table( handler_args: HandlerArgs, explain_options: ExplainOptions, source: Arc, @@ -1001,8 +1000,7 @@ pub(super) async fn handle_create_table_plan( resolved_table_name, database_id, schema_id, - ) - .await?; + )?; ((plan, None, table), TableJobType::SharedCdcSource) } diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index c24c48c82a2a..f83e1420eb9c 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -29,15 +29,12 @@ use risingwave_common::hash::{ParallelUnitMapping, VirtualNode}; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::epoch::Epoch; -use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::util::stream_graph_visitor::{ visit_fragment, visit_stream_node, visit_stream_node_cont_mut, }; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::dispatch_source_prop; use risingwave_connector::error::ConnectorError; -use risingwave_connector::source::cdc::external::postgres::PostgresExternalTable; -use risingwave_connector::source::cdc::external::ExternalTableConfig; use risingwave_connector::source::cdc::CdcSourceType; use risingwave_connector::source::{ ConnectorProperties, SourceEnumeratorContext, SourceProperties, SplitEnumerator, @@ -59,7 +56,6 @@ use risingwave_pb::ddl_service::{ }; use risingwave_pb::meta::table_fragments::PbFragment; use risingwave_pb::meta::PbTableParallelism; -use risingwave_pb::plan_common::ColumnCatalog; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ Dispatcher, DispatcherType, FragmentTypeFlag, MergeNode, PbStreamFragmentGraph, From 64678d08b1fe2769775fba1da96c2a8395fc87ae Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 4 Jun 2024 16:51:53 +0800 Subject: [PATCH 33/36] minor --- src/compute/tests/cdc_tests.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index 628c2ca22cf4..720c07b7d22d 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -33,8 +33,9 @@ use risingwave_common::types::{Datum, JsonbVal}; use risingwave_common::util::epoch::{test_epoch, EpochExt}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_connector::source::cdc::external::mock_external_table::MockExternalTableReader; +use risingwave_connector::source::cdc::external::mysql::MySqlOffset; use risingwave_connector::source::cdc::external::{ - DebeziumOffset, DebeziumSourceOffset, ExternalTableReaderImpl, MySqlOffset, SchemaTableName, + DebeziumOffset, DebeziumSourceOffset, ExternalTableReaderImpl, SchemaTableName, }; use risingwave_connector::source::cdc::DebeziumCdcSplit; use risingwave_connector::source::SplitImpl; From c455b70fdc6b151fe14d4ae5abac6eb6b01ce4d3 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 4 Jun 2024 17:17:42 +0800 Subject: [PATCH 34/36] fix --- .../src/executor/backfill/cdc/upstream_table/snapshot.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs b/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs index 548b6a5feec7..a3944fcb6e97 100644 --- a/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs +++ b/src/stream/src/executor/backfill/cdc/upstream_table/snapshot.rs @@ -229,9 +229,8 @@ mod tests { use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; - use risingwave_connector::source::cdc::external::{ - ExternalTableReader, MySqlExternalTableReader, SchemaTableName, - }; + use risingwave_connector::source::cdc::external::mysql::MySqlExternalTableReader; + use risingwave_connector::source::cdc::external::{ExternalTableReader, SchemaTableName}; use crate::executor::backfill::utils::{get_new_pos, iter_chunks}; From b75c77945ac5b796365d6cf7f494028cc997640b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 6 Jun 2024 17:36:27 +0800 Subject: [PATCH 35/36] minor --- src/frontend/src/handler/create_table.rs | 23 ++++++++++++----------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index f981534d4fa2..edeb4ae5441d 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -1020,6 +1020,18 @@ fn sanity_check_for_cdc_table( constraints: &Vec, source_watermarks: &Vec, ) -> Result<()> { + for c in column_defs { + for op in &c.options { + if let ColumnOption::GeneratedColumns(_) = op.option { + return Err(ErrorCode::NotSupported( + "generated column defined on the table created from a CDC source".into(), + "Remove the generated column in the column list".into(), + ) + .into()); + } + } + } + // wildcard cannot be used with column definitions if wildcard_idx.is_some() && !column_defs.is_empty() { return Err(ErrorCode::NotSupported( @@ -1067,17 +1079,6 @@ fn sanity_check_for_cdc_table( ) .into()); } - for c in column_defs { - for op in &c.options { - if let ColumnOption::GeneratedColumns(_) = op.option { - return Err(ErrorCode::NotSupported( - "generated column defined on the table created from a CDC source".into(), - "Remove the generated column in the column list".into(), - ) - .into()); - } - } - } Ok(()) } From a9525a42ab42da4e2b9a554f000ee63f52ef2854 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 10 Jun 2024 14:34:28 +0800 Subject: [PATCH 36/36] minor --- e2e_test/source/cdc_inline/auto_schema_map_mysql.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt index 87e59736e037..5d27c73765e8 100644 --- a/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt +++ b/e2e_test/source/cdc_inline/auto_schema_map_mysql.slt @@ -2,7 +2,7 @@ control substitution on # test case need to cover all data types system ok -mysql --protocol=tcp -u root -e "DROP DATABASE IF EXISTS mytest; CREATE DATABASE mytest;" +mysql -e "DROP DATABASE IF EXISTS mytest; CREATE DATABASE mytest;" system ok mysql --protocol=tcp -u root mytest -e "