From 0cb68c0ad3e8c487382999b623fb2b197fd4b873 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 26 Nov 2024 12:52:00 +0800 Subject: [PATCH 01/56] ignore sqrt error --- src/tests/sqlsmith/src/validation.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/tests/sqlsmith/src/validation.rs b/src/tests/sqlsmith/src/validation.rs index 99580616c15d..c49ee05cbdd9 100644 --- a/src/tests/sqlsmith/src/validation.rs +++ b/src/tests/sqlsmith/src/validation.rs @@ -104,6 +104,10 @@ pub fn is_neg_exp_error(db_error: &str) -> bool { db_error.contains("zero raised to a negative power is undefined") } +pub fn is_neg_input_error(db_error: &str) -> bool { + db_error.contains("input cannot be negative value") +} + /// Certain errors are permitted to occur. This is because: /// 1. It is more complex to generate queries without these errors. /// 2. These errors seldom occur, skipping them won't affect overall effectiveness of sqlsmith. @@ -121,4 +125,5 @@ pub fn is_permissible_error(db_error: &str) -> bool { || is_overlay_start_error(db_error) || is_broken_channel_error(db_error) || is_neg_exp_error(db_error) + || is_neg_input_error(db_error) } From 9b2383c39273cf9c0a0412b6e5aa117581756874 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 8 Nov 2024 19:03:57 +0800 Subject: [PATCH 02/56] add postgres sink --- src/connector/src/sink/mod.rs | 1 + src/connector/src/sink/postgres.rs | 0 2 files changed, 1 insertion(+) create mode 100644 src/connector/src/sink/postgres.rs diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 03b2fc68476c..4f5c66e647c7 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -112,6 +112,7 @@ macro_rules! for_all_sinks { { GooglePubSub, $crate::sink::google_pubsub::GooglePubSubSink }, { Nats, $crate::sink::nats::NatsSink }, { Jdbc, $crate::sink::remote::JdbcSink }, + { Postgres, $crate::sink::postgres::PostgresSink } // { ElasticSearchJava, $crate::sink::remote::ElasticSearchJavaSink }, // { OpensearchJava, $crate::sink::remote::OpenSearchJavaSink }, { ElasticSearch, $crate::sink::elasticsearch_opensearch::elasticsearch::ElasticSearchSink }, diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs new file mode 100644 index 000000000000..e69de29bb2d1 From 4cbbff560bf16012c421252be3b3b392938efb8f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 11 Nov 2024 12:04:53 +0800 Subject: [PATCH 03/56] skeleton --- src/connector/src/sink/mod.rs | 10 +- src/connector/src/sink/postgres.rs | 662 +++++++++++++++++++++++++++++ 2 files changed, 671 insertions(+), 1 deletion(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 4f5c66e647c7..33d4bf298bf9 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -38,6 +38,7 @@ pub mod pulsar; pub mod redis; pub mod remote; pub mod sqlserver; +pub mod postgres; pub mod starrocks; pub mod test_sink; pub mod trivial; @@ -112,7 +113,6 @@ macro_rules! for_all_sinks { { GooglePubSub, $crate::sink::google_pubsub::GooglePubSubSink }, { Nats, $crate::sink::nats::NatsSink }, { Jdbc, $crate::sink::remote::JdbcSink }, - { Postgres, $crate::sink::postgres::PostgresSink } // { ElasticSearchJava, $crate::sink::remote::ElasticSearchJavaSink }, // { OpensearchJava, $crate::sink::remote::OpenSearchJavaSink }, { ElasticSearch, $crate::sink::elasticsearch_opensearch::elasticsearch::ElasticSearchSink }, @@ -134,6 +134,8 @@ macro_rules! for_all_sinks { { DynamoDb, $crate::sink::dynamodb::DynamoDbSink }, { Mongodb, $crate::sink::mongodb::MongodbSink }, { SqlServer, $crate::sink::sqlserver::SqlServerSink }, + { Postgres, $crate::sink::postgres::PostgresSink }, + { Test, $crate::sink::test_sink::TestSink }, { Table, $crate::sink::trivial::TableSink } } @@ -867,6 +869,12 @@ pub enum SinkError { #[backtrace] anyhow::Error, ), + #[error("Postgres error: {0}")] + Postgres( + #[source] + #[backtrace] + anyhow::Error, + ), #[error(transparent)] Connector( #[from] diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index e69de29bb2d1..55e247682b54 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -0,0 +1,662 @@ +// 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::{BTreeMap, HashMap}; +use std::sync::Arc; + +use anyhow::{anyhow, Context}; +use async_trait::async_trait; +use risingwave_common::array::{Op, RowRef, StreamChunk}; +use risingwave_common::bitmap::Bitmap; +use risingwave_common::catalog::Schema; +use risingwave_common::row::{OwnedRow, Row}; +use risingwave_common::types::{DataType, Decimal}; +use serde_derive::Deserialize; +use serde_with::{serde_as, DisplayFromStr}; +use simd_json::prelude::ArrayTrait; +use tiberius::numeric::Numeric; +use tiberius::{AuthMethod, Client, ColumnData, Config, Query}; +use tokio::net::TcpStream; +use tokio_util::compat::TokioAsyncWriteCompatExt; +use with_options::WithOptions; + +use super::{ + SinkError, SinkWriterMetrics, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, +}; +use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; +use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriterParam}; + +pub const POSTGRES_SINK: &str = "postgres"; + +fn default_max_batch_rows() -> usize { + 1024 +} + +#[serde_as] +#[derive(Clone, Debug, Deserialize, WithOptions)] +pub struct PostgresConfig { + #[serde(rename = "postgres.host")] + pub host: String, + #[serde(rename = "postgres.port")] + #[serde_as(as = "DisplayFromStr")] + pub port: u16, + #[serde(rename = "postgres.user")] + pub user: String, + #[serde(rename = "postgres.password")] + pub password: String, + #[serde(rename = "postgres.database")] + pub database: String, + #[serde(rename = "postgres.table")] + pub table: String, + #[serde( + rename = "postgres.max_batch_rows", + default = "default_max_batch_rows" + )] + #[serde_as(as = "DisplayFromStr")] + pub max_batch_rows: usize, + pub r#type: String, // accept "append-only" or "upsert" +} + +impl PostgresConfig { + pub fn from_btreemap(properties: BTreeMap) -> Result { + let config = + serde_json::from_value::(serde_json::to_value(properties).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) + } +} + +#[derive(Debug)] +pub struct PostgresSink { + pub config: PostgresConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, +} + +impl PostgresSink { + pub fn new( + mut config: PostgresConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + Ok(Self { + config, + schema, + pk_indices, + is_append_only, + }) + } +} + +impl TryFrom for PostgresSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = PostgresConfig::from_btreemap(param.properties)?; + PostgresSink::new( + config, + schema, + param.downstream_pk, + param.sink_type.is_append_only(), + ) + } +} + +impl Sink for PostgresSink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = LogSinkerOf; + + const SINK_NAME: &'static str = POSTGRES_SINK; + + async fn validate(&self) -> Result<()> { + if !self.is_append_only && self.pk_indices.is_empty() { + return Err(SinkError::Config(anyhow!( + "Primary key not defined for upsert Postgres sink (please define in `primary_key` field)"))); + } + + // NOTE(kwannoel): postgresql types should be superset of rw. + // for f in self.schema.fields() { + // check_data_type_compatibility(&f.data_type)?; + // } + + // Query table metadata from SQL Server. + let mut sql_server_table_metadata = HashMap::new(); + let mut sql_client = PostgresClient::new(&self.config).await?; + let query_table_metadata_error = || { + SinkError::Postgres(anyhow!(format!( + "Postgres table {} metadata error", + self.config.table + ))) + }; + + // Query the column information from the `information_schema.columns` table + let table_name = "your_table_name"; + let rows = sqlx::query( + r#" + SELECT column_name, data_type, is_nullable + FROM information_schema.columns + WHERE table_name = $1 + ORDER BY ordinal_position; + "# + ) + .bind(table_name) + .fetch_all(&pool) + .await?; + + // Print the schema details + for row in rows { + let column_name: &str = row.get("column_name"); + let data_type: &str = row.get("data_type"); + let is_nullable: &str = row.get("is_nullable"); + + println!("Column: {}, Type: {}, Nullable: {}", column_name, data_type, is_nullable); + } + + // Validate Column name and Primary Key + for (idx, col) in self.schema.fields().iter().enumerate() { + let rw_is_pk = self.pk_indices.contains(&idx); + match sql_server_table_metadata.get(&col.name) { + None => { + return Err(SinkError::Postgres(anyhow!(format!( + "column {} not found in the downstream SQL Server table {}", + col.name, self.config.table + )))); + } + Some(sql_server_is_pk) => { + if self.is_append_only { + continue; + } + if rw_is_pk && !*sql_server_is_pk { + return Err(SinkError::Postgres(anyhow!(format!( + "column {} specified in primary_key mismatches with the downstream SQL Server table {} PK", + col.name, self.config.table, + )))); + } + if !rw_is_pk && *sql_server_is_pk { + return Err(SinkError::Postgres(anyhow!(format!( + "column {} unspecified in primary_key mismatches with the downstream SQL Server table {} PK", + col.name, self.config.table, + )))); + } + } + } + } + + if !self.is_append_only { + let sql_server_pk_count = sql_server_table_metadata + .values() + .filter(|is_pk| **is_pk) + .count(); + if sql_server_pk_count != self.pk_indices.len() { + return Err(SinkError::Postgres(anyhow!(format!( + "primary key does not match between RisingWave sink ({}) and SQL Server table {} ({})", + self.pk_indices.len(), + self.config.table, + sql_server_pk_count, + )))); + } + } + + Ok(()) + } + + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok(PostgresSinkWriter::new( + self.config.clone(), + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + ) + .await? + .into_log_sinker(SinkWriterMetrics::new(&writer_param))) + } +} + +enum SqlOp { + Insert(OwnedRow), + Merge(OwnedRow), + Delete(OwnedRow), +} + +pub struct PostgresSinkWriter { + config: PostgresConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + sql_client: PostgresClient, + ops: Vec, +} + +impl PostgresSinkWriter { + async fn new( + config: PostgresConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + let sql_client = PostgresClient::new(&config).await?; + let writer = Self { + config, + schema, + pk_indices, + is_append_only, + sql_client, + ops: vec![], + }; + Ok(writer) + } + + async fn delete_one(&mut self, row: RowRef<'_>) -> Result<()> { + if self.ops.len() + 1 >= self.config.max_batch_rows { + self.flush().await?; + } + self.ops.push(SqlOp::Delete(row.into_owned_row())); + Ok(()) + } + + async fn upsert_one(&mut self, row: RowRef<'_>) -> Result<()> { + if self.ops.len() + 1 >= self.config.max_batch_rows { + self.flush().await?; + } + self.ops.push(SqlOp::Merge(row.into_owned_row())); + Ok(()) + } + + async fn insert_one(&mut self, row: RowRef<'_>) -> Result<()> { + if self.ops.len() + 1 >= self.config.max_batch_rows { + self.flush().await?; + } + self.ops.push(SqlOp::Insert(row.into_owned_row())); + Ok(()) + } + + async fn flush(&mut self) -> Result<()> { + use std::fmt::Write; + if self.ops.is_empty() { + return Ok(()); + } + let mut query_str = String::new(); + let col_num = self.schema.fields.len(); + let mut next_param_id = 1; + let non_pk_col_indices = (0..col_num) + .filter(|idx| !self.pk_indices.contains(idx)) + .collect::>(); + let all_col_names = self + .schema + .fields + .iter() + .map(|f| format!("[{}]", f.name)) + .collect::>() + .join(","); + let all_source_col_names = self + .schema + .fields + .iter() + .map(|f| format!("[SOURCE].[{}]", f.name)) + .collect::>() + .join(","); + let pk_match = self + .pk_indices + .iter() + .map(|idx| { + format!( + "[SOURCE].[{}]=[TARGET].[{}]", + &self.schema[*idx].name, &self.schema[*idx].name + ) + }) + .collect::>() + .join(" AND "); + let param_placeholders = |param_id: &mut usize| { + let params = (*param_id..(*param_id + col_num)) + .map(|i| format!("@P{}", i)) + .collect::>() + .join(","); + *param_id += col_num; + params + }; + let set_all_source_col = non_pk_col_indices + .iter() + .map(|idx| { + format!( + "[{}]=[SOURCE].[{}]", + &self.schema[*idx].name, &self.schema[*idx].name + ) + }) + .collect::>() + .join(","); + // TODO: avoid repeating the SQL + for op in &self.ops { + match op { + SqlOp::Insert(_) => { + write!( + &mut query_str, + "INSERT INTO [{}] ({}) VALUES ({});", + self.config.table, + all_col_names, + param_placeholders(&mut next_param_id), + ) + .unwrap(); + } + SqlOp::Merge(_) => { + write!( + &mut query_str, + r#"MERGE [{}] AS [TARGET] + USING (VALUES ({})) AS [SOURCE] ({}) + ON {} + WHEN MATCHED THEN UPDATE SET {} + WHEN NOT MATCHED THEN INSERT ({}) VALUES ({});"#, + self.config.table, + param_placeholders(&mut next_param_id), + all_col_names, + pk_match, + set_all_source_col, + all_col_names, + all_source_col_names, + ) + .unwrap(); + } + SqlOp::Delete(_) => { + write!( + &mut query_str, + r#"DELETE FROM [{}] WHERE {};"#, + self.config.table, + self.pk_indices + .iter() + .map(|idx| { + let condition = + format!("[{}]=@P{}", self.schema[*idx].name, next_param_id); + next_param_id += 1; + condition + }) + .collect::>() + .join(" AND "), + ) + .unwrap(); + } + } + } + + let mut query = Query::new(query_str); + for op in self.ops.drain(..) { + match op { + SqlOp::Insert(row) => { + bind_params(&mut query, row, &self.schema, 0..col_num)?; + } + SqlOp::Merge(row) => { + bind_params(&mut query, row, &self.schema, 0..col_num)?; + } + SqlOp::Delete(row) => { + bind_params( + &mut query, + row, + &self.schema, + self.pk_indices.iter().copied(), + )?; + } + } + } + query.execute(&mut self.sql_client.inner_client).await?; + Ok(()) + } +} + +#[async_trait] +impl SinkWriter for PostgresSinkWriter { + async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { + Ok(()) + } + + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + for (op, row) in chunk.rows() { + match op { + Op::Insert => { + if self.is_append_only { + self.insert_one(row).await?; + } else { + self.upsert_one(row).await?; + } + } + Op::UpdateInsert => { + debug_assert!(!self.is_append_only); + self.upsert_one(row).await?; + } + Op::Delete => { + debug_assert!(!self.is_append_only); + self.delete_one(row).await?; + } + Op::UpdateDelete => {} + } + } + Ok(()) + } + + async fn barrier(&mut self, is_checkpoint: bool) -> Result { + if is_checkpoint { + self.flush().await?; + } + Ok(()) + } + + async fn abort(&mut self) -> Result<()> { + Ok(()) + } + + async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { + Ok(()) + } +} + +#[derive(Debug)] +pub struct PostgresClient { + pub inner_client: Client>, +} + +impl PostgresClient { + async fn new(msconfig: &PostgresConfig) -> Result { + let mut config = Config::new(); + config.host(&msconfig.host); + config.port(msconfig.port); + config.authentication(AuthMethod::sql_server(&msconfig.user, &msconfig.password)); + config.database(&msconfig.database); + config.trust_cert(); + Self::new_with_config(config).await + } + + pub async fn new_with_config(mut config: Config) -> Result { + let tcp = TcpStream::connect(config.get_addr()) + .await + .context("failed to connect to sql server") + .map_err(SinkError::Postgres)?; + tcp.set_nodelay(true) + .context("failed to setting nodelay when connecting to sql server") + .map_err(SinkError::Postgres)?; + + let client = match Client::connect(config.clone(), tcp.compat_write()).await { + // Connection successful. + Ok(client) => client, + // The server wants us to redirect to a different address + Err(tiberius::error::Error::Routing { host, port }) => { + config.host(&host); + config.port(port); + let tcp = TcpStream::connect(config.get_addr()) + .await + .context("failed to connect to sql server after routing") + .map_err(SinkError::Postgres)?; + tcp.set_nodelay(true) + .context( + "failed to setting nodelay when connecting to sql server after routing", + ) + .map_err(SinkError::Postgres)?; + // we should not have more than one redirect, so we'll short-circuit here. + Client::connect(config, tcp.compat_write()).await? + } + Err(e) => return Err(e.into()), + }; + + Ok(Self { + inner_client: client, + }) + } +} + +fn bind_params( + query: &mut Query<'_>, + row: impl Row, + schema: &Schema, + col_indices: impl Iterator, +) -> Result<()> { + use risingwave_common::types::ScalarRefImpl; + for col_idx in col_indices { + match row.datum_at(col_idx) { + Some(data_ref) => match data_ref { + ScalarRefImpl::Int16(v) => query.bind(v), + ScalarRefImpl::Int32(v) => query.bind(v), + ScalarRefImpl::Int64(v) => query.bind(v), + ScalarRefImpl::Float32(v) => query.bind(v.into_inner()), + ScalarRefImpl::Float64(v) => query.bind(v.into_inner()), + ScalarRefImpl::Utf8(v) => query.bind(v.to_owned()), + ScalarRefImpl::Bool(v) => query.bind(v), + ScalarRefImpl::Decimal(v) => match v { + Decimal::Normalized(d) => { + query.bind(decimal_to_sql(&d)); + } + Decimal::NaN | Decimal::PositiveInf | Decimal::NegativeInf => { + tracing::warn!( + "Inf, -Inf, Nan in RisingWave decimal is converted into SQL Server null!" + ); + query.bind(None as Option); + } + }, + ScalarRefImpl::Date(v) => query.bind(v.0), + ScalarRefImpl::Timestamp(v) => query.bind(v.0), + ScalarRefImpl::Timestamptz(v) => query.bind(v.timestamp_micros()), + ScalarRefImpl::Time(v) => query.bind(v.0), + ScalarRefImpl::Bytea(v) => query.bind(v.to_vec()), + ScalarRefImpl::Interval(_) => return Err(data_type_not_supported("Interval")), + ScalarRefImpl::Jsonb(_) => return Err(data_type_not_supported("Jsonb")), + ScalarRefImpl::Struct(_) => return Err(data_type_not_supported("Struct")), + ScalarRefImpl::List(_) => return Err(data_type_not_supported("List")), + ScalarRefImpl::Int256(_) => return Err(data_type_not_supported("Int256")), + ScalarRefImpl::Serial(_) => return Err(data_type_not_supported("Serial")), + ScalarRefImpl::Map(_) => return Err(data_type_not_supported("Map")), + }, + None => match schema[col_idx].data_type { + DataType::Boolean => { + query.bind(None as Option); + } + DataType::Int16 => { + query.bind(None as Option); + } + DataType::Int32 => { + query.bind(None as Option); + } + DataType::Int64 => { + query.bind(None as Option); + } + DataType::Float32 => { + query.bind(None as Option); + } + DataType::Float64 => { + query.bind(None as Option); + } + DataType::Decimal => { + query.bind(None as Option); + } + DataType::Date => { + query.bind(None as Option); + } + DataType::Time => { + query.bind(None as Option); + } + DataType::Timestamp => { + query.bind(None as Option); + } + DataType::Timestamptz => { + query.bind(None as Option); + } + DataType::Varchar => { + query.bind(None as Option); + } + DataType::Bytea => { + query.bind(None as Option>); + } + DataType::Interval => return Err(data_type_not_supported("Interval")), + DataType::Struct(_) => return Err(data_type_not_supported("Struct")), + DataType::List(_) => return Err(data_type_not_supported("List")), + DataType::Jsonb => return Err(data_type_not_supported("Jsonb")), + DataType::Serial => return Err(data_type_not_supported("Serial")), + DataType::Int256 => return Err(data_type_not_supported("Int256")), + DataType::Map(_) => return Err(data_type_not_supported("Map")), + }, + }; + } + Ok(()) +} + +fn data_type_not_supported(data_type_name: &str) -> SinkError { + SinkError::Postgres(anyhow!(format!( + "{data_type_name} is not supported in SQL Server" + ))) +} + +fn check_data_type_compatibility(data_type: &DataType) -> Result<()> { + match data_type { + DataType::Boolean + | DataType::Int16 + | DataType::Int32 + | DataType::Int64 + | DataType::Float32 + | DataType::Float64 + | DataType::Decimal + | DataType::Date + | DataType::Varchar + | DataType::Time + | DataType::Timestamp + | DataType::Timestamptz + | DataType::Bytea => Ok(()), + DataType::Interval => Err(data_type_not_supported("Interval")), + DataType::Struct(_) => Err(data_type_not_supported("Struct")), + DataType::List(_) => Err(data_type_not_supported("List")), + DataType::Jsonb => Err(data_type_not_supported("Jsonb")), + DataType::Serial => Err(data_type_not_supported("Serial")), + DataType::Int256 => Err(data_type_not_supported("Int256")), + DataType::Map(_) => Err(data_type_not_supported("Map")), + } +} + +/// The implementation is copied from tiberius crate. +fn decimal_to_sql(decimal: &rust_decimal::Decimal) -> Numeric { + let unpacked = decimal.unpack(); + + let mut value = (((unpacked.hi as u128) << 64) + + ((unpacked.mid as u128) << 32) + + unpacked.lo as u128) as i128; + + if decimal.is_sign_negative() { + value = -value; + } + + Numeric::new_with_scale(value, decimal.scale() as u8) +} From 6e0a00b276f71ae03b42221c3d3b94f2d18ba874 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 14 Nov 2024 10:38:11 +0800 Subject: [PATCH 04/56] stub validation logic --- src/connector/src/sink/postgres.rs | 154 ++++++++++++++--------------- 1 file changed, 77 insertions(+), 77 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 55e247682b54..92cc0b17b647 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -141,83 +141,83 @@ impl Sink for PostgresSink { // check_data_type_compatibility(&f.data_type)?; // } - // Query table metadata from SQL Server. - let mut sql_server_table_metadata = HashMap::new(); - let mut sql_client = PostgresClient::new(&self.config).await?; - let query_table_metadata_error = || { - SinkError::Postgres(anyhow!(format!( - "Postgres table {} metadata error", - self.config.table - ))) - }; - - // Query the column information from the `information_schema.columns` table - let table_name = "your_table_name"; - let rows = sqlx::query( - r#" - SELECT column_name, data_type, is_nullable - FROM information_schema.columns - WHERE table_name = $1 - ORDER BY ordinal_position; - "# - ) - .bind(table_name) - .fetch_all(&pool) - .await?; - - // Print the schema details - for row in rows { - let column_name: &str = row.get("column_name"); - let data_type: &str = row.get("data_type"); - let is_nullable: &str = row.get("is_nullable"); - - println!("Column: {}, Type: {}, Nullable: {}", column_name, data_type, is_nullable); - } - - // Validate Column name and Primary Key - for (idx, col) in self.schema.fields().iter().enumerate() { - let rw_is_pk = self.pk_indices.contains(&idx); - match sql_server_table_metadata.get(&col.name) { - None => { - return Err(SinkError::Postgres(anyhow!(format!( - "column {} not found in the downstream SQL Server table {}", - col.name, self.config.table - )))); - } - Some(sql_server_is_pk) => { - if self.is_append_only { - continue; - } - if rw_is_pk && !*sql_server_is_pk { - return Err(SinkError::Postgres(anyhow!(format!( - "column {} specified in primary_key mismatches with the downstream SQL Server table {} PK", - col.name, self.config.table, - )))); - } - if !rw_is_pk && *sql_server_is_pk { - return Err(SinkError::Postgres(anyhow!(format!( - "column {} unspecified in primary_key mismatches with the downstream SQL Server table {} PK", - col.name, self.config.table, - )))); - } - } - } - } - - if !self.is_append_only { - let sql_server_pk_count = sql_server_table_metadata - .values() - .filter(|is_pk| **is_pk) - .count(); - if sql_server_pk_count != self.pk_indices.len() { - return Err(SinkError::Postgres(anyhow!(format!( - "primary key does not match between RisingWave sink ({}) and SQL Server table {} ({})", - self.pk_indices.len(), - self.config.table, - sql_server_pk_count, - )))); - } - } + // // Query table metadata from SQL Server. + // let mut sql_server_table_metadata = HashMap::new(); + // let mut sql_client = PostgresClient::new(&self.config).await?; + // let query_table_metadata_error = || { + // SinkError::Postgres(anyhow!(format!( + // "Postgres table {} metadata error", + // self.config.table + // ))) + // }; + // + // // Query the column information from the `information_schema.columns` table + // let table_name = "your_table_name"; + // let rows = sqlx::query( + // r#" + // SELECT column_name, data_type, is_nullable + // FROM information_schema.columns + // WHERE table_name = $1 + // ORDER BY ordinal_position; + // "# + // ) + // .bind(table_name) + // .fetch_all(&pool) + // .await?; + // + // // Print the schema details + // for row in rows { + // let column_name: &str = row.get("column_name"); + // let data_type: &str = row.get("data_type"); + // let is_nullable: &str = row.get("is_nullable"); + // + // println!("Column: {}, Type: {}, Nullable: {}", column_name, data_type, is_nullable); + // } + // + // // Validate Column name and Primary Key + // for (idx, col) in self.schema.fields().iter().enumerate() { + // let rw_is_pk = self.pk_indices.contains(&idx); + // match sql_server_table_metadata.get(&col.name) { + // None => { + // return Err(SinkError::Postgres(anyhow!(format!( + // "column {} not found in the downstream SQL Server table {}", + // col.name, self.config.table + // )))); + // } + // Some(sql_server_is_pk) => { + // if self.is_append_only { + // continue; + // } + // if rw_is_pk && !*sql_server_is_pk { + // return Err(SinkError::Postgres(anyhow!(format!( + // "column {} specified in primary_key mismatches with the downstream SQL Server table {} PK", + // col.name, self.config.table, + // )))); + // } + // if !rw_is_pk && *sql_server_is_pk { + // return Err(SinkError::Postgres(anyhow!(format!( + // "column {} unspecified in primary_key mismatches with the downstream SQL Server table {} PK", + // col.name, self.config.table, + // )))); + // } + // } + // } + // } + // + // if !self.is_append_only { + // let sql_server_pk_count = sql_server_table_metadata + // .values() + // .filter(|is_pk| **is_pk) + // .count(); + // if sql_server_pk_count != self.pk_indices.len() { + // return Err(SinkError::Postgres(anyhow!(format!( + // "primary key does not match between RisingWave sink ({}) and SQL Server table {} ({})", + // self.pk_indices.len(), + // self.config.table, + // sql_server_pk_count, + // )))); + // } + // } Ok(()) } From 816082d5a0a54b5cef4bb19d2d2da99335cbff8e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 14 Nov 2024 19:34:41 +0800 Subject: [PATCH 05/56] remove more unnecessary stuff + make compile --- src/connector/src/sink/mod.rs | 2 +- src/connector/src/sink/postgres.rs | 374 +---------------------------- 2 files changed, 5 insertions(+), 371 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 33d4bf298bf9..047e9be4998a 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -34,11 +34,11 @@ pub mod mock_coordination_client; pub mod mongodb; pub mod mqtt; pub mod nats; +pub mod postgres; pub mod pulsar; pub mod redis; pub mod remote; pub mod sqlserver; -pub mod postgres; pub mod starrocks; pub mod test_sink; pub mod trivial; diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 92cc0b17b647..e410fed4a7f4 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -25,8 +25,6 @@ use risingwave_common::types::{DataType, Decimal}; use serde_derive::Deserialize; use serde_with::{serde_as, DisplayFromStr}; use simd_json::prelude::ArrayTrait; -use tiberius::numeric::Numeric; -use tiberius::{AuthMethod, Client, ColumnData, Config, Query}; use tokio::net::TcpStream; use tokio_util::compat::TokioAsyncWriteCompatExt; use with_options::WithOptions; @@ -59,10 +57,7 @@ pub struct PostgresConfig { pub database: String, #[serde(rename = "postgres.table")] pub table: String, - #[serde( - rename = "postgres.max_batch_rows", - default = "default_max_batch_rows" - )] + #[serde(rename = "postgres.max_batch_rows", default = "default_max_batch_rows")] #[serde_as(as = "DisplayFromStr")] pub max_batch_rows: usize, pub r#type: String, // accept "append-only" or "upsert" @@ -136,89 +131,6 @@ impl Sink for PostgresSink { "Primary key not defined for upsert Postgres sink (please define in `primary_key` field)"))); } - // NOTE(kwannoel): postgresql types should be superset of rw. - // for f in self.schema.fields() { - // check_data_type_compatibility(&f.data_type)?; - // } - - // // Query table metadata from SQL Server. - // let mut sql_server_table_metadata = HashMap::new(); - // let mut sql_client = PostgresClient::new(&self.config).await?; - // let query_table_metadata_error = || { - // SinkError::Postgres(anyhow!(format!( - // "Postgres table {} metadata error", - // self.config.table - // ))) - // }; - // - // // Query the column information from the `information_schema.columns` table - // let table_name = "your_table_name"; - // let rows = sqlx::query( - // r#" - // SELECT column_name, data_type, is_nullable - // FROM information_schema.columns - // WHERE table_name = $1 - // ORDER BY ordinal_position; - // "# - // ) - // .bind(table_name) - // .fetch_all(&pool) - // .await?; - // - // // Print the schema details - // for row in rows { - // let column_name: &str = row.get("column_name"); - // let data_type: &str = row.get("data_type"); - // let is_nullable: &str = row.get("is_nullable"); - // - // println!("Column: {}, Type: {}, Nullable: {}", column_name, data_type, is_nullable); - // } - // - // // Validate Column name and Primary Key - // for (idx, col) in self.schema.fields().iter().enumerate() { - // let rw_is_pk = self.pk_indices.contains(&idx); - // match sql_server_table_metadata.get(&col.name) { - // None => { - // return Err(SinkError::Postgres(anyhow!(format!( - // "column {} not found in the downstream SQL Server table {}", - // col.name, self.config.table - // )))); - // } - // Some(sql_server_is_pk) => { - // if self.is_append_only { - // continue; - // } - // if rw_is_pk && !*sql_server_is_pk { - // return Err(SinkError::Postgres(anyhow!(format!( - // "column {} specified in primary_key mismatches with the downstream SQL Server table {} PK", - // col.name, self.config.table, - // )))); - // } - // if !rw_is_pk && *sql_server_is_pk { - // return Err(SinkError::Postgres(anyhow!(format!( - // "column {} unspecified in primary_key mismatches with the downstream SQL Server table {} PK", - // col.name, self.config.table, - // )))); - // } - // } - // } - // } - // - // if !self.is_append_only { - // let sql_server_pk_count = sql_server_table_metadata - // .values() - // .filter(|is_pk| **is_pk) - // .count(); - // if sql_server_pk_count != self.pk_indices.len() { - // return Err(SinkError::Postgres(anyhow!(format!( - // "primary key does not match between RisingWave sink ({}) and SQL Server table {} ({})", - // self.pk_indices.len(), - // self.config.table, - // sql_server_pk_count, - // )))); - // } - // } - Ok(()) } @@ -293,131 +205,6 @@ impl PostgresSinkWriter { } async fn flush(&mut self) -> Result<()> { - use std::fmt::Write; - if self.ops.is_empty() { - return Ok(()); - } - let mut query_str = String::new(); - let col_num = self.schema.fields.len(); - let mut next_param_id = 1; - let non_pk_col_indices = (0..col_num) - .filter(|idx| !self.pk_indices.contains(idx)) - .collect::>(); - let all_col_names = self - .schema - .fields - .iter() - .map(|f| format!("[{}]", f.name)) - .collect::>() - .join(","); - let all_source_col_names = self - .schema - .fields - .iter() - .map(|f| format!("[SOURCE].[{}]", f.name)) - .collect::>() - .join(","); - let pk_match = self - .pk_indices - .iter() - .map(|idx| { - format!( - "[SOURCE].[{}]=[TARGET].[{}]", - &self.schema[*idx].name, &self.schema[*idx].name - ) - }) - .collect::>() - .join(" AND "); - let param_placeholders = |param_id: &mut usize| { - let params = (*param_id..(*param_id + col_num)) - .map(|i| format!("@P{}", i)) - .collect::>() - .join(","); - *param_id += col_num; - params - }; - let set_all_source_col = non_pk_col_indices - .iter() - .map(|idx| { - format!( - "[{}]=[SOURCE].[{}]", - &self.schema[*idx].name, &self.schema[*idx].name - ) - }) - .collect::>() - .join(","); - // TODO: avoid repeating the SQL - for op in &self.ops { - match op { - SqlOp::Insert(_) => { - write!( - &mut query_str, - "INSERT INTO [{}] ({}) VALUES ({});", - self.config.table, - all_col_names, - param_placeholders(&mut next_param_id), - ) - .unwrap(); - } - SqlOp::Merge(_) => { - write!( - &mut query_str, - r#"MERGE [{}] AS [TARGET] - USING (VALUES ({})) AS [SOURCE] ({}) - ON {} - WHEN MATCHED THEN UPDATE SET {} - WHEN NOT MATCHED THEN INSERT ({}) VALUES ({});"#, - self.config.table, - param_placeholders(&mut next_param_id), - all_col_names, - pk_match, - set_all_source_col, - all_col_names, - all_source_col_names, - ) - .unwrap(); - } - SqlOp::Delete(_) => { - write!( - &mut query_str, - r#"DELETE FROM [{}] WHERE {};"#, - self.config.table, - self.pk_indices - .iter() - .map(|idx| { - let condition = - format!("[{}]=@P{}", self.schema[*idx].name, next_param_id); - next_param_id += 1; - condition - }) - .collect::>() - .join(" AND "), - ) - .unwrap(); - } - } - } - - let mut query = Query::new(query_str); - for op in self.ops.drain(..) { - match op { - SqlOp::Insert(row) => { - bind_params(&mut query, row, &self.schema, 0..col_num)?; - } - SqlOp::Merge(row) => { - bind_params(&mut query, row, &self.schema, 0..col_num)?; - } - SqlOp::Delete(row) => { - bind_params( - &mut query, - row, - &self.schema, - self.pk_indices.iter().copied(), - )?; - } - } - } - query.execute(&mut self.sql_client.inner_client).await?; Ok(()) } } @@ -469,152 +256,14 @@ impl SinkWriter for PostgresSinkWriter { } #[derive(Debug)] -pub struct PostgresClient { - pub inner_client: Client>, -} +pub struct PostgresClient {} impl PostgresClient { - async fn new(msconfig: &PostgresConfig) -> Result { - let mut config = Config::new(); - config.host(&msconfig.host); - config.port(msconfig.port); - config.authentication(AuthMethod::sql_server(&msconfig.user, &msconfig.password)); - config.database(&msconfig.database); - config.trust_cert(); - Self::new_with_config(config).await - } - - pub async fn new_with_config(mut config: Config) -> Result { - let tcp = TcpStream::connect(config.get_addr()) - .await - .context("failed to connect to sql server") - .map_err(SinkError::Postgres)?; - tcp.set_nodelay(true) - .context("failed to setting nodelay when connecting to sql server") - .map_err(SinkError::Postgres)?; - - let client = match Client::connect(config.clone(), tcp.compat_write()).await { - // Connection successful. - Ok(client) => client, - // The server wants us to redirect to a different address - Err(tiberius::error::Error::Routing { host, port }) => { - config.host(&host); - config.port(port); - let tcp = TcpStream::connect(config.get_addr()) - .await - .context("failed to connect to sql server after routing") - .map_err(SinkError::Postgres)?; - tcp.set_nodelay(true) - .context( - "failed to setting nodelay when connecting to sql server after routing", - ) - .map_err(SinkError::Postgres)?; - // we should not have more than one redirect, so we'll short-circuit here. - Client::connect(config, tcp.compat_write()).await? - } - Err(e) => return Err(e.into()), - }; - - Ok(Self { - inner_client: client, - }) + async fn new(pg_config: &PostgresConfig) -> Result { + todo!() } } -fn bind_params( - query: &mut Query<'_>, - row: impl Row, - schema: &Schema, - col_indices: impl Iterator, -) -> Result<()> { - use risingwave_common::types::ScalarRefImpl; - for col_idx in col_indices { - match row.datum_at(col_idx) { - Some(data_ref) => match data_ref { - ScalarRefImpl::Int16(v) => query.bind(v), - ScalarRefImpl::Int32(v) => query.bind(v), - ScalarRefImpl::Int64(v) => query.bind(v), - ScalarRefImpl::Float32(v) => query.bind(v.into_inner()), - ScalarRefImpl::Float64(v) => query.bind(v.into_inner()), - ScalarRefImpl::Utf8(v) => query.bind(v.to_owned()), - ScalarRefImpl::Bool(v) => query.bind(v), - ScalarRefImpl::Decimal(v) => match v { - Decimal::Normalized(d) => { - query.bind(decimal_to_sql(&d)); - } - Decimal::NaN | Decimal::PositiveInf | Decimal::NegativeInf => { - tracing::warn!( - "Inf, -Inf, Nan in RisingWave decimal is converted into SQL Server null!" - ); - query.bind(None as Option); - } - }, - ScalarRefImpl::Date(v) => query.bind(v.0), - ScalarRefImpl::Timestamp(v) => query.bind(v.0), - ScalarRefImpl::Timestamptz(v) => query.bind(v.timestamp_micros()), - ScalarRefImpl::Time(v) => query.bind(v.0), - ScalarRefImpl::Bytea(v) => query.bind(v.to_vec()), - ScalarRefImpl::Interval(_) => return Err(data_type_not_supported("Interval")), - ScalarRefImpl::Jsonb(_) => return Err(data_type_not_supported("Jsonb")), - ScalarRefImpl::Struct(_) => return Err(data_type_not_supported("Struct")), - ScalarRefImpl::List(_) => return Err(data_type_not_supported("List")), - ScalarRefImpl::Int256(_) => return Err(data_type_not_supported("Int256")), - ScalarRefImpl::Serial(_) => return Err(data_type_not_supported("Serial")), - ScalarRefImpl::Map(_) => return Err(data_type_not_supported("Map")), - }, - None => match schema[col_idx].data_type { - DataType::Boolean => { - query.bind(None as Option); - } - DataType::Int16 => { - query.bind(None as Option); - } - DataType::Int32 => { - query.bind(None as Option); - } - DataType::Int64 => { - query.bind(None as Option); - } - DataType::Float32 => { - query.bind(None as Option); - } - DataType::Float64 => { - query.bind(None as Option); - } - DataType::Decimal => { - query.bind(None as Option); - } - DataType::Date => { - query.bind(None as Option); - } - DataType::Time => { - query.bind(None as Option); - } - DataType::Timestamp => { - query.bind(None as Option); - } - DataType::Timestamptz => { - query.bind(None as Option); - } - DataType::Varchar => { - query.bind(None as Option); - } - DataType::Bytea => { - query.bind(None as Option>); - } - DataType::Interval => return Err(data_type_not_supported("Interval")), - DataType::Struct(_) => return Err(data_type_not_supported("Struct")), - DataType::List(_) => return Err(data_type_not_supported("List")), - DataType::Jsonb => return Err(data_type_not_supported("Jsonb")), - DataType::Serial => return Err(data_type_not_supported("Serial")), - DataType::Int256 => return Err(data_type_not_supported("Int256")), - DataType::Map(_) => return Err(data_type_not_supported("Map")), - }, - }; - } - Ok(()) -} - fn data_type_not_supported(data_type_name: &str) -> SinkError { SinkError::Postgres(anyhow!(format!( "{data_type_name} is not supported in SQL Server" @@ -645,18 +294,3 @@ fn check_data_type_compatibility(data_type: &DataType) -> Result<()> { DataType::Map(_) => Err(data_type_not_supported("Map")), } } - -/// The implementation is copied from tiberius crate. -fn decimal_to_sql(decimal: &rust_decimal::Decimal) -> Numeric { - let unpacked = decimal.unpack(); - - let mut value = (((unpacked.hi as u128) << 64) - + ((unpacked.mid as u128) << 32) - + unpacked.lo as u128) as i128; - - if decimal.is_sign_negative() { - value = -value; - } - - Numeric::new_with_scale(value, decimal.scale() as u8) -} From e0d174dca4fe93e7bcd4459f3ecd318d40fc3aa5 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 14 Nov 2024 20:03:49 +0800 Subject: [PATCH 06/56] init postgres client --- src/connector/src/sink/postgres.rs | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index e410fed4a7f4..79e8b6f0f9d6 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -256,11 +256,26 @@ impl SinkWriter for PostgresSinkWriter { } #[derive(Debug)] -pub struct PostgresClient {} +pub struct PostgresClient { + client: tokio_postgres::Client, +} impl PostgresClient { async fn new(pg_config: &PostgresConfig) -> Result { - todo!() + let connection_string = format!( + "host={} port={} user={} password={} dbname={}", + pg_config.host, pg_config.port, pg_config.user, pg_config.password, pg_config.database + ); + let (client, connection) = + tokio_postgres::connect(&connection_string, tokio_postgres::NoTls) + .await + .context("Failed to connect to Postgres for Sinking")?; + tokio::spawn(async move { + if let Err(e) = connection.await { + tracing::error!("connection error: {}", e); + } + }); + Ok(Self { client }) } } From 6a398b48d2a044c4cb692bc7b21a297d41d4c708 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 15 Nov 2024 12:06:32 +0800 Subject: [PATCH 07/56] handle flush --- src/connector/src/sink/mod.rs | 6 + src/connector/src/sink/postgres.rs | 171 ++++++++++++++++------------- 2 files changed, 103 insertions(+), 74 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 047e9be4998a..76dd4d4a9eac 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -960,3 +960,9 @@ impl From<::opensearch::Error> for SinkError { SinkError::ElasticSearchOpenSearch(anyhow!(err)) } } + +impl From for SinkError { + fn from(err: tokio_postgres::Error) -> Self { + SinkError::Postgres(anyhow!(err)) + } +} diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 79e8b6f0f9d6..bf9c63bf887b 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -12,20 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cell::Ref; use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; use anyhow::{anyhow, Context}; use async_trait::async_trait; +use itertools::Itertools; use risingwave_common::array::{Op, RowRef, StreamChunk}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::Schema; use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::{DataType, Decimal}; +use risingwave_common::types::{DataType, Datum, Decimal}; use serde_derive::Deserialize; use serde_with::{serde_as, DisplayFromStr}; use simd_json::prelude::ArrayTrait; use tokio::net::TcpStream; +use tokio_postgres::types::ToSql; use tokio_util::compat::TokioAsyncWriteCompatExt; use with_options::WithOptions; @@ -146,10 +149,29 @@ impl Sink for PostgresSink { } } -enum SqlOp { - Insert(OwnedRow), - Merge(OwnedRow), - Delete(OwnedRow), +struct Buffer { + buffer: Vec, + size: usize, +} + +impl Buffer { + fn new() -> Self { + Self { + buffer: Vec::new(), + size: 0, + } + } + + fn push(&mut self, chunk: StreamChunk) -> usize { + self.size += chunk.cardinality(); + self.buffer.push(chunk); + self.size + } + + fn drain(&mut self) -> Vec { + self.size = 0; + std::mem::take(&mut self.buffer) + } } pub struct PostgresSinkWriter { @@ -157,8 +179,8 @@ pub struct PostgresSinkWriter { schema: Schema, pk_indices: Vec, is_append_only: bool, - sql_client: PostgresClient, - ops: Vec, + client: tokio_postgres::Client, + buffer: Buffer, } impl PostgresSinkWriter { @@ -168,43 +190,84 @@ impl PostgresSinkWriter { pk_indices: Vec, is_append_only: bool, ) -> Result { - let sql_client = PostgresClient::new(&config).await?; + let client = { + let connection_string = format!( + "host={} port={} user={} password={} dbname={}", + config.host, config.port, config.user, config.password, config.database + ); + let (client, connection) = + tokio_postgres::connect(&connection_string, tokio_postgres::NoTls) + .await + .context("Failed to connect to Postgres for Sinking")?; + tokio::spawn(async move { + if let Err(e) = connection.await { + tracing::error!("connection error: {}", e); + } + }); + client + }; + let writer = Self { config, schema, pk_indices, is_append_only, - sql_client, - ops: vec![], + client, + buffer: Buffer::new(), }; Ok(writer) } - async fn delete_one(&mut self, row: RowRef<'_>) -> Result<()> { - if self.ops.len() + 1 >= self.config.max_batch_rows { - self.flush().await?; - } - self.ops.push(SqlOp::Delete(row.into_owned_row())); - Ok(()) - } - - async fn upsert_one(&mut self, row: RowRef<'_>) -> Result<()> { - if self.ops.len() + 1 >= self.config.max_batch_rows { - self.flush().await?; - } - self.ops.push(SqlOp::Merge(row.into_owned_row())); - Ok(()) - } - - async fn insert_one(&mut self, row: RowRef<'_>) -> Result<()> { - if self.ops.len() + 1 >= self.config.max_batch_rows { - self.flush().await?; + async fn flush(&mut self) -> Result<()> { + // let mut delete_sql = format!("DELETE FROM {} WHERE ", self.config.table); + // NOTE(kwannoel): Use merge rather than update. + // Downstream DB may just clean old record, so it can't be updated. + // let mut update_sql = format!("MERGE {} SET ", self.config.table); + + let table_name = &self.config.table; + let parameters: String = { + let param_len = self.schema.fields().len(); + (0..param_len) + .map(|i| { + // let data_type = field.data_type(); + // check_data_type_compatibility(data_type)?; + format!("${}", i + 1) + }) + .collect_vec() + .join(",") + }; + let insert_sql = format!("INSERT INTO {table_name} VALUES ({parameters})"); + let insert_statement = self + .client + .prepare(&insert_sql) // TODO(kwannoel): use prepare_typed instead + .await + .context("Failed to prepare insert statement")?; + for chunk in self.buffer.drain() { + for (op, row) in chunk.rows() { + match op { + Op::Insert => self.flush_row(row, &insert_statement).await?, + Op::UpdateInsert => {} + Op::Delete => {} + Op::UpdateDelete => {} + } + } } - self.ops.push(SqlOp::Insert(row.into_owned_row())); Ok(()) } - async fn flush(&mut self) -> Result<()> { + async fn flush_row( + &self, + row: RowRef<'_>, + statement: &tokio_postgres::Statement, + ) -> Result<()> { + let owned = row.into_owned_row(); + let params = owned + .as_inner() + .iter() + .map(|s| s as &(dyn ToSql + Sync)) + .collect_vec(); + let params_ref = ¶ms[..]; + self.client.execute(statement, params_ref).await?; Ok(()) } } @@ -216,25 +279,9 @@ impl SinkWriter for PostgresSinkWriter { } async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - for (op, row) in chunk.rows() { - match op { - Op::Insert => { - if self.is_append_only { - self.insert_one(row).await?; - } else { - self.upsert_one(row).await?; - } - } - Op::UpdateInsert => { - debug_assert!(!self.is_append_only); - self.upsert_one(row).await?; - } - Op::Delete => { - debug_assert!(!self.is_append_only); - self.delete_one(row).await?; - } - Op::UpdateDelete => {} - } + let cardinality = self.buffer.push(chunk); + if cardinality >= self.config.max_batch_rows { + self.flush().await?; } Ok(()) } @@ -255,30 +302,6 @@ impl SinkWriter for PostgresSinkWriter { } } -#[derive(Debug)] -pub struct PostgresClient { - client: tokio_postgres::Client, -} - -impl PostgresClient { - async fn new(pg_config: &PostgresConfig) -> Result { - let connection_string = format!( - "host={} port={} user={} password={} dbname={}", - pg_config.host, pg_config.port, pg_config.user, pg_config.password, pg_config.database - ); - let (client, connection) = - tokio_postgres::connect(&connection_string, tokio_postgres::NoTls) - .await - .context("Failed to connect to Postgres for Sinking")?; - tokio::spawn(async move { - if let Err(e) = connection.await { - tracing::error!("connection error: {}", e); - } - }); - Ok(Self { client }) - } -} - fn data_type_not_supported(data_type_name: &str) -> SinkError { SinkError::Postgres(anyhow!(format!( "{data_type_name} is not supported in SQL Server" From 890dcb1a4acd84b88e1be3d2f6893bce9fb669e1 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 15 Nov 2024 16:16:48 +0800 Subject: [PATCH 08/56] remove unnecessary serde --- src/connector/src/sink/postgres.rs | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index bf9c63bf887b..9919bb37045d 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -44,24 +44,14 @@ fn default_max_batch_rows() -> usize { 1024 } -#[serde_as] #[derive(Clone, Debug, Deserialize, WithOptions)] pub struct PostgresConfig { - #[serde(rename = "postgres.host")] pub host: String, - #[serde(rename = "postgres.port")] - #[serde_as(as = "DisplayFromStr")] pub port: u16, - #[serde(rename = "postgres.user")] pub user: String, - #[serde(rename = "postgres.password")] pub password: String, - #[serde(rename = "postgres.database")] pub database: String, - #[serde(rename = "postgres.table")] pub table: String, - #[serde(rename = "postgres.max_batch_rows", default = "default_max_batch_rows")] - #[serde_as(as = "DisplayFromStr")] pub max_batch_rows: usize, pub r#type: String, // accept "append-only" or "upsert" } From 5746e81096ed6eaa6f0e4f3d6f27beb96ac95297 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 15 Nov 2024 23:39:14 +0800 Subject: [PATCH 09/56] add password for risedev.yml:full profile --- risedev.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/risedev.yml b/risedev.yml index 8e3668dcb49c..5f7e43d1b75d 100644 --- a/risedev.yml +++ b/risedev.yml @@ -117,6 +117,7 @@ profile: - use: postgres port: 8432 user: postgres + password: postgres database: metadata - use: meta-node meta-backend: postgres From 5953687820a8a532bfd40f1d15798b4b0007c6e7 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 15 Nov 2024 23:39:43 +0800 Subject: [PATCH 10/56] convert numeric with_options for connector --- src/connector/src/sink/postgres.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 9919bb37045d..7e86443294d0 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -44,14 +44,17 @@ fn default_max_batch_rows() -> usize { 1024 } +#[serde_as] #[derive(Clone, Debug, Deserialize, WithOptions)] pub struct PostgresConfig { pub host: String, + #[serde_as(as = "DisplayFromStr")] pub port: u16, pub user: String, pub password: String, pub database: String, pub table: String, + #[serde_as(as = "DisplayFromStr")] pub max_batch_rows: usize, pub r#type: String, // accept "append-only" or "upsert" } From 0ebd17b9a0032e73fc4e8e8c7753b82650b9a995 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 15 Nov 2024 23:39:49 +0800 Subject: [PATCH 11/56] add test --- e2e_test/sink/postgres_sink.slt | 81 +++++++++++++++++++++++++++++++++ 1 file changed, 81 insertions(+) create mode 100644 e2e_test/sink/postgres_sink.slt diff --git a/e2e_test/sink/postgres_sink.slt b/e2e_test/sink/postgres_sink.slt new file mode 100644 index 000000000000..b66f0a13da11 --- /dev/null +++ b/e2e_test/sink/postgres_sink.slt @@ -0,0 +1,81 @@ +control substitution on + +system ok +PGDATABASE=sink_test createdb + +system ok +PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +)" + +statement ok +CREATE TABLE rw_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +); + +statement ok +CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + max_batch_rows=1024, + type='upsert', + primary_key='id', +); + +statement ok +INSERT INTO rw_types_table (id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column) VALUES + (1, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}'), + (2, 'Varcharvalue2', 'Textvalue2', 234, 567, 890, 'NAN'::decimal, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}'), + (3, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, '+INF'::decimal, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}'); + +statement ok +flush; + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by id;'); +---- +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +2 Varcharvalue2 Textvalue2 234 567 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} +3 Varcharvalue1 Textvalue1 123 456 789 Infinity 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} + +statement ok +DROP SINK postgres_rw_types_sink; + +statement ok +DROP TABLE rw_types_table; + +system ok +PGDATABASE=postgres psql -c "DROP DATABASE sink_test" \ No newline at end of file From 582a8765acc0eea65d383a850d7ac331f27855f8 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 18 Nov 2024 21:12:08 +0800 Subject: [PATCH 12/56] handle deletes --- src/connector/src/sink/postgres.rs | 83 +++++++++++++++++++----------- 1 file changed, 54 insertions(+), 29 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 7e86443294d0..4eedf9d13e6a 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -22,7 +22,7 @@ use itertools::Itertools; use risingwave_common::array::{Op, RowRef, StreamChunk}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::Schema; -use risingwave_common::row::{OwnedRow, Row}; +use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::{DataType, Datum, Decimal}; use serde_derive::Deserialize; use serde_with::{serde_as, DisplayFromStr}; @@ -218,51 +218,76 @@ impl PostgresSinkWriter { // let mut update_sql = format!("MERGE {} SET ", self.config.table); let table_name = &self.config.table; - let parameters: String = { - let param_len = self.schema.fields().len(); - (0..param_len) + let pk_fields: Vec<_> = self + .pk_indices + .iter() + .map(|i| self.schema.fields()[*i].clone()) + .collect(); + + let insert_statement = { + let parameters: String = (0..self.schema.fields().len()) .map(|i| { // let data_type = field.data_type(); // check_data_type_compatibility(data_type)?; format!("${}", i + 1) }) .collect_vec() - .join(",") + .join(","); + let insert_sql = format!("INSERT INTO {table_name} VALUES ({parameters})"); + self.client + .prepare(&insert_sql) // TODO(kwannoel): use prepare_typed instead + .await + .context("Failed to prepare insert statement")? + }; + + let delete_statement = { + let parameters: String = pk_fields + .iter() + .enumerate() + .map(|(i, field)| { + let data_type = field.data_type(); + // check_data_type_compatibility(&data_type)?; + format!("{} = ${}", &field.name, i + 1) + }) + .collect_vec() + .join(" AND "); + let delete_sql = format!("DELETE FROM {table_name} WHERE {parameters}"); + self.client + .prepare(&delete_sql) // TODO: use prepare_typed instead + .await + .context("Failed to prepare delete statement")? }; - let insert_sql = format!("INSERT INTO {table_name} VALUES ({parameters})"); - let insert_statement = self - .client - .prepare(&insert_sql) // TODO(kwannoel): use prepare_typed instead - .await - .context("Failed to prepare insert statement")?; + for chunk in self.buffer.drain() { for (op, row) in chunk.rows() { match op { - Op::Insert => self.flush_row(row, &insert_statement).await?, + Op::Insert => { + let owned = row.into_owned_row(); + let params = owned + .as_inner() + .iter() + .map(|s| s as &(dyn ToSql + Sync)) + .collect_vec(); + let params_ref = ¶ms[..]; + self.client.execute(&insert_statement, params_ref).await?; + } Op::UpdateInsert => {} Op::Delete => {} - Op::UpdateDelete => {} + Op::UpdateDelete => { + let owned = row.project(&self.pk_indices).into_owned_row(); + let params = owned + .as_inner() + .iter() + .map(|s| s as &(dyn ToSql + Sync)) + .collect_vec(); + let params_ref = ¶ms[..]; + self.client.execute(&delete_statement, params_ref).await?; + } } } } Ok(()) } - - async fn flush_row( - &self, - row: RowRef<'_>, - statement: &tokio_postgres::Statement, - ) -> Result<()> { - let owned = row.into_owned_row(); - let params = owned - .as_inner() - .iter() - .map(|s| s as &(dyn ToSql + Sync)) - .collect_vec(); - let params_ref = ¶ms[..]; - self.client.execute(statement, params_ref).await?; - Ok(()) - } } #[async_trait] From 5c6c6fd3b87369327efae2ecb37acb6f5eaee05a Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 11:08:00 +0800 Subject: [PATCH 13/56] handle update --- src/connector/src/sink/postgres.rs | 63 ++++++++++++++++++++++++++++-- 1 file changed, 60 insertions(+), 3 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 4eedf9d13e6a..7d2113b144db 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -258,6 +258,54 @@ impl PostgresSinkWriter { .context("Failed to prepare delete statement")? }; + let merge_statement = { + let named_parameters: String = self + .schema + .fields + .iter() + .enumerate() + .map(|(i, field)| { + // let data_type = field.data_type(); + // check_data_type_compatibility(&data_type)?; + format!("${} as {}", i + 1, &field.name) + }) + .collect_vec() + .join(","); + let conditions: String = pk_fields + .iter() + .map(|pk_field| format!("source.{} = target.{}", pk_field.name, pk_field.name)) + .collect_vec() + .join(" AND "); + let update_vars: String = self + .schema + .fields + .iter() + .enumerate() + .map(|(i, field)| format!("target.{} = source.{}", field.name, field.name)) + .collect_vec() + .join(","); + let insert_vars: String = self + .schema + .fields + .iter() + .map(|field| format!("source.{}", field.name)) + .collect_vec() + .join(","); + let merge_sql = format!( + " + MERGE INTO {table_name} target + USING (SELECT {named_parameters} FROM {table_name}) AS source + ON ({conditions}) + WHEN MATCHED THEN UPDATE SET ({update_vars}) + WHEN NOT MATCHED THEN INSERT VALUES ({insert_vars}) + " + ); + self.client + .prepare(&merge_sql) // TODO: use prepare_typed instead + .await + .context("Failed to prepare merge statement")? + }; + for chunk in self.buffer.drain() { for (op, row) in chunk.rows() { match op { @@ -271,9 +319,17 @@ impl PostgresSinkWriter { let params_ref = ¶ms[..]; self.client.execute(&insert_statement, params_ref).await?; } - Op::UpdateInsert => {} - Op::Delete => {} - Op::UpdateDelete => { + Op::UpdateInsert => { + let owned = row.into_owned_row(); + let params = owned + .as_inner() + .iter() + .map(|s| s as &(dyn ToSql + Sync)) + .collect_vec(); + let params_ref = ¶ms[..]; + self.client.execute(&merge_statement, params_ref).await?; + } + Op::Delete => { let owned = row.project(&self.pk_indices).into_owned_row(); let params = owned .as_inner() @@ -283,6 +339,7 @@ impl PostgresSinkWriter { let params_ref = ¶ms[..]; self.client.execute(&delete_statement, params_ref).await?; } + Op::UpdateDelete => {} } } } From e2c6e775c5940cf64ae2882cc5d4f804be8585dd Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 16:38:42 +0800 Subject: [PATCH 14/56] prepared typed --- src/common/src/types/postgres_type.rs | 7 ++++ src/connector/src/sink/postgres.rs | 54 +++++++++++++++++++++------ 2 files changed, 50 insertions(+), 11 deletions(-) diff --git a/src/common/src/types/postgres_type.rs b/src/common/src/types/postgres_type.rs index c84f3e19f309..32fd5a3f944d 100644 --- a/src/common/src/types/postgres_type.rs +++ b/src/common/src/types/postgres_type.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use postgres_types::Type as PgType; + use super::DataType; /// `DataType` information extracted from PostgreSQL `pg_type` @@ -149,4 +151,9 @@ impl DataType { } for_all_base_types! { impl_pg_name } } + + pub fn to_pg_type(&self) -> PgType { + let oid = self.to_oid(); + PgType::from_oid(oid as u32).unwrap() + } } diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 7d2113b144db..e457914cf398 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -225,6 +225,13 @@ impl PostgresSinkWriter { .collect(); let insert_statement = { + let insert_types = self + .schema + .fields() + .iter() + .map(|field| field.data_type().to_pg_type()) + .collect_vec(); + let parameters: String = (0..self.schema.fields().len()) .map(|i| { // let data_type = field.data_type(); @@ -234,13 +241,20 @@ impl PostgresSinkWriter { .collect_vec() .join(","); let insert_sql = format!("INSERT INTO {table_name} VALUES ({parameters})"); - self.client - .prepare(&insert_sql) // TODO(kwannoel): use prepare_typed instead + let insert_statement = self + .client + .prepare_typed(&insert_sql, &insert_types) // TODO(kwannoel): use prepare_typed instead .await - .context("Failed to prepare insert statement")? + .context("Failed to prepare insert statement")?; + insert_statement }; let delete_statement = { + let delete_types = self + .pk_indices + .iter() + .map(|i| self.schema.fields()[*i].data_type().to_pg_type()) + .collect_vec(); let parameters: String = pk_fields .iter() .enumerate() @@ -252,13 +266,22 @@ impl PostgresSinkWriter { .collect_vec() .join(" AND "); let delete_sql = format!("DELETE FROM {table_name} WHERE {parameters}"); - self.client - .prepare(&delete_sql) // TODO: use prepare_typed instead + let delete_statement = self + .client + .prepare_typed(&delete_sql, &delete_types) // TODO: use prepare_typed instead .await - .context("Failed to prepare delete statement")? + .context("Failed to prepare delete statement")?; + delete_statement }; let merge_statement = { + let merge_types = self + .schema + .fields + .iter() + .map(|field| field.data_type().to_pg_type()) + .collect_vec(); + let named_parameters: String = self .schema .fields @@ -281,7 +304,14 @@ impl PostgresSinkWriter { .fields .iter() .enumerate() - .map(|(i, field)| format!("target.{} = source.{}", field.name, field.name)) + .map(|(i, field)| format!("{} = source.{}", field.name, field.name)) + .collect_vec() + .join(","); + let insert_columns: String = self + .schema + .fields + .iter() + .map(|field| field.name.clone()) .collect_vec() .join(","); let insert_vars: String = self @@ -294,14 +324,16 @@ impl PostgresSinkWriter { let merge_sql = format!( " MERGE INTO {table_name} target - USING (SELECT {named_parameters} FROM {table_name}) AS source + USING (SELECT {named_parameters}) AS source ON ({conditions}) - WHEN MATCHED THEN UPDATE SET ({update_vars}) - WHEN NOT MATCHED THEN INSERT VALUES ({insert_vars}) + WHEN MATCHED + THEN UPDATE SET {update_vars} + WHEN NOT MATCHED + THEN INSERT ({insert_columns}) VALUES ({insert_vars}) " ); self.client - .prepare(&merge_sql) // TODO: use prepare_typed instead + .prepare_typed(&merge_sql, &merge_types) // TODO: use prepare_typed instead .await .context("Failed to prepare merge statement")? }; From bcd2b70d0978d8007b6bdb6058ef021ef0a3d297 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 16:39:04 +0800 Subject: [PATCH 15/56] add test --- e2e_test/sink/postgres_sink.slt | 61 +++++++++++++++++++++++++++++++++ 1 file changed, 61 insertions(+) diff --git a/e2e_test/sink/postgres_sink.slt b/e2e_test/sink/postgres_sink.slt index b66f0a13da11..51a96ff37139 100644 --- a/e2e_test/sink/postgres_sink.slt +++ b/e2e_test/sink/postgres_sink.slt @@ -1,5 +1,11 @@ control substitution on +system ok +PGDATABASE=postgres psql -c "DROP DATABASE IF EXISTS sink_test WITH (FORCE)" + +statement ok +drop table if exists rw_types_table cascade; + system ok PGDATABASE=sink_test createdb @@ -55,6 +61,8 @@ CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( primary_key='id', ); +################### test insert + statement ok INSERT INTO rw_types_table (id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column) VALUES (1, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}'), @@ -71,6 +79,59 @@ select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:post 2 Varcharvalue2 Textvalue2 234 567 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} 3 Varcharvalue1 Textvalue1 123 456 789 Infinity 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +################### test upsert (update) + +statement ok +UPDATE rw_types_table SET varchar_column = 'Varcharvalue3', smallint_column = '300' WHERE id = 3; + +statement ok +flush; + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by id;'); +---- +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +2 Varcharvalue2 Textvalue2 234 567 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} +3 Varcharvalue3 Textvalue1 123 300 789 Infinity 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} + +################### test delete + +statement ok +DELETE FROM rw_types_table WHERE id = 3; + +statement ok +flush; + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by id;'); +---- +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +2 Varcharvalue2 Textvalue2 234 567 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} + +################### test upsert (insert) + +system ok +PGDATABASE=sink_test psql -c "DELETE FROM pg_types_table WHERE id = 2" + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by id;'); +---- +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} + +statement ok +UPDATE rw_types_table SET varchar_column = 'Varcharvalue4', smallint_column = '400' WHERE id = 2; + +statement ok +flush; + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by id;'); +---- +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +2 Varcharvalue4 Textvalue2 234 400 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} + +################### cleanup + statement ok DROP SINK postgres_rw_types_sink; From 0f8bbb0a59fc0d8cdebd04dceffa053dc22bcf4b Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 16:49:27 +0800 Subject: [PATCH 16/56] add to_sql for ScalarRefImpl --- src/common/src/types/jsonb.rs | 19 ++++++++++++++++ src/common/src/types/to_sql.rs | 41 ++++++++++++++++++++++++++++++++++ 2 files changed, 60 insertions(+) diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 4b25741fbe96..2dbc65ec481a 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -619,3 +619,22 @@ impl<'a> FromSql<'a> for JsonbVal { matches!(*ty, Type::JSONB) } } + +impl ToSql for JsonbRef<'_> { + accepts!(JSONB); + + to_sql_checked!(); + + fn to_sql( + &self, + _ty: &Type, + out: &mut BytesMut, + ) -> Result> + where + Self: Sized, + { + out.put_u8(1); + write!(out, "{}", self.0).unwrap(); + Ok(IsNull::No) + } +} diff --git a/src/common/src/types/to_sql.rs b/src/common/src/types/to_sql.rs index 57aab11daf4d..de9441d8aaf3 100644 --- a/src/common/src/types/to_sql.rs +++ b/src/common/src/types/to_sql.rs @@ -16,6 +16,7 @@ use std::error::Error; use bytes::BytesMut; use postgres_types::{to_sql_checked, IsNull, ToSql, Type}; +use risingwave_common::types::ScalarRefImpl; use crate::types::ScalarImpl; @@ -58,3 +59,43 @@ impl ToSql for ScalarImpl { true } } + +impl ToSql for ScalarRefImpl<'_> { + to_sql_checked!(); + + fn to_sql(&self, ty: &Type, out: &mut BytesMut) -> Result> + where + Self: Sized, + { + match self { + ScalarRefImpl::Int16(v) => v.to_sql(ty, out), + ScalarRefImpl::Int32(v) => v.to_sql(ty, out), + ScalarRefImpl::Int64(v) => v.to_sql(ty, out), + ScalarRefImpl::Serial(v) => v.to_sql(ty, out), + ScalarRefImpl::Float32(v) => v.to_sql(ty, out), + ScalarRefImpl::Float64(v) => v.to_sql(ty, out), + ScalarRefImpl::Utf8(v) => v.to_sql(ty, out), + ScalarRefImpl::Bool(v) => v.to_sql(ty, out), + ScalarRefImpl::Decimal(v) => v.to_sql(ty, out), + ScalarRefImpl::Interval(v) => v.to_sql(ty, out), + ScalarRefImpl::Date(v) => v.to_sql(ty, out), + ScalarRefImpl::Timestamp(v) => v.to_sql(ty, out), + ScalarRefImpl::Timestamptz(v) => v.to_sql(ty, out), + ScalarRefImpl::Time(v) => v.to_sql(ty, out), + ScalarRefImpl::Bytea(v) => (&**v).to_sql(ty, out), + ScalarRefImpl::Jsonb(v) => v.to_sql(ty, out), + ScalarRefImpl::Int256(_) | ScalarRefImpl::Struct(_) | ScalarRefImpl::List(_) => { + bail_not_implemented!("the postgres encoding for {ty} is unsupported") + } + ScalarRefImpl::Map(_) => todo!(), + } + } + + // return true to accept all types + fn accepts(_ty: &Type) -> bool + where + Self: Sized, + { + true + } +} From 81fdcca2562fcad7bd97daacab342894a5069e80 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 16:49:43 +0800 Subject: [PATCH 17/56] separate sql string formatters into separate functions --- src/connector/src/sink/postgres.rs | 74 ++++++++++++++++++++++++++++++ 1 file changed, 74 insertions(+) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index e457914cf398..60cfc0eb0005 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -439,3 +439,77 @@ fn check_data_type_compatibility(data_type: &DataType) -> Result<()> { DataType::Map(_) => Err(data_type_not_supported("Map")), } } + +fn create_insert_sql(schema: &Schema, table_name: &str) -> String { + let columns: String = schema + .fields() + .iter() + .map(|field| field.name.clone()) + .collect_vec() + .join(","); + let parameters: String = (0..schema.fields().len()) + .map(|i| format!("${}", i + 1)) + .collect_vec() + .join(","); + format!("INSERT INTO {table_name} ({columns}) VALUES ({parameters})") +} + +fn create_merge_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> String { + let named_parameters: String = schema + .fields() + .iter() + .enumerate() + .map(|(i, field)| format!("${} as {}", i + 1, &field.name)) + .collect_vec() + .join(","); + let conditions: String = pk_indices + .iter() + .map(|i| { + format!( + "source.{} = target.{}", + schema.fields()[*i].name, + schema.fields()[*i].name + ) + }) + .collect_vec() + .join(" AND "); + let update_vars: String = schema + .fields() + .iter() + .enumerate() + .map(|(i, field)| format!("{} = source.{}", field.name, field.name)) + .collect_vec() + .join(","); + let insert_columns: String = schema + .fields() + .iter() + .map(|field| field.name.clone()) + .collect_vec() + .join(","); + let insert_vars: String = schema + .fields() + .iter() + .map(|field| format!("source.{}", field.name)) + .collect_vec() + .join(","); + format!( + " + MERGE INTO {table_name} target + USING (SELECT {named_parameters}) AS source + ON ({conditions}) + WHEN MATCHED + THEN UPDATE SET {update_vars} + WHEN NOT MATCHED + THEN INSERT ({insert_columns}) VALUES ({insert_vars}) + " + ) +} + +fn create_delete_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> String { + let parameters: String = pk_indices + .iter() + .map(|i| format!("{} = ${}", schema.fields()[*i].name, i + 1)) + .collect_vec() + .join(" AND "); + format!("DELETE FROM {table_name} WHERE {parameters}") +} From e679b06396cf3e6f9c2514ba0b880fbf74f12af3 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 18:03:54 +0800 Subject: [PATCH 18/56] add tests --- src/connector/src/sink/postgres.rs | 100 +++++++++++++++++++++++++++-- 1 file changed, 94 insertions(+), 6 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 60cfc0eb0005..47e04d5d7e83 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -127,6 +127,8 @@ impl Sink for PostgresSink { "Primary key not defined for upsert Postgres sink (please define in `primary_key` field)"))); } + // TODO(kwannoel): Add more validation - see sqlserver. Check type compatibility, etc. + Ok(()) } @@ -446,11 +448,11 @@ fn create_insert_sql(schema: &Schema, table_name: &str) -> String { .iter() .map(|field| field.name.clone()) .collect_vec() - .join(","); + .join(", "); let parameters: String = (0..schema.fields().len()) .map(|i| format!("${}", i + 1)) .collect_vec() - .join(","); + .join(", "); format!("INSERT INTO {table_name} ({columns}) VALUES ({parameters})") } @@ -477,21 +479,22 @@ fn create_merge_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> .fields() .iter() .enumerate() - .map(|(i, field)| format!("{} = source.{}", field.name, field.name)) + .filter(|(i, _)| !pk_indices.contains(i)) + .map(|(_, field)| format!("{} = source.{}", field.name, field.name)) .collect_vec() - .join(","); + .join(", "); let insert_columns: String = schema .fields() .iter() .map(|field| field.name.clone()) .collect_vec() - .join(","); + .join(", "); let insert_vars: String = schema .fields() .iter() .map(|field| format!("source.{}", field.name)) .collect_vec() - .join(","); + .join(", "); format!( " MERGE INTO {table_name} target @@ -513,3 +516,88 @@ fn create_delete_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> .join(" AND "); format!("DELETE FROM {table_name} WHERE {parameters}") } + +#[cfg(test)] +mod tests { + use std::fmt::Display; + use super::*; + use expect_test::{expect, Expect}; + use risingwave_common::catalog::Field; + + fn check(actual: impl Display, expect: Expect) { + let actual = format!("{}", actual); + expect.assert_eq(&actual); + } + + #[test] + fn test_create_insert_sql() { + let schema = Schema::new(vec![ + Field { + data_type: DataType::Int32, + name: "a".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }, + Field { + data_type: DataType::Int32, + name: "b".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + } + ]); + let table_name = "test_table"; + let sql = create_insert_sql(&schema, table_name); + check(sql, expect!["INSERT INTO test_table (a, b) VALUES ($1, $2)"]); + } + + #[test] + fn test_create_delete_sql() { + let schema = Schema::new(vec![ + Field { + data_type: DataType::Int32, + name: "a".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }, + Field { + data_type: DataType::Int32, + name: "b".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + } + ]); + let table_name = "test_table"; + let sql = create_delete_sql(&schema, table_name, &[1]); + check(sql, expect!["DELETE FROM test_table WHERE b = $2"]); + } + + #[test] + fn test_create_merge_sql() { + let schema = Schema::new(vec![ + Field { + data_type: DataType::Int32, + name: "a".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }, + Field { + data_type: DataType::Int32, + name: "b".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + } + ]); + let table_name = "test_table"; + let sql = create_merge_sql(&schema, table_name, &[1]); + check(sql, expect![[r#" + + MERGE INTO test_table target + USING (SELECT $1 as a,$2 as b) AS source + ON (source.b = target.b) + WHEN MATCHED + THEN UPDATE SET a = source.a + WHEN NOT MATCHED + THEN INSERT (a, b) VALUES (source.a, source.b) + "#]]); + } +} \ No newline at end of file From e4cfab70e14d1d2366ef6901f19877fc2f5be579 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 18:11:44 +0800 Subject: [PATCH 19/56] use insert, delete, merge formatters --- src/connector/src/sink/postgres.rs | 203 ++++++++++------------------- 1 file changed, 70 insertions(+), 133 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 47e04d5d7e83..4e2b5f546cc1 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -29,6 +29,7 @@ use serde_with::{serde_as, DisplayFromStr}; use simd_json::prelude::ArrayTrait; use tokio::net::TcpStream; use tokio_postgres::types::ToSql; +use tokio_postgres::Statement; use tokio_util::compat::TokioAsyncWriteCompatExt; use with_options::WithOptions; @@ -176,6 +177,9 @@ pub struct PostgresSinkWriter { is_append_only: bool, client: tokio_postgres::Client, buffer: Buffer, + insert_statement: Statement, + delete_statement: Statement, + merge_statement: Statement, } impl PostgresSinkWriter { @@ -202,144 +206,59 @@ impl PostgresSinkWriter { client }; - let writer = Self { - config, - schema, - pk_indices, - is_append_only, - client, - buffer: Buffer::new(), - }; - Ok(writer) - } - - async fn flush(&mut self) -> Result<()> { - // let mut delete_sql = format!("DELETE FROM {} WHERE ", self.config.table); - // NOTE(kwannoel): Use merge rather than update. - // Downstream DB may just clean old record, so it can't be updated. - // let mut update_sql = format!("MERGE {} SET ", self.config.table); - - let table_name = &self.config.table; - let pk_fields: Vec<_> = self - .pk_indices - .iter() - .map(|i| self.schema.fields()[*i].clone()) - .collect(); - let insert_statement = { - let insert_types = self - .schema + let insert_types = schema .fields() .iter() .map(|field| field.data_type().to_pg_type()) .collect_vec(); - - let parameters: String = (0..self.schema.fields().len()) - .map(|i| { - // let data_type = field.data_type(); - // check_data_type_compatibility(data_type)?; - format!("${}", i + 1) - }) - .collect_vec() - .join(","); - let insert_sql = format!("INSERT INTO {table_name} VALUES ({parameters})"); - let insert_statement = self - .client - .prepare_typed(&insert_sql, &insert_types) // TODO(kwannoel): use prepare_typed instead + let insert_sql = create_insert_sql(&schema, &config.table); + client + .prepare_typed(&insert_sql, &insert_types) .await - .context("Failed to prepare insert statement")?; - insert_statement + .context("Failed to prepare insert statement")? }; let delete_statement = { - let delete_types = self - .pk_indices + let delete_types = pk_indices .iter() - .map(|i| self.schema.fields()[*i].data_type().to_pg_type()) + .map(|i| schema.fields()[*i].data_type().to_pg_type()) .collect_vec(); - let parameters: String = pk_fields - .iter() - .enumerate() - .map(|(i, field)| { - let data_type = field.data_type(); - // check_data_type_compatibility(&data_type)?; - format!("{} = ${}", &field.name, i + 1) - }) - .collect_vec() - .join(" AND "); - let delete_sql = format!("DELETE FROM {table_name} WHERE {parameters}"); - let delete_statement = self - .client - .prepare_typed(&delete_sql, &delete_types) // TODO: use prepare_typed instead + let delete_sql = create_delete_sql(&schema, &config.table, &pk_indices); + client + .prepare_typed(&delete_sql, &delete_types) .await - .context("Failed to prepare delete statement")?; - delete_statement + .context("Failed to prepare delete statement")? }; let merge_statement = { - let merge_types = self - .schema + let merge_types = schema .fields .iter() .map(|field| field.data_type().to_pg_type()) .collect_vec(); - - let named_parameters: String = self - .schema - .fields - .iter() - .enumerate() - .map(|(i, field)| { - // let data_type = field.data_type(); - // check_data_type_compatibility(&data_type)?; - format!("${} as {}", i + 1, &field.name) - }) - .collect_vec() - .join(","); - let conditions: String = pk_fields - .iter() - .map(|pk_field| format!("source.{} = target.{}", pk_field.name, pk_field.name)) - .collect_vec() - .join(" AND "); - let update_vars: String = self - .schema - .fields - .iter() - .enumerate() - .map(|(i, field)| format!("{} = source.{}", field.name, field.name)) - .collect_vec() - .join(","); - let insert_columns: String = self - .schema - .fields - .iter() - .map(|field| field.name.clone()) - .collect_vec() - .join(","); - let insert_vars: String = self - .schema - .fields - .iter() - .map(|field| format!("source.{}", field.name)) - .collect_vec() - .join(","); - let merge_sql = format!( - " - MERGE INTO {table_name} target - USING (SELECT {named_parameters}) AS source - ON ({conditions}) - WHEN MATCHED - THEN UPDATE SET {update_vars} - WHEN NOT MATCHED - THEN INSERT ({insert_columns}) VALUES ({insert_vars}) - " - ); - self.client - .prepare_typed(&merge_sql, &merge_types) // TODO: use prepare_typed instead + let merge_sql = create_merge_sql(&schema, &config.table, &pk_indices); + client + .prepare_typed(&merge_sql, &merge_types) .await .context("Failed to prepare merge statement")? }; + let writer = Self { + config, + schema, + pk_indices, + is_append_only, + client, + buffer: Buffer::new(), + insert_statement, + delete_statement, + merge_statement, + }; + Ok(writer) + } + + async fn flush(&mut self) -> Result<()> { for chunk in self.buffer.drain() { for (op, row) in chunk.rows() { match op { @@ -351,7 +270,9 @@ impl PostgresSinkWriter { .map(|s| s as &(dyn ToSql + Sync)) .collect_vec(); let params_ref = ¶ms[..]; - self.client.execute(&insert_statement, params_ref).await?; + self.client + .execute(&self.insert_statement, params_ref) + .await?; } Op::UpdateInsert => { let owned = row.into_owned_row(); @@ -361,7 +282,13 @@ impl PostgresSinkWriter { .map(|s| s as &(dyn ToSql + Sync)) .collect_vec(); let params_ref = ¶ms[..]; - self.client.execute(&merge_statement, params_ref).await?; + // NOTE(kwannoel): Here we use `MERGE` rather than `UPDATE/INSERT` directly. + // This is because the downstream db could have cleaned the old record, + // in that case it needs to be `INSERTED` rather than UPDATED. + // On the other hand, if the record is there, it should be `UPDATED`. + self.client + .execute(&self.merge_statement, params_ref) + .await?; } Op::Delete => { let owned = row.project(&self.pk_indices).into_owned_row(); @@ -371,7 +298,9 @@ impl PostgresSinkWriter { .map(|s| s as &(dyn ToSql + Sync)) .collect_vec(); let params_ref = ¶ms[..]; - self.client.execute(&delete_statement, params_ref).await?; + self.client + .execute(&self.delete_statement, params_ref) + .await?; } Op::UpdateDelete => {} } @@ -520,10 +449,12 @@ fn create_delete_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> #[cfg(test)] mod tests { use std::fmt::Display; - use super::*; + use expect_test::{expect, Expect}; use risingwave_common::catalog::Field; + use super::*; + fn check(actual: impl Display, expect: Expect) { let actual = format!("{}", actual); expect.assert_eq(&actual); @@ -532,22 +463,25 @@ mod tests { #[test] fn test_create_insert_sql() { let schema = Schema::new(vec![ - Field { + Field { data_type: DataType::Int32, name: "a".to_string(), sub_fields: vec![], type_name: "".to_string(), - }, - Field { - data_type: DataType::Int32, - name: "b".to_string(), - sub_fields: vec![], - type_name: "".to_string(), - } + }, + Field { + data_type: DataType::Int32, + name: "b".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }, ]); let table_name = "test_table"; let sql = create_insert_sql(&schema, table_name); - check(sql, expect!["INSERT INTO test_table (a, b) VALUES ($1, $2)"]); + check( + sql, + expect!["INSERT INTO test_table (a, b) VALUES ($1, $2)"], + ); } #[test] @@ -564,7 +498,7 @@ mod tests { name: "b".to_string(), sub_fields: vec![], type_name: "".to_string(), - } + }, ]); let table_name = "test_table"; let sql = create_delete_sql(&schema, table_name, &[1]); @@ -585,11 +519,13 @@ mod tests { name: "b".to_string(), sub_fields: vec![], type_name: "".to_string(), - } + }, ]); let table_name = "test_table"; let sql = create_merge_sql(&schema, table_name, &[1]); - check(sql, expect![[r#" + check( + sql, + expect![[r#" MERGE INTO test_table target USING (SELECT $1 as a,$2 as b) AS source @@ -598,6 +534,7 @@ mod tests { THEN UPDATE SET a = source.a WHEN NOT MATCHED THEN INSERT (a, b) VALUES (source.a, source.b) - "#]]); + "#]], + ); } -} \ No newline at end of file +} From e8e7bb51a2792ccd3cfbf90a74ecf27dd67a0b13 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 18:30:02 +0800 Subject: [PATCH 20/56] directly iterate on row --- src/connector/src/sink/postgres.rs | 30 ++++++------------------------ 1 file changed, 6 insertions(+), 24 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 4e2b5f546cc1..dfa56b3722e0 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -263,43 +263,25 @@ impl PostgresSinkWriter { for (op, row) in chunk.rows() { match op { Op::Insert => { - let owned = row.into_owned_row(); - let params = owned - .as_inner() - .iter() - .map(|s| s as &(dyn ToSql + Sync)) - .collect_vec(); - let params_ref = ¶ms[..]; self.client - .execute(&self.insert_statement, params_ref) + .execute_raw(&self.insert_statement, row.iter()) .await?; } Op::UpdateInsert => { - let owned = row.into_owned_row(); - let params = owned - .as_inner() - .iter() - .map(|s| s as &(dyn ToSql + Sync)) - .collect_vec(); - let params_ref = ¶ms[..]; // NOTE(kwannoel): Here we use `MERGE` rather than `UPDATE/INSERT` directly. // This is because the downstream db could have cleaned the old record, // in that case it needs to be `INSERTED` rather than UPDATED. // On the other hand, if the record is there, it should be `UPDATED`. self.client - .execute(&self.merge_statement, params_ref) + .execute_raw(&self.merge_statement, row.iter()) .await?; } Op::Delete => { - let owned = row.project(&self.pk_indices).into_owned_row(); - let params = owned - .as_inner() - .iter() - .map(|s| s as &(dyn ToSql + Sync)) - .collect_vec(); - let params_ref = ¶ms[..]; self.client - .execute(&self.delete_statement, params_ref) + .execute_raw( + &self.delete_statement, + row.project(&self.pk_indices).iter(), + ) .await?; } Op::UpdateDelete => {} From 0e33eb8d80c06d56e43c9368d380608a070bbec5 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 20:16:08 +0800 Subject: [PATCH 21/56] cf --- src/connector/src/sink/postgres.rs | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index dfa56b3722e0..742190c45aa1 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -12,25 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::cell::Ref; -use std::collections::{BTreeMap, HashMap}; +use std::collections::BTreeMap; use std::sync::Arc; use anyhow::{anyhow, Context}; use async_trait::async_trait; use itertools::Itertools; -use risingwave_common::array::{Op, RowRef, StreamChunk}; +use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::Schema; -use risingwave_common::row::{OwnedRow, Row, RowExt}; -use risingwave_common::types::{DataType, Datum, Decimal}; +use risingwave_common::row::{Row, RowExt}; +use risingwave_common::types::DataType; use serde_derive::Deserialize; use serde_with::{serde_as, DisplayFromStr}; use simd_json::prelude::ArrayTrait; -use tokio::net::TcpStream; -use tokio_postgres::types::ToSql; use tokio_postgres::Statement; -use tokio_util::compat::TokioAsyncWriteCompatExt; use with_options::WithOptions; use super::{ @@ -87,7 +83,7 @@ pub struct PostgresSink { impl PostgresSink { pub fn new( - mut config: PostgresConfig, + config: PostgresConfig, schema: Schema, pk_indices: Vec, is_append_only: bool, From aa94edd07bff1642ed5fc6dcd4d624cfa8e4d1bd Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 20:24:11 +0800 Subject: [PATCH 22/56] handle max_batch_rows --- src/connector/src/sink/postgres.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 742190c45aa1..29bd59db80d1 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -37,10 +37,6 @@ use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWrite pub const POSTGRES_SINK: &str = "postgres"; -fn default_max_batch_rows() -> usize { - 1024 -} - #[serde_as] #[derive(Clone, Debug, Deserialize, WithOptions)] pub struct PostgresConfig { @@ -51,11 +47,16 @@ pub struct PostgresConfig { pub password: String, pub database: String, pub table: String, + #[serde(default = "default_max_batch_rows")] #[serde_as(as = "DisplayFromStr")] pub max_batch_rows: usize, pub r#type: String, // accept "append-only" or "upsert" } +fn default_max_batch_rows() -> usize { + 1024 +} + impl PostgresConfig { pub fn from_btreemap(properties: BTreeMap) -> Result { let config = From 5e085771a635e303b8e91cabc594dbbe833c229e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 20:27:41 +0800 Subject: [PATCH 23/56] remove unused schema field --- src/connector/src/sink/postgres.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 29bd59db80d1..2632d27229e2 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -169,7 +169,6 @@ impl Buffer { pub struct PostgresSinkWriter { config: PostgresConfig, - schema: Schema, pk_indices: Vec, is_append_only: bool, client: tokio_postgres::Client, @@ -243,7 +242,6 @@ impl PostgresSinkWriter { let writer = Self { config, - schema, pk_indices, is_append_only, client, From 5ce0024b5c97307b78fd624555f7fd1aeade5d79 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 20:53:25 +0800 Subject: [PATCH 24/56] handle append-only --- src/connector/src/sink/postgres.rs | 66 +++++++++++++++++++----------- 1 file changed, 42 insertions(+), 24 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 2632d27229e2..694e156aecc2 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -254,35 +254,53 @@ impl PostgresSinkWriter { } async fn flush(&mut self) -> Result<()> { - for chunk in self.buffer.drain() { - for (op, row) in chunk.rows() { - match op { - Op::Insert => { - self.client - .execute_raw(&self.insert_statement, row.iter()) - .await?; + if self.is_append_only { + for chunk in self.buffer.drain() { + for (op, row) in chunk.rows() { + match op { + Op::Insert => { + self.client + .execute_raw(&self.insert_statement, row.iter()) + .await?; + } + Op::UpdateInsert | Op::Delete | Op::UpdateDelete => { + debug_assert!(!self.is_append_only); + } } - Op::UpdateInsert => { - // NOTE(kwannoel): Here we use `MERGE` rather than `UPDATE/INSERT` directly. - // This is because the downstream db could have cleaned the old record, - // in that case it needs to be `INSERTED` rather than UPDATED. - // On the other hand, if the record is there, it should be `UPDATED`. - self.client - .execute_raw(&self.merge_statement, row.iter()) - .await?; - } - Op::Delete => { - self.client - .execute_raw( - &self.delete_statement, - row.project(&self.pk_indices).iter(), - ) - .await?; + } + } + } else { + for chunk in self.buffer.drain() { + for (op, row) in chunk.rows() { + match op { + Op::Insert => { + self.client + .execute_raw(&self.insert_statement, row.iter()) + .await?; + } + Op::UpdateInsert => { + // NOTE(kwannoel): Here we use `MERGE` rather than `UPDATE/INSERT` directly. + // This is because the downstream db could have cleaned the old record, + // in that case it needs to be `INSERTED` rather than UPDATED. + // On the other hand, if the record is there, it should be `UPDATED`. + self.client + .execute_raw(&self.merge_statement, row.iter()) + .await?; + } + Op::Delete => { + self.client + .execute_raw( + &self.delete_statement, + row.project(&self.pk_indices).iter(), + ) + .await?; + } + Op::UpdateDelete => {} } - Op::UpdateDelete => {} } } } + Ok(()) } } From 0e4b2223cfb55c1af675403b24961c0050ffa31e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 21:40:44 +0800 Subject: [PATCH 25/56] test append-only + fix bug --- e2e_test/sink/postgres_sink.slt | 87 +++++++++++++++++++++++++++++- src/connector/src/sink/postgres.rs | 24 +++++---- 2 files changed, 99 insertions(+), 12 deletions(-) diff --git a/e2e_test/sink/postgres_sink.slt b/e2e_test/sink/postgres_sink.slt index 51a96ff37139..cc7a002e7ba4 100644 --- a/e2e_test/sink/postgres_sink.slt +++ b/e2e_test/sink/postgres_sink.slt @@ -6,6 +6,9 @@ PGDATABASE=postgres psql -c "DROP DATABASE IF EXISTS sink_test WITH (FORCE)" statement ok drop table if exists rw_types_table cascade; +statement ok +drop table if exists rw_types_table_append_only cascade; + system ok PGDATABASE=sink_test createdb @@ -56,7 +59,6 @@ CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( password='$PGPASSWORD', database='sink_test', table='pg_types_table', - max_batch_rows=1024, type='upsert', primary_key='id', ); @@ -130,7 +132,7 @@ select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:post 1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} 2 Varcharvalue4 Textvalue2 234 400 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} -################### cleanup +################### cleanup sink statement ok DROP SINK postgres_rw_types_sink; @@ -138,5 +140,86 @@ DROP SINK postgres_rw_types_sink; statement ok DROP TABLE rw_types_table; +system ok +PGDATABASE=sink_test psql -c "DROP TABLE pg_types_table" + +################### test append-only sink + +system ok +PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +)" + +statement ok +CREATE TABLE rw_types_table_append_only ( + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +) APPEND ONLY; + +statement ok +CREATE SINK postgres_rw_types_sink_append_only FROM rw_types_table_append_only WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='append-only', +); + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table;'); +---- + +statement ok +INSERT INTO rw_types_table_append_only (varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column) VALUES + ('Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}'), + ('Varcharvalue2', 'Textvalue2', 234, 567, 890, 'NAN'::decimal, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}'), + ('Varcharvalue3', 'Textvalue1', 333, 456, 789, '+INF'::decimal, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}'); + +statement ok +flush; + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by integer_column;'); +---- +Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +Varcharvalue2 Textvalue2 234 567 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} +Varcharvalue3 Textvalue1 333 456 789 Infinity 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} + +################### cleanup append-only sink + +statement ok +DROP SINK postgres_rw_types_sink_append_only; + +statement ok +DROP TABLE rw_types_table_append_only; + system ok PGDATABASE=postgres psql -c "DROP DATABASE sink_test" \ No newline at end of file diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 694e156aecc2..3aa6708848d9 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -174,8 +174,8 @@ pub struct PostgresSinkWriter { client: tokio_postgres::Client, buffer: Buffer, insert_statement: Statement, - delete_statement: Statement, - merge_statement: Statement, + delete_statement: Option, + merge_statement: Option, } impl PostgresSinkWriter { @@ -215,29 +215,33 @@ impl PostgresSinkWriter { .context("Failed to prepare insert statement")? }; - let delete_statement = { + let delete_statement = if is_append_only { + None + } else { let delete_types = pk_indices .iter() .map(|i| schema.fields()[*i].data_type().to_pg_type()) .collect_vec(); let delete_sql = create_delete_sql(&schema, &config.table, &pk_indices); - client + Some(client .prepare_typed(&delete_sql, &delete_types) .await - .context("Failed to prepare delete statement")? + .context("Failed to prepare delete statement")?) }; - let merge_statement = { + let merge_statement = if is_append_only { + None + } else { let merge_types = schema .fields .iter() .map(|field| field.data_type().to_pg_type()) .collect_vec(); let merge_sql = create_merge_sql(&schema, &config.table, &pk_indices); - client + Some(client .prepare_typed(&merge_sql, &merge_types) .await - .context("Failed to prepare merge statement")? + .context("Failed to prepare merge statement")?) }; let writer = Self { @@ -284,13 +288,13 @@ impl PostgresSinkWriter { // in that case it needs to be `INSERTED` rather than UPDATED. // On the other hand, if the record is there, it should be `UPDATED`. self.client - .execute_raw(&self.merge_statement, row.iter()) + .execute_raw(self.merge_statement.as_ref().unwrap(), row.iter()) .await?; } Op::Delete => { self.client .execute_raw( - &self.delete_statement, + self.delete_statement.as_ref().unwrap(), row.project(&self.pk_indices).iter(), ) .await?; From be52c6bf15f45829abb3a9c4c092f22785f54339 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 22:48:08 +0800 Subject: [PATCH 26/56] handle datatype validation --- src/connector/src/sink/postgres.rs | 96 ++++++++++++++++++++++++++---- 1 file changed, 84 insertions(+), 12 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 3aa6708848d9..b541744a13d1 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -125,7 +125,75 @@ impl Sink for PostgresSink { "Primary key not defined for upsert Postgres sink (please define in `primary_key` field)"))); } - // TODO(kwannoel): Add more validation - see sqlserver. Check type compatibility, etc. + for field in self.schema.fields() { + check_data_type_compatibility(field.data_type())?; + } + + // Verify pg table schema matches rw table schema, and pk indices are valid + let table_name = &self.config.table; + let connection_string = format!( + "host={} port={} user={} password={} dbname={}", + self.config.host, + self.config.port, + self.config.user, + self.config.password, + self.config.database + ); + let (client, connection) = + tokio_postgres::connect(&connection_string, tokio_postgres::NoTls) + .await + .context("Failed to connect to Postgres for Sinking")?; + tokio::spawn(async move { + if let Err(e) = connection.await { + tracing::error!("connection error: {}", e); + } + }); + + let result = client + .query( + " + SELECT a.attname as col_name, i.indisprimary AS is_pk + FROM pg_index i + JOIN pg_attribute a ON a.attrelid = i.indrelid + AND a.attnum = ANY(i.indkey) + WHERE i.indrelid = $1::regclass", + &[&table_name], + ) + .await + .context("Failed to query Postgres for Sinking")?; + + let mut pg_schema = BTreeMap::new(); + for row in result { + let col_name: String = row.get(0); + let is_pk: bool = row.get(1); + pg_schema.insert(col_name, is_pk); + } + + for (i, field) in self.schema.fields().iter().enumerate() { + let col_name = &field.name; + let is_pk = pg_schema.get(col_name); + match is_pk { + None => return Err(SinkError::Config(anyhow!( + "Column `{}` not found in Postgres table `{}`", + col_name, + table_name + ))), + Some(is_pk) => + match (*is_pk, self.pk_indices.contains(&i)) { + (false, false) | (true, true) => continue, + (false, true) => return Err(SinkError::Config(anyhow!( + "Column `{}` in Postgres table `{}` is not a primary key, but RW schema defines it as a primary key", + col_name, + table_name + ))), + (true, false) => return Err(SinkError::Config(anyhow!( + "Column `{}` in Postgres table `{}` is a primary key, but RW schema does not define it as a primary key", + col_name, + table_name + ))), + } + } + } Ok(()) } @@ -223,10 +291,12 @@ impl PostgresSinkWriter { .map(|i| schema.fields()[*i].data_type().to_pg_type()) .collect_vec(); let delete_sql = create_delete_sql(&schema, &config.table, &pk_indices); - Some(client - .prepare_typed(&delete_sql, &delete_types) - .await - .context("Failed to prepare delete statement")?) + Some( + client + .prepare_typed(&delete_sql, &delete_types) + .await + .context("Failed to prepare delete statement")?, + ) }; let merge_statement = if is_append_only { @@ -238,10 +308,12 @@ impl PostgresSinkWriter { .map(|field| field.data_type().to_pg_type()) .collect_vec(); let merge_sql = create_merge_sql(&schema, &config.table, &pk_indices); - Some(client - .prepare_typed(&merge_sql, &merge_types) - .await - .context("Failed to prepare merge statement")?) + Some( + client + .prepare_typed(&merge_sql, &merge_types) + .await + .context("Failed to prepare merge statement")?, + ) }; let writer = Self { @@ -345,7 +417,7 @@ fn data_type_not_supported(data_type_name: &str) -> SinkError { ))) } -fn check_data_type_compatibility(data_type: &DataType) -> Result<()> { +fn check_data_type_compatibility(data_type: DataType) -> Result<()> { match data_type { DataType::Boolean | DataType::Int16 @@ -359,11 +431,11 @@ fn check_data_type_compatibility(data_type: &DataType) -> Result<()> { | DataType::Time | DataType::Timestamp | DataType::Timestamptz + | DataType::Jsonb + | DataType::Interval | DataType::Bytea => Ok(()), - DataType::Interval => Err(data_type_not_supported("Interval")), DataType::Struct(_) => Err(data_type_not_supported("Struct")), DataType::List(_) => Err(data_type_not_supported("List")), - DataType::Jsonb => Err(data_type_not_supported("Jsonb")), DataType::Serial => Err(data_type_not_supported("Serial")), DataType::Int256 => Err(data_type_not_supported("Int256")), DataType::Map(_) => Err(data_type_not_supported("Map")), From 704a09ad5a36954bd5d196963a329485e9f232bd Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 23:12:28 +0800 Subject: [PATCH 27/56] fix bug in validate + increase timeout for risedev:postgres service --- src/connector/src/sink/postgres.rs | 25 +++++++++++++++---- .../src/task/task_log_ready_check.rs | 2 +- 2 files changed, 21 insertions(+), 6 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index b541744a13d1..f103fed63723 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -152,11 +152,26 @@ impl Sink for PostgresSink { let result = client .query( " - SELECT a.attname as col_name, i.indisprimary AS is_pk - FROM pg_index i - JOIN pg_attribute a ON a.attrelid = i.indrelid - AND a.attnum = ANY(i.indkey) - WHERE i.indrelid = $1::regclass", + SELECT + column_name, + EXISTS ( + SELECT 1 + FROM pg_index i + WHERE i.indrelid = c.table_name::regclass + AND i.indisprimary + AND column_name = ANY( + SELECT a.attname + FROM pg_attribute a + WHERE a.attrelid = i.indrelid + AND a.attnum = ANY(i.indkey) + ) + ) AS is_primary_key + FROM + information_schema.columns c + WHERE + table_name = $1 + ORDER BY + ordinal_position;", &[&table_name], ) .await diff --git a/src/risedevtool/src/task/task_log_ready_check.rs b/src/risedevtool/src/task/task_log_ready_check.rs index cc55c6142b47..ef27169abedb 100644 --- a/src/risedevtool/src/task/task_log_ready_check.rs +++ b/src/risedevtool/src/task/task_log_ready_check.rs @@ -84,7 +84,7 @@ where &mut self.log, self.status_file.as_ref().unwrap(), self.id.as_ref().unwrap(), - Some(Duration::from_secs(30)), + Some(Duration::from_secs(60)), true, )?; From c4e08e13e502cf76bd5f63d53a1757452f37b4ef Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 23:14:23 +0800 Subject: [PATCH 28/56] test out-of-order pg and rw table --- e2e_test/sink/postgres_sink.slt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/e2e_test/sink/postgres_sink.slt b/e2e_test/sink/postgres_sink.slt index cc7a002e7ba4..01f6e29b90d5 100644 --- a/e2e_test/sink/postgres_sink.slt +++ b/e2e_test/sink/postgres_sink.slt @@ -36,8 +36,8 @@ CREATE TABLE rw_types_table ( id BIGINT PRIMARY KEY, varchar_column VARCHAR, text_column TEXT, - integer_column INTEGER, smallint_column SMALLINT, + integer_column INTEGER, bigint_column BIGINT, decimal_column DECIMAL, real_column REAL, @@ -45,9 +45,9 @@ CREATE TABLE rw_types_table ( boolean_column BOOLEAN, date_column DATE, time_column TIME, - timestamp_column TIMESTAMP, interval_column INTERVAL, - jsonb_column JSONB + jsonb_column JSONB, + timestamp_column TIMESTAMP ); statement ok From eebf885d1a5eb7fd6d78b06021e44f345a656681 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 19 Nov 2024 23:31:11 +0800 Subject: [PATCH 29/56] test more error states --- e2e_test/sink/postgres_sink.slt | 137 ++++++++++++++++++++++++++++++++ 1 file changed, 137 insertions(+) diff --git a/e2e_test/sink/postgres_sink.slt b/e2e_test/sink/postgres_sink.slt index 01f6e29b90d5..5b721b53a719 100644 --- a/e2e_test/sink/postgres_sink.slt +++ b/e2e_test/sink/postgres_sink.slt @@ -12,6 +12,126 @@ drop table if exists rw_types_table_append_only cascade; system ok PGDATABASE=sink_test createdb +################### test table pk can mismatch + +system ok +PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +)" + +statement ok +CREATE TABLE rw_types_table ( + id BIGINT, + varchar_column VARCHAR, + text_column TEXT, + smallint_column SMALLINT, + integer_column INTEGER, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + interval_column INTERVAL, + jsonb_column JSONB, + timestamp_column TIMESTAMP +); + +statement ok +CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='upsert', + primary_key='id', +); + +statement ok +drop table rw_types_table cascade; + +system ok +PGDATABASE=sink_test psql -c "DROP TABLE pg_types_table"; + +################### test table col name cannot mismatch + +system ok +PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +)" + +statement ok +CREATE TABLE rw_types_table ( + id BIGINT, + varchar_column_mismatch_name VARCHAR, + text_column TEXT, + smallint_column SMALLINT, + integer_column INTEGER, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + interval_column INTERVAL, + jsonb_column JSONB, + timestamp_column TIMESTAMP +); + +statement error +CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='upsert', + primary_key='id', +); + +statement ok +drop table rw_types_table cascade; + +system ok +PGDATABASE=sink_test psql -c "DROP TABLE pg_types_table"; + +################### test table col can be out of order + system ok PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( id BIGINT PRIMARY KEY, @@ -50,6 +170,23 @@ CREATE TABLE rw_types_table ( timestamp_column TIMESTAMP ); +################### test sink with_options:primary_key mismatch should fail + +statement error +CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='upsert', + primary_key='id, date_column', +); + +################### test pk match should work + statement ok CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( connector='postgres', From 62dcf788ec1dc165f454ae43b847e9a61c8cf8c7 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 20 Nov 2024 09:22:19 +0800 Subject: [PATCH 30/56] test --- src/connector/with_options_sink.yaml | 27 +++++++++++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 88f2e64cce2e..713ac58ee836 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -905,6 +905,33 @@ NatsConfig: - name: r#type field_type: String required: true +PostgresConfig: + fields: + - name: host + field_type: String + required: true + - name: port + field_type: u16 + required: true + - name: user + field_type: String + required: true + - name: password + field_type: String + required: true + - name: database + field_type: String + required: true + - name: table + field_type: String + required: true + - name: max_batch_rows + field_type: usize + required: false + default: '1024' + - name: r#type + field_type: String + required: true PulsarConfig: fields: - name: properties.retry.max From 670e7409eb9145a9d4287aa4bc054cb8ab68549e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 20 Nov 2024 09:37:02 +0800 Subject: [PATCH 31/56] handle connection error --- src/connector/src/sink/postgres.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index f103fed63723..b503c3a92e0d 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -26,6 +26,7 @@ use risingwave_common::types::DataType; use serde_derive::Deserialize; use serde_with::{serde_as, DisplayFromStr}; use simd_json::prelude::ArrayTrait; +use thiserror_ext::AsReport; use tokio_postgres::Statement; use with_options::WithOptions; @@ -144,8 +145,8 @@ impl Sink for PostgresSink { .await .context("Failed to connect to Postgres for Sinking")?; tokio::spawn(async move { - if let Err(e) = connection.await { - tracing::error!("connection error: {}", e); + if let Err(error) = connection.await { + tracing::error!("postgres sink connection error: {:?}", error.as_report()); } }); @@ -278,8 +279,8 @@ impl PostgresSinkWriter { .await .context("Failed to connect to Postgres for Sinking")?; tokio::spawn(async move { - if let Err(e) = connection.await { - tracing::error!("connection error: {}", e); + if let Err(error) = connection.await { + tracing::error!("postgres sink connection error: {:?}", error.as_report()); } }); client From 3fd129d47489ae818178cd371d36baaf86bb16ba Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 20 Nov 2024 10:09:17 +0800 Subject: [PATCH 32/56] fix formatting --- src/connector/src/sink/postgres.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index b503c3a92e0d..2f920c879228 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -614,13 +614,13 @@ mod tests { sql, expect![[r#" - MERGE INTO test_table target - USING (SELECT $1 as a,$2 as b) AS source - ON (source.b = target.b) - WHEN MATCHED - THEN UPDATE SET a = source.a - WHEN NOT MATCHED - THEN INSERT (a, b) VALUES (source.a, source.b) + MERGE INTO test_table target + USING (SELECT $1 as a,$2 as b) AS source + ON (source.b = target.b) + WHEN MATCHED + THEN UPDATE SET a = source.a + WHEN NOT MATCHED + THEN INSERT (a, b) VALUES (source.a, source.b) "#]], ); } From b6b0f1e4507c755ba5b32b5c2ba2670f881830ec Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 20 Nov 2024 11:45:56 +0800 Subject: [PATCH 33/56] fix --- src/connector/src/sink/postgres.rs | 25 +++++++++++-------------- 1 file changed, 11 insertions(+), 14 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 2f920c879228..62084280ebc2 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -519,8 +519,7 @@ fn create_merge_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> WHEN MATCHED THEN UPDATE SET {update_vars} WHEN NOT MATCHED - THEN INSERT ({insert_columns}) VALUES ({insert_vars}) - " + THEN INSERT ({insert_columns}) VALUES ({insert_vars})" ) } @@ -543,7 +542,7 @@ mod tests { use super::*; fn check(actual: impl Display, expect: Expect) { - let actual = format!("{}", actual); + let actual = actual.to_string(); expect.assert_eq(&actual); } @@ -610,18 +609,16 @@ mod tests { ]); let table_name = "test_table"; let sql = create_merge_sql(&schema, table_name, &[1]); - check( + assert_eq!( sql, - expect![[r#" - - MERGE INTO test_table target - USING (SELECT $1 as a,$2 as b) AS source - ON (source.b = target.b) - WHEN MATCHED - THEN UPDATE SET a = source.a - WHEN NOT MATCHED - THEN INSERT (a, b) VALUES (source.a, source.b) - "#]], + " + MERGE INTO test_table target + USING (SELECT $1 as a,$2 as b) AS source + ON (source.b = target.b) + WHEN MATCHED + THEN UPDATE SET a = source.a + WHEN NOT MATCHED + THEN INSERT (a, b) VALUES (source.a, source.b)".to_string() ); } } From 8e7da05bbad047ff8ccbcd18627ad1e278176b99 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 20 Nov 2024 11:57:39 +0800 Subject: [PATCH 34/56] fmt --- src/connector/src/sink/postgres.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 62084280ebc2..bfc41423d759 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -611,14 +611,15 @@ mod tests { let sql = create_merge_sql(&schema, table_name, &[1]); assert_eq!( sql, - " + " MERGE INTO test_table target USING (SELECT $1 as a,$2 as b) AS source ON (source.b = target.b) WHEN MATCHED THEN UPDATE SET a = source.a WHEN NOT MATCHED - THEN INSERT (a, b) VALUES (source.a, source.b)".to_string() + THEN INSERT (a, b) VALUES (source.a, source.b)" + .to_string() ); } } From d84c4c4c8e7493167ce971bb9ca26fa7c4545348 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 20 Nov 2024 19:25:57 +0800 Subject: [PATCH 35/56] fix --- src/connector/src/sink/postgres.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index bfc41423d759..ac57281d757a 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -146,7 +146,7 @@ impl Sink for PostgresSink { .context("Failed to connect to Postgres for Sinking")?; tokio::spawn(async move { if let Err(error) = connection.await { - tracing::error!("postgres sink connection error: {:?}", error.as_report()); + tracing::error!(error = %error.as_report(), "postgres sink connection error"); } }); @@ -280,7 +280,7 @@ impl PostgresSinkWriter { .context("Failed to connect to Postgres for Sinking")?; tokio::spawn(async move { if let Err(error) = connection.await { - tracing::error!("postgres sink connection error: {:?}", error.as_report()); + tracing::error!(error = %error.as_report(), "postgres sink connection error"); } }); client From a892bd876b68b6c4a8607b523032fc1db7adeba8 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 21 Nov 2024 12:56:58 +0800 Subject: [PATCH 36/56] use on conflict do update --- src/connector/src/sink/postgres.rs | 74 ++++++++---------------------- 1 file changed, 20 insertions(+), 54 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index ac57281d757a..5be40794ebf7 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -323,7 +323,7 @@ impl PostgresSinkWriter { .iter() .map(|field| field.data_type().to_pg_type()) .collect_vec(); - let merge_sql = create_merge_sql(&schema, &config.table, &pk_indices); + let merge_sql = create_upsert_sql(&schema, &config.table, &pk_indices); Some( client .prepare_typed(&merge_sql, &merge_types) @@ -472,55 +472,32 @@ fn create_insert_sql(schema: &Schema, table_name: &str) -> String { format!("INSERT INTO {table_name} ({columns}) VALUES ({parameters})") } -fn create_merge_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> String { - let named_parameters: String = schema +fn create_upsert_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> String { + let columns: String = schema .fields() .iter() - .enumerate() - .map(|(i, field)| format!("${} as {}", i + 1, &field.name)) - .collect_vec() - .join(","); - let conditions: String = pk_indices - .iter() - .map(|i| { - format!( - "source.{} = target.{}", - schema.fields()[*i].name, - schema.fields()[*i].name - ) - }) + .map(|field| field.name.clone()) .collect_vec() - .join(" AND "); - let update_vars: String = schema - .fields() - .iter() - .enumerate() - .filter(|(i, _)| !pk_indices.contains(i)) - .map(|(_, field)| format!("{} = source.{}", field.name, field.name)) + .join(", "); + let parameters: String = (0..schema.fields().len()) + .map(|i| format!("${}", i + 1)) .collect_vec() .join(", "); - let insert_columns: String = schema - .fields() + let pk_columns = pk_indices .iter() - .map(|field| field.name.clone()) + .map(|i| schema.fields()[*i].name.clone()) .collect_vec() .join(", "); - let insert_vars: String = schema - .fields() - .iter() - .map(|field| format!("source.{}", field.name)) + let update_parameters: String = (0..schema.fields().len()) + .filter(|i| !pk_indices.contains(i)) + .map(|i| { + let column = schema.fields()[i].name.clone(); + let param = format!("${}", i + 1); + format!("{column} = {param}") + }) .collect_vec() .join(", "); - format!( - " - MERGE INTO {table_name} target - USING (SELECT {named_parameters}) AS source - ON ({conditions}) - WHEN MATCHED - THEN UPDATE SET {update_vars} - WHEN NOT MATCHED - THEN INSERT ({insert_columns}) VALUES ({insert_vars})" - ) + format!("INSERT INTO {table_name} ({columns}) VALUES ({parameters}) on conflict ({pk_columns}) do update set {update_parameters}") } fn create_delete_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> String { @@ -592,7 +569,7 @@ mod tests { } #[test] - fn test_create_merge_sql() { + fn test_create_upsert_sql() { let schema = Schema::new(vec![ Field { data_type: DataType::Int32, @@ -608,18 +585,7 @@ mod tests { }, ]); let table_name = "test_table"; - let sql = create_merge_sql(&schema, table_name, &[1]); - assert_eq!( - sql, - " - MERGE INTO test_table target - USING (SELECT $1 as a,$2 as b) AS source - ON (source.b = target.b) - WHEN MATCHED - THEN UPDATE SET a = source.a - WHEN NOT MATCHED - THEN INSERT (a, b) VALUES (source.a, source.b)" - .to_string() - ); + let sql = create_upsert_sql(&schema, table_name, &[1]); + check(sql, expect!["INSERT INTO test_table (a, b) VALUES ($1, $2) on conflict do update set a = $1"]); } } From 640d10706ec0646dd0f6ca86acb70d3a2a94e057 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 21 Nov 2024 13:10:26 +0800 Subject: [PATCH 37/56] add schema fields --- src/connector/src/sink/postgres.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 5be40794ebf7..7259155c3197 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -48,6 +48,8 @@ pub struct PostgresConfig { pub password: String, pub database: String, pub table: String, + #[serde(default = "default_schema")] + pub schema: String, #[serde(default = "default_max_batch_rows")] #[serde_as(as = "DisplayFromStr")] pub max_batch_rows: usize, @@ -58,6 +60,10 @@ fn default_max_batch_rows() -> usize { 1024 } +fn default_schema() -> String { + "public".to_string() +} + impl PostgresConfig { pub fn from_btreemap(properties: BTreeMap) -> Result { let config = From fbd4f08cee9b3683154a0fe0a7d96192e4ee8b4c Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 21 Nov 2024 13:57:54 +0800 Subject: [PATCH 38/56] fmt --- src/connector/src/sink/postgres.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 7259155c3197..46769c2f78d4 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -592,6 +592,11 @@ mod tests { ]); let table_name = "test_table"; let sql = create_upsert_sql(&schema, table_name, &[1]); - check(sql, expect!["INSERT INTO test_table (a, b) VALUES ($1, $2) on conflict do update set a = $1"]); + check( + sql, + expect![ + "INSERT INTO test_table (a, b) VALUES ($1, $2) on conflict do update set a = $1" + ], + ); } } From 23cb74f67ee70172889f955c88b8a97f438a906d Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 24 Nov 2024 20:36:44 +0800 Subject: [PATCH 39/56] refactor connector create_pg_client logic --- .../maybe_tls_connector.rs | 0 src/connector/src/connector_common/mod.rs | 4 + .../src/connector_common/postgres.rs | 145 ++++++++++++++++++ src/connector/src/source/cdc/external/mod.rs | 41 +---- .../src/source/cdc/external/postgres.rs | 87 ++--------- 5 files changed, 162 insertions(+), 115 deletions(-) rename src/connector/src/{source/cdc/external => connector_common}/maybe_tls_connector.rs (100%) create mode 100644 src/connector/src/connector_common/postgres.rs diff --git a/src/connector/src/source/cdc/external/maybe_tls_connector.rs b/src/connector/src/connector_common/maybe_tls_connector.rs similarity index 100% rename from src/connector/src/source/cdc/external/maybe_tls_connector.rs rename to src/connector/src/connector_common/maybe_tls_connector.rs diff --git a/src/connector/src/connector_common/mod.rs b/src/connector/src/connector_common/mod.rs index 57b614fdf548..8c51dbe2afac 100644 --- a/src/connector/src/connector_common/mod.rs +++ b/src/connector/src/connector_common/mod.rs @@ -25,4 +25,8 @@ pub use common::{ }; mod iceberg; +mod maybe_tls_connector; +mod postgres; + pub use iceberg::IcebergCommon; +pub use postgres::{create_pg_client, SslMode}; diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs new file mode 100644 index 000000000000..7ec338b29d8f --- /dev/null +++ b/src/connector/src/connector_common/postgres.rs @@ -0,0 +1,145 @@ +// 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::fmt; + +use anyhow::anyhow; +use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; +use postgres_openssl::MakeTlsConnector; +use serde_derive::Deserialize; +use thiserror_ext::AsReport; +use tokio_postgres::{Client as PgClient, NoTls}; + +use super::maybe_tls_connector::MaybeMakeTlsConnector; + +#[derive(Debug, Clone, PartialEq, Deserialize)] +#[serde(rename_all = "lowercase")] +pub enum SslMode { + #[serde(alias = "disable")] + Disabled, + #[serde(alias = "prefer")] + Preferred, + #[serde(alias = "require")] + Required, + /// verify that the server is trustworthy by checking the certificate chain + /// up to the root certificate stored on the client. + #[serde(alias = "verify-ca")] + VerifyCa, + /// Besides verify the certificate, will also verify that the serverhost name + /// matches the name stored in the server certificate. + #[serde(alias = "verify-full")] + VerifyFull, +} + +impl Default for SslMode { + fn default() -> Self { + // default to `disabled` for backward compatibility + Self::Disabled + } +} + +impl fmt::Display for SslMode { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.write_str(match self { + SslMode::Disabled => "disabled", + SslMode::Preferred => "preferred", + SslMode::Required => "required", + SslMode::VerifyCa => "verify-ca", + SslMode::VerifyFull => "verify-full", + }) + } +} + +pub async fn create_pg_client( + user: &str, + password: &str, + host: &str, + port: &str, + database: &str, + ssl_mode: &SslMode, + ssl_root_cert: &Option, +) -> anyhow::Result { + let mut pg_config = tokio_postgres::Config::new(); + pg_config + .user(user) + .password(password) + .host(host) + .port(port.parse::().unwrap()) + .dbname(database); + + let (_verify_ca, verify_hostname) = match ssl_mode { + SslMode::VerifyCa => (true, false), + SslMode::VerifyFull => (true, true), + _ => (false, false), + }; + + #[cfg(not(madsim))] + let connector = match ssl_mode { + SslMode::Disabled => { + pg_config.ssl_mode(tokio_postgres::config::SslMode::Disable); + MaybeMakeTlsConnector::NoTls(NoTls) + } + SslMode::Preferred => { + pg_config.ssl_mode(tokio_postgres::config::SslMode::Prefer); + match SslConnector::builder(SslMethod::tls()) { + Ok(mut builder) => { + // disable certificate verification for `prefer` + builder.set_verify(SslVerifyMode::NONE); + MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())) + } + Err(e) => { + tracing::warn!(error = %e.as_report(), "SSL connector error"); + MaybeMakeTlsConnector::NoTls(NoTls) + } + } + } + SslMode::Required => { + pg_config.ssl_mode(tokio_postgres::config::SslMode::Require); + let mut builder = SslConnector::builder(SslMethod::tls())?; + // disable certificate verification for `require` + builder.set_verify(SslVerifyMode::NONE); + MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())) + } + + SslMode::VerifyCa | SslMode::VerifyFull => { + pg_config.ssl_mode(tokio_postgres::config::SslMode::Require); + let mut builder = SslConnector::builder(SslMethod::tls())?; + if let Some(ssl_root_cert) = ssl_root_cert { + builder.set_ca_file(ssl_root_cert).map_err(|e| { + anyhow!(format!("bad ssl root cert error: {}", e.to_report_string())) + })?; + } + let mut connector = MakeTlsConnector::new(builder.build()); + if !verify_hostname { + connector.set_callback(|config, _| { + config.set_verify_hostname(false); + Ok(()) + }); + } + MaybeMakeTlsConnector::Tls(connector) + } + }; + #[cfg(madsim)] + let connector = NoTls; + + let (client, connection) = pg_config.connect(connector).await?; + + tokio::spawn(async move { + if let Err(e) = connection.await { + tracing::error!(error = %e.as_report(), "postgres connection error"); + } + }); + + Ok(client) +} diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index fbfa66ef0e7c..b5d12487f2cb 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -17,11 +17,9 @@ pub mod postgres; pub mod sql_server; #[cfg(not(madsim))] -mod maybe_tls_connector; pub mod mysql; use std::collections::{BTreeMap, HashMap}; -use std::fmt; use anyhow::anyhow; use futures::pin_mut; @@ -34,6 +32,7 @@ use risingwave_common::secret::LocalSecretManager; use risingwave_pb::secret::PbSecretRef; use serde_derive::{Deserialize, Serialize}; +use crate::connector_common::SslMode; use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::mysql_row_to_owned_row; use crate::source::cdc::external::mock_external_table::MockExternalTableReader; @@ -263,44 +262,6 @@ impl ExternalTableConfig { } } -#[derive(Debug, Clone, PartialEq, Deserialize)] -#[serde(rename_all = "lowercase")] -pub enum SslMode { - #[serde(alias = "disable")] - Disabled, - #[serde(alias = "prefer")] - Preferred, - #[serde(alias = "require")] - Required, - /// verify that the server is trustworthy by checking the certificate chain - /// up to the root certificate stored on the client. - #[serde(alias = "verify-ca")] - VerifyCa, - /// Besides verify the certificate, will also verify that the serverhost name - /// matches the name stored in the server certificate. - #[serde(alias = "verify-full")] - VerifyFull, -} - -impl Default for SslMode { - fn default() -> Self { - // default to `disabled` for backward compatibility - Self::Disabled - } -} - -impl fmt::Display for SslMode { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.write_str(match self { - SslMode::Disabled => "disabled", - SslMode::Preferred => "preferred", - SslMode::Required => "required", - SslMode::VerifyCa => "verify-ca", - SslMode::VerifyFull => "verify-full", - }) - } -} - impl ExternalTableReader for ExternalTableReaderImpl { async fn current_cdc_offset(&self) -> ConnectorResult { match self { diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 112fd16e6bff..202be3a5ee0e 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -20,8 +20,6 @@ use futures::stream::BoxStream; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; -use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; -use postgres_openssl::MakeTlsConnector; use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema}; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, ScalarImpl, StructType}; @@ -33,13 +31,12 @@ use sqlx::postgres::{PgConnectOptions, PgSslMode}; use sqlx::PgPool; use thiserror_ext::AsReport; use tokio_postgres::types::PgLsn; -use tokio_postgres::NoTls; +use crate::connector_common::create_pg_client; +#[cfg(not(madsim))] use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::postgres_row_to_owned_row; use crate::parser::scalar_adapter::ScalarAdapter; -#[cfg(not(madsim))] -use crate::source::cdc::external::maybe_tls_connector::MaybeMakeTlsConnector; use crate::source::cdc::external::{ CdcOffset, CdcOffsetParseFunc, DebeziumOffset, ExternalTableConfig, ExternalTableReader, SchemaTableName, SslMode, @@ -312,76 +309,16 @@ impl PostgresExternalTableReader { "create postgres external table reader" ); - let mut pg_config = tokio_postgres::Config::new(); - pg_config - .user(&config.username) - .password(&config.password) - .host(&config.host) - .port(config.port.parse::().unwrap()) - .dbname(&config.database); - - let (_verify_ca, verify_hostname) = match config.ssl_mode { - SslMode::VerifyCa => (true, false), - SslMode::VerifyFull => (true, true), - _ => (false, false), - }; - - #[cfg(not(madsim))] - let connector = match config.ssl_mode { - SslMode::Disabled => { - pg_config.ssl_mode(tokio_postgres::config::SslMode::Disable); - MaybeMakeTlsConnector::NoTls(NoTls) - } - SslMode::Preferred => { - pg_config.ssl_mode(tokio_postgres::config::SslMode::Prefer); - match SslConnector::builder(SslMethod::tls()) { - Ok(mut builder) => { - // disable certificate verification for `prefer` - builder.set_verify(SslVerifyMode::NONE); - MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())) - } - Err(e) => { - tracing::warn!(error = %e.as_report(), "SSL connector error"); - MaybeMakeTlsConnector::NoTls(NoTls) - } - } - } - SslMode::Required => { - pg_config.ssl_mode(tokio_postgres::config::SslMode::Require); - let mut builder = SslConnector::builder(SslMethod::tls())?; - // disable certificate verification for `require` - builder.set_verify(SslVerifyMode::NONE); - MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())) - } - - SslMode::VerifyCa | SslMode::VerifyFull => { - pg_config.ssl_mode(tokio_postgres::config::SslMode::Require); - let mut builder = SslConnector::builder(SslMethod::tls())?; - if let Some(ssl_root_cert) = config.ssl_root_cert { - builder.set_ca_file(ssl_root_cert).map_err(|e| { - anyhow!(format!("bad ssl root cert error: {}", e.to_report_string())) - })?; - } - let mut connector = MakeTlsConnector::new(builder.build()); - if !verify_hostname { - connector.set_callback(|config, _| { - config.set_verify_hostname(false); - Ok(()) - }); - } - MaybeMakeTlsConnector::Tls(connector) - } - }; - #[cfg(madsim)] - let connector = NoTls; - - let (client, connection) = pg_config.connect(connector).await?; - - tokio::spawn(async move { - if let Err(e) = connection.await { - tracing::error!(error = %e.as_report(), "postgres connection error"); - } - }); + let client = create_pg_client( + &config.username, + &config.password, + &config.host, + &config.port, + &config.database, + &config.ssl_mode, + &config.ssl_root_cert, + ) + .await?; let field_names = rw_schema .fields From 5c842b05b4be3a25303859f552956b618e9f16cb Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 24 Nov 2024 21:03:11 +0800 Subject: [PATCH 40/56] handle ssl options --- .../src/connector_common/postgres.rs | 3 +- src/connector/src/sink/postgres.rs | 34 +++++++++---------- src/connector/src/source/cdc/external/mod.rs | 7 +++- 3 files changed, 23 insertions(+), 21 deletions(-) diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs index 7ec338b29d8f..6ace11db5a1e 100644 --- a/src/connector/src/connector_common/postgres.rs +++ b/src/connector/src/connector_common/postgres.rs @@ -44,8 +44,7 @@ pub enum SslMode { impl Default for SslMode { fn default() -> Self { - // default to `disabled` for backward compatibility - Self::Disabled + Self::Preferred } } diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 46769c2f78d4..cad4bb4b6833 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -33,13 +33,14 @@ use with_options::WithOptions; use super::{ SinkError, SinkWriterMetrics, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; +use crate::connector_common::{create_pg_client, SslMode}; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriterParam}; pub const POSTGRES_SINK: &str = "postgres"; #[serde_as] -#[derive(Clone, Debug, Deserialize, WithOptions)] +#[derive(Clone, Debug, Deserialize)] pub struct PostgresConfig { pub host: String, #[serde_as(as = "DisplayFromStr")] @@ -50,6 +51,10 @@ pub struct PostgresConfig { pub table: String, #[serde(default = "default_schema")] pub schema: String, + #[serde(default = "Default::default")] + pub ssl_mode: SslMode, + #[serde(rename = "ssl.root.cert")] + pub ssl_root_cert: Option, #[serde(default = "default_max_batch_rows")] #[serde_as(as = "DisplayFromStr")] pub max_batch_rows: usize, @@ -138,23 +143,16 @@ impl Sink for PostgresSink { // Verify pg table schema matches rw table schema, and pk indices are valid let table_name = &self.config.table; - let connection_string = format!( - "host={} port={} user={} password={} dbname={}", - self.config.host, - self.config.port, - self.config.user, - self.config.password, - self.config.database - ); - let (client, connection) = - tokio_postgres::connect(&connection_string, tokio_postgres::NoTls) - .await - .context("Failed to connect to Postgres for Sinking")?; - tokio::spawn(async move { - if let Err(error) = connection.await { - tracing::error!(error = %error.as_report(), "postgres sink connection error"); - } - }); + let client = create_pg_client( + &self.config.user, + &self.config.password, + &self.config.host, + &self.config.port.to_string(), + &self.config.database, + &self.config.ssl_mode, + &self.config.ssl_root_cert, + ) + .await?; let result = client .query( diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index b5d12487f2cb..ad1fcaca454f 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -235,7 +235,7 @@ pub struct ExternalTableConfig { /// `ssl.mode` specifies the SSL/TLS encryption level for secure communication with Postgres. /// Choices include `disabled`, `preferred`, and `required`. /// This field is optional. - #[serde(rename = "ssl.mode", default = "Default::default")] + #[serde(rename = "ssl.mode", default = "postgres_ssl_mode_default")] #[serde(alias = "debezium.database.sslmode")] pub ssl_mode: SslMode, @@ -249,6 +249,11 @@ pub struct ExternalTableConfig { pub encrypt: String, } +fn postgres_ssl_mode_default() -> SslMode { + // NOTE(StrikeW): Default to `disabled` for backward compatibility + SslMode::Disabled +} + impl ExternalTableConfig { pub fn try_from_btreemap( connect_properties: BTreeMap, From b5397db91abccf2a34a90b36bec66890bd02b859 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 24 Nov 2024 22:17:27 +0800 Subject: [PATCH 41/56] add sanity check --- src/connector/src/sink/postgres.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index cad4bb4b6833..4bca537abc69 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -366,8 +366,10 @@ impl PostgresSinkWriter { } } } else { + let mut unmatched_update_insert = 0; for chunk in self.buffer.drain() { for (op, row) in chunk.rows() { + let mut expect_update_delete = false; match op { Op::Insert => { self.client @@ -375,6 +377,7 @@ impl PostgresSinkWriter { .await?; } Op::UpdateInsert => { + unmatched_update_insert += 1; // NOTE(kwannoel): Here we use `MERGE` rather than `UPDATE/INSERT` directly. // This is because the downstream db could have cleaned the old record, // in that case it needs to be `INSERTED` rather than UPDATED. @@ -384,6 +387,7 @@ impl PostgresSinkWriter { .await?; } Op::Delete => { + unmatched_update_insert -= 1; self.client .execute_raw( self.delete_statement.as_ref().unwrap(), @@ -395,6 +399,7 @@ impl PostgresSinkWriter { } } } + assert_eq!(unmatched_update_insert, 0); } Ok(()) From d123d6f6775fb00606aadd8825017524ae01b588 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 24 Nov 2024 22:30:45 +0800 Subject: [PATCH 42/56] use pg_client method to construct --- src/connector/src/sink/postgres.rs | 26 ++++++++++---------------- 1 file changed, 10 insertions(+), 16 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 4bca537abc69..d6aac79d6a92 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -273,22 +273,16 @@ impl PostgresSinkWriter { pk_indices: Vec, is_append_only: bool, ) -> Result { - let client = { - let connection_string = format!( - "host={} port={} user={} password={} dbname={}", - config.host, config.port, config.user, config.password, config.database - ); - let (client, connection) = - tokio_postgres::connect(&connection_string, tokio_postgres::NoTls) - .await - .context("Failed to connect to Postgres for Sinking")?; - tokio::spawn(async move { - if let Err(error) = connection.await { - tracing::error!(error = %error.as_report(), "postgres sink connection error"); - } - }); - client - }; + let client = create_pg_client( + &config.user, + &config.password, + &config.host, + &config.port.to_string(), + &config.database, + &config.ssl_mode, + &config.ssl_root_cert, + ) + .await?; let insert_statement = { let insert_types = schema From b04feb9dacbee1c8394365fa1e14f358959bfe00 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 24 Nov 2024 23:05:40 +0800 Subject: [PATCH 43/56] move table reader and functions to common --- src/connector/src/connector_common/mod.rs | 4 +- .../src/connector_common/postgres.rs | 192 ++++++++++++++++++ src/connector/src/parser/scalar_adapter.rs | 4 + src/connector/src/sink/postgres.rs | 12 +- src/connector/src/source/cdc/external/mod.rs | 6 +- .../src/source/cdc/external/postgres.rs | 186 +---------------- 6 files changed, 213 insertions(+), 191 deletions(-) diff --git a/src/connector/src/connector_common/mod.rs b/src/connector/src/connector_common/mod.rs index 8c51dbe2afac..e7f43bd3f902 100644 --- a/src/connector/src/connector_common/mod.rs +++ b/src/connector/src/connector_common/mod.rs @@ -26,7 +26,7 @@ pub use common::{ mod iceberg; mod maybe_tls_connector; -mod postgres; +pub mod postgres; pub use iceberg::IcebergCommon; -pub use postgres::{create_pg_client, SslMode}; +pub use postgres::{create_pg_client, PostgresExternalTable, SslMode}; diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs index 6ace11db5a1e..cd9d8bbdf341 100644 --- a/src/connector/src/connector_common/postgres.rs +++ b/src/connector/src/connector_common/postgres.rs @@ -12,16 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::fmt; use anyhow::anyhow; use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; use postgres_openssl::MakeTlsConnector; +use risingwave_common::catalog::{ColumnDesc, ColumnId}; +use risingwave_common::types::{DataType, ScalarImpl, StructType}; +use sea_schema::postgres::def::{ColumnType, TableInfo}; +use sea_schema::postgres::discovery::SchemaDiscovery; use serde_derive::Deserialize; +use sqlx::postgres::{PgConnectOptions, PgSslMode}; +use sqlx::PgPool; use thiserror_ext::AsReport; use tokio_postgres::{Client as PgClient, NoTls}; use super::maybe_tls_connector::MaybeMakeTlsConnector; +use crate::error::ConnectorResult; +use crate::source::cdc::external::ExternalTableConfig; #[derive(Debug, Clone, PartialEq, Deserialize)] #[serde(rename_all = "lowercase")] @@ -48,6 +57,102 @@ impl Default for SslMode { } } +pub struct PostgresExternalTable { + column_descs: Vec, + pk_names: Vec, +} + +impl PostgresExternalTable { + pub async fn connect(config: ExternalTableConfig) -> ConnectorResult { + tracing::debug!("connect to postgres external table"); + let mut options = PgConnectOptions::new() + .username(&config.username) + .password(&config.password) + .host(&config.host) + .port(config.port.parse::().unwrap()) + .database(&config.database) + .ssl_mode(match config.ssl_mode { + SslMode::Disabled => PgSslMode::Disable, + SslMode::Preferred => PgSslMode::Prefer, + SslMode::Required => PgSslMode::Require, + SslMode::VerifyCa => PgSslMode::VerifyCa, + SslMode::VerifyFull => PgSslMode::VerifyFull, + }); + + if config.ssl_mode == SslMode::VerifyCa || config.ssl_mode == SslMode::VerifyFull { + if let Some(ref root_cert) = config.ssl_root_cert { + options = options.ssl_root_cert(root_cert.as_str()); + } + } + + 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 { + let data_type = type_to_rw_type(&col.col_type)?; + let column_desc = if let Some(ref default_expr) = col.default { + // parse the value of "column_default" field in information_schema.columns, + // non number data type will be stored as "'value'::type" + let val_text = default_expr + .0 + .split("::") + .map(|s| s.trim_matches('\'')) + .next() + .expect("default value expression"); + + match ScalarImpl::from_text(val_text, &data_type) { + Ok(scalar) => ColumnDesc::named_with_default_value( + col.name.clone(), + ColumnId::placeholder(), + data_type.clone(), + Some(scalar), + ), + Err(err) => { + tracing::warn!(error=%err.as_report(), "failed to parse postgres default value expression, only constant is supported"); + ColumnDesc::named(col.name.clone(), ColumnId::placeholder(), data_type) + } + } + } else { + ColumnDesc::named(col.name.clone(), ColumnId::placeholder(), data_type) + }; + column_descs.push(column_desc); + } + + 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 { + column_descs, + pk_names, + }) + } + + pub fn column_descs(&self) -> &Vec { + &self.column_descs + } + + pub fn pk_names(&self) -> &Vec { + &self.pk_names + } +} + impl fmt::Display for SslMode { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { f.write_str(match self { @@ -142,3 +247,90 @@ pub async fn create_pg_client( Ok(client) } + +pub 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::VarBit(_) => { + return Err(anyhow!("VARBIT 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(name) => { + // NOTES: user-defined enum type is classified as `Unknown` + tracing::warn!("Unknown Postgres data type: {name}, map to varchar"); + DataType::Varchar + } + }; + + Ok(dtype) +} diff --git a/src/connector/src/parser/scalar_adapter.rs b/src/connector/src/parser/scalar_adapter.rs index 0f5d2d6d6d93..e01d72e4285c 100644 --- a/src/connector/src/parser/scalar_adapter.rs +++ b/src/connector/src/parser/scalar_adapter.rs @@ -17,6 +17,7 @@ use std::str::FromStr; use anyhow::anyhow; use bytes::BytesMut; use pg_bigdecimal::PgNumeric; +use risingwave_common::row::Row; use risingwave_common::types::{DataType, Decimal, Int256, ListValue, ScalarImpl, ScalarRefImpl}; use thiserror_ext::AsReport; use tokio_postgres::types::{to_sql_checked, FromSql, IsNull, Kind, ToSql, Type}; @@ -377,3 +378,6 @@ fn rw_numeric_to_pg_numeric(val: Decimal) -> PgNumeric { Decimal::NaN => PgNumeric::NaN, } } +// pub fn row_to_scalar_iter<'a>( +// row: impl Row +// ) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index d6aac79d6a92..d19ec54b0251 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -34,6 +34,7 @@ use super::{ SinkError, SinkWriterMetrics, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; use crate::connector_common::{create_pg_client, SslMode}; +use crate::parser::scalar_adapter::ScalarAdapter; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriterParam}; @@ -367,7 +368,16 @@ impl PostgresSinkWriter { match op { Op::Insert => { self.client - .execute_raw(&self.insert_statement, row.iter()) + .execute_raw( + &self.insert_statement, + row.iter(), + // row + // .iter() + // .enumerate() + // .map(|(i, d)| d.map(|d| { + // ScalarAdapter::from_scalar(d) + // }))) + ) .await?; } Op::UpdateInsert => { diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index ad1fcaca454f..84216ae4762b 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -32,16 +32,14 @@ use risingwave_common::secret::LocalSecretManager; use risingwave_pb::secret::PbSecretRef; use serde_derive::{Deserialize, Serialize}; -use crate::connector_common::SslMode; +use crate::connector_common::{PostgresExternalTable, SslMode}; 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::mysql::{ MySqlExternalTable, MySqlExternalTableReader, MySqlOffset, }; -use crate::source::cdc::external::postgres::{ - PostgresExternalTable, PostgresExternalTableReader, PostgresOffset, -}; +use crate::source::cdc::external::postgres::{PostgresExternalTableReader, PostgresOffset}; use crate::source::cdc::external::sql_server::{ SqlServerExternalTable, SqlServerExternalTableReader, SqlServerOffset, }; diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 202be3a5ee0e..435889ff09b2 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -32,7 +32,8 @@ use sqlx::PgPool; use thiserror_ext::AsReport; use tokio_postgres::types::PgLsn; -use crate::connector_common::create_pg_client; +use crate::connector_common::postgres::type_to_rw_type; +use crate::connector_common::{create_pg_client, PostgresExternalTable}; #[cfg(not(madsim))] use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::postgres_row_to_owned_row; @@ -75,189 +76,6 @@ impl PostgresOffset { } } -pub struct PostgresExternalTable { - column_descs: Vec, - pk_names: Vec, -} - -impl PostgresExternalTable { - pub async fn connect(config: ExternalTableConfig) -> ConnectorResult { - tracing::debug!("connect to postgres external table"); - let mut options = PgConnectOptions::new() - .username(&config.username) - .password(&config.password) - .host(&config.host) - .port(config.port.parse::().unwrap()) - .database(&config.database) - .ssl_mode(match config.ssl_mode { - SslMode::Disabled => PgSslMode::Disable, - SslMode::Preferred => PgSslMode::Prefer, - SslMode::Required => PgSslMode::Require, - SslMode::VerifyCa => PgSslMode::VerifyCa, - SslMode::VerifyFull => PgSslMode::VerifyFull, - }); - - if config.ssl_mode == SslMode::VerifyCa || config.ssl_mode == SslMode::VerifyFull { - if let Some(ref root_cert) = config.ssl_root_cert { - options = options.ssl_root_cert(root_cert.as_str()); - } - } - - 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 { - let data_type = type_to_rw_type(&col.col_type)?; - let column_desc = if let Some(ref default_expr) = col.default { - // parse the value of "column_default" field in information_schema.columns, - // non number data type will be stored as "'value'::type" - let val_text = default_expr - .0 - .split("::") - .map(|s| s.trim_matches('\'')) - .next() - .expect("default value expression"); - - match ScalarImpl::from_text(val_text, &data_type) { - Ok(scalar) => ColumnDesc::named_with_default_value( - col.name.clone(), - ColumnId::placeholder(), - data_type.clone(), - Some(scalar), - ), - Err(err) => { - tracing::warn!(error=%err.as_report(), "failed to parse postgres default value expression, only constant is supported"); - ColumnDesc::named(col.name.clone(), ColumnId::placeholder(), data_type) - } - } - } else { - ColumnDesc::named(col.name.clone(), ColumnId::placeholder(), data_type) - }; - column_descs.push(column_desc); - } - - 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 { - column_descs, - pk_names, - }) - } - - pub fn column_descs(&self) -> &Vec { - &self.column_descs - } - - pub fn pk_names(&self) -> &Vec { - &self.pk_names - } -} - -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::VarBit(_) => { - return Err(anyhow!("VARBIT 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(name) => { - // NOTES: user-defined enum type is classified as `Unknown` - tracing::warn!("Unknown Postgres data type: {name}, map to varchar"); - DataType::Varchar - } - }; - - Ok(dtype) -} - pub struct PostgresExternalTableReader { rw_schema: Schema, field_names: String, From 7f979da6688fd311f166f8883fbfc54bf5e995d8 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 24 Nov 2024 23:12:08 +0800 Subject: [PATCH 44/56] decouple from pg config --- .../src/connector_common/postgres.rs | 32 ++++++++++++------- src/connector/src/source/cdc/external/mod.rs | 13 +++++++- .../src/source/cdc/external/postgres.rs | 14 +++++++- 3 files changed, 46 insertions(+), 13 deletions(-) diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs index cd9d8bbdf341..20720d8e4e7e 100644 --- a/src/connector/src/connector_common/postgres.rs +++ b/src/connector/src/connector_common/postgres.rs @@ -63,15 +63,25 @@ pub struct PostgresExternalTable { } impl PostgresExternalTable { - pub async fn connect(config: ExternalTableConfig) -> ConnectorResult { + pub async fn connect( + username: &str, + password: &str, + host: &str, + port: &str, + database: &str, + schema: &str, + table: &str, + ssl_mode: &SslMode, + ssl_root_cert: &Option, + ) -> ConnectorResult { tracing::debug!("connect to postgres external table"); let mut options = PgConnectOptions::new() - .username(&config.username) - .password(&config.password) - .host(&config.host) - .port(config.port.parse::().unwrap()) - .database(&config.database) - .ssl_mode(match config.ssl_mode { + .username(username) + .password(password) + .host(host) + .port(port.parse::().unwrap()) + .database(database) + .ssl_mode(match ssl_mode { SslMode::Disabled => PgSslMode::Disable, SslMode::Preferred => PgSslMode::Prefer, SslMode::Required => PgSslMode::Require, @@ -79,20 +89,20 @@ impl PostgresExternalTable { SslMode::VerifyFull => PgSslMode::VerifyFull, }); - if config.ssl_mode == SslMode::VerifyCa || config.ssl_mode == SslMode::VerifyFull { - if let Some(ref root_cert) = config.ssl_root_cert { + if *ssl_mode == SslMode::VerifyCa || *ssl_mode == SslMode::VerifyFull { + if let Some(ref root_cert) = ssl_root_cert { options = options.ssl_root_cert(root_cert.as_str()); } } let connection = PgPool::connect_with(options).await?; - let schema_discovery = SchemaDiscovery::new(connection, config.schema.as_str()); + let schema_discovery = SchemaDiscovery::new(connection, schema); // fetch column schema and primary key let empty_map = HashMap::new(); let table_schema = schema_discovery .discover_table( TableInfo { - name: config.table.clone(), + name: table.to_string(), of_type: None, }, &empty_map, diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index 84216ae4762b..916bb1b81d39 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -346,7 +346,18 @@ impl ExternalTableImpl { MySqlExternalTable::connect(config).await?, )), CdcSourceType::Postgres => Ok(ExternalTableImpl::Postgres( - PostgresExternalTable::connect(config).await?, + PostgresExternalTable::connect( + &config.username, + &config.password, + &config.host, + &config.port, + &config.database, + &config.schema, + &config.table, + &config.ssl_mode, + &config.ssl_root_cert, + ) + .await?, )), CdcSourceType::SqlServer => Ok(ExternalTableImpl::SqlServer( SqlServerExternalTable::connect(config).await?, diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 435889ff09b2..da4cc2f697b0 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -298,7 +298,19 @@ mod tests { encrypt: "false".to_string(), }; - let table = PostgresExternalTable::connect(config).await.unwrap(); + let table = PostgresExternalTable::connect( + &config.host, + &config.port, + &config.username, + &config.password, + &config.database, + &config.schema, + &config.table, + &config.ssl_mode, + &config.ssl_root_cert, + ) + .await + .unwrap(); println!("columns: {:?}", &table.column_descs); println!("primary keys: {:?}", &table.pk_names); From 70065f2580b149ab48e17b701904b58cdef9d743 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 24 Nov 2024 23:43:50 +0800 Subject: [PATCH 45/56] use PostgresExternalTable to validate --- .../src/connector_common/postgres.rs | 4 +- src/connector/src/sink/postgres.rs | 153 +++++++++--------- src/connector/src/source/cdc/external/mod.rs | 2 +- .../src/source/cdc/external/postgres.rs | 4 +- 4 files changed, 84 insertions(+), 79 deletions(-) diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs index 20720d8e4e7e..d450c65f5ea5 100644 --- a/src/connector/src/connector_common/postgres.rs +++ b/src/connector/src/connector_common/postgres.rs @@ -67,7 +67,7 @@ impl PostgresExternalTable { username: &str, password: &str, host: &str, - port: &str, + port: u16, database: &str, schema: &str, table: &str, @@ -79,7 +79,7 @@ impl PostgresExternalTable { .username(username) .password(password) .host(host) - .port(port.parse::().unwrap()) + .port(port) .database(database) .ssl_mode(match ssl_mode { SslMode::Disabled => PgSslMode::Disable, diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index d19ec54b0251..98696757fa0b 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashSet}; use std::sync::Arc; use anyhow::{anyhow, Context}; @@ -23,6 +23,7 @@ use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::Schema; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::DataType; +use risingwave_common::util::iter_util::ZipEqDebug; use serde_derive::Deserialize; use serde_with::{serde_as, DisplayFromStr}; use simd_json::prelude::ArrayTrait; @@ -33,7 +34,7 @@ use with_options::WithOptions; use super::{ SinkError, SinkWriterMetrics, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; -use crate::connector_common::{create_pg_client, SslMode}; +use crate::connector_common::{create_pg_client, PostgresExternalTable, SslMode}; use crate::parser::scalar_adapter::ScalarAdapter; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriterParam}; @@ -138,81 +139,85 @@ impl Sink for PostgresSink { "Primary key not defined for upsert Postgres sink (please define in `primary_key` field)"))); } - for field in self.schema.fields() { - check_data_type_compatibility(field.data_type())?; - } - - // Verify pg table schema matches rw table schema, and pk indices are valid - let table_name = &self.config.table; - let client = create_pg_client( - &self.config.user, - &self.config.password, - &self.config.host, - &self.config.port.to_string(), - &self.config.database, - &self.config.ssl_mode, - &self.config.ssl_root_cert, - ) - .await?; - - let result = client - .query( - " - SELECT - column_name, - EXISTS ( - SELECT 1 - FROM pg_index i - WHERE i.indrelid = c.table_name::regclass - AND i.indisprimary - AND column_name = ANY( - SELECT a.attname - FROM pg_attribute a - WHERE a.attrelid = i.indrelid - AND a.attnum = ANY(i.indkey) - ) - ) AS is_primary_key - FROM - information_schema.columns c - WHERE - table_name = $1 - ORDER BY - ordinal_position;", - &[&table_name], + // Verify our sink schema is compatible with Postgres + { + let pg_table = PostgresExternalTable::connect( + &self.config.user, + &self.config.password, + &self.config.host, + self.config.port, + &self.config.database, + &self.config.schema, + &self.config.table, + &self.config.ssl_mode, + &self.config.ssl_root_cert, ) - .await - .context("Failed to query Postgres for Sinking")?; - - let mut pg_schema = BTreeMap::new(); - for row in result { - let col_name: String = row.get(0); - let is_pk: bool = row.get(1); - pg_schema.insert(col_name, is_pk); - } + .await?; + + // Check that names and types match, order of columns doesn't matter. + { + let pg_columns = pg_table.column_descs(); + let sink_columns = self.schema.fields(); + if pg_columns.len() != sink_columns.len() { + return Err(SinkError::Config(anyhow!( + "Column count mismatch: Postgres table has {} columns, but sink schema has {} columns", + pg_columns.len(), + sink_columns.len() + ))); + } - for (i, field) in self.schema.fields().iter().enumerate() { - let col_name = &field.name; - let is_pk = pg_schema.get(col_name); - match is_pk { - None => return Err(SinkError::Config(anyhow!( - "Column `{}` not found in Postgres table `{}`", - col_name, - table_name - ))), - Some(is_pk) => - match (*is_pk, self.pk_indices.contains(&i)) { - (false, false) | (true, true) => continue, - (false, true) => return Err(SinkError::Config(anyhow!( - "Column `{}` in Postgres table `{}` is not a primary key, but RW schema defines it as a primary key", - col_name, - table_name - ))), - (true, false) => return Err(SinkError::Config(anyhow!( - "Column `{}` in Postgres table `{}` is a primary key, but RW schema does not define it as a primary key", - col_name, - table_name - ))), + let pg_columns_lookup = pg_columns + .iter() + .map(|c| (c.name.clone(), c.data_type.clone())) + .collect::>(); + for sink_column in sink_columns { + let pg_column = pg_columns_lookup.get(&sink_column.name); + match pg_column { + None => { + return Err(SinkError::Config(anyhow!( + "Column `{}` not found in Postgres table `{}`", + sink_column.name, + self.config.table + ))) + } + Some(pg_column) => { + if pg_column != &sink_column.data_type() { + return Err(SinkError::Config(anyhow!( + "Column `{}` in Postgres table `{}` has type `{}`, but sink schema defines it as type `{}`", + sink_column.name, + self.config.table, + pg_column, + sink_column.data_type() + ))); + } + } + } + } + } + + // check that pk matches + { + let pg_pk_names = pg_table.pk_names(); + let sink_pk_names = self + .pk_indices + .iter() + .map(|i| &self.schema.fields()[*i].name) + .collect::>(); + if pg_pk_names.len() != sink_pk_names.len() { + return Err(SinkError::Config(anyhow!( + "Primary key mismatch: Postgres table has primary key on columns {:?}, but sink schema defines primary key on columns {:?}", + pg_pk_names, + sink_pk_names + ))); + } + for name in pg_pk_names { + if !sink_pk_names.contains(name) { + return Err(SinkError::Config(anyhow!( + "Primary key mismatch: Postgres table has primary key on column `{}`, but sink schema does not define it as a primary key", + name + ))); } + } } } diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index 916bb1b81d39..871ea8b0f080 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -350,7 +350,7 @@ impl ExternalTableImpl { &config.username, &config.password, &config.host, - &config.port, + config.port.parse::().unwrap(), &config.database, &config.schema, &config.table, diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index da4cc2f697b0..c79cd3ace277 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -299,10 +299,10 @@ mod tests { }; let table = PostgresExternalTable::connect( - &config.host, - &config.port, &config.username, &config.password, + &config.host, + (&config.port).parse::().unwrap(), &config.database, &config.schema, &config.table, From c87cd7cbb15900daa2d750d3b24cc85f5f0e08f6 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 00:44:00 +0800 Subject: [PATCH 46/56] reuse scalar adapter --- src/connector/src/sink/postgres.rs | 41 ++++++++++++++++++++++-------- 1 file changed, 31 insertions(+), 10 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 98696757fa0b..2dce9bdd8038 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use anyhow::{anyhow, Context}; use async_trait::async_trait; use itertools::Itertools; +use risingwave_common::array::data_chunk_iter::RowRefIter; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::Schema; @@ -41,6 +42,20 @@ use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWrite pub const POSTGRES_SINK: &str = "postgres"; +macro_rules! rw_row_to_pg_values { + ($row:expr, $statement:expr) => { + $row.iter().enumerate().map(|(i, d)| { + d.and_then(|d| { + let ty = &$statement.params()[i]; + match ScalarAdapter::from_scalar(d, ty) { + Ok(scalar) => Some(scalar), + Err(e) => None, + } + }) + }) + }; +} + #[serde_as] #[derive(Clone, Debug, Deserialize)] pub struct PostgresConfig { @@ -356,7 +371,10 @@ impl PostgresSinkWriter { match op { Op::Insert => { self.client - .execute_raw(&self.insert_statement, row.iter()) + .execute_raw( + &self.insert_statement, + rw_row_to_pg_values!(row, self.insert_statement), + ) .await?; } Op::UpdateInsert | Op::Delete | Op::UpdateDelete => { @@ -375,13 +393,7 @@ impl PostgresSinkWriter { self.client .execute_raw( &self.insert_statement, - row.iter(), - // row - // .iter() - // .enumerate() - // .map(|(i, d)| d.map(|d| { - // ScalarAdapter::from_scalar(d) - // }))) + rw_row_to_pg_values!(row, self.insert_statement), ) .await?; } @@ -392,7 +404,13 @@ impl PostgresSinkWriter { // in that case it needs to be `INSERTED` rather than UPDATED. // On the other hand, if the record is there, it should be `UPDATED`. self.client - .execute_raw(self.merge_statement.as_ref().unwrap(), row.iter()) + .execute_raw( + self.merge_statement.as_ref().unwrap(), + rw_row_to_pg_values!( + row, + self.merge_statement.as_ref().unwrap() + ), + ) .await?; } Op::Delete => { @@ -400,7 +418,10 @@ impl PostgresSinkWriter { self.client .execute_raw( self.delete_statement.as_ref().unwrap(), - row.project(&self.pk_indices).iter(), + rw_row_to_pg_values!( + row.project(&self.pk_indices), + self.delete_statement.as_ref().unwrap() + ), ) .await?; } From 38291194ebf9751b14af1fa2fd11385d264d5ead Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 01:02:42 +0800 Subject: [PATCH 47/56] fix update sanity check + fix non-append-only pk check --- src/connector/src/connector_common/postgres.rs | 5 +++-- src/connector/src/sink/postgres.rs | 6 ++++-- src/connector/src/source/cdc/external/mod.rs | 1 + src/connector/src/source/cdc/external/postgres.rs | 1 + 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs index d450c65f5ea5..1da128cfb545 100644 --- a/src/connector/src/connector_common/postgres.rs +++ b/src/connector/src/connector_common/postgres.rs @@ -73,6 +73,7 @@ impl PostgresExternalTable { table: &str, ssl_mode: &SslMode, ssl_root_cert: &Option, + is_append_only: bool, ) -> ConnectorResult { tracing::debug!("connect to postgres external table"); let mut options = PgConnectOptions::new() @@ -140,8 +141,8 @@ impl PostgresExternalTable { column_descs.push(column_desc); } - if table_schema.primary_key_constraints.is_empty() { - return Err(anyhow!("Postgres table doesn't define the primary key").into()); + if !is_append_only && table_schema.primary_key_constraints.is_empty() { + return Err(anyhow!("Postgres table should define the primary key for non-append-only tables").into()); } let mut pk_names = vec![]; table_schema.primary_key_constraints.iter().for_each(|pk| { diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 2dce9bdd8038..f0f1aaf8fea2 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -166,6 +166,7 @@ impl Sink for PostgresSink { &self.config.table, &self.config.ssl_mode, &self.config.ssl_root_cert, + self.is_append_only, ) .await?; @@ -414,7 +415,6 @@ impl PostgresSinkWriter { .await?; } Op::Delete => { - unmatched_update_insert -= 1; self.client .execute_raw( self.delete_statement.as_ref().unwrap(), @@ -425,7 +425,9 @@ impl PostgresSinkWriter { ) .await?; } - Op::UpdateDelete => {} + Op::UpdateDelete => { + unmatched_update_insert -= 1; + } } } } diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index 871ea8b0f080..874e90bc4fc1 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -356,6 +356,7 @@ impl ExternalTableImpl { &config.table, &config.ssl_mode, &config.ssl_root_cert, + false, ) .await?, )), diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index c79cd3ace277..091193f29c32 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -308,6 +308,7 @@ mod tests { &config.table, &config.ssl_mode, &config.ssl_root_cert, + false, ) .await .unwrap(); From 8caa1f0d55a3796abaabc288d7e3ded73230f8b2 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 14:41:18 +0800 Subject: [PATCH 48/56] loosen validation --- e2e_test/sink/postgres_sink.slt | 75 +++++++++++++++++++ .../src/connector_common/postgres.rs | 5 +- src/connector/src/parser/scalar_adapter.rs | 19 ++++- src/connector/src/sink/postgres.rs | 4 +- 4 files changed, 97 insertions(+), 6 deletions(-) diff --git a/e2e_test/sink/postgres_sink.slt b/e2e_test/sink/postgres_sink.slt index 5b721b53a719..673a3235a871 100644 --- a/e2e_test/sink/postgres_sink.slt +++ b/e2e_test/sink/postgres_sink.slt @@ -358,5 +358,80 @@ DROP SINK postgres_rw_types_sink_append_only; statement ok DROP TABLE rw_types_table_append_only; +system ok +PGDATABASE=sink_test psql -c "DROP TABLE pg_types_table" + +################### Test pg special types + +system ok +PGDATABASE=sink_test psql -c "CREATE TYPE MY_ENUM AS ENUM ('a', 'b', 'c');" + +system ok +PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB, + uuid_column UUID, + numeric_column NUMERIC, + myenum_column MY_ENUM, + numeric_decimal_array_column NUMERIC[], + numeric_utf8_array_column NUMERIC[], + enum_array_column MY_ENUM[], + int_column int[] +)" + +statement ok +CREATE TABLE rw_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB, + uuid_column VARCHAR, + numeric_column VARCHAR, + myenum_column VARCHAR, + numeric_decimal_array_column DECIMAL[], + numeric_utf8_array_column VARCHAR[], + enum_array_column VARCHAR[], + int_column int[] +) + +statement ok +CREATE SINK rw_types_to_pg_types FROM rw_types_table WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='upsert', + primary_key='id', +); + +################### Drop DB + system ok PGDATABASE=postgres psql -c "DROP DATABASE sink_test" \ No newline at end of file diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs index 1da128cfb545..6dfa30714e8c 100644 --- a/src/connector/src/connector_common/postgres.rs +++ b/src/connector/src/connector_common/postgres.rs @@ -142,7 +142,10 @@ impl PostgresExternalTable { } if !is_append_only && table_schema.primary_key_constraints.is_empty() { - return Err(anyhow!("Postgres table should define the primary key for non-append-only tables").into()); + return Err(anyhow!( + "Postgres table should define the primary key for non-append-only tables" + ) + .into()); } let mut pk_names = vec![]; table_schema.primary_key_constraints.iter().for_each(|pk| { diff --git a/src/connector/src/parser/scalar_adapter.rs b/src/connector/src/parser/scalar_adapter.rs index e01d72e4285c..3f37cdbadca2 100644 --- a/src/connector/src/parser/scalar_adapter.rs +++ b/src/connector/src/parser/scalar_adapter.rs @@ -316,6 +316,22 @@ impl ScalarAdapter { } } +pub fn validate_pg_type_to_rw_type(pg_type: &DataType, rw_type: &DataType) -> bool { + if pg_type == rw_type { + return true; + } + match rw_type { + DataType::Varchar => matches!(pg_type, DataType::Decimal | DataType::Int256), + DataType::List(box DataType::Varchar) => { + matches!( + pg_type, + DataType::List(box (DataType::Decimal | DataType::Int256)) + ) + } + _ => false, + } +} + fn pg_numeric_is_special(val: &PgNumeric) -> bool { matches!( val, @@ -378,6 +394,3 @@ fn rw_numeric_to_pg_numeric(val: Decimal) -> PgNumeric { Decimal::NaN => PgNumeric::NaN, } } -// pub fn row_to_scalar_iter<'a>( -// row: impl Row -// ) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index f0f1aaf8fea2..11a389045f3f 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -36,7 +36,7 @@ use super::{ SinkError, SinkWriterMetrics, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; use crate::connector_common::{create_pg_client, PostgresExternalTable, SslMode}; -use crate::parser::scalar_adapter::ScalarAdapter; +use crate::parser::scalar_adapter::{validate_pg_type_to_rw_type, ScalarAdapter}; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriterParam}; @@ -197,7 +197,7 @@ impl Sink for PostgresSink { ))) } Some(pg_column) => { - if pg_column != &sink_column.data_type() { + if !validate_pg_type_to_rw_type(pg_column, &sink_column.data_type()) { return Err(SinkError::Config(anyhow!( "Column `{}` in Postgres table `{}` has type `{}`, but sink schema defines it as type `{}`", sink_column.name, From e5b47f60a223146802f7843f1d15a42f33cee476 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 15:11:44 +0800 Subject: [PATCH 49/56] fix --- e2e_test/sink/postgres_sink.slt | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/e2e_test/sink/postgres_sink.slt b/e2e_test/sink/postgres_sink.slt index 673a3235a871..a5c029e71a44 100644 --- a/e2e_test/sink/postgres_sink.slt +++ b/e2e_test/sink/postgres_sink.slt @@ -431,6 +431,16 @@ CREATE SINK rw_types_to_pg_types FROM rw_types_table WITH ( primary_key='id', ); +statement ok +INSERT INTO rw_types_table (id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column, uuid_column, numeric_column, myenum_column, numeric_decimal_array_column, numeric_utf8_array_column, enum_array_column, int_column) VALUES + (1, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', '448be8d9-297f-4514-85c6-a360e82ae331', '123', 'a', ARRAY[1, 2, 3], ARRAY['1', '2', '3'], ARRAY['a', 'b', 'c'], ARRAY[1, 2, 3]), + (2, 'Varcharvalue2', 'Textvalue2', 234, 567, 890, 'NAN'::decimal, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}', '14b4431e-203a-452b-a331-4d8a0d8f952e', '456', 'b', ARRAY[10, 20, 30], ARRAY['10', '20', '30'], ARRAY['a', 'b', 'c'], ARRAY[10, 20, 30]), + (3, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, '+INF'::decimal, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', '024d2719-ca29-45e1-bc58-4ed38777f4bf', '789', 'c', ARRAY[100, 200, 300], ARRAY['100', '200', '300'], ARRAY['a', 'b', 'c'], ARRAY[100, 200, 300]); + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column from pg_types_table order by integer_column;'); +---- + ################### Drop DB system ok From 46fd3dbd8f4facefe1e8064f4ffc74de7bd48167 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 20:58:33 +0800 Subject: [PATCH 50/56] handle edge cases --- e2e_test/sink/postgres_sink.slt | 18 +++- .../src/connector_common/postgres.rs | 85 ++++++++++++++++++- src/connector/src/sink/postgres.rs | 51 ++++++++--- 3 files changed, 137 insertions(+), 17 deletions(-) diff --git a/e2e_test/sink/postgres_sink.slt b/e2e_test/sink/postgres_sink.slt index a5c029e71a44..c680d3992d68 100644 --- a/e2e_test/sink/postgres_sink.slt +++ b/e2e_test/sink/postgres_sink.slt @@ -437,9 +437,23 @@ INSERT INTO rw_types_table (id, varchar_column, text_column, integer_column, sma (2, 'Varcharvalue2', 'Textvalue2', 234, 567, 890, 'NAN'::decimal, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}', '14b4431e-203a-452b-a331-4d8a0d8f952e', '456', 'b', ARRAY[10, 20, 30], ARRAY['10', '20', '30'], ARRAY['a', 'b', 'c'], ARRAY[10, 20, 30]), (3, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, '+INF'::decimal, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', '024d2719-ca29-45e1-bc58-4ed38777f4bf', '789', 'c', ARRAY[100, 200, 300], ARRAY['100', '200', '300'], ARRAY['a', 'b', 'c'], ARRAY[100, 200, 300]); -query I -select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column from pg_types_table order by integer_column;'); +statement ok +flush; + +# NOTE(kwannoel): postgres_query doesn't support struct etc... yet. +system ok +PGDATABASE=sink_test psql --tuples-only -c "select * from pg_types_table order by id;" ---- + 1 | Varcharvalue1 | Textvalue1 | 123 | 456 | 789 | 12.34 | 56.78 | 90.12 | t | 2023-05-22 | 12:34:56 | 2023-05-22 12:34:56 | 1 day | {"key": "value"} | 448be8d9-297f-4514-85c6-a360e82ae331 | 123 | a | {1,2,3} | {1,2,3} | {a,b,c} | {1,2,3} + 2 | Varcharvalue2 | Textvalue2 | 234 | 567 | 890 | NaN | 67.89 | 1.23 | f | 2023-05-23 | 23:45:01 | 2023-05-23 23:45:01 | 2 days | {"key": "value2"} | 14b4431e-203a-452b-a331-4d8a0d8f952e | 456 | b | {10,20,30} | {10,20,30} | {a,b,c} | {10,20,30} + 3 | Varcharvalue1 | Textvalue1 | 123 | 456 | 789 | Infinity | 56.78 | 90.12 | t | 2023-05-22 | 12:34:56 | 2023-05-22 12:34:56 | 1 day | {"key": "value"} | 024d2719-ca29-45e1-bc58-4ed38777f4bf | 789 | c | {100,200,300} | {100,200,300} | {a,b,c} | {100,200,300} + + +statement ok +DROP SINK rw_types_to_pg_types; + +statement ok +DROP TABLE rw_types_table; ################### Drop DB diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs index 6dfa30714e8c..b1358a6efe4a 100644 --- a/src/connector/src/connector_common/postgres.rs +++ b/src/connector/src/connector_common/postgres.rs @@ -15,12 +15,15 @@ use std::collections::HashMap; use std::fmt; +use tokio_postgres::types::Kind as PgKind; + use anyhow::anyhow; use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; use postgres_openssl::MakeTlsConnector; +use risingwave_common::bail; use risingwave_common::catalog::{ColumnDesc, ColumnId}; use risingwave_common::types::{DataType, ScalarImpl, StructType}; -use sea_schema::postgres::def::{ColumnType, TableInfo}; +use sea_schema::postgres::def::{ColumnType, TableInfo, Type as SeaType}; use sea_schema::postgres::discovery::SchemaDiscovery; use serde_derive::Deserialize; use sqlx::postgres::{PgConnectOptions, PgSslMode}; @@ -60,6 +63,7 @@ impl Default for SslMode { pub struct PostgresExternalTable { column_descs: Vec, pk_names: Vec, + column_name_to_pg_type: HashMap, } impl PostgresExternalTable { @@ -110,6 +114,7 @@ impl PostgresExternalTable { ) .await?; + let mut column_name_to_pg_type = HashMap::new(); let mut column_descs = vec![]; for col in &table_schema.columns { let data_type = type_to_rw_type(&col.col_type)?; @@ -138,6 +143,10 @@ impl PostgresExternalTable { } else { ColumnDesc::named(col.name.clone(), ColumnId::placeholder(), data_type) }; + { + let pg_type = Self::discovered_type_to_pg_type(&col.col_type)?; + column_name_to_pg_type.insert(col.name.clone(), pg_type); + } column_descs.push(column_desc); } @@ -155,6 +164,7 @@ impl PostgresExternalTable { Ok(Self { column_descs, pk_names, + column_name_to_pg_type, }) } @@ -165,6 +175,79 @@ impl PostgresExternalTable { pub fn pk_names(&self) -> &Vec { &self.pk_names } + + pub fn column_name_to_pg_type(&self) -> &HashMap { + &self.column_name_to_pg_type + } + + fn discovered_type_to_pg_type( + discovered_type: &SeaType, + ) -> anyhow::Result { + use tokio_postgres::types::Type as PgType; + match discovered_type { + SeaType::SmallInt => Ok(PgType::INT2), + SeaType::Integer => Ok(PgType::INT4), + SeaType::BigInt => Ok(PgType::INT8), + SeaType::Decimal(_) => Ok(PgType::NUMERIC), + SeaType::Numeric(_) => Ok(PgType::NUMERIC), + SeaType::Real => Ok(PgType::FLOAT4), + SeaType::DoublePrecision => Ok(PgType::FLOAT8), + SeaType::Varchar(_) => Ok(PgType::VARCHAR), + SeaType::Char(_) => Ok(PgType::CHAR), + SeaType::Text => Ok(PgType::TEXT), + SeaType::Bytea => Ok(PgType::BYTEA), + SeaType::Timestamp(_) => Ok(PgType::TIMESTAMP), + SeaType::TimestampWithTimeZone(_) => Ok(PgType::TIMESTAMPTZ), + SeaType::Date => Ok(PgType::DATE), + SeaType::Time(_) => Ok(PgType::TIME), + SeaType::TimeWithTimeZone(_) => Ok(PgType::TIMETZ), + SeaType::Interval(_) => Ok(PgType::INTERVAL), + SeaType::Boolean => Ok(PgType::BOOL), + SeaType::Point => Ok(PgType::POINT), + SeaType::Uuid => Ok(PgType::UUID), + SeaType::JsonBinary => Ok(PgType::JSONB), + SeaType::Array(t) => { + let Some(t) = t.col_type.as_ref() else { + bail!("missing array type") + }; + match t.as_ref() { + // RW only supports 1 level of nesting. + SeaType::SmallInt => Ok(PgType::INT2_ARRAY), + SeaType::Integer => Ok(PgType::INT4_ARRAY), + SeaType::BigInt => Ok(PgType::INT8_ARRAY), + SeaType::Decimal(_) => Ok(PgType::NUMERIC_ARRAY), + SeaType::Numeric(_) => Ok(PgType::NUMERIC_ARRAY), + SeaType::Real => Ok(PgType::FLOAT4_ARRAY), + SeaType::DoublePrecision => Ok(PgType::FLOAT8_ARRAY), + SeaType::Varchar(_) => Ok(PgType::VARCHAR_ARRAY), + SeaType::Char(_) => Ok(PgType::CHAR_ARRAY), + SeaType::Text => Ok(PgType::TEXT_ARRAY), + SeaType::Bytea => Ok(PgType::BYTEA_ARRAY), + SeaType::Timestamp(_) => Ok(PgType::TIMESTAMP_ARRAY), + SeaType::TimestampWithTimeZone(_) => Ok(PgType::TIMESTAMPTZ_ARRAY), + SeaType::Date => Ok(PgType::DATE_ARRAY), + SeaType::Time(_) => Ok(PgType::TIME_ARRAY), + SeaType::TimeWithTimeZone(_) => Ok(PgType::TIMETZ_ARRAY), + SeaType::Interval(_) => Ok(PgType::INTERVAL_ARRAY), + SeaType::Boolean => Ok(PgType::BOOL_ARRAY), + SeaType::Point => Ok(PgType::POINT_ARRAY), + SeaType::Uuid => Ok(PgType::UUID_ARRAY), + SeaType::JsonBinary => Ok(PgType::JSONB_ARRAY), + SeaType::Array(_) => bail!("nested array type is not supported"), + SeaType::Unknown(name) => { + // Treat as enum type + Ok(PgType::new(name.clone(), 0, PgKind::Array(PgType::new(name.clone(), 0, PgKind::Enum(vec![]), "".into())), "".into())) + } + _ => bail!("unsupported array type: {:?}", t), + } + } + SeaType::Unknown(name) => { + // Treat as enum type + Ok(PgType::new(name.clone(), 0, PgKind::Enum(vec![]), "".into())) + } + _ => bail!("unsupported type: {:?}", discovered_type), + } + } } impl fmt::Display for SslMode { diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 11a389045f3f..30a9a0bc041d 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -291,7 +291,7 @@ pub struct PostgresSinkWriter { impl PostgresSinkWriter { async fn new( config: PostgresConfig, - schema: Schema, + mut schema: Schema, pk_indices: Vec, is_append_only: bool, ) -> Result { @@ -306,15 +306,43 @@ impl PostgresSinkWriter { ) .await?; + // Rewrite schema types for serialization + let schema_types = { + let pg_table = PostgresExternalTable::connect( + &config.user, + &config.password, + &config.host, + config.port, + &config.database, + &config.schema, + &config.table, + &config.ssl_mode, + &config.ssl_root_cert, + is_append_only, + ) + .await?; + let name_to_type = pg_table.column_name_to_pg_type(); + let mut schema_types = Vec::with_capacity(schema.fields.len()); + for field in &mut schema.fields[..] { + let field_name = &field.name; + let actual_data_type = name_to_type.get(field_name).map(|t| (*t).clone()); + let actual_data_type = actual_data_type + .ok_or_else(|| { + SinkError::Config(anyhow!( + "Column `{}` not found in sink schema", + field_name + )) + })? + .clone(); + schema_types.push(actual_data_type); + } + schema_types + }; + let insert_statement = { - let insert_types = schema - .fields() - .iter() - .map(|field| field.data_type().to_pg_type()) - .collect_vec(); let insert_sql = create_insert_sql(&schema, &config.table); client - .prepare_typed(&insert_sql, &insert_types) + .prepare_typed(&insert_sql, &schema_types) .await .context("Failed to prepare insert statement")? }; @@ -324,7 +352,7 @@ impl PostgresSinkWriter { } else { let delete_types = pk_indices .iter() - .map(|i| schema.fields()[*i].data_type().to_pg_type()) + .map(|i| schema_types[*i].clone()) .collect_vec(); let delete_sql = create_delete_sql(&schema, &config.table, &pk_indices); Some( @@ -338,15 +366,10 @@ impl PostgresSinkWriter { let merge_statement = if is_append_only { None } else { - let merge_types = schema - .fields - .iter() - .map(|field| field.data_type().to_pg_type()) - .collect_vec(); let merge_sql = create_upsert_sql(&schema, &config.table, &pk_indices); Some( client - .prepare_typed(&merge_sql, &merge_types) + .prepare_typed(&merge_sql, &schema_types) .await .context("Failed to prepare merge statement")?, ) From 364722a9517c9287f9ef7fcebd9682deb106e5b7 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 21:07:19 +0800 Subject: [PATCH 51/56] fmt + fix warn --- .../src/connector_common/postgres.rs | 23 ++++++++--- src/connector/src/parser/scalar_adapter.rs | 1 - src/connector/src/sink/postgres.rs | 41 ++----------------- .../src/source/cdc/external/postgres.rs | 16 ++------ 4 files changed, 26 insertions(+), 55 deletions(-) diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs index b1358a6efe4a..2701ec142b92 100644 --- a/src/connector/src/connector_common/postgres.rs +++ b/src/connector/src/connector_common/postgres.rs @@ -15,8 +15,6 @@ use std::collections::HashMap; use std::fmt; -use tokio_postgres::types::Kind as PgKind; - use anyhow::anyhow; use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; use postgres_openssl::MakeTlsConnector; @@ -29,11 +27,11 @@ use serde_derive::Deserialize; use sqlx::postgres::{PgConnectOptions, PgSslMode}; use sqlx::PgPool; use thiserror_ext::AsReport; +use tokio_postgres::types::Kind as PgKind; use tokio_postgres::{Client as PgClient, NoTls}; use super::maybe_tls_connector::MaybeMakeTlsConnector; use crate::error::ConnectorResult; -use crate::source::cdc::external::ExternalTableConfig; #[derive(Debug, Clone, PartialEq, Deserialize)] #[serde(rename_all = "lowercase")] @@ -236,14 +234,29 @@ impl PostgresExternalTable { SeaType::Array(_) => bail!("nested array type is not supported"), SeaType::Unknown(name) => { // Treat as enum type - Ok(PgType::new(name.clone(), 0, PgKind::Array(PgType::new(name.clone(), 0, PgKind::Enum(vec![]), "".into())), "".into())) + Ok(PgType::new( + name.clone(), + 0, + PgKind::Array(PgType::new( + name.clone(), + 0, + PgKind::Enum(vec![]), + "".into(), + )), + "".into(), + )) } _ => bail!("unsupported array type: {:?}", t), } } SeaType::Unknown(name) => { // Treat as enum type - Ok(PgType::new(name.clone(), 0, PgKind::Enum(vec![]), "".into())) + Ok(PgType::new( + name.clone(), + 0, + PgKind::Enum(vec![]), + "".into(), + )) } _ => bail!("unsupported type: {:?}", discovered_type), } diff --git a/src/connector/src/parser/scalar_adapter.rs b/src/connector/src/parser/scalar_adapter.rs index 3f37cdbadca2..004ebe2cdfcf 100644 --- a/src/connector/src/parser/scalar_adapter.rs +++ b/src/connector/src/parser/scalar_adapter.rs @@ -17,7 +17,6 @@ use std::str::FromStr; use anyhow::anyhow; use bytes::BytesMut; use pg_bigdecimal::PgNumeric; -use risingwave_common::row::Row; use risingwave_common::types::{DataType, Decimal, Int256, ListValue, ScalarImpl, ScalarRefImpl}; use thiserror_ext::AsReport; use tokio_postgres::types::{to_sql_checked, FromSql, IsNull, Kind, ToSql, Type}; diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 30a9a0bc041d..43f36988a185 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -18,19 +18,15 @@ use std::sync::Arc; use anyhow::{anyhow, Context}; use async_trait::async_trait; use itertools::Itertools; -use risingwave_common::array::data_chunk_iter::RowRefIter; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::Schema; use risingwave_common::row::{Row, RowExt}; -use risingwave_common::types::DataType; -use risingwave_common::util::iter_util::ZipEqDebug; use serde_derive::Deserialize; use serde_with::{serde_as, DisplayFromStr}; use simd_json::prelude::ArrayTrait; use thiserror_ext::AsReport; use tokio_postgres::Statement; -use with_options::WithOptions; use super::{ SinkError, SinkWriterMetrics, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, @@ -49,7 +45,10 @@ macro_rules! rw_row_to_pg_values { let ty = &$statement.params()[i]; match ScalarAdapter::from_scalar(d, ty) { Ok(scalar) => Some(scalar), - Err(e) => None, + Err(e) => { + tracing::error!(error=%e.as_report(), scalar=?d, "Failed to convert scalar to pg value"); + None + } } }) }) @@ -411,7 +410,6 @@ impl PostgresSinkWriter { let mut unmatched_update_insert = 0; for chunk in self.buffer.drain() { for (op, row) in chunk.rows() { - let mut expect_update_delete = false; match op { Op::Insert => { self.client @@ -491,37 +489,6 @@ impl SinkWriter for PostgresSinkWriter { } } -fn data_type_not_supported(data_type_name: &str) -> SinkError { - SinkError::Postgres(anyhow!(format!( - "{data_type_name} is not supported in SQL Server" - ))) -} - -fn check_data_type_compatibility(data_type: DataType) -> Result<()> { - match data_type { - DataType::Boolean - | DataType::Int16 - | DataType::Int32 - | DataType::Int64 - | DataType::Float32 - | DataType::Float64 - | DataType::Decimal - | DataType::Date - | DataType::Varchar - | DataType::Time - | DataType::Timestamp - | DataType::Timestamptz - | DataType::Jsonb - | DataType::Interval - | DataType::Bytea => Ok(()), - DataType::Struct(_) => Err(data_type_not_supported("Struct")), - DataType::List(_) => Err(data_type_not_supported("List")), - DataType::Serial => Err(data_type_not_supported("Serial")), - DataType::Int256 => Err(data_type_not_supported("Int256")), - DataType::Map(_) => Err(data_type_not_supported("Map")), - } -} - fn create_insert_sql(schema: &Schema, table_name: &str) -> String { let columns: String = schema .fields() diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 091193f29c32..40f027d27858 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -13,34 +13,26 @@ // limitations under the License. use std::cmp::Ordering; -use std::collections::HashMap; -use anyhow::{anyhow, Context}; +use anyhow::Context; use futures::stream::BoxStream; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema}; +use risingwave_common::catalog::Schema; use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::{DataType, ScalarImpl, StructType}; use risingwave_common::util::iter_util::ZipEqFast; -use sea_schema::postgres::def::{ColumnType, TableInfo}; -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 crate::connector_common::postgres::type_to_rw_type; -use crate::connector_common::{create_pg_client, PostgresExternalTable}; +use crate::connector_common::create_pg_client; #[cfg(not(madsim))] use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::postgres_row_to_owned_row; use crate::parser::scalar_adapter::ScalarAdapter; use crate::source::cdc::external::{ CdcOffset, CdcOffsetParseFunc, DebeziumOffset, ExternalTableConfig, ExternalTableReader, - SchemaTableName, SslMode, + SchemaTableName, }; #[derive(Debug, Clone, Default, PartialEq, Serialize, Deserialize)] From fac753a87789d5c4dc189aec68050a121eb25b37 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 21:25:34 +0800 Subject: [PATCH 52/56] fix unit test --- src/connector/src/sink/postgres.rs | 1 + .../src/source/cdc/external/postgres.rs | 7 ++--- src/connector/with_options_sink.yaml | 27 ------------------- 3 files changed, 5 insertions(+), 30 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index 43f36988a185..a1a789316717 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -546,6 +546,7 @@ mod tests { use expect_test::{expect, Expect}; use risingwave_common::catalog::Field; + use risingwave_common::types::DataType; use super::*; diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 40f027d27858..9466190a3fa2 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -268,8 +268,9 @@ mod tests { use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; + use crate::connector_common::PostgresExternalTable; use crate::source::cdc::external::postgres::{ - PostgresExternalTable, PostgresExternalTableReader, PostgresOffset, + PostgresExternalTableReader, PostgresOffset, }; use crate::source::cdc::external::{ExternalTableConfig, ExternalTableReader, SchemaTableName}; @@ -305,8 +306,8 @@ mod tests { .await .unwrap(); - println!("columns: {:?}", &table.column_descs); - println!("primary keys: {:?}", &table.pk_names); + println!("columns: {:?}", &table.column_descs()); + println!("primary keys: {:?}", &table.pk_names()); } #[test] diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 713ac58ee836..88f2e64cce2e 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -905,33 +905,6 @@ NatsConfig: - name: r#type field_type: String required: true -PostgresConfig: - fields: - - name: host - field_type: String - required: true - - name: port - field_type: u16 - required: true - - name: user - field_type: String - required: true - - name: password - field_type: String - required: true - - name: database - field_type: String - required: true - - name: table - field_type: String - required: true - - name: max_batch_rows - field_type: usize - required: false - default: '1024' - - name: r#type - field_type: String - required: true PulsarConfig: fields: - name: properties.retry.max From 7e75e4ce22577d516a9d0efbb2990cc52ec09884 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 21:57:42 +0800 Subject: [PATCH 53/56] fix test + check --- src/connector/src/sink/postgres.rs | 4 +--- src/connector/src/source/cdc/external/postgres.rs | 4 +--- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index a1a789316717..b7568a44ea82 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -620,9 +620,7 @@ mod tests { let sql = create_upsert_sql(&schema, table_name, &[1]); check( sql, - expect![ - "INSERT INTO test_table (a, b) VALUES ($1, $2) on conflict do update set a = $1" - ], + expect!["INSERT INTO test_table (a, b) VALUES ($1, $2) on conflict (b) do update set a = $1"], ); } } diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 9466190a3fa2..830241817006 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -269,9 +269,7 @@ mod tests { use risingwave_common::types::{DataType, ScalarImpl}; use crate::connector_common::PostgresExternalTable; - use crate::source::cdc::external::postgres::{ - PostgresExternalTableReader, PostgresOffset, - }; + use crate::source::cdc::external::postgres::{PostgresExternalTableReader, PostgresOffset}; use crate::source::cdc::external::{ExternalTableConfig, ExternalTableReader, SchemaTableName}; #[ignore] From 6f93bb0220c2d2e1544867db99404343495c25bd Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 22:26:02 +0800 Subject: [PATCH 54/56] fix compile errors for madsim + fix lint --- src/connector/src/connector_common/mod.rs | 1 + src/connector/src/source/cdc/external/mod.rs | 1 - src/connector/src/source/cdc/external/postgres.rs | 2 +- 3 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/connector/src/connector_common/mod.rs b/src/connector/src/connector_common/mod.rs index e7f43bd3f902..a4a2c5a02e3f 100644 --- a/src/connector/src/connector_common/mod.rs +++ b/src/connector/src/connector_common/mod.rs @@ -25,6 +25,7 @@ pub use common::{ }; mod iceberg; +#[cfg(not(madsim))] mod maybe_tls_connector; pub mod postgres; diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index 874e90bc4fc1..115b1be99d21 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -16,7 +16,6 @@ pub mod mock_external_table; pub mod postgres; pub mod sql_server; -#[cfg(not(madsim))] pub mod mysql; use std::collections::{BTreeMap, HashMap}; diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 830241817006..d3684030030b 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -293,7 +293,7 @@ mod tests { &config.username, &config.password, &config.host, - (&config.port).parse::().unwrap(), + config.port.parse::().unwrap(), &config.database, &config.schema, &config.table, From dc7f788d493e75f4949e1f9a705a4479be537893 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 22:46:30 +0800 Subject: [PATCH 55/56] more madsim fix --- src/connector/src/connector_common/postgres.rs | 1 + src/connector/src/source/cdc/external/postgres.rs | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs index 2701ec142b92..73c8241aae83 100644 --- a/src/connector/src/connector_common/postgres.rs +++ b/src/connector/src/connector_common/postgres.rs @@ -30,6 +30,7 @@ use thiserror_ext::AsReport; use tokio_postgres::types::Kind as PgKind; use tokio_postgres::{Client as PgClient, NoTls}; +#[cfg(not(madsim))] use super::maybe_tls_connector::MaybeMakeTlsConnector; use crate::error::ConnectorResult; diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index d3684030030b..ecc503934bb6 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -26,7 +26,6 @@ use serde_derive::{Deserialize, Serialize}; use tokio_postgres::types::PgLsn; use crate::connector_common::create_pg_client; -#[cfg(not(madsim))] use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::postgres_row_to_owned_row; use crate::parser::scalar_adapter::ScalarAdapter; From e57cbd55410f69b5b318b225cc49e29d8e6f4036 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 25 Nov 2024 23:02:57 +0800 Subject: [PATCH 56/56] fix docs --- .../src/connector_common/postgres.rs | 3 +++ src/connector/src/sink/postgres.rs | 20 ++++++++----------- 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs index 73c8241aae83..fc79bf9b961e 100644 --- a/src/connector/src/connector_common/postgres.rs +++ b/src/connector/src/connector_common/postgres.rs @@ -179,6 +179,9 @@ impl PostgresExternalTable { &self.column_name_to_pg_type } + // We use `sea-schema` for table schema discovery. + // So we have to map `sea-schema` pg types + // to `tokio-postgres` pg types (which we use for query binding). fn discovered_type_to_pg_type( discovered_type: &SeaType, ) -> anyhow::Result { diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs index b7568a44ea82..3f722e1016ef 100644 --- a/src/connector/src/sink/postgres.rs +++ b/src/connector/src/sink/postgres.rs @@ -284,7 +284,7 @@ pub struct PostgresSinkWriter { buffer: Buffer, insert_statement: Statement, delete_statement: Option, - merge_statement: Option, + upsert_statement: Option, } impl PostgresSinkWriter { @@ -362,15 +362,15 @@ impl PostgresSinkWriter { ) }; - let merge_statement = if is_append_only { + let upsert_statement = if is_append_only { None } else { - let merge_sql = create_upsert_sql(&schema, &config.table, &pk_indices); + let upsert_sql = create_upsert_sql(&schema, &config.table, &pk_indices); Some( client - .prepare_typed(&merge_sql, &schema_types) + .prepare_typed(&upsert_sql, &schema_types) .await - .context("Failed to prepare merge statement")?, + .context("Failed to prepare upsert statement")?, ) }; @@ -382,7 +382,7 @@ impl PostgresSinkWriter { buffer: Buffer::new(), insert_statement, delete_statement, - merge_statement, + upsert_statement, }; Ok(writer) } @@ -421,16 +421,12 @@ impl PostgresSinkWriter { } Op::UpdateInsert => { unmatched_update_insert += 1; - // NOTE(kwannoel): Here we use `MERGE` rather than `UPDATE/INSERT` directly. - // This is because the downstream db could have cleaned the old record, - // in that case it needs to be `INSERTED` rather than UPDATED. - // On the other hand, if the record is there, it should be `UPDATED`. self.client .execute_raw( - self.merge_statement.as_ref().unwrap(), + self.upsert_statement.as_ref().unwrap(), rw_row_to_pg_values!( row, - self.merge_statement.as_ref().unwrap() + self.upsert_statement.as_ref().unwrap() ), ) .await?;