Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

feat: skip rw's generated column when validating source #16522

Merged
merged 6 commits into from
Apr 28, 2024
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
28 changes: 28 additions & 0 deletions e2e_test/source/cdc/cdc.validate.postgres.slt
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,9 @@ create table shipments (
slot.name = 'shipments'
) format canal encode csv;

statement ok
drop table shipments;

statement ok
explain create table numeric_to_rw_int256 (
id int,
Expand Down Expand Up @@ -217,3 +220,28 @@ explain create table enum_to_varchar (
table.name = 'enum_table',
slot.name = 'enum_to_varchar'
);

# generated column
statement ok
explain create table shipments (
shipment_id INTEGER,
order_id INTEGER,
origin STRING,
destination STRING,
is_arrived boolean,
generated_c INGEGER AS shipment_id + order_id,
PRIMARY KEY (shipment_id)
) with (
connector = 'postgres-cdc',
hostname = 'db',
port = '5432',
username = 'postgres',
password = 'postgres',
database.name = 'cdc_test',
schema.name = 'public',
table.name = 'shipments',
slot.name = 'shipments'
);

statement ok
drop table shipments;
32 changes: 18 additions & 14 deletions src/connector/src/source/cdc/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,9 +23,9 @@ use std::marker::PhantomData;

pub use enumerator::*;
use itertools::Itertools;
use risingwave_common::catalog::{ColumnDesc, Field, Schema};
use risingwave_pb::catalog::PbSource;
use risingwave_pb::connector_service::{PbSourceType, PbTableSchema, SourceType, TableSchema};
use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn;
use risingwave_pb::plan_common::ExternalTableDesc;
use simd_json::prelude::ArrayTrait;
pub use source::*;
Expand Down Expand Up @@ -144,6 +144,12 @@ where
.columns
.iter()
.flat_map(|col| &col.column_desc)
.filter(|col| {
!matches!(
col.generated_or_default_column,
Some(GeneratedOrDefaultColumn::GeneratedColumn(_))
)
})
.cloned()
.collect(),
pk_indices,
Expand All @@ -159,7 +165,17 @@ where
table_desc.connect_properties.clone().into_iter().collect();

let table_schema = TableSchema {
columns: table_desc.columns.clone(),
columns: table_desc
.columns
.iter()
.filter(|col| {
!matches!(
col.generated_or_default_column,
Some(GeneratedOrDefaultColumn::GeneratedColumn(_))
)
})
.cloned()
.collect(),
pk_indices: table_desc.stream_key.clone(),
};

Expand All @@ -181,16 +197,4 @@ impl<T: CdcSourceTypeTrait> CdcProperties<T> {
pub fn get_source_type_pb(&self) -> SourceType {
SourceType::from(T::source_type())
}

pub fn schema(&self) -> Schema {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

this function is not used.

Schema {
fields: self
.table_schema
.columns
.iter()
.map(ColumnDesc::from)
.map(Field::from)
.collect(),
}
}
}
10 changes: 10 additions & 0 deletions src/rpc_client/src/connector_client.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ use risingwave_pb::connector_service::sink_writer_stream_request::{
};
use risingwave_pb::connector_service::sink_writer_stream_response::CommitResponse;
use risingwave_pb::connector_service::*;
use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn;
use thiserror_ext::AsReport;
use tokio_stream::wrappers::ReceiverStream;
use tonic::transport::{Channel, Endpoint};
Expand Down Expand Up @@ -238,6 +239,15 @@ impl ConnectorClient {
is_source_job: bool,
is_backfill_table: bool,
) -> Result<()> {
let table_schema = table_schema.map(|mut table_schema| {
table_schema.columns.retain(|c| {
!matches!(
c.generated_or_default_column,
Some(GeneratedOrDefaultColumn::GeneratedColumn(_))
)
});
table_schema
});
let response = self
.rpc_client
.clone()
Expand Down
Loading