From 250c4aae7ce2e5f3a7ceb2fcfa3078afdad71aae Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 22 Oct 2024 17:20:46 +0800 Subject: [PATCH 01/48] add mysql_query --- src/frontend/src/binder/expr/function/mod.rs | 11 ++ src/frontend/src/expr/table_function.rs | 122 +++++++++++++++++++ 2 files changed, 133 insertions(+) diff --git a/src/frontend/src/binder/expr/function/mod.rs b/src/frontend/src/binder/expr/function/mod.rs index ddc21c6ee7ac7..8dc41b1c9bd90 100644 --- a/src/frontend/src/binder/expr/function/mod.rs +++ b/src/frontend/src/binder/expr/function/mod.rs @@ -333,6 +333,17 @@ impl Binder { .context("postgres_query error")? .into()); } + // `mysql_query` table function + if func_name.eq("mysql_query") { + reject_syntax!( + arg_list.variadic, + "`VARIADIC` is not allowed in table function call" + ); + self.ensure_table_function_allowed()?; + return Ok(TableFunction::new_mysql_query(args) + .context("postgres_query error")? + .into()); + } // UDTF if let Some(ref udf) = udf && udf.kind.is_table() diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 5f22398cc5834..6be5cd88d534b 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -350,6 +350,128 @@ impl TableFunction { } } + pub fn new_mysql_query(args: Vec) -> RwResult { + static MYSQL_ARGS_LEN: usize = 7; + let args = { + if args.len() != MYSQL_ARGS_LEN { + return Err(BindError("postgres_query function only accepts 7 arguments: mysql_query(hostname varchar, port varchar, username varchar, password varchar, database_name varchar, server_id varchar, postgres_query varchar)".to_string()).into()); + } + let mut cast_args = Vec::with_capacity(MYSQL_ARGS_LEN); + for arg in args { + let arg = arg.cast_implicit(DataType::Varchar)?; + cast_args.push(arg); + } + cast_args + }; + let evaled_args = { + let mut evaled_args: Vec = Vec::with_capacity(MYSQL_ARGS_LEN); + for arg in &args { + match arg.try_fold_const() { + Some(Ok(value)) => { + let Some(scalar) = value else { + return Err(BindError( + "postgres_query function does not accept null arguments" + .to_string(), + ) + .into()); + }; + evaled_args.push(scalar.into_utf8().into()); + } + Some(Err(err)) => { + return Err(err); + } + None => { + return Err(BindError( + "postgres_query function only accepts constant arguments".to_string(), + ) + .into()); + } + } + } + evaled_args + }; + + #[cfg(madsim)] + { + return Err(crate::error::ErrorCode::BindError( + "postgres_query can't be used in the madsim mode".to_string(), + ) + .into()); + } + + #[cfg(not(madsim))] + { + let schema = tokio::task::block_in_place(|| { + RUNTIME.block_on(async { + let (client, connection) = tokio_postgres::connect( + format!( + "host={} port={} user={} password={} dbname={}", + evaled_args[0], + evaled_args[1], + evaled_args[2], + evaled_args[3], + evaled_args[4] + ) + .as_str(), + tokio_postgres::NoTls, + ) + .await?; + + tokio::spawn(async move { + if let Err(e) = connection.await { + tracing::error!( + "postgres_query_executor: connection error: {:?}", + e.as_report() + ); + } + }); + + let statement = client.prepare(evaled_args[5].as_str()).await?; + + let mut rw_types = vec![]; + for column in statement.columns() { + let name = column.name().to_string(); + let data_type = match *column.type_() { + TokioPgType::BOOL => DataType::Boolean, + TokioPgType::INT2 => DataType::Int16, + TokioPgType::INT4 => DataType::Int32, + TokioPgType::INT8 => DataType::Int64, + TokioPgType::FLOAT4 => DataType::Float32, + TokioPgType::FLOAT8 => DataType::Float64, + TokioPgType::NUMERIC => DataType::Decimal, + TokioPgType::DATE => DataType::Date, + TokioPgType::TIME => DataType::Time, + TokioPgType::TIMESTAMP => DataType::Timestamp, + TokioPgType::TIMESTAMPTZ => DataType::Timestamptz, + TokioPgType::TEXT | TokioPgType::VARCHAR => DataType::Varchar, + TokioPgType::INTERVAL => DataType::Interval, + TokioPgType::JSONB => DataType::Jsonb, + TokioPgType::BYTEA => DataType::Bytea, + _ => { + return Err(crate::error::ErrorCode::BindError( + format!("unsupported column type: {}", column.type_()) + .to_string(), + ) + .into()); + } + }; + rw_types.push((name, data_type)); + } + Ok::(DataType::Struct( + StructType::new(rw_types), + )) + }) + })?; + + Ok(TableFunction { + args, + return_type: schema, + function_type: TableFunctionType::PostgresQuery, + user_defined: None, + }) + } + } + pub fn to_protobuf(&self) -> PbTableFunction { PbTableFunction { function_type: self.function_type as i32, From 82935fd30072b14095ee1b2e4ec50ff6a87cabae Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 23 Oct 2024 14:21:09 +0800 Subject: [PATCH 02/48] fmt --- src/frontend/src/expr/table_function.rs | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 6be5cd88d534b..223520b34bff5 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -354,7 +354,7 @@ impl TableFunction { static MYSQL_ARGS_LEN: usize = 7; let args = { if args.len() != MYSQL_ARGS_LEN { - return Err(BindError("postgres_query function only accepts 7 arguments: mysql_query(hostname varchar, port varchar, username varchar, password varchar, database_name varchar, server_id varchar, postgres_query varchar)".to_string()).into()); + return Err(BindError("mysql_query function only accepts 7 arguments: mysql_query(hostname varchar, port varchar, username varchar, password varchar, database_name varchar, server_id varchar, mysql_query varchar)".to_string()).into()); } let mut cast_args = Vec::with_capacity(MYSQL_ARGS_LEN); for arg in args { @@ -370,10 +370,9 @@ impl TableFunction { Some(Ok(value)) => { let Some(scalar) = value else { return Err(BindError( - "postgres_query function does not accept null arguments" - .to_string(), + "mysql_query function does not accept null arguments".to_string(), ) - .into()); + .into()); }; evaled_args.push(scalar.into_utf8().into()); } @@ -382,9 +381,9 @@ impl TableFunction { } None => { return Err(BindError( - "postgres_query function only accepts constant arguments".to_string(), + "mysql_query function only accepts constant arguments".to_string(), ) - .into()); + .into()); } } } @@ -396,7 +395,7 @@ impl TableFunction { return Err(crate::error::ErrorCode::BindError( "postgres_query can't be used in the madsim mode".to_string(), ) - .into()); + .into()); } #[cfg(not(madsim))] @@ -412,10 +411,10 @@ impl TableFunction { evaled_args[3], evaled_args[4] ) - .as_str(), + .as_str(), tokio_postgres::NoTls, ) - .await?; + .await?; tokio::spawn(async move { if let Err(e) = connection.await { @@ -452,7 +451,7 @@ impl TableFunction { format!("unsupported column type: {}", column.type_()) .to_string(), ) - .into()); + .into()); } }; rw_types.push((name, data_type)); From 12ce8f0c0be472afcb30849e07cee7acdef107f7 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 23 Oct 2024 14:47:27 +0800 Subject: [PATCH 03/48] use mysql_async in binder --- Cargo.lock | 1 + src/frontend/Cargo.toml | 3 + src/frontend/src/expr/table_function.rs | 78 +++++++++++-------------- 3 files changed, 39 insertions(+), 43 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 2f767d12de213..76309eca2c92e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -11346,6 +11346,7 @@ dependencies = [ "maplit", "md5", "memcomparable", + "mysql_async", "num-integer", "parking_lot 0.12.1", "parse-display", diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index bbb2951e162f1..76ee2aa076e8e 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -45,6 +45,9 @@ linkme = { version = "0.3", features = ["used_linker"] } maplit = "1" md5 = "0.7.0" memcomparable = "0.2" +mysql_async = { version = "0.34", default-features = false, features = [ + "default", +] } num-integer = "0.1" parking_lot = { workspace = true } parse-display = "0.10" diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 223520b34bff5..4a2d353328b65 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -15,6 +15,8 @@ use std::sync::{Arc, LazyLock}; use itertools::Itertools; +use mysql_async::consts::ColumnType as MySqlColumnType; +use mysql_async::prelude::*; use risingwave_common::array::arrow::IcebergArrowConvert; use risingwave_common::types::{DataType, ScalarImpl, StructType}; use risingwave_connector::source::iceberg::{create_parquet_stream_builder, list_s3_directory}; @@ -22,8 +24,8 @@ pub use risingwave_pb::expr::table_function::PbType as TableFunctionType; use risingwave_pb::expr::PbTableFunction; use thiserror_ext::AsReport; use tokio::runtime::Runtime; -use tokio_postgres; use tokio_postgres::types::Type as TokioPgType; +use {mysql_async, tokio_postgres}; use super::{infer_type, Expr, ExprImpl, ExprRewriter, Literal, RwResult}; use crate::catalog::function_catalog::{FunctionCatalog, FunctionKind}; @@ -351,10 +353,10 @@ impl TableFunction { } pub fn new_mysql_query(args: Vec) -> RwResult { - static MYSQL_ARGS_LEN: usize = 7; + static MYSQL_ARGS_LEN: usize = 6; let args = { if args.len() != MYSQL_ARGS_LEN { - return Err(BindError("mysql_query function only accepts 7 arguments: mysql_query(hostname varchar, port varchar, username varchar, password varchar, database_name varchar, server_id varchar, mysql_query varchar)".to_string()).into()); + return Err(BindError("mysql_query function only accepts 6 arguments: mysql_query(hostname varchar, port varchar, username varchar, password varchar, database_name varchar, mysql_query varchar)".to_string()).into()); } let mut cast_args = Vec::with_capacity(MYSQL_ARGS_LEN); for arg in args { @@ -402,53 +404,43 @@ impl TableFunction { { let schema = tokio::task::block_in_place(|| { RUNTIME.block_on(async { - let (client, connection) = tokio_postgres::connect( - format!( - "host={} port={} user={} password={} dbname={}", - evaled_args[0], - evaled_args[1], - evaled_args[2], - evaled_args[3], - evaled_args[4] - ) - .as_str(), - tokio_postgres::NoTls, - ) - .await?; + let database_opts: mysql_async::Opts = mysql_async::OptsBuilder::default() + .ip_or_hostname(evaled_args[0].clone()) + .tcp_port(evaled_args[1].parse::().unwrap()) + .user(Some(evaled_args[2].clone())) + .pass(Some(evaled_args[3].clone())) + .db_name(Some(evaled_args[4].clone())) + .into(); - tokio::spawn(async move { - if let Err(e) = connection.await { - tracing::error!( - "postgres_query_executor: connection error: {:?}", - e.as_report() - ); - } - }); + let pool = mysql_async::Pool::new(database_opts); + let mut conn = pool.get_conn().await?; - let statement = client.prepare(evaled_args[5].as_str()).await?; + let query = evaled_args[6].clone(); + let statement = conn.prep(query).await?; let mut rw_types = vec![]; + #[allow(clippy::never_loop)] for column in statement.columns() { - let name = column.name().to_string(); - let data_type = match *column.type_() { - TokioPgType::BOOL => DataType::Boolean, - TokioPgType::INT2 => DataType::Int16, - TokioPgType::INT4 => DataType::Int32, - TokioPgType::INT8 => DataType::Int64, - TokioPgType::FLOAT4 => DataType::Float32, - TokioPgType::FLOAT8 => DataType::Float64, - TokioPgType::NUMERIC => DataType::Decimal, - TokioPgType::DATE => DataType::Date, - TokioPgType::TIME => DataType::Time, - TokioPgType::TIMESTAMP => DataType::Timestamp, - TokioPgType::TIMESTAMPTZ => DataType::Timestamptz, - TokioPgType::TEXT | TokioPgType::VARCHAR => DataType::Varchar, - TokioPgType::INTERVAL => DataType::Interval, - TokioPgType::JSONB => DataType::Jsonb, - TokioPgType::BYTEA => DataType::Bytea, + let name = column.name_str().to_string(); + let data_type = match column.column_type() { + // TokioPgType::BOOL => DataType::Boolean, + // TokioPgType::INT2 => DataType::Int16, + // TokioPgType::INT4 => DataType::Int32, + // TokioPgType::INT8 => DataType::Int64, + // TokioPgType::FLOAT4 => DataType::Float32, + // TokioPgType::FLOAT8 => DataType::Float64, + // TokioPgType::NUMERIC => DataType::Decimal, + // TokioPgType::DATE => DataType::Date, + // TokioPgType::TIME => DataType::Time, + // TokioPgType::TIMESTAMP => DataType::Timestamp, + // TokioPgType::TIMESTAMPTZ => DataType::Timestamptz, + // TokioPgType::TEXT | TokioPgType::VARCHAR => DataType::Varchar, + // TokioPgType::INTERVAL => DataType::Interval, + // TokioPgType::JSONB => DataType::Jsonb, + // TokioPgType::BYTEA => DataType::Bytea, _ => { return Err(crate::error::ErrorCode::BindError( - format!("unsupported column type: {}", column.type_()) + format!("unsupported column type: {:?}", column.column_type()) .to_string(), ) .into()); From 63202d968fd2a314efaced0f8a7b2d8ae1620f93 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 24 Oct 2024 16:04:02 +0800 Subject: [PATCH 04/48] add mysql table type --- proto/expr.proto | 2 ++ src/frontend/src/expr/table_function.rs | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/proto/expr.proto b/proto/expr.proto index 43e2002933a47..5330843512849 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -365,6 +365,8 @@ message TableFunction { FILE_SCAN = 19; // postgres query POSTGRES_QUERY = 20; + // mysql query + MYSQL_QUERY = 21; // User defined table function USER_DEFINED = 100; } diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 4a2d353328b65..9dbc452b3f75f 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -457,7 +457,7 @@ impl TableFunction { Ok(TableFunction { args, return_type: schema, - function_type: TableFunctionType::PostgresQuery, + function_type: TableFunctionType::MysqlQuery, user_defined: None, }) } From db4062b24319f0facec8c8b9bf0725cb97a80866 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 24 Oct 2024 16:41:17 +0800 Subject: [PATCH 05/48] bind types --- src/frontend/src/expr/table_function.rs | 26 +++++++++++-------------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 9dbc452b3f75f..8bed6fef96070 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -423,21 +423,17 @@ impl TableFunction { for column in statement.columns() { let name = column.name_str().to_string(); let data_type = match column.column_type() { - // TokioPgType::BOOL => DataType::Boolean, - // TokioPgType::INT2 => DataType::Int16, - // TokioPgType::INT4 => DataType::Int32, - // TokioPgType::INT8 => DataType::Int64, - // TokioPgType::FLOAT4 => DataType::Float32, - // TokioPgType::FLOAT8 => DataType::Float64, - // TokioPgType::NUMERIC => DataType::Decimal, - // TokioPgType::DATE => DataType::Date, - // TokioPgType::TIME => DataType::Time, - // TokioPgType::TIMESTAMP => DataType::Timestamp, - // TokioPgType::TIMESTAMPTZ => DataType::Timestamptz, - // TokioPgType::TEXT | TokioPgType::VARCHAR => DataType::Varchar, - // TokioPgType::INTERVAL => DataType::Interval, - // TokioPgType::JSONB => DataType::Jsonb, - // TokioPgType::BYTEA => DataType::Bytea, + MySqlColumnType::MYSQL_TYPE_BIT => DataType::Boolean, + MySqlColumnType::MYSQL_TYPE_SHORT => DataType::Int16, + MySqlColumnType::MYSQL_TYPE_LONG => DataType::Int32, + MySqlColumnType::MYSQL_TYPE_LONGLONG => DataType::Int64, + MySqlColumnType::MYSQL_TYPE_FLOAT => DataType::Float32, + MySqlColumnType::MYSQL_TYPE_DOUBLE => DataType::Float64, + MySqlColumnType::MYSQL_TYPE_DECIMAL => DataType::Decimal, + MySqlColumnType::MYSQL_TYPE_DATE => DataType::Date, + MySqlColumnType::MYSQL_TYPE_TIME => DataType::Time, + MySqlColumnType::MYSQL_TYPE_TIMESTAMP => DataType::Timestamp, + MySqlColumnType::MYSQL_TYPE_VARCHAR => DataType::Varchar, _ => { return Err(crate::error::ErrorCode::BindError( format!("unsupported column type: {:?}", column.column_type()) From a8d3efc8fcd0cd14dba0881ca8348ab4f1ca8a44 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 24 Oct 2024 20:34:59 +0800 Subject: [PATCH 06/48] add optimizer rule --- src/frontend/src/optimizer/rule/mod.rs | 3 + .../table_function_to_mysql_query_rule.rs | 92 +++++++++++++++++++ 2 files changed, 95 insertions(+) create mode 100644 src/frontend/src/optimizer/rule/table_function_to_mysql_query_rule.rs diff --git a/src/frontend/src/optimizer/rule/mod.rs b/src/frontend/src/optimizer/rule/mod.rs index 56d79bf7b408b..7468f1c96524c 100644 --- a/src/frontend/src/optimizer/rule/mod.rs +++ b/src/frontend/src/optimizer/rule/mod.rs @@ -161,6 +161,7 @@ mod pull_up_correlated_predicate_agg_rule; mod source_to_iceberg_scan_rule; mod source_to_kafka_scan_rule; mod table_function_to_file_scan_rule; +mod table_function_to_mysql_query_rule; mod table_function_to_postgres_query_rule; mod values_extract_project_rule; @@ -169,6 +170,7 @@ pub use pull_up_correlated_predicate_agg_rule::*; pub use source_to_iceberg_scan_rule::*; pub use source_to_kafka_scan_rule::*; pub use table_function_to_file_scan_rule::*; +pub use table_function_to_mysql_query_rule::*; pub use table_function_to_postgres_query_rule::*; pub use values_extract_project_rule::*; @@ -234,6 +236,7 @@ macro_rules! for_all_rules { , { TableFunctionToProjectSetRule } , { TableFunctionToFileScanRule } , { TableFunctionToPostgresQueryRule } + , { TableFunctionToMySqlQueryRule } , { ApplyLimitTransposeRule } , { CommonSubExprExtractRule } , { BatchProjectMergeRule } diff --git a/src/frontend/src/optimizer/rule/table_function_to_mysql_query_rule.rs b/src/frontend/src/optimizer/rule/table_function_to_mysql_query_rule.rs new file mode 100644 index 0000000000000..3a6ccdbc643a4 --- /dev/null +++ b/src/frontend/src/optimizer/rule/table_function_to_mysql_query_rule.rs @@ -0,0 +1,92 @@ +// 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 itertools::Itertools; +use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::util::iter_util::ZipEqDebug; + +use super::{BoxedRule, Rule}; +use crate::expr::{Expr, TableFunctionType}; +use crate::optimizer::plan_node::generic::GenericPlanRef; +// use crate::optimizer::plan_node::{LogicalMySqlQuery, LogicalTableFunction}; +use crate::optimizer::plan_node::LogicalTableFunction; +use crate::optimizer::PlanRef; + +/// Transform a special `TableFunction` (with `MYSQL_QUERY` table function type) into a `LogicalMySqlQuery` +pub struct TableFunctionToMySqlQueryRule {} +impl Rule for TableFunctionToMySqlQueryRule { + fn apply(&self, plan: PlanRef) -> Option { + let logical_table_function: &LogicalTableFunction = plan.as_logical_table_function()?; + if logical_table_function.table_function.function_type != TableFunctionType::MysqlQuery { + return None; + } + assert!(!logical_table_function.with_ordinality); + let table_function_return_type = logical_table_function.table_function().return_type(); + + if let DataType::Struct(st) = table_function_return_type.clone() { + let fields = st + .types() + .zip_eq_debug(st.names()) + .map(|(data_type, name)| Field::with_name(data_type.clone(), name.to_string())) + .collect_vec(); + + let schema = Schema::new(fields); + + assert_eq!(logical_table_function.table_function().args.len(), 6); + let mut eval_args = vec![]; + for arg in &logical_table_function.table_function().args { + assert_eq!(arg.return_type(), DataType::Varchar); + let value = arg.try_fold_const().unwrap().unwrap(); + match value { + Some(ScalarImpl::Utf8(s)) => { + eval_args.push(s.to_string()); + } + _ => { + unreachable!("must be a varchar") + } + } + } + let hostname = eval_args[0].clone(); + let port = eval_args[1].clone(); + let username = eval_args[2].clone(); + let password = eval_args[3].clone(); + let database = eval_args[4].clone(); + let query = eval_args[5].clone(); + + None + // Some( + // LogicalMySqlQuery::new( + // logical_table_function.ctx(), + // schema, + // hostname, + // port, + // username, + // password, + // database, + // query, + // ) + // .into(), + // ) + } else { + unreachable!("TableFunction return type should be struct") + } + } +} + +impl TableFunctionToMySqlQueryRule { + pub fn create() -> BoxedRule { + Box::new(TableFunctionToMySqlQueryRule {}) + } +} From 8f6e323e62597d9b6523c2fc59fa87def7df635c Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 25 Oct 2024 10:11:50 +0800 Subject: [PATCH 07/48] minor --- src/frontend/src/binder/expr/function/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/binder/expr/function/mod.rs b/src/frontend/src/binder/expr/function/mod.rs index 8dc41b1c9bd90..f7a4007ffd467 100644 --- a/src/frontend/src/binder/expr/function/mod.rs +++ b/src/frontend/src/binder/expr/function/mod.rs @@ -341,7 +341,7 @@ impl Binder { ); self.ensure_table_function_allowed()?; return Ok(TableFunction::new_mysql_query(args) - .context("postgres_query error")? + .context("mysql_query error")? .into()); } // UDTF From 546370efdda8904588128defd764d6de7801fe07 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 25 Oct 2024 10:26:22 +0800 Subject: [PATCH 08/48] add error context --- src/frontend/src/expr/table_function.rs | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 8bed6fef96070..5bfeea349e9ee 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -14,6 +14,7 @@ use std::sync::{Arc, LazyLock}; +use anyhow::Context; use itertools::Itertools; use mysql_async::consts::ColumnType as MySqlColumnType; use mysql_async::prelude::*; @@ -413,10 +414,16 @@ impl TableFunction { .into(); let pool = mysql_async::Pool::new(database_opts); - let mut conn = pool.get_conn().await?; + let mut conn = pool + .get_conn() + .await + .context("failed to connect to mysql in binder")?; let query = evaled_args[6].clone(); - let statement = conn.prep(query).await?; + let statement = conn + .prep(query) + .await + .context("failed to prepare mysql_query in binder")?; let mut rw_types = vec![]; #[allow(clippy::never_loop)] From 8258dffde1b588d74e5c4a742462ebfe23b27a84 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 25 Oct 2024 10:59:14 +0800 Subject: [PATCH 09/48] add mysql plan nodes + batch executor skeleton --- proto/batch_plan.proto | 12 ++ src/batch/src/executor/mod.rs | 3 + src/batch/src/executor/mysql_query.rs | 194 ++++++++++++++++++ .../optimizer/plan_node/batch_mysql_query.rs | 96 +++++++++ .../src/optimizer/plan_node/generic/mod.rs | 3 + .../plan_node/generic/mysql_query.rs | 67 ++++++ .../plan_node/logical_mysql_query.rs | 115 +++++++++++ .../plan_node/logical_postgres_query.rs | 4 +- src/frontend/src/optimizer/plan_node/mod.rs | 10 + 9 files changed, 502 insertions(+), 2 deletions(-) create mode 100644 src/batch/src/executor/mysql_query.rs create mode 100644 src/frontend/src/optimizer/plan_node/batch_mysql_query.rs create mode 100644 src/frontend/src/optimizer/plan_node/generic/mysql_query.rs create mode 100644 src/frontend/src/optimizer/plan_node/logical_mysql_query.rs diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index f881f6546fae5..b46230b2438d6 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -104,6 +104,17 @@ message PostgresQueryNode { string query = 7; } +// NOTE(kwannoel): This will only be used in batch mode. We can change the definition as needed. +message MySqlQueryNode { + repeated plan_common.ColumnDesc columns = 1; + string hostname = 2; + string port = 3; + string username = 4; + string password = 5; + string database = 6; + string query = 7; +} + message ProjectNode { repeated expr.ExprNode select_list = 1; } @@ -386,6 +397,7 @@ message PlanNode { FileScanNode file_scan = 38; IcebergScanNode iceberg_scan = 39; PostgresQueryNode postgres_query = 40; + MySqlQueryNode mysql_query = 41; // The following nodes are used for testing. bool block_executor = 100; bool busy_loop_executor = 101; diff --git a/src/batch/src/executor/mod.rs b/src/batch/src/executor/mod.rs index c3bd373198df7..ce84065d9d41c 100644 --- a/src/batch/src/executor/mod.rs +++ b/src/batch/src/executor/mod.rs @@ -29,6 +29,7 @@ mod managed; mod max_one_row; mod merge_sort; mod merge_sort_exchange; +mod mysql_query; mod order_by; mod postgres_query; mod project; @@ -65,6 +66,7 @@ pub use managed::*; pub use max_one_row::*; pub use merge_sort::*; pub use merge_sort_exchange::*; +pub use mysql_query::*; pub use order_by::*; pub use postgres_query::*; pub use project::*; @@ -247,6 +249,7 @@ impl<'a, C: BatchTaskContext> ExecutorBuilder<'a, C> { NodeBody::FileScan => FileScanExecutorBuilder, NodeBody::IcebergScan => IcebergScanExecutorBuilder, NodeBody::PostgresQuery => PostgresQueryExecutorBuilder, + NodeBody::MysqlQuery => MySqlQueryExecutorBuilder, // Follow NodeBody only used for test NodeBody::BlockExecutor => BlockExecutorBuilder, NodeBody::BusyLoopExecutor => BusyLoopExecutorBuilder, diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs new file mode 100644 index 0000000000000..4deb63ba9118c --- /dev/null +++ b/src/batch/src/executor/mysql_query.rs @@ -0,0 +1,194 @@ +// 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 anyhow::Context; +use futures_async_stream::try_stream; +use futures_util::stream::StreamExt; +use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::{DataType, Datum, Decimal, ScalarImpl}; +use risingwave_common::util::chunk_coalesce::DataChunkBuilder; +use risingwave_pb::batch_plan::plan_node::NodeBody; +use thiserror_ext::AsReport; + +use crate::error::BatchError; +use crate::executor::{BoxedExecutor, BoxedExecutorBuilder, DataChunk, Executor, ExecutorBuilder}; +use crate::task::BatchTaskContext; + +/// `MySqlQuery` executor. Runs a query against a MySql database. +pub struct MySqlQueryExecutor { + schema: Schema, + host: String, + port: String, + username: String, + password: String, + database: String, + query: String, + identity: String, +} + +impl Executor for MySqlQueryExecutor { + fn schema(&self) -> &risingwave_common::catalog::Schema { + &self.schema + } + + fn identity(&self) -> &str { + &self.identity + } + + fn execute(self: Box) -> super::BoxedDataChunkStream { + self.do_execute().boxed() + } +} +// pub fn mysql_row_to_owned_row( +// row: tokio_mysql::Row, +// schema: &Schema, +// ) -> Result { +// let mut datums = vec![]; +// for i in 0..schema.fields.len() { +// let rw_field = &schema.fields[i]; +// let name = rw_field.name.as_str(); +// let datum = mysql_cell_to_scalar_impl(&row, &rw_field.data_type, i, name)?; +// datums.push(datum); +// } +// Ok(OwnedRow::new(datums)) +// } + +// TODO(kwannoel): Support more types, see mysql connector's ScalarAdapter. +// fn mysql_cell_to_scalar_impl( +// row: &tokio_mysql::Row, +// data_type: &DataType, +// i: usize, +// name: &str, +// ) -> Result { +// let datum = match data_type { +// DataType::Boolean +// | DataType::Int16 +// | DataType::Int32 +// | DataType::Int64 +// | DataType::Float32 +// | DataType::Float64 +// | DataType::Date +// | DataType::Time +// | DataType::Timestamp +// | DataType::Timestamptz +// | DataType::Jsonb +// | DataType::Interval +// | DataType::Varchar +// | DataType::Bytea => { +// // ScalarAdapter is also fine. But ScalarImpl is more efficient +// row.try_get::<_, Option>(i)? +// } +// DataType::Decimal => { +// // Decimal is more efficient than PgNumeric in ScalarAdapter +// let val = row.try_get::<_, Option>(i)?; +// val.map(ScalarImpl::from) +// } +// _ => { +// tracing::warn!(name, ?data_type, "unsupported data type, set to null"); +// None +// } +// }; +// Ok(datum) +// } + +impl MySqlQueryExecutor { + pub fn new( + schema: Schema, + host: String, + port: String, + username: String, + password: String, + database: String, + query: String, + identity: String, + ) -> Self { + Self { + schema, + host, + port, + username, + password, + database, + query, + identity, + } + } + + #[try_stream(ok = DataChunk, error = BatchError)] + async fn do_execute(self: Box) { + tracing::debug!("mysql_query_executor: started"); + // let conn_str = format!( + // "host={} port={} user={} password={} dbname={}", + // self.host, self.port, self.username, self.password, self.database + // ); + // let (client, conn) = tokio_mysql::connect(&conn_str, tokio_mysql::NoTls).await?; + // + // tokio::spawn(async move { + // if let Err(e) = conn.await { + // tracing::error!( + // "mysql_query_executor: connection error: {:?}", + // e.as_report() + // ); + // } + // }); + // + // let params: &[&str] = &[]; + // let row_stream = client + // .query_raw(&self.query, params) + // .await + // .context("mysql_query received error from remote server")?; + // let mut builder = DataChunkBuilder::new(self.schema.data_types(), 1024); + // tracing::debug!("mysql_query_executor: query executed, start deserializing rows"); + // // deserialize the rows + // #[for_await] + // for row in row_stream { + // let row = row?; + // let owned_row = mysql_row_to_owned_row(row, &self.schema)?; + // if let Some(chunk) = builder.append_one_row(owned_row) { + // yield chunk; + // } + // } + // if let Some(chunk) = builder.consume_all() { + // yield chunk; + // } + return Ok(()); + } +} + +pub struct MySqlQueryExecutorBuilder {} + +#[async_trait::async_trait] +impl BoxedExecutorBuilder for MySqlQueryExecutorBuilder { + async fn new_boxed_executor( + source: &ExecutorBuilder<'_, C>, + _inputs: Vec, + ) -> crate::error::Result { + let mysql_query_node = try_match_expand!( + source.plan_node().get_node_body().unwrap(), + NodeBody::MysqlQuery + )?; + + Ok(Box::new(MySqlQueryExecutor::new( + Schema::from_iter(mysql_query_node.columns.iter().map(Field::from)), + mysql_query_node.hostname.clone(), + mysql_query_node.port.clone(), + mysql_query_node.username.clone(), + mysql_query_node.password.clone(), + mysql_query_node.database.clone(), + mysql_query_node.query.clone(), + source.plan_node().get_identity().clone(), + ))) + } +} diff --git a/src/frontend/src/optimizer/plan_node/batch_mysql_query.rs b/src/frontend/src/optimizer/plan_node/batch_mysql_query.rs new file mode 100644 index 0000000000000..308b1e82c63f3 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/batch_mysql_query.rs @@ -0,0 +1,96 @@ +// 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 pretty_xmlish::XmlNode; +use risingwave_pb::batch_plan::plan_node::NodeBody; +use risingwave_pb::batch_plan::MySqlQueryNode; + +use super::batch::prelude::*; +use super::utils::{childless_record, column_names_pretty, Distill}; +use super::{ + generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch, +}; +use crate::error::Result; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::property::{Distribution, Order}; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct BatchMySqlQuery { + pub base: PlanBase, + pub core: generic::MySqlQuery, +} + +impl BatchMySqlQuery { + pub fn new(core: generic::MySqlQuery) -> Self { + let base = PlanBase::new_batch_with_core(&core, Distribution::Single, Order::any()); + + Self { base, core } + } + + pub fn column_names(&self) -> Vec<&str> { + self.schema().names_str() + } + + pub fn clone_with_dist(&self) -> Self { + let base = self.base.clone_with_new_distribution(Distribution::Single); + Self { + base, + core: self.core.clone(), + } + } +} + +impl_plan_tree_node_for_leaf! { BatchMySqlQuery } + +impl Distill for BatchMySqlQuery { + fn distill<'a>(&self) -> XmlNode<'a> { + let fields = vec![("columns", column_names_pretty(self.schema()))]; + childless_record("BatchMySqlQuery", fields) + } +} + +impl ToLocalBatch for BatchMySqlQuery { + fn to_local(&self) -> Result { + Ok(self.clone_with_dist().into()) + } +} + +impl ToDistributedBatch for BatchMySqlQuery { + fn to_distributed(&self) -> Result { + Ok(self.clone_with_dist().into()) + } +} + +impl ToBatchPb for BatchMySqlQuery { + fn to_batch_prost_body(&self) -> NodeBody { + NodeBody::MysqlQuery(MySqlQueryNode { + columns: self + .core + .columns() + .iter() + .map(|c| c.to_protobuf()) + .collect(), + hostname: self.core.hostname.clone(), + port: self.core.port.clone(), + username: self.core.username.clone(), + password: self.core.password.clone(), + database: self.core.database.clone(), + query: self.core.query.clone(), + }) + } +} + +impl ExprRewritable for BatchMySqlQuery {} + +impl ExprVisitable for BatchMySqlQuery {} diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index 6a076025b906c..c35a367e8ccec 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -92,6 +92,9 @@ pub use file_scan::*; mod postgres_query; pub use postgres_query::*; +mod mysql_query; +pub use mysql_query::*; + pub trait DistillUnit { fn distill_with_name<'a>(&self, name: impl Into>) -> XmlNode<'a>; } diff --git a/src/frontend/src/optimizer/plan_node/generic/mysql_query.rs b/src/frontend/src/optimizer/plan_node/generic/mysql_query.rs new file mode 100644 index 0000000000000..03bbfa0b229eb --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/generic/mysql_query.rs @@ -0,0 +1,67 @@ +// 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 educe::Educe; +use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema}; + +use super::GenericPlanNode; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::property::FunctionalDependencySet; + +#[derive(Debug, Clone, Educe)] +#[educe(PartialEq, Eq, Hash)] +pub struct MySqlQuery { + pub schema: Schema, + pub hostname: String, + pub port: String, + pub username: String, + pub password: String, + pub database: String, + pub query: String, + + #[educe(PartialEq(ignore))] + #[educe(Hash(ignore))] + pub ctx: OptimizerContextRef, +} + +impl GenericPlanNode for MySqlQuery { + fn schema(&self) -> Schema { + self.schema.clone() + } + + fn stream_key(&self) -> Option> { + None + } + + fn ctx(&self) -> OptimizerContextRef { + self.ctx.clone() + } + + fn functional_dependency(&self) -> FunctionalDependencySet { + FunctionalDependencySet::new(self.schema.len()) + } +} + +impl MySqlQuery { + pub fn columns(&self) -> Vec { + self.schema + .fields + .iter() + .enumerate() + .map(|(i, f)| { + ColumnDesc::named(f.name.clone(), ColumnId::new(i as i32), f.data_type.clone()) + }) + .collect() + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_mysql_query.rs b/src/frontend/src/optimizer/plan_node/logical_mysql_query.rs new file mode 100644 index 0000000000000..1512fe60120a3 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_mysql_query.rs @@ -0,0 +1,115 @@ +// 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 pretty_xmlish::XmlNode; +use risingwave_common::bail; +use risingwave_common::catalog::Schema; + +use super::generic::GenericPlanRef; +use super::utils::{childless_record, Distill}; +use super::{ + generic, BatchMySqlQuery, ColPrunable, ExprRewritable, Logical, LogicalProject, PlanBase, + PlanRef, PredicatePushdown, ToBatch, ToStream, +}; +use crate::error::Result; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::utils::column_names_pretty; +use crate::optimizer::plan_node::{ + ColumnPruningContext, LogicalFilter, PredicatePushdownContext, RewriteStreamContext, + ToStreamContext, +}; +use crate::utils::{ColIndexMapping, Condition}; +use crate::OptimizerContextRef; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LogicalMySqlQuery { + pub base: PlanBase, + pub core: generic::MySqlQuery, +} + +impl LogicalMySqlQuery { + pub fn new( + ctx: OptimizerContextRef, + schema: Schema, + hostname: String, + port: String, + username: String, + password: String, + database: String, + query: String, + ) -> Self { + let core = generic::MySqlQuery { + schema, + hostname, + port, + username, + password, + database, + query, + ctx, + }; + + let base = PlanBase::new_logical_with_core(&core); + + LogicalMySqlQuery { base, core } + } +} + +impl_plan_tree_node_for_leaf! {LogicalMySqlQuery} +impl Distill for LogicalMySqlQuery { + fn distill<'a>(&self) -> XmlNode<'a> { + let fields = vec![("columns", column_names_pretty(self.schema()))]; + childless_record("LogicalMySqlQuery", fields) + } +} + +impl ColPrunable for LogicalMySqlQuery { + fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { + LogicalProject::with_out_col_idx(self.clone().into(), required_cols.iter().cloned()).into() + } +} + +impl ExprRewritable for LogicalMySqlQuery {} + +impl ExprVisitable for LogicalMySqlQuery {} + +impl PredicatePushdown for LogicalMySqlQuery { + fn predicate_pushdown( + &self, + predicate: Condition, + _ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + // No pushdown. + LogicalFilter::create(self.clone().into(), predicate) + } +} + +impl ToBatch for LogicalMySqlQuery { + fn to_batch(&self) -> Result { + Ok(BatchMySqlQuery::new(self.core.clone()).into()) + } +} + +impl ToStream for LogicalMySqlQuery { + fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { + bail!("mysql_query function is not supported in streaming mode") + } + + fn logical_rewrite_for_stream( + &self, + _ctx: &mut RewriteStreamContext, + ) -> Result<(PlanRef, ColIndexMapping)> { + bail!("mysql_query function is not supported in streaming mode") + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_postgres_query.rs b/src/frontend/src/optimizer/plan_node/logical_postgres_query.rs index 9082bd86a3f37..d3d793fb8ba01 100644 --- a/src/frontend/src/optimizer/plan_node/logical_postgres_query.rs +++ b/src/frontend/src/optimizer/plan_node/logical_postgres_query.rs @@ -103,13 +103,13 @@ impl ToBatch for LogicalPostgresQuery { impl ToStream for LogicalPostgresQuery { fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { - bail!("file_scan function is not supported in streaming mode") + bail!("postgres_query function is not supported in streaming mode") } fn logical_rewrite_for_stream( &self, _ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping)> { - bail!("file_scan function is not supported in streaming mode") + bail!("postgres_query function is not supported in streaming mode") } } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 9b814ab8289c2..432475b52809b 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -935,10 +935,14 @@ mod batch_file_scan; mod batch_iceberg_scan; mod batch_kafka_scan; mod batch_postgres_query; + +mod batch_mysql_query; mod derive; mod logical_file_scan; mod logical_iceberg_scan; mod logical_postgres_query; + +mod logical_mysql_query; mod stream_cdc_table_scan; mod stream_share; mod stream_temporal_join; @@ -961,6 +965,7 @@ pub use batch_limit::BatchLimit; pub use batch_log_seq_scan::BatchLogSeqScan; pub use batch_lookup_join::BatchLookupJoin; pub use batch_max_one_row::BatchMaxOneRow; +pub use batch_mysql_query::BatchMySqlQuery; pub use batch_nested_loop_join::BatchNestedLoopJoin; pub use batch_over_window::BatchOverWindow; pub use batch_postgres_query::BatchPostgresQuery; @@ -997,6 +1002,7 @@ pub use logical_kafka_scan::LogicalKafkaScan; pub use logical_limit::LogicalLimit; pub use logical_max_one_row::LogicalMaxOneRow; pub use logical_multi_join::{LogicalMultiJoin, LogicalMultiJoinBuilder}; +pub use logical_mysql_query::LogicalMySqlQuery; pub use logical_now::LogicalNow; pub use logical_over_window::LogicalOverWindow; pub use logical_postgres_query::LogicalPostgresQuery; @@ -1112,6 +1118,7 @@ macro_rules! for_all_plan_nodes { , { Logical, ChangeLog } , { Logical, FileScan } , { Logical, PostgresQuery } + , { Logical, MySqlQuery } , { Batch, SimpleAgg } , { Batch, HashAgg } , { Batch, SortAgg } @@ -1144,6 +1151,7 @@ macro_rules! for_all_plan_nodes { , { Batch, IcebergScan } , { Batch, FileScan } , { Batch, PostgresQuery } + , { Batch, MySqlQuery } , { Stream, Project } , { Stream, Filter } , { Stream, TableScan } @@ -1226,6 +1234,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, ChangeLog } , { Logical, FileScan } , { Logical, PostgresQuery } + , { Logical, MySqlQuery } } }; } @@ -1267,6 +1276,7 @@ macro_rules! for_batch_plan_nodes { , { Batch, IcebergScan } , { Batch, FileScan } , { Batch, PostgresQuery } + , { Batch, MySqlQuery } } }; } From 2486af45da6a0453273c636ef57263e945048c99 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 25 Oct 2024 11:01:28 +0800 Subject: [PATCH 10/48] link rule to logical plan node --- .../table_function_to_mysql_query_rule.rs | 29 +++++++++---------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/src/frontend/src/optimizer/rule/table_function_to_mysql_query_rule.rs b/src/frontend/src/optimizer/rule/table_function_to_mysql_query_rule.rs index 3a6ccdbc643a4..0ad534825790c 100644 --- a/src/frontend/src/optimizer/rule/table_function_to_mysql_query_rule.rs +++ b/src/frontend/src/optimizer/rule/table_function_to_mysql_query_rule.rs @@ -21,7 +21,7 @@ use super::{BoxedRule, Rule}; use crate::expr::{Expr, TableFunctionType}; use crate::optimizer::plan_node::generic::GenericPlanRef; // use crate::optimizer::plan_node::{LogicalMySqlQuery, LogicalTableFunction}; -use crate::optimizer::plan_node::LogicalTableFunction; +use crate::optimizer::plan_node::{LogicalMySqlQuery, LogicalTableFunction}; use crate::optimizer::PlanRef; /// Transform a special `TableFunction` (with `MYSQL_QUERY` table function type) into a `LogicalMySqlQuery` @@ -65,20 +65,19 @@ impl Rule for TableFunctionToMySqlQueryRule { let database = eval_args[4].clone(); let query = eval_args[5].clone(); - None - // Some( - // LogicalMySqlQuery::new( - // logical_table_function.ctx(), - // schema, - // hostname, - // port, - // username, - // password, - // database, - // query, - // ) - // .into(), - // ) + Some( + LogicalMySqlQuery::new( + logical_table_function.ctx(), + schema, + hostname, + port, + username, + password, + database, + query, + ) + .into(), + ) } else { unreachable!("TableFunction return type should be struct") } From 23cd0ee0db96bb8c83a3f486ee58402ba5e56441 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 25 Oct 2024 23:01:54 +0800 Subject: [PATCH 11/48] instantiate connection in mysql executor --- Cargo.lock | 1 + Cargo.toml | 3 +++ src/batch/Cargo.toml | 1 + src/batch/src/executor/mysql_query.rs | 31 ++++++++++++++------------- src/connector/Cargo.toml | 4 +--- 5 files changed, 22 insertions(+), 18 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 76309eca2c92e..059aa168b58aa 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10476,6 +10476,7 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "memcomparable", + "mysql_async", "opendal 0.49.2", "parking_lot 0.12.1", "parquet 53.0.0", diff --git a/Cargo.toml b/Cargo.toml index 80166c384efb5..49e421f4d8219 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -161,6 +161,9 @@ deltalake = { version = "0.20.1", features = [ itertools = "0.13.0" jsonbb = "0.1.4" lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "2682b85" } +mysql_async = { version = "0.34", default-features = false, features = [ + "default", +] } parquet = { version = "53", features = ["async"] } thiserror-ext = "0.1.2" tikv-jemalloc-ctl = { git = "https://github.com/risingwavelabs/jemallocator.git", rev = "64a2d9" } diff --git a/src/batch/Cargo.toml b/src/batch/Cargo.toml index 46c4aa7b9de6e..9d9082d33cb85 100644 --- a/src/batch/Cargo.toml +++ b/src/batch/Cargo.toml @@ -29,6 +29,7 @@ hytra = "0.1.2" iceberg = { workspace = true } itertools = { workspace = true } memcomparable = "0.2" +mysql_async = { workspace = true } opendal = { workspace = true } parking_lot = { workspace = true } parquet = { workspace = true } diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index 4deb63ba9118c..335a8d0a28702 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -15,6 +15,8 @@ use anyhow::Context; use futures_async_stream::try_stream; use futures_util::stream::StreamExt; +use mysql_async; +use mysql_async::prelude::*; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, Datum, Decimal, ScalarImpl}; @@ -129,21 +131,20 @@ impl MySqlQueryExecutor { #[try_stream(ok = DataChunk, error = BatchError)] async fn do_execute(self: Box) { tracing::debug!("mysql_query_executor: started"); - // let conn_str = format!( - // "host={} port={} user={} password={} dbname={}", - // self.host, self.port, self.username, self.password, self.database - // ); - // let (client, conn) = tokio_mysql::connect(&conn_str, tokio_mysql::NoTls).await?; - // - // tokio::spawn(async move { - // if let Err(e) = conn.await { - // tracing::error!( - // "mysql_query_executor: connection error: {:?}", - // e.as_report() - // ); - // } - // }); - // + let database_opts: mysql_async::Opts = mysql_async::OptsBuilder::default() + .ip_or_hostname(self.host) + .tcp_port(self.port.parse::().unwrap()) // FIXME + .user(Some(self.username)) + .pass(Some(self.password)) + .db_name(Some(self.database)) + .into(); + + let pool = mysql_async::Pool::new(database_opts); + let mut conn = pool + .get_conn() + .await + .context("failed to connect to mysql in binder")?; + // let params: &[&str] = &[]; // let row_stream = client // .query_raw(&self.query, params) diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 9fd49fea88c7b..44fb2d7ba840f 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -73,9 +73,7 @@ jsonwebtoken = "9.2.0" maplit = "1.0.2" moka = { version = "0.12.0", features = ["future"] } mongodb = { version = "2.8.2", features = ["tokio-runtime"] } -mysql_async = { version = "0.34", default-features = false, features = [ - "default", -] } +mysql_async = { workspace = true } mysql_common = { version = "0.32", default-features = false, features = [ "chrono", ] } From c3715b6beccb2d31fdc1bb09f20548952cbffba4 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 28 Oct 2024 12:05:13 +0800 Subject: [PATCH 12/48] handle MySql serde --- src/batch/src/error.rs | 7 +- src/batch/src/executor/mysql_query.rs | 144 +++++++++++++------------- 2 files changed, 78 insertions(+), 73 deletions(-) diff --git a/src/batch/src/error.rs b/src/batch/src/error.rs index e3e53ee449fac..688fa1d12c1d2 100644 --- a/src/batch/src/error.rs +++ b/src/batch/src/error.rs @@ -17,6 +17,7 @@ use std::sync::Arc; pub use anyhow::anyhow; +use mysql_async::{Error as MySqlError, FromValueError as MySqlValueConversionError}; use parquet::errors::ParquetError; use risingwave_common::array::ArrayError; use risingwave_common::error::{def_anyhow_newtype, def_anyhow_variant, BoxedError}; @@ -29,7 +30,7 @@ use risingwave_rpc_client::error::{RpcError, ToTonicStatus}; use risingwave_storage::error::StorageError; use thiserror::Error; use thiserror_ext::Construct; -use tokio_postgres; +use tokio_postgres::Error as PostgresError; use tonic::Status; use crate::worker_manager::worker_node_manager::FragmentId; @@ -192,7 +193,9 @@ def_anyhow_variant! { pub BatchExternalSystemError, BatchError ExternalSystemError, - tokio_postgres::Error => "Postgres error", + PostgresError => "Postgres error", iceberg::Error => "Iceberg error", ParquetError => "Parquet error", + MySqlError => "MySQL error", + MySqlValueConversionError => "MySQL value conversion error" } diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index 335a8d0a28702..3464c80563492 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::Context; +use anyhow::{anyhow, Context}; use futures_async_stream::try_stream; use futures_util::stream::StreamExt; use mysql_async; @@ -53,57 +53,55 @@ impl Executor for MySqlQueryExecutor { self.do_execute().boxed() } } -// pub fn mysql_row_to_owned_row( -// row: tokio_mysql::Row, -// schema: &Schema, -// ) -> Result { -// let mut datums = vec![]; -// for i in 0..schema.fields.len() { -// let rw_field = &schema.fields[i]; -// let name = rw_field.name.as_str(); -// let datum = mysql_cell_to_scalar_impl(&row, &rw_field.data_type, i, name)?; -// datums.push(datum); -// } -// Ok(OwnedRow::new(datums)) -// } +pub fn mysql_row_to_owned_row( + mut row: mysql_async::Row, + schema: &Schema, +) -> Result { + let mut datums = vec![]; + for i in 0..schema.fields.len() { + let rw_field = &schema.fields[i]; + let name = rw_field.name.as_str(); + let datum = mysql_cell_to_scalar_impl(&mut row, &rw_field.data_type, i, name)?; + datums.push(datum); + } + Ok(OwnedRow::new(datums)) +} -// TODO(kwannoel): Support more types, see mysql connector's ScalarAdapter. -// fn mysql_cell_to_scalar_impl( -// row: &tokio_mysql::Row, -// data_type: &DataType, -// i: usize, -// name: &str, -// ) -> Result { -// let datum = match data_type { -// DataType::Boolean -// | DataType::Int16 -// | DataType::Int32 -// | DataType::Int64 -// | DataType::Float32 -// | DataType::Float64 -// | DataType::Date -// | DataType::Time -// | DataType::Timestamp -// | DataType::Timestamptz -// | DataType::Jsonb -// | DataType::Interval -// | DataType::Varchar -// | DataType::Bytea => { -// // ScalarAdapter is also fine. But ScalarImpl is more efficient -// row.try_get::<_, Option>(i)? -// } -// DataType::Decimal => { -// // Decimal is more efficient than PgNumeric in ScalarAdapter -// let val = row.try_get::<_, Option>(i)?; -// val.map(ScalarImpl::from) -// } -// _ => { -// tracing::warn!(name, ?data_type, "unsupported data type, set to null"); -// None -// } -// }; -// Ok(datum) -// } +fn mysql_cell_to_scalar_impl( + row: &mut mysql_async::Row, + data_type: &DataType, + i: usize, + name: &str, +) -> Result { + let datum = match data_type { + DataType::Boolean => { + let val = row.take_opt::, _>(i); + match val { + None => bail!("missing value for column {}, at index {}", name, i), + Some(Ok(Some(val))) => Some(ScalarImpl::from(val)), + Some(Ok(None)) => None, + Some(Err(e)) => return Err(e.into()), + } + } + // | DataType::Int16 + // | DataType::Int32 + // | DataType::Int64 + // | DataType::Float32 + // | DataType::Float64 + // | DataType::Decimal, + // | DataType::Date + // | DataType::Time + // | DataType::Timestamp + // | DataType::Varchar => { + // // ScalarAdapter is also fine. But ScalarImpl is more efficient + // row.try_get::<_, Option>(i)? + // } + _ => { + bail!("unsupported data type: {}", data_type) + } + }; + Ok(datum) +} impl MySqlQueryExecutor { pub fn new( @@ -143,27 +141,31 @@ impl MySqlQueryExecutor { let mut conn = pool .get_conn() .await - .context("failed to connect to mysql in binder")?; + .context("failed to connect to mysql in batch executor")?; - // let params: &[&str] = &[]; - // let row_stream = client - // .query_raw(&self.query, params) - // .await - // .context("mysql_query received error from remote server")?; - // let mut builder = DataChunkBuilder::new(self.schema.data_types(), 1024); - // tracing::debug!("mysql_query_executor: query executed, start deserializing rows"); - // // deserialize the rows - // #[for_await] - // for row in row_stream { - // let row = row?; - // let owned_row = mysql_row_to_owned_row(row, &self.schema)?; - // if let Some(chunk) = builder.append_one_row(owned_row) { - // yield chunk; - // } - // } - // if let Some(chunk) = builder.consume_all() { - // yield chunk; - // } + let query = self.query; + let mut query_iter = conn + .query_iter(query) + .await + .context("failed to execute my_sql_query in batch executor")?; + let Some(row_stream) = query_iter.stream::().await? else { + bail!("failed to get row stream from mysql query") + }; + + let mut builder = DataChunkBuilder::new(self.schema.data_types(), 1024); + tracing::debug!("mysql_query_executor: query executed, start deserializing rows"); + // deserialize the rows + #[for_await] + for row in row_stream { + let mut row = row?; + let owned_row = mysql_row_to_owned_row(row, &self.schema)?; + if let Some(chunk) = builder.append_one_row(owned_row) { + yield chunk; + } + } + if let Some(chunk) = builder.consume_all() { + yield chunk; + } return Ok(()); } } From 8c68e2fac7b3d719f3e3a608d0377129edfa11ca Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 28 Oct 2024 14:38:08 +0800 Subject: [PATCH 13/48] introduce macro --- src/batch/src/executor/mysql_query.rs | 22 +++++++++++++--------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index 3464c80563492..8c858009c9df2 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -67,6 +67,18 @@ pub fn mysql_row_to_owned_row( Ok(OwnedRow::new(datums)) } +macro_rules! mysql_value_to_scalar { + ($row:ident, $i:ident, $name:ident, $ty:ty, $variant:ident) => {{ + let val = $row.take_opt::, _>($i); + match val { + None => bail!("missing value for column {}, at index {}", $name, $i), + Some(Ok(Some(val))) => Some(ScalarImpl::$variant(val)), + Some(Ok(None)) => None, + Some(Err(e)) => return Err(e.into()), + } + }}; +} + fn mysql_cell_to_scalar_impl( row: &mut mysql_async::Row, data_type: &DataType, @@ -74,15 +86,7 @@ fn mysql_cell_to_scalar_impl( name: &str, ) -> Result { let datum = match data_type { - DataType::Boolean => { - let val = row.take_opt::, _>(i); - match val { - None => bail!("missing value for column {}, at index {}", name, i), - Some(Ok(Some(val))) => Some(ScalarImpl::from(val)), - Some(Ok(None)) => None, - Some(Err(e)) => return Err(e.into()), - } - } + DataType::Boolean => mysql_value_to_scalar!(row, i, name, bool, Bool), // | DataType::Int16 // | DataType::Int32 // | DataType::Int64 From ee9f146c0ce25b99997b8ec7aa9a5d9b0205ae6f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 28 Oct 2024 15:02:57 +0800 Subject: [PATCH 14/48] convert all mysql types to rw types --- Cargo.lock | 2 ++ src/batch/Cargo.toml | 2 ++ src/batch/src/executor/mysql_query.rs | 30 +++++++++++++-------------- 3 files changed, 19 insertions(+), 15 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 059aa168b58aa..275024641e22e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10463,6 +10463,7 @@ dependencies = [ "async-recursion", "async-trait", "bytes", + "chrono", "criterion", "either", "foyer", @@ -10494,6 +10495,7 @@ dependencies = [ "risingwave_pb", "risingwave_rpc_client", "risingwave_storage", + "rust_decimal", "rw_futures_util", "scopeguard", "serde_json", diff --git a/src/batch/Cargo.toml b/src/batch/Cargo.toml index 9d9082d33cb85..ee6f757e17376 100644 --- a/src/batch/Cargo.toml +++ b/src/batch/Cargo.toml @@ -19,6 +19,7 @@ assert_matches = "1" async-recursion = "1" async-trait = "0.1" bytes = "1" +chrono = "0.4" either = "1" foyer = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } @@ -46,6 +47,7 @@ risingwave_hummock_sdk = { workspace = true } risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } risingwave_storage = { workspace = true } +rust_decimal = "1" rw_futures_util = { workspace = true } scopeguard = "1" serde_json = "1" diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index 8c858009c9df2..d06009cddbb46 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -15,14 +15,15 @@ use anyhow::{anyhow, Context}; use futures_async_stream::try_stream; use futures_util::stream::StreamExt; -use mysql_async; use mysql_async::prelude::*; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, Datum, Decimal, ScalarImpl}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_pb::batch_plan::plan_node::NodeBody; +use rust_decimal::Decimal as RustDecimal; use thiserror_ext::AsReport; +use {chrono, mysql_async}; use crate::error::BatchError; use crate::executor::{BoxedExecutor, BoxedExecutorBuilder, DataChunk, Executor, ExecutorBuilder}; @@ -72,7 +73,7 @@ macro_rules! mysql_value_to_scalar { let val = $row.take_opt::, _>($i); match val { None => bail!("missing value for column {}, at index {}", $name, $i), - Some(Ok(Some(val))) => Some(ScalarImpl::$variant(val)), + Some(Ok(Some(val))) => Some(ScalarImpl::$variant(val.into())), Some(Ok(None)) => None, Some(Err(e)) => return Err(e.into()), } @@ -87,19 +88,18 @@ fn mysql_cell_to_scalar_impl( ) -> Result { let datum = match data_type { DataType::Boolean => mysql_value_to_scalar!(row, i, name, bool, Bool), - // | DataType::Int16 - // | DataType::Int32 - // | DataType::Int64 - // | DataType::Float32 - // | DataType::Float64 - // | DataType::Decimal, - // | DataType::Date - // | DataType::Time - // | DataType::Timestamp - // | DataType::Varchar => { - // // ScalarAdapter is also fine. But ScalarImpl is more efficient - // row.try_get::<_, Option>(i)? - // } + DataType::Int16 => mysql_value_to_scalar!(row, i, name, i16, Int16), + DataType::Int32 => mysql_value_to_scalar!(row, i, name, i32, Int32), + DataType::Int64 => mysql_value_to_scalar!(row, i, name, i64, Int64), + DataType::Float32 => mysql_value_to_scalar!(row, i, name, f32, Float32), + DataType::Float64 => mysql_value_to_scalar!(row, i, name, f64, Float64), + DataType::Decimal => mysql_value_to_scalar!(row, i, name, RustDecimal, Decimal), + DataType::Date => mysql_value_to_scalar!(row, i, name, chrono::NaiveDate, Date), + DataType::Time => mysql_value_to_scalar!(row, i, name, chrono::NaiveTime, Time), + DataType::Timestamp => { + mysql_value_to_scalar!(row, i, name, chrono::NaiveDateTime, Timestamp) + } + DataType::Varchar => mysql_value_to_scalar!(row, i, name, String, Utf8), _ => { bail!("unsupported data type: {}", data_type) } From f5925d504385088f36747a9a3d354d2d6b55f745 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 28 Oct 2024 15:32:11 +0800 Subject: [PATCH 15/48] add e2e mysql_query slt test --- e2e_test/source_inline/tvf/mysql_query.slt | 40 ++++++++++++++++++++++ 1 file changed, 40 insertions(+) create mode 100644 e2e_test/source_inline/tvf/mysql_query.slt diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt new file mode 100644 index 0000000000000..d32aaddd1bf8c --- /dev/null +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -0,0 +1,40 @@ +control substitution on + +system ok +mysql -e "DROP DATABASE IF EXISTS mysql_tvf; CREATE DATABASE msql_tvf;" + +system ok +mysql -e " +USE mysql_tvf; +CREATE TABLE test ( + id bigint primary key, + v1 bool, + v2 smallint, + v3 integer, + v4 bigint, + v5 float, + v6 double, + v7 numeric, + v8 date, + v9 time, + v10 timestamp, + v11 varchar +); +INSERT INTO test SELECT + 1 as id, + true as v1, + 1 as v2, + 1 as v3, + 1 as v4, + 1.23 as v5, + 1.23 as v6, + 1.0 as v7, + '2021-01-01' as v8, + '00:00:00' as v9, + '2021-01-01 00:00:00' as v10, + 'text' as v11; +" + +query II +select * from mysql_query('$MYSQL_HOST', '$MYSQL_TCP_PORT', '$RISEDEV_MYSQL_USER', '$MYSQL_PWD', 'mysql_tvf', 'select * from test where id > 90;'); +---- From fb18cacac74eccc2714a99d049fa0df67276194f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 28 Oct 2024 15:42:59 +0800 Subject: [PATCH 16/48] fix warn --- src/batch/src/executor/mysql_query.rs | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index d06009cddbb46..296afa0b1e62b 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -12,24 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Context}; +use anyhow::Context; use futures_async_stream::try_stream; use futures_util::stream::StreamExt; use mysql_async::prelude::*; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, Datum, Decimal, ScalarImpl}; +use risingwave_common::types::{DataType, Datum, ScalarImpl}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_pb::batch_plan::plan_node::NodeBody; use rust_decimal::Decimal as RustDecimal; -use thiserror_ext::AsReport; use {chrono, mysql_async}; use crate::error::BatchError; use crate::executor::{BoxedExecutor, BoxedExecutorBuilder, DataChunk, Executor, ExecutorBuilder}; use crate::task::BatchTaskContext; -/// `MySqlQuery` executor. Runs a query against a MySql database. +/// `MySqlQuery` executor. Runs a query against a `MySql` database. pub struct MySqlQueryExecutor { schema: Schema, host: String, @@ -161,7 +160,7 @@ impl MySqlQueryExecutor { // deserialize the rows #[for_await] for row in row_stream { - let mut row = row?; + let row = row?; let owned_row = mysql_row_to_owned_row(row, &self.schema)?; if let Some(chunk) = builder.append_one_row(owned_row) { yield chunk; From 86d8d455bfe07a50fe963fdcf09deb6569c13f58 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 28 Oct 2024 23:28:03 +0800 Subject: [PATCH 17/48] rename db --- e2e_test/source_inline/tvf/mysql_query.slt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index d32aaddd1bf8c..d581912b867b2 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -1,11 +1,11 @@ control substitution on system ok -mysql -e "DROP DATABASE IF EXISTS mysql_tvf; CREATE DATABASE msql_tvf;" +mysql -e "DROP DATABASE IF EXISTS tvf; CREATE DATABASE tvf;" system ok mysql -e " -USE mysql_tvf; +USE tvf; CREATE TABLE test ( id bigint primary key, v1 bool, @@ -36,5 +36,5 @@ INSERT INTO test SELECT " query II -select * from mysql_query('$MYSQL_HOST', '$MYSQL_TCP_PORT', '$RISEDEV_MYSQL_USER', '$MYSQL_PWD', 'mysql_tvf', 'select * from test where id > 90;'); +select * from mysql_query('$MYSQL_HOST', '$MYSQL_TCP_PORT', '$RISEDEV_MYSQL_USER', '$MYSQL_PWD', 'tvf', 'select * from test where id > 90;'); ---- From e1002dfcfadcf1dde9b564f726a64e81da2fb8d5 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 00:18:43 +0800 Subject: [PATCH 18/48] missing comma? --- e2e_test/source_inline/tvf/mysql_query.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index d581912b867b2..01aa030d974fb 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -18,7 +18,7 @@ CREATE TABLE test ( v8 date, v9 time, v10 timestamp, - v11 varchar + v11 varchar, ); INSERT INTO test SELECT 1 as id, From 07aa7a6048c232a5c41b8164c0432b0e90af1b28 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 00:20:27 +0800 Subject: [PATCH 19/48] add varchar len --- e2e_test/source_inline/tvf/mysql_query.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index 01aa030d974fb..b133c03a374c7 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -18,7 +18,7 @@ CREATE TABLE test ( v8 date, v9 time, v10 timestamp, - v11 varchar, + v11 varchar(255) ); INSERT INTO test SELECT 1 as id, From 90085a8e65267d97501f1c2dee4c209179ba065d Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 11:30:20 +0800 Subject: [PATCH 20/48] fix --- src/frontend/src/expr/table_function.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 5bfeea349e9ee..5a62aea858af1 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -419,7 +419,7 @@ impl TableFunction { .await .context("failed to connect to mysql in binder")?; - let query = evaled_args[6].clone(); + let query = evaled_args[5].clone(); let statement = conn .prep(query) .await From 522ad3473c9c012d34db03a85b2bb326a4a4de60 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 11:50:10 +0800 Subject: [PATCH 21/48] fix --- e2e_test/source_inline/tvf/mysql_query.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index b133c03a374c7..24b917a94c347 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -8,7 +8,7 @@ mysql -e " USE tvf; CREATE TABLE test ( id bigint primary key, - v1 bool, + v1 bit, v2 smallint, v3 integer, v4 bigint, From b50c38cb2b1bc2e30fe85bd84e934e37184b41da Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 15:04:04 +0800 Subject: [PATCH 22/48] fix --- src/frontend/src/expr/table_function.rs | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 5a62aea858af1..55351277464dc 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -430,16 +430,22 @@ impl TableFunction { for column in statement.columns() { let name = column.name_str().to_string(); let data_type = match column.column_type() { + // Numeric types MySqlColumnType::MYSQL_TYPE_BIT => DataType::Boolean, MySqlColumnType::MYSQL_TYPE_SHORT => DataType::Int16, MySqlColumnType::MYSQL_TYPE_LONG => DataType::Int32, MySqlColumnType::MYSQL_TYPE_LONGLONG => DataType::Int64, MySqlColumnType::MYSQL_TYPE_FLOAT => DataType::Float32, MySqlColumnType::MYSQL_TYPE_DOUBLE => DataType::Float64, - MySqlColumnType::MYSQL_TYPE_DECIMAL => DataType::Decimal, + MySqlColumnType::MYSQL_TYPE_NEWDECIMAL + | MySqlColumnType::MYSQL_TYPE_DECIMAL => DataType::Decimal, + + // Date time types MySqlColumnType::MYSQL_TYPE_DATE => DataType::Date, MySqlColumnType::MYSQL_TYPE_TIME => DataType::Time, MySqlColumnType::MYSQL_TYPE_TIMESTAMP => DataType::Timestamp, + + // String types MySqlColumnType::MYSQL_TYPE_VARCHAR => DataType::Varchar, _ => { return Err(crate::error::ErrorCode::BindError( From 406f3b0015ef2f353ed08cbdf3f87f6fc1f267df Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 15:48:59 +0800 Subject: [PATCH 23/48] fix --- src/frontend/src/expr/table_function.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 55351277464dc..918aeb1b47253 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -446,7 +446,9 @@ impl TableFunction { MySqlColumnType::MYSQL_TYPE_TIMESTAMP => DataType::Timestamp, // String types - MySqlColumnType::MYSQL_TYPE_VARCHAR => DataType::Varchar, + MySqlColumnType::MYSQL_TYPE_VARCHAR + | MySqlColumnType::MYSQL_TYPE_STRING + | MySqlColumnType::MYSQL_TYPE_VAR_STRING => DataType::Varchar, _ => { return Err(crate::error::ErrorCode::BindError( format!("unsupported column type: {:?}", column.column_type()) From 03bf9077eabbe829dda38eb01f7da1eb0bc1f626 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 16:10:13 +0800 Subject: [PATCH 24/48] add TableFunctionToMySqlQueryRule --- src/frontend/src/optimizer/logical_optimization.rs | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index 766cde4ecfc7e..e413188154569 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -136,6 +136,8 @@ static TABLE_FUNCTION_CONVERT: LazyLock = LazyLock::new(|| { TableFunctionToFileScanRule::create(), // Apply postgres query rule next TableFunctionToPostgresQueryRule::create(), + // Apply mysql query rule next + TableFunctionToMySqlQueryRule::create(), // Apply project set rule last TableFunctionToProjectSetRule::create(), ], @@ -159,6 +161,14 @@ static TABLE_FUNCTION_TO_POSTGRES_QUERY: LazyLock = LazyLock: ) }); +static TABLE_FUNCTION_TO_MYSQL_QUERY: LazyLock = LazyLock::new(|| { + OptimizationStage::new( + "Table Function To MySQL", + vec![TableFunctionToMySqlQueryRule::create()], + ApplyOrder::TopDown, + ) +}); + static VALUES_EXTRACT_PROJECT: LazyLock = LazyLock::new(|| { OptimizationStage::new( "Values Extract Project", @@ -713,6 +723,7 @@ impl LogicalOptimizer { // Table function should be converted into `file_scan` before `project_set`. plan = plan.optimize_by_rules(&TABLE_FUNCTION_TO_FILE_SCAN); plan = plan.optimize_by_rules(&TABLE_FUNCTION_TO_POSTGRES_QUERY); + plan = plan.optimize_by_rules(&TABLE_FUNCTION_TO_MYSQL_QUERY); // In order to unnest a table function, we need to convert it into a `project_set` first. plan = plan.optimize_by_rules(&TABLE_FUNCTION_CONVERT); From e420c8cd70812520be3337ef5970b13f8dc478ce Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 20:48:12 +0800 Subject: [PATCH 25/48] fix --- e2e_test/source_inline/tvf/mysql_query.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index 24b917a94c347..0274a22cdfe2a 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -36,5 +36,5 @@ INSERT INTO test SELECT " query II -select * from mysql_query('$MYSQL_HOST', '$MYSQL_TCP_PORT', '$RISEDEV_MYSQL_USER', '$MYSQL_PWD', 'tvf', 'select * from test where id > 90;'); +select * from mysql_query('$MYSQL_HOST', '$MYSQL_TCP_PORT', '$RISEDEV_MYSQL_USER', '$MYSQL_PWD', 'tvf', 'select * from test;'); ---- From d3923af33f44ffcdf4654e9421d67b4dac0a47f2 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 21:20:28 +0800 Subject: [PATCH 26/48] handle more types --- src/batch/src/executor/mysql_query.rs | 10 +++++++++- src/frontend/src/expr/table_function.rs | 23 +++++++++++++++++++++-- 2 files changed, 30 insertions(+), 3 deletions(-) diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index 296afa0b1e62b..c57ef06997bea 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -86,7 +86,15 @@ fn mysql_cell_to_scalar_impl( name: &str, ) -> Result { let datum = match data_type { - DataType::Boolean => mysql_value_to_scalar!(row, i, name, bool, Bool), + DataType::Boolean => { + let val = row.take_opt::, _>(i); + match val { + None => bail!("missing value for column {}, at index {}", name, i), + Some(Ok(Some(val))) => Some(ScalarImpl::Bool(val != 0)), + Some(Ok(None)) => None, + Some(Err(e)) => return Err(e.into()), + } + } DataType::Int16 => mysql_value_to_scalar!(row, i, name, i16, Int16), DataType::Int32 => mysql_value_to_scalar!(row, i, name, i32, Int32), DataType::Int64 => mysql_value_to_scalar!(row, i, name, i64, Int64), diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 918aeb1b47253..6d816321d74ff 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -432,13 +432,22 @@ impl TableFunction { let data_type = match column.column_type() { // Numeric types MySqlColumnType::MYSQL_TYPE_BIT => DataType::Boolean, + MySqlColumnType::MYSQL_TYPE_TINY => { + if column.column_length() == 1 { + // Bool/Boolean is an alias for TINYINT(1) + DataType::Boolean + } else { + DataType::Int16 + } + } MySqlColumnType::MYSQL_TYPE_SHORT => DataType::Int16, + MySqlColumnType::MYSQL_TYPE_INT24 => DataType::Int32, MySqlColumnType::MYSQL_TYPE_LONG => DataType::Int32, MySqlColumnType::MYSQL_TYPE_LONGLONG => DataType::Int64, MySqlColumnType::MYSQL_TYPE_FLOAT => DataType::Float32, MySqlColumnType::MYSQL_TYPE_DOUBLE => DataType::Float64, - MySqlColumnType::MYSQL_TYPE_NEWDECIMAL - | MySqlColumnType::MYSQL_TYPE_DECIMAL => DataType::Decimal, + MySqlColumnType::MYSQL_TYPE_NEWDECIMAL => DataType::Decimal, + MySqlColumnType::MYSQL_TYPE_DECIMAL => DataType::Decimal, // Date time types MySqlColumnType::MYSQL_TYPE_DATE => DataType::Date, @@ -449,6 +458,16 @@ impl TableFunction { MySqlColumnType::MYSQL_TYPE_VARCHAR | MySqlColumnType::MYSQL_TYPE_STRING | MySqlColumnType::MYSQL_TYPE_VAR_STRING => DataType::Varchar, + + // JSON types + MySqlColumnType::MYSQL_TYPE_JSON => DataType::Jsonb, + + // Binary types + MySqlColumnType::MYSQL_TYPE_BLOB + | MySqlColumnType::MYSQL_TYPE_TINY_BLOB + | MySqlColumnType::MYSQL_TYPE_MEDIUM_BLOB + | MySqlColumnType::MYSQL_TYPE_LONG_BLOB => DataType::Bytea, + _ => { return Err(crate::error::ErrorCode::BindError( format!("unsupported column type: {:?}", column.column_type()) From 90958d50156083a32e57d4d7acbcdc57aed31850 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 21:45:35 +0800 Subject: [PATCH 27/48] try again --- e2e_test/source_inline/tvf/mysql_query.slt | 2 +- src/batch/src/executor/mysql_query.rs | 10 +--------- 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index 0274a22cdfe2a..3336adf108ed4 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -22,7 +22,7 @@ CREATE TABLE test ( ); INSERT INTO test SELECT 1 as id, - true as v1, + 1 as v1, 1 as v2, 1 as v3, 1 as v4, diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index c57ef06997bea..296afa0b1e62b 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -86,15 +86,7 @@ fn mysql_cell_to_scalar_impl( name: &str, ) -> Result { let datum = match data_type { - DataType::Boolean => { - let val = row.take_opt::, _>(i); - match val { - None => bail!("missing value for column {}, at index {}", name, i), - Some(Ok(Some(val))) => Some(ScalarImpl::Bool(val != 0)), - Some(Ok(None)) => None, - Some(Err(e)) => return Err(e.into()), - } - } + DataType::Boolean => mysql_value_to_scalar!(row, i, name, bool, Bool), DataType::Int16 => mysql_value_to_scalar!(row, i, name, i16, Int16), DataType::Int32 => mysql_value_to_scalar!(row, i, name, i32, Int32), DataType::Int64 => mysql_value_to_scalar!(row, i, name, i64, Int64), From ea78a3df2d1c8e6fbe7767956e26c20bccb6a3ff Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 22:13:39 +0800 Subject: [PATCH 28/48] add more context to error --- src/batch/src/error.rs | 3 +-- src/batch/src/executor/mysql_query.rs | 13 +++++++++++-- src/error/src/anyhow.rs | 2 +- 3 files changed, 13 insertions(+), 5 deletions(-) diff --git a/src/batch/src/error.rs b/src/batch/src/error.rs index 688fa1d12c1d2..b2e4d055956da 100644 --- a/src/batch/src/error.rs +++ b/src/batch/src/error.rs @@ -17,7 +17,7 @@ use std::sync::Arc; pub use anyhow::anyhow; -use mysql_async::{Error as MySqlError, FromValueError as MySqlValueConversionError}; +use mysql_async::Error as MySqlError; use parquet::errors::ParquetError; use risingwave_common::array::ArrayError; use risingwave_common::error::{def_anyhow_newtype, def_anyhow_variant, BoxedError}; @@ -197,5 +197,4 @@ def_anyhow_variant! { iceberg::Error => "Iceberg error", ParquetError => "Parquet error", MySqlError => "MySQL error", - MySqlValueConversionError => "MySQL value conversion error" } diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index 296afa0b1e62b..a6ce1ff734066 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -24,7 +24,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use rust_decimal::Decimal as RustDecimal; use {chrono, mysql_async}; -use crate::error::BatchError; +use crate::error::{BatchError, BatchExternalSystemError}; use crate::executor::{BoxedExecutor, BoxedExecutorBuilder, DataChunk, Executor, ExecutorBuilder}; use crate::task::BatchTaskContext; @@ -74,7 +74,16 @@ macro_rules! mysql_value_to_scalar { None => bail!("missing value for column {}, at index {}", $name, $i), Some(Ok(Some(val))) => Some(ScalarImpl::$variant(val.into())), Some(Ok(None)) => None, - Some(Err(e)) => return Err(e.into()), + Some(Err(e)) => { + let e: anyhow::Error = anyhow::Error::new(e.clone()).context(format!( + "failed to deserialize column {} at index {} as value of rust_type: {}", + $name, + $i, + stringify!($ty), + )); + let e = BatchExternalSystemError(e); + return Err(e.into()); + } } }}; } diff --git a/src/error/src/anyhow.rs b/src/error/src/anyhow.rs index 08203c176fcbc..30a46259ffdb5 100644 --- a/src/error/src/anyhow.rs +++ b/src/error/src/anyhow.rs @@ -126,7 +126,7 @@ macro_rules! def_anyhow_newtype { ) => { #[derive(::thiserror::Error, ::std::fmt::Debug)] #[error(transparent)] - $(#[$attr])* $vis struct $name(#[from] #[backtrace] ::anyhow::Error); + $(#[$attr])* $vis struct $name(#[from] #[backtrace] pub ::anyhow::Error); impl $name { /// Unwrap the newtype to get the inner [`anyhow::Error`]. From 2e4c15c6de00aa201de8c1f753f7cddf72baa38f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 22:51:57 +0800 Subject: [PATCH 29/48] refine --- src/batch/src/executor/mysql_query.rs | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index a6ce1ff734066..ceccc5f0cf320 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -75,12 +75,14 @@ macro_rules! mysql_value_to_scalar { Some(Ok(Some(val))) => Some(ScalarImpl::$variant(val.into())), Some(Ok(None)) => None, Some(Err(e)) => { - let e: anyhow::Error = anyhow::Error::new(e.clone()).context(format!( - "failed to deserialize column {} at index {} as value of rust_type: {}", - $name, - $i, - stringify!($ty), - )); + let e: anyhow::Error = anyhow::Error::new(e.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: {}", + $name, + $i, + stringify!($ty), + )); let e = BatchExternalSystemError(e); return Err(e.into()); } From e909dd48c76e072b5f7acfcb68e9793a286e8094 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Tue, 29 Oct 2024 22:52:15 +0800 Subject: [PATCH 30/48] try fix --- e2e_test/source_inline/tvf/mysql_query.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index 3336adf108ed4..94e6e9c01f130 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -8,7 +8,7 @@ mysql -e " USE tvf; CREATE TABLE test ( id bigint primary key, - v1 bit, + v1 bool, v2 smallint, v3 integer, v4 bigint, @@ -21,7 +21,7 @@ CREATE TABLE test ( v11 varchar(255) ); INSERT INTO test SELECT - 1 as id, + true as id, 1 as v1, 1 as v2, 1 as v3, From 30908a07e8f88516864cf975e9e0f7627dea63df Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 12:34:20 +0800 Subject: [PATCH 31/48] fix test --- e2e_test/source_inline/tvf/mysql_query.slt | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index 94e6e9c01f130..fe4e7853e6494 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -21,8 +21,8 @@ CREATE TABLE test ( v11 varchar(255) ); INSERT INTO test SELECT - true as id, - 1 as v1, + 1 as id, + true as v1, 1 as v2, 1 as v3, 1 as v4, @@ -38,3 +38,4 @@ INSERT INTO test SELECT query II select * from mysql_query('$MYSQL_HOST', '$MYSQL_TCP_PORT', '$RISEDEV_MYSQL_USER', '$MYSQL_PWD', 'tvf', 'select * from test;'); ---- +1 t 1 1 1 1.23 1.23 1 2021-01-01 00:00:00 2021-01-01 00:00:00 text \ No newline at end of file From 26062515cd16c9a8a72d03761d1769e8648a8724 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 12:40:15 +0800 Subject: [PATCH 32/48] explicitly bind unsupported types --- src/frontend/src/expr/table_function.rs | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 6d816321d74ff..1a90f53a7f492 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -431,7 +431,6 @@ impl TableFunction { let name = column.name_str().to_string(); let data_type = match column.column_type() { // Numeric types - MySqlColumnType::MYSQL_TYPE_BIT => DataType::Boolean, MySqlColumnType::MYSQL_TYPE_TINY => { if column.column_length() == 1 { // Bool/Boolean is an alias for TINYINT(1) @@ -468,7 +467,19 @@ impl TableFunction { | MySqlColumnType::MYSQL_TYPE_MEDIUM_BLOB | MySqlColumnType::MYSQL_TYPE_LONG_BLOB => DataType::Bytea, - _ => { + MySqlColumnType::MYSQL_TYPE_UNKNOWN + | MySqlColumnType::MYSQL_TYPE_BIT + | MySqlColumnType::MYSQL_TYPE_TYPED_ARRAY + | MySqlColumnType::MYSQL_TYPE_ENUM + | MySqlColumnType::MYSQL_TYPE_SET + | MySqlColumnType::MYSQL_TYPE_GEOMETRY + | MySqlColumnType::MYSQL_TYPE_NULL + | MySqlColumnType::MYSQL_TYPE_TIMESTAMP2 + | MySqlColumnType::MYSQL_TYPE_DATETIME + | MySqlColumnType::MYSQL_TYPE_DATETIME2 + | MySqlColumnType::MYSQL_TYPE_TIME2 + | MySqlColumnType::MYSQL_TYPE_YEAR + | MySqlColumnType::MYSQL_TYPE_NEWDATE => { return Err(crate::error::ErrorCode::BindError( format!("unsupported column type: {:?}", column.column_type()) .to_string(), From 20b224f4e0d2499ce7862e1e5033bbb833399d63 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 12:49:46 +0800 Subject: [PATCH 33/48] support more types --- src/frontend/src/expr/table_function.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 1a90f53a7f492..c0950e640a5eb 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -449,9 +449,15 @@ impl TableFunction { MySqlColumnType::MYSQL_TYPE_DECIMAL => DataType::Decimal, // Date time types + MySqlColumnType::MYSQL_TYPE_YEAR => DataType::Int32, MySqlColumnType::MYSQL_TYPE_DATE => DataType::Date, + MySqlColumnType::MYSQL_TYPE_NEWDATE => DataType::Date, MySqlColumnType::MYSQL_TYPE_TIME => DataType::Time, - MySqlColumnType::MYSQL_TYPE_TIMESTAMP => DataType::Timestamp, + MySqlColumnType::MYSQL_TYPE_TIME2 => DataType::Time, + MySqlColumnType::MYSQL_TYPE_DATETIME => DataType::Timestamp, + MySqlColumnType::MYSQL_TYPE_DATETIME2 => DataType::Timestamp, + MySqlColumnType::MYSQL_TYPE_TIMESTAMP => DataType::Timestamptz, + MySqlColumnType::MYSQL_TYPE_TIMESTAMP2 => DataType::Timestamptz, // String types MySqlColumnType::MYSQL_TYPE_VARCHAR @@ -473,13 +479,7 @@ impl TableFunction { | MySqlColumnType::MYSQL_TYPE_ENUM | MySqlColumnType::MYSQL_TYPE_SET | MySqlColumnType::MYSQL_TYPE_GEOMETRY - | MySqlColumnType::MYSQL_TYPE_NULL - | MySqlColumnType::MYSQL_TYPE_TIMESTAMP2 - | MySqlColumnType::MYSQL_TYPE_DATETIME - | MySqlColumnType::MYSQL_TYPE_DATETIME2 - | MySqlColumnType::MYSQL_TYPE_TIME2 - | MySqlColumnType::MYSQL_TYPE_YEAR - | MySqlColumnType::MYSQL_TYPE_NEWDATE => { + | MySqlColumnType::MYSQL_TYPE_NULL => { return Err(crate::error::ErrorCode::BindError( format!("unsupported column type: {:?}", column.column_type()) .to_string(), From adcf79b25b631ec9f0c09c1cce936a56ec5ca85c Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 15:38:08 +0800 Subject: [PATCH 34/48] extract common logic --- Cargo.lock | 1 + src/common/Cargo.toml | 1 + src/common/src/lib.rs | 2 + src/common/src/mysql.rs | 164 ++++++++++++++++++++++++++++++++++++++++ 4 files changed, 168 insertions(+) create mode 100644 src/common/src/mysql.rs diff --git a/Cargo.lock b/Cargo.lock index 275024641e22e..fd77180f9d678 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10663,6 +10663,7 @@ dependencies = [ "madsim-tokio", "memcomparable", "more-asserts", + "mysql_async", "num-integer", "num-traits", "number_prefix", diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 8b341fb621fe6..27d19402dc6ac 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -59,6 +59,7 @@ itoa = "1.0" jsonbb = { workspace = true } lru = { workspace = true } memcomparable = { version = "0.2", features = ["decimal"] } +mysql_async = { workspace = true } num-integer = "0.1" num-traits = "0.2" number_prefix = "0.4.0" diff --git a/src/common/src/lib.rs b/src/common/src/lib.rs index b47076c645949..86b2585ca9529 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -80,6 +80,7 @@ pub use { risingwave_license as license, }; pub mod lru; +pub mod mysql; pub mod opts; pub mod range; pub mod row; @@ -91,6 +92,7 @@ pub mod test_utils; pub mod transaction; pub mod types; pub mod vnode_mapping; + pub mod test_prelude { pub use super::array::{DataChunkTestExt, StreamChunkTestExt}; pub use super::catalog::test_utils::ColumnDescTestExt; diff --git a/src/common/src/mysql.rs b/src/common/src/mysql.rs new file mode 100644 index 0000000000000..55f303b5beba2 --- /dev/null +++ b/src/common/src/mysql.rs @@ -0,0 +1,164 @@ +// 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 anyhow::anyhow; +use chrono::NaiveDate; +use mysql_async::Row as MysqlRow; +use risingwave_common::types::{ + DataType, Date, Decimal, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, +}; +use rust_decimal::Decimal as RustDecimal; + +use crate::types::Datum; + +macro_rules! handle_data_type { + ($row:expr, $i:expr, $name:expr, $type:ty) => {{ + let datum_opt = $row.take_opt::, _>($i); + datum_opt.map(|res| match res { + Ok(val) => Ok(val.map(|v| ScalarImpl::from(v))), + Err(e) => Err(anyhow::Error::new(e.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: {}", + $name, + $i, + stringify!($ty), + ))), + }) + }}; + ($row:expr, $i:expr, $name:expr, $type:ty, $rw_type:ty) => {{ + let datum_opt = $row.take_opt::, _>($i); + datum_opt.map(|res| match res { + Ok(val) => Ok(val.map(|v| ScalarImpl::from(<$rw_type>::from(v)))), + Err(e) => Err(anyhow::Error::new(e.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: {}", + $name, + $i, + stringify!($ty), + ))), + }) + }}; +} + +/// The decoding result can be interpreted as follows: +/// None => No value was found in the column. +/// Some(Ok(value)) => The value was found and successfully decoded. +/// Some(Err(error)) => The value was found but could not be decoded, +/// either because it was not supported, +/// or there was an error during conversion. +/// This error is wrapped via anyhow, so it is opaque. +pub fn mysql_datum_to_rw_datum( + mysql_row: &mut MysqlRow, + mysql_datum_index: usize, + column_name: &str, + rw_data_type: DataType, +) -> Option> { + match rw_data_type { + DataType::Boolean => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, bool) + } + DataType::Int16 => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, i16) + } + DataType::Int32 => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, i32) + } + DataType::Int64 => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, i64) + } + DataType::Float32 => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, f32) + } + DataType::Float64 => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, f64) + } + DataType::Decimal => { + handle_data_type!( + mysql_row, + mysql_datum_index, + column_name, + RustDecimal, + Decimal + ) + } + DataType::Varchar => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, String) + } + DataType::Date => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, NaiveDate, Date) + } + DataType::Time => { + handle_data_type!( + mysql_row, + mysql_datum_index, + column_name, + chrono::NaiveTime, + Time + ) + } + DataType::Timestamp => { + handle_data_type!( + mysql_row, + mysql_datum_index, + column_name, + chrono::NaiveDateTime, + Timestamp + ) + } + DataType::Timestamptz => mysql_row + .take_opt::, _>(mysql_datum_index) + .map(|res| match res { + Ok(val) => Ok(val.map(|v| { + ScalarImpl::from(Timestamptz::from_micros(v.and_utc().timestamp_micros())) + })), + Err(err) => Err(anyhow::Error::new(err.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: chrono::NaiveDateTime", + column_name, mysql_datum_index, + ))), + }), + DataType::Bytea => mysql_row + .take_opt::>, _>(mysql_datum_index) + .map(|res| match res { + Ok(val) => Ok(val.map(ScalarImpl::from)), + Err(err) => Err(anyhow::Error::new(err.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: Vec", + column_name, mysql_datum_index, + ))), + }), + DataType::Jsonb => { + handle_data_type!( + mysql_row, + mysql_datum_index, + column_name, + serde_json::Value, + JsonbVal + ) + } + DataType::Interval + | DataType::Struct(_) + | DataType::List(_) + | DataType::Int256 + | DataType::Serial + | DataType::Map(_) => Some(Err(anyhow!( + "unsupported data type: {}, set to null", + rw_data_type + ))), + } +} From 90f7d714868dfbfbf76cd708ef76e626f1741fd9 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 15:46:38 +0800 Subject: [PATCH 35/48] reuse common parsing logic --- src/batch/src/executor/mysql_query.rs | 63 ++------------ src/common/src/mysql.rs | 70 +++++++++------- src/connector/src/parser/mysql.rs | 113 ++------------------------ 3 files changed, 54 insertions(+), 192 deletions(-) diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index ceccc5f0cf320..edde81a699511 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -15,14 +15,13 @@ use anyhow::Context; use futures_async_stream::try_stream; use futures_util::stream::StreamExt; +use mysql_async; use mysql_async::prelude::*; use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::mysql::mysql_datum_to_rw_datum; use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, Datum, ScalarImpl}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_pb::batch_plan::plan_node::NodeBody; -use rust_decimal::Decimal as RustDecimal; -use {chrono, mysql_async}; use crate::error::{BatchError, BatchExternalSystemError}; use crate::executor::{BoxedExecutor, BoxedExecutorBuilder, DataChunk, Executor, ExecutorBuilder}; @@ -61,60 +60,16 @@ pub fn mysql_row_to_owned_row( for i in 0..schema.fields.len() { let rw_field = &schema.fields[i]; let name = rw_field.name.as_str(); - let datum = mysql_cell_to_scalar_impl(&mut row, &rw_field.data_type, i, name)?; - datums.push(datum); - } - Ok(OwnedRow::new(datums)) -} - -macro_rules! mysql_value_to_scalar { - ($row:ident, $i:ident, $name:ident, $ty:ty, $variant:ident) => {{ - let val = $row.take_opt::, _>($i); - match val { - None => bail!("missing value for column {}, at index {}", $name, $i), - Some(Ok(Some(val))) => Some(ScalarImpl::$variant(val.into())), - Some(Ok(None)) => None, - Some(Err(e)) => { - let e: anyhow::Error = anyhow::Error::new(e.clone()) - .context("failed to deserialize MySQL value into rust value") - .context(format!( - "column: {}, index: {}, rust_type: {}", - $name, - $i, - stringify!($ty), - )); + let datum = match mysql_datum_to_rw_datum(&mut row, i, name, &rw_field.data_type) { + Ok(val) => val, + Err(e) => { let e = BatchExternalSystemError(e); return Err(e.into()); } - } - }}; -} - -fn mysql_cell_to_scalar_impl( - row: &mut mysql_async::Row, - data_type: &DataType, - i: usize, - name: &str, -) -> Result { - let datum = match data_type { - DataType::Boolean => mysql_value_to_scalar!(row, i, name, bool, Bool), - DataType::Int16 => mysql_value_to_scalar!(row, i, name, i16, Int16), - DataType::Int32 => mysql_value_to_scalar!(row, i, name, i32, Int32), - DataType::Int64 => mysql_value_to_scalar!(row, i, name, i64, Int64), - DataType::Float32 => mysql_value_to_scalar!(row, i, name, f32, Float32), - DataType::Float64 => mysql_value_to_scalar!(row, i, name, f64, Float64), - DataType::Decimal => mysql_value_to_scalar!(row, i, name, RustDecimal, Decimal), - DataType::Date => mysql_value_to_scalar!(row, i, name, chrono::NaiveDate, Date), - DataType::Time => mysql_value_to_scalar!(row, i, name, chrono::NaiveTime, Time), - DataType::Timestamp => { - mysql_value_to_scalar!(row, i, name, chrono::NaiveDateTime, Timestamp) - } - DataType::Varchar => mysql_value_to_scalar!(row, i, name, String, Utf8), - _ => { - bail!("unsupported data type: {}", data_type) - } - }; - Ok(datum) + }; + datums.push(datum); + } + Ok(OwnedRow::new(datums)) } impl MySqlQueryExecutor { diff --git a/src/common/src/mysql.rs b/src/common/src/mysql.rs index 55f303b5beba2..8d6cd18b2d8c4 100644 --- a/src/common/src/mysql.rs +++ b/src/common/src/mysql.rs @@ -24,10 +24,10 @@ use crate::types::Datum; macro_rules! handle_data_type { ($row:expr, $i:expr, $name:expr, $type:ty) => {{ - let datum_opt = $row.take_opt::, _>($i); - datum_opt.map(|res| match res { - Ok(val) => Ok(val.map(|v| ScalarImpl::from(v))), - Err(e) => Err(anyhow::Error::new(e.clone()) + match $row.take_opt::, _>($i) { + None => bail!("no value found at column: {}, index: {}", $name, $i), + Some(Ok(val)) => Ok(val.map(|v| ScalarImpl::from(v))), + Some(Err(e)) => Err(anyhow::Error::new(e.clone()) .context("failed to deserialize MySQL value into rust value") .context(format!( "column: {}, index: {}, rust_type: {}", @@ -35,13 +35,13 @@ macro_rules! handle_data_type { $i, stringify!($ty), ))), - }) + } }}; ($row:expr, $i:expr, $name:expr, $type:ty, $rw_type:ty) => {{ - let datum_opt = $row.take_opt::, _>($i); - datum_opt.map(|res| match res { - Ok(val) => Ok(val.map(|v| ScalarImpl::from(<$rw_type>::from(v)))), - Err(e) => Err(anyhow::Error::new(e.clone()) + match $row.take_opt::, _>($i) { + None => bail!("no value found at column: {}, index: {}", $name, $i), + Some(Ok(val)) => Ok(val.map(|v| ScalarImpl::from(<$rw_type>::from(v)))), + Some(Err(e)) => Err(anyhow::Error::new(e.clone()) .context("failed to deserialize MySQL value into rust value") .context(format!( "column: {}, index: {}, rust_type: {}", @@ -49,7 +49,7 @@ macro_rules! handle_data_type { $i, stringify!($ty), ))), - }) + } }}; } @@ -64,8 +64,8 @@ pub fn mysql_datum_to_rw_datum( mysql_row: &mut MysqlRow, mysql_datum_index: usize, column_name: &str, - rw_data_type: DataType, -) -> Option> { + rw_data_type: &DataType, +) -> Result { match rw_data_type { DataType::Boolean => { handle_data_type!(mysql_row, mysql_datum_index, column_name, bool) @@ -118,30 +118,38 @@ pub fn mysql_datum_to_rw_datum( Timestamp ) } - DataType::Timestamptz => mysql_row - .take_opt::, _>(mysql_datum_index) - .map(|res| match res { - Ok(val) => Ok(val.map(|v| { + DataType::Timestamptz => { + match mysql_row.take_opt::, _>(mysql_datum_index) { + None => bail!( + "no value found at column: {}, index: {}", + column_name, + mysql_datum_index + ), + Some(Ok(val)) => Ok(val.map(|v| { ScalarImpl::from(Timestamptz::from_micros(v.and_utc().timestamp_micros())) })), - Err(err) => Err(anyhow::Error::new(err.clone()) + Some(Err(err)) => Err(anyhow::Error::new(err.clone()) .context("failed to deserialize MySQL value into rust value") .context(format!( "column: {}, index: {}, rust_type: chrono::NaiveDateTime", column_name, mysql_datum_index, ))), - }), - DataType::Bytea => mysql_row - .take_opt::>, _>(mysql_datum_index) - .map(|res| match res { - Ok(val) => Ok(val.map(ScalarImpl::from)), - Err(err) => Err(anyhow::Error::new(err.clone()) - .context("failed to deserialize MySQL value into rust value") - .context(format!( - "column: {}, index: {}, rust_type: Vec", - column_name, mysql_datum_index, - ))), - }), + } + } + DataType::Bytea => match mysql_row.take_opt::>, _>(mysql_datum_index) { + None => bail!( + "no value found at column: {}, index: {}", + column_name, + mysql_datum_index + ), + Some(Ok(val)) => Ok(val.map(ScalarImpl::from)), + Some(Err(err)) => Err(anyhow::Error::new(err.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: Vec", + column_name, mysql_datum_index, + ))), + }, DataType::Jsonb => { handle_data_type!( mysql_row, @@ -156,9 +164,9 @@ pub fn mysql_datum_to_rw_datum( | DataType::List(_) | DataType::Int256 | DataType::Serial - | DataType::Map(_) => Some(Err(anyhow!( + | DataType::Map(_) => Err(anyhow!( "unsupported data type: {}, set to null", rw_data_type - ))), + )), } } diff --git a/src/connector/src/parser/mysql.rs b/src/connector/src/parser/mysql.rs index fe9b77c643de7..6aabe9d3ccafe 100644 --- a/src/connector/src/parser/mysql.rs +++ b/src/connector/src/parser/mysql.rs @@ -14,128 +14,27 @@ use std::sync::LazyLock; -use chrono::NaiveDate; use mysql_async::Row as MysqlRow; use risingwave_common::catalog::Schema; use risingwave_common::log::LogSuppresser; use risingwave_common::row::OwnedRow; -use risingwave_common::types::{ - DataType, Date, Decimal, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, -}; -use rust_decimal::Decimal as RustDecimal; use thiserror_ext::AsReport; use crate::parser::util::log_error; static LOG_SUPPERSSER: LazyLock = LazyLock::new(LogSuppresser::default); - -macro_rules! handle_data_type { - ($row:expr, $i:expr, $name:expr, $type:ty) => {{ - let res = $row.take_opt::, _>($i).unwrap_or(Ok(None)); - match res { - Ok(val) => val.map(|v| ScalarImpl::from(v)), - Err(err) => { - log_error!($name, err, "parse column failed"); - None - } - } - }}; - ($row:expr, $i:expr, $name:expr, $type:ty, $rw_type:ty) => {{ - let res = $row.take_opt::, _>($i).unwrap_or(Ok(None)); - match res { - Ok(val) => val.map(|v| ScalarImpl::from(<$rw_type>::from(v))), - Err(err) => { - log_error!($name, err, "parse column failed"); - None - } - } - }}; -} +use risingwave_common::mysql::mysql_datum_to_rw_datum; pub fn mysql_row_to_owned_row(mysql_row: &mut MysqlRow, schema: &Schema) -> OwnedRow { let mut datums = vec![]; for i in 0..schema.fields.len() { let rw_field = &schema.fields[i]; let name = rw_field.name.as_str(); - let datum = { - match rw_field.data_type { - DataType::Boolean => { - handle_data_type!(mysql_row, i, name, bool) - } - DataType::Int16 => { - handle_data_type!(mysql_row, i, name, i16) - } - DataType::Int32 => { - handle_data_type!(mysql_row, i, name, i32) - } - DataType::Int64 => { - handle_data_type!(mysql_row, i, name, i64) - } - DataType::Float32 => { - handle_data_type!(mysql_row, i, name, f32) - } - DataType::Float64 => { - handle_data_type!(mysql_row, i, name, f64) - } - DataType::Decimal => { - handle_data_type!(mysql_row, i, name, RustDecimal, Decimal) - } - DataType::Varchar => { - handle_data_type!(mysql_row, i, name, String) - } - DataType::Date => { - handle_data_type!(mysql_row, i, name, NaiveDate, Date) - } - DataType::Time => { - handle_data_type!(mysql_row, i, name, chrono::NaiveTime, Time) - } - DataType::Timestamp => { - handle_data_type!(mysql_row, i, name, chrono::NaiveDateTime, Timestamp) - } - DataType::Timestamptz => { - let res = mysql_row - .take_opt::, _>(i) - .unwrap_or(Ok(None)); - match res { - Ok(val) => val.map(|v| { - ScalarImpl::from(Timestamptz::from_micros( - v.and_utc().timestamp_micros(), - )) - }), - Err(err) => { - log_error!(name, err, "parse column failed"); - None - } - } - } - DataType::Bytea => { - let res = mysql_row - .take_opt::>, _>(i) - .unwrap_or(Ok(None)); - match res { - Ok(val) => val.map(|v| ScalarImpl::from(v.into_boxed_slice())), - Err(err) => { - log_error!(name, err, "parse column failed"); - None - } - } - } - DataType::Jsonb => { - handle_data_type!(mysql_row, i, name, serde_json::Value, JsonbVal) - } - DataType::Interval - | DataType::Struct(_) - | DataType::List(_) - | DataType::Int256 - | DataType::Serial - | DataType::Map(_) => { - // Interval, Struct, List, Int256 are not supported - // XXX: is this branch reachable? - if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { - tracing::warn!(column = rw_field.name, ?rw_field.data_type, suppressed_count, "unsupported data type, set to null"); - } - None - } + let datum = match mysql_datum_to_rw_datum(mysql_row, i, name, &rw_field.data_type) { + Ok(val) => val, + Err(e) => { + log_error!(name, e, "parse column failed"); + None } }; datums.push(datum); From 48503b0c543f164b8fb77c97c33351d9c6a1336a Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 16:53:02 +0800 Subject: [PATCH 36/48] add risedev profile for local inline tests --- risedev.yml | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/risedev.yml b/risedev.yml index 0ec9b8b09af79..8e3668dcb49c2 100644 --- a/risedev.yml +++ b/risedev.yml @@ -959,6 +959,25 @@ profile: address: schemaregistry port: 8082 + local-inline-source-test: + config-path: src/config/ci-recovery.toml + steps: + - use: minio + - use: sqlite + - use: meta-node + meta-backend: sqlite + - use: compute-node + enable-tiered-cache: true + - use: frontend + - use: compactor + - use: pubsub + persist-data: true + - use: kafka + persist-data: true + - use: schema-registry + - use: mysql + - use: postgres + ci-inline-source-test: config-path: src/config/ci-recovery.toml steps: From 38d447c1e17e4de575e0344a9383e9c6d57c0dac Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 16:53:36 +0800 Subject: [PATCH 37/48] make handling of tinyint straightforward --- src/common/src/mysql.rs | 8 +++----- src/frontend/src/expr/table_function.rs | 11 +++-------- 2 files changed, 6 insertions(+), 13 deletions(-) diff --git a/src/common/src/mysql.rs b/src/common/src/mysql.rs index 8d6cd18b2d8c4..60853212c6cb8 100644 --- a/src/common/src/mysql.rs +++ b/src/common/src/mysql.rs @@ -33,7 +33,7 @@ macro_rules! handle_data_type { "column: {}, index: {}, rust_type: {}", $name, $i, - stringify!($ty), + stringify!($type), ))), } }}; @@ -67,9 +67,6 @@ pub fn mysql_datum_to_rw_datum( rw_data_type: &DataType, ) -> Result { match rw_data_type { - DataType::Boolean => { - handle_data_type!(mysql_row, mysql_datum_index, column_name, bool) - } DataType::Int16 => { handle_data_type!(mysql_row, mysql_datum_index, column_name, i16) } @@ -159,7 +156,8 @@ pub fn mysql_datum_to_rw_datum( JsonbVal ) } - DataType::Interval + DataType::Boolean + | DataType::Interval | DataType::Struct(_) | DataType::List(_) | DataType::Int256 diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index c0950e640a5eb..c3ba0a622353b 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -431,14 +431,9 @@ impl TableFunction { let name = column.name_str().to_string(); let data_type = match column.column_type() { // Numeric types - MySqlColumnType::MYSQL_TYPE_TINY => { - if column.column_length() == 1 { - // Bool/Boolean is an alias for TINYINT(1) - DataType::Boolean - } else { - DataType::Int16 - } - } + // NOTE(kwannoel): Although `bool/boolean` is a synonym of TINY(1) in MySQL, + // we treat it as Int16 here. It is better to be straightforward in our conversion. + MySqlColumnType::MYSQL_TYPE_TINY => DataType::Int16, MySqlColumnType::MYSQL_TYPE_SHORT => DataType::Int16, MySqlColumnType::MYSQL_TYPE_INT24 => DataType::Int32, MySqlColumnType::MYSQL_TYPE_LONG => DataType::Int32, From f3ec0ff1dab94fd54d1c00b4d83d4a5491b35396 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 17:21:36 +0800 Subject: [PATCH 38/48] handle more types --- src/common/src/mysql.rs | 27 +++++++++++++++++++++++-- src/frontend/src/expr/table_function.rs | 9 +++++++-- 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/src/common/src/mysql.rs b/src/common/src/mysql.rs index 60853212c6cb8..c1df361bf3a7b 100644 --- a/src/common/src/mysql.rs +++ b/src/common/src/mysql.rs @@ -67,6 +67,30 @@ pub fn mysql_datum_to_rw_datum( rw_data_type: &DataType, ) -> Result { match rw_data_type { + DataType::Boolean => { + // Bit + match mysql_row.take_opt::>, _>(mysql_datum_index) { + None => bail!( + "no value found at column: {}, index: {}", + column_name, + mysql_datum_index + ), + Some(Ok(val)) => match val { + None => Ok(None), + Some(val) => match val.as_slice() { + [0] => Ok(Some(ScalarImpl::from(false))), + [1] => Ok(Some(ScalarImpl::from(true))), + _ => Err(anyhow!("invalid value for boolean: {:?}", val)), + }, + }, + Some(Err(e)) => Err(anyhow::Error::new(e.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: Vec", + column_name, mysql_datum_index, + ))), + } + } DataType::Int16 => { handle_data_type!(mysql_row, mysql_datum_index, column_name, i16) } @@ -156,8 +180,7 @@ pub fn mysql_datum_to_rw_datum( JsonbVal ) } - DataType::Boolean - | DataType::Interval + DataType::Interval | DataType::Struct(_) | DataType::List(_) | DataType::Int256 diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index c3ba0a622353b..ef398141ac5d2 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -430,6 +430,11 @@ impl TableFunction { for column in statement.columns() { let name = column.name_str().to_string(); let data_type = match column.column_type() { + // Boolean types + MySqlColumnType::MYSQL_TYPE_BIT if column.column_length() == 1 => { + DataType::Boolean + } + // Numeric types // NOTE(kwannoel): Although `bool/boolean` is a synonym of TINY(1) in MySQL, // we treat it as Int16 here. It is better to be straightforward in our conversion. @@ -463,13 +468,13 @@ impl TableFunction { MySqlColumnType::MYSQL_TYPE_JSON => DataType::Jsonb, // Binary types - MySqlColumnType::MYSQL_TYPE_BLOB + MySqlColumnType::MYSQL_TYPE_BIT + | MySqlColumnType::MYSQL_TYPE_BLOB | MySqlColumnType::MYSQL_TYPE_TINY_BLOB | MySqlColumnType::MYSQL_TYPE_MEDIUM_BLOB | MySqlColumnType::MYSQL_TYPE_LONG_BLOB => DataType::Bytea, MySqlColumnType::MYSQL_TYPE_UNKNOWN - | MySqlColumnType::MYSQL_TYPE_BIT | MySqlColumnType::MYSQL_TYPE_TYPED_ARRAY | MySqlColumnType::MYSQL_TYPE_ENUM | MySqlColumnType::MYSQL_TYPE_SET From 02d983d9aabb437a034aa9b58f401dd57e816df7 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 17:21:41 +0800 Subject: [PATCH 39/48] test all types --- e2e_test/source_inline/tvf/mysql_query.slt | 72 +++++++++++++++------- 1 file changed, 50 insertions(+), 22 deletions(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index fe4e7853e6494..babfbfea07724 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -8,34 +8,62 @@ mysql -e " USE tvf; CREATE TABLE test ( id bigint primary key, + v0 bit, v1 bool, - v2 smallint, - v3 integer, - v4 bigint, - v5 float, - v6 double, - v7 numeric, - v8 date, - v9 time, - v10 timestamp, - v11 varchar(255) + v2 tinyint(1), + v3 tinyint(2), + v4 smallint, + v5 mediumint, + v6 integer, + v7 bigint, + v8 float, + v9 double, + v10 numeric(4, 2), + v11 decimal(4, 2), + v12 char(255), + v13 varchar(255), + v14 bit(10), + v15 tinyblob, + v16 blob, + v17 mediumblob, + v18 longblob, + v19 date, + v20 time, + v21 timestamp ); INSERT INTO test SELECT 1 as id, + true as v0, true as v1, - 1 as v2, - 1 as v3, - 1 as v4, - 1.23 as v5, - 1.23 as v6, - 1.0 as v7, - '2021-01-01' as v8, - '00:00:00' as v9, - '2021-01-01 00:00:00' as v10, - 'text' as v11; + 2 as v2, + 3 as v3, + 4 as v4, + 5 as v5, + 6 as v6, + 7 as v7, + 1.08 as v8, + 1.09 as v9, + 1.10 as v10, + 1.11 as v11, + 'char' as v12, + 'varchar' as v13, + b'1010' as v14, + x'16' as v15, + x'17' as v16, + x'18' as v17, + x'19' as v18, + '2021-01-01' as v19, + '12:34:56' as v20, + '2021-01-01 12:34:56' as v21; " -query II +query select * from mysql_query('$MYSQL_HOST', '$MYSQL_TCP_PORT', '$RISEDEV_MYSQL_USER', '$MYSQL_PWD', 'tvf', 'select * from test;'); ---- -1 t 1 1 1 1.23 1.23 1 2021-01-01 00:00:00 2021-01-01 00:00:00 text \ No newline at end of file +1 t 1 2 3 4 5 6 7 1.08 1.09 1.10 1.11 char varchar \x000a \x16 \x17 \x18 \x19 2021-01-01 12:34:56 2021-01-01 12:34:56+00:00 + +system ok +mysql -e " +USE tvf; +DROP DATABASE tvf; +" From e7d607044b21d1b51d32fec86ae46ef79be50d9b Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 17:35:25 +0800 Subject: [PATCH 40/48] test jsonb --- e2e_test/source_inline/tvf/mysql_query.slt | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index babfbfea07724..d6055c5b64df5 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -29,7 +29,8 @@ CREATE TABLE test ( v18 longblob, v19 date, v20 time, - v21 timestamp + v21 timestamp, + v22 json ); INSERT INTO test SELECT 1 as id, @@ -54,16 +55,17 @@ INSERT INTO test SELECT x'19' as v18, '2021-01-01' as v19, '12:34:56' as v20, - '2021-01-01 12:34:56' as v21; + '2021-01-01 12:34:56' as v21, + JSON_OBJECT('key1', 1, 'key2', 'abc'); " query select * from mysql_query('$MYSQL_HOST', '$MYSQL_TCP_PORT', '$RISEDEV_MYSQL_USER', '$MYSQL_PWD', 'tvf', 'select * from test;'); ---- -1 t 1 2 3 4 5 6 7 1.08 1.09 1.10 1.11 char varchar \x000a \x16 \x17 \x18 \x19 2021-01-01 12:34:56 2021-01-01 12:34:56+00:00 +1 t 1 2 3 4 5 6 7 1.08 1.09 1.10 1.11 char varchar \x000a \x16 \x17 \x18 \x19 2021-01-01 12:34:56 2021-01-01 12:34:56+00:00 {"key1": 1, "key2": "abc"} system ok mysql -e " USE tvf; DROP DATABASE tvf; -" +" \ No newline at end of file From 5f31f3d21496e51f765f6bc419559283900c2f5d Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 18:00:45 +0800 Subject: [PATCH 41/48] add chrono feature --- Cargo.lock | 1 + src/common/Cargo.toml | 3 +++ 2 files changed, 4 insertions(+) diff --git a/Cargo.lock b/Cargo.lock index fd77180f9d678..889caaab6f9f9 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10664,6 +10664,7 @@ dependencies = [ "memcomparable", "more-asserts", "mysql_async", + "mysql_common", "num-integer", "num-traits", "number_prefix", diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 27d19402dc6ac..7facfde900cc3 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -60,6 +60,9 @@ jsonbb = { workspace = true } lru = { workspace = true } memcomparable = { version = "0.2", features = ["decimal"] } mysql_async = { workspace = true } +mysql_common = { version = "0.32", default-features = false, features = [ + "chrono", +] } num-integer = "0.1" num-traits = "0.2" number_prefix = "0.4.0" From bbbca68625dde36afc3d85e9d0db3f95c4f712e7 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 18:09:51 +0800 Subject: [PATCH 42/48] test null --- e2e_test/source_inline/tvf/mysql_query.slt | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/e2e_test/source_inline/tvf/mysql_query.slt b/e2e_test/source_inline/tvf/mysql_query.slt index d6055c5b64df5..56acf0598244c 100644 --- a/e2e_test/source_inline/tvf/mysql_query.slt +++ b/e2e_test/source_inline/tvf/mysql_query.slt @@ -30,7 +30,8 @@ CREATE TABLE test ( v19 date, v20 time, v21 timestamp, - v22 json + v22 json, + v23 int ); INSERT INTO test SELECT 1 as id, @@ -56,13 +57,14 @@ INSERT INTO test SELECT '2021-01-01' as v19, '12:34:56' as v20, '2021-01-01 12:34:56' as v21, - JSON_OBJECT('key1', 1, 'key2', 'abc'); + JSON_OBJECT('key1', 1, 'key2', 'abc') as v22, + null as v23; " query select * from mysql_query('$MYSQL_HOST', '$MYSQL_TCP_PORT', '$RISEDEV_MYSQL_USER', '$MYSQL_PWD', 'tvf', 'select * from test;'); ---- -1 t 1 2 3 4 5 6 7 1.08 1.09 1.10 1.11 char varchar \x000a \x16 \x17 \x18 \x19 2021-01-01 12:34:56 2021-01-01 12:34:56+00:00 {"key1": 1, "key2": "abc"} +1 t 1 2 3 4 5 6 7 1.08 1.09 1.10 1.11 char varchar \x000a \x16 \x17 \x18 \x19 2021-01-01 12:34:56 2021-01-01 12:34:56+00:00 {"key1": 1, "key2": "abc"} NULL system ok mysql -e " From 5b54d1b13f1269fe7257af7bb62c68868c16656d Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 18:09:59 +0800 Subject: [PATCH 43/48] cleanup docs --- src/batch/src/error.rs | 3 ++- src/common/src/mysql.rs | 12 +++++------- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/src/batch/src/error.rs b/src/batch/src/error.rs index b2e4d055956da..b0723277a6f5e 100644 --- a/src/batch/src/error.rs +++ b/src/batch/src/error.rs @@ -17,6 +17,7 @@ use std::sync::Arc; pub use anyhow::anyhow; +use iceberg::Error as IcebergError; use mysql_async::Error as MySqlError; use parquet::errors::ParquetError; use risingwave_common::array::ArrayError; @@ -194,7 +195,7 @@ def_anyhow_variant! { BatchError ExternalSystemError, PostgresError => "Postgres error", - iceberg::Error => "Iceberg error", + IcebergError => "Iceberg error", ParquetError => "Parquet error", MySqlError => "MySQL error", } diff --git a/src/common/src/mysql.rs b/src/common/src/mysql.rs index c1df361bf3a7b..c3ed3d9903764 100644 --- a/src/common/src/mysql.rs +++ b/src/common/src/mysql.rs @@ -54,12 +54,10 @@ macro_rules! handle_data_type { } /// The decoding result can be interpreted as follows: -/// None => No value was found in the column. -/// Some(Ok(value)) => The value was found and successfully decoded. -/// Some(Err(error)) => The value was found but could not be decoded, -/// either because it was not supported, -/// or there was an error during conversion. -/// This error is wrapped via anyhow, so it is opaque. +/// Ok(value) => The value was found and successfully decoded. +/// Err(error) => The value was found but could not be decoded, +/// either because it was not supported, +/// or there was an error during conversion. pub fn mysql_datum_to_rw_datum( mysql_row: &mut MysqlRow, mysql_datum_index: usize, @@ -68,7 +66,7 @@ pub fn mysql_datum_to_rw_datum( ) -> Result { match rw_data_type { DataType::Boolean => { - // Bit + // Bit(1) match mysql_row.take_opt::>, _>(mysql_datum_index) { None => bail!( "no value found at column: {}, index: {}", From db7d3bd649b7188bac8f69b5611296e3e2394f3c Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 30 Oct 2024 18:38:44 +0800 Subject: [PATCH 44/48] fix source test --- e2e_test/source_legacy/cdc_inline/auto_schema_map_mysql.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e_test/source_legacy/cdc_inline/auto_schema_map_mysql.slt b/e2e_test/source_legacy/cdc_inline/auto_schema_map_mysql.slt index 08afa5d1988a7..e1b5016774a0c 100644 --- a/e2e_test/source_legacy/cdc_inline/auto_schema_map_mysql.slt +++ b/e2e_test/source_legacy/cdc_inline/auto_schema_map_mysql.slt @@ -174,8 +174,8 @@ SELECT c_binary_255 FROM rw_mysql_types_test order by c_boolean; ---- -0 NULL NULL NULL -8388608 -2147483647 9223372036854775806 -10 -10000 -10000 c d \x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 -1 NULL -128 -32767 -8388608 -2147483647 -9223372036854775807 -10 -10000 -10000 a b \x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 +0 f NULL NULL -8388608 -2147483647 9223372036854775806 -10 -10000 -10000 c d \x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 +1 t -128 -32767 -8388608 -2147483647 -9223372036854775807 -10 -10000 -10000 a b \x000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 query TTTTTTTT SELECT From aff5786e4716784a7cb512adb31cd1358ceb7ae7 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 1 Nov 2024 10:06:00 +0800 Subject: [PATCH 45/48] use chunk_size instead of magic value --- src/batch/src/executor/mysql_query.rs | 6 +++++- src/batch/src/executor/postgres_query.rs | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index edde81a699511..35c55eaf59497 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -37,6 +37,7 @@ pub struct MySqlQueryExecutor { database: String, query: String, identity: String, + chunk_size: usize, } impl Executor for MySqlQueryExecutor { @@ -82,6 +83,7 @@ impl MySqlQueryExecutor { database: String, query: String, identity: String, + chunk_size: usize, ) -> Self { Self { schema, @@ -92,6 +94,7 @@ impl MySqlQueryExecutor { database, query, identity, + chunk_size, } } @@ -121,7 +124,7 @@ impl MySqlQueryExecutor { bail!("failed to get row stream from mysql query") }; - let mut builder = DataChunkBuilder::new(self.schema.data_types(), 1024); + let mut builder = DataChunkBuilder::new(self.schema.data_types(), self.chunk_size); tracing::debug!("mysql_query_executor: query executed, start deserializing rows"); // deserialize the rows #[for_await] @@ -161,6 +164,7 @@ impl BoxedExecutorBuilder for MySqlQueryExecutorBuilder { mysql_query_node.database.clone(), mysql_query_node.query.clone(), source.plan_node().get_identity().clone(), + source.context.get_config().developer.chunk_size, ))) } } diff --git a/src/batch/src/executor/postgres_query.rs b/src/batch/src/executor/postgres_query.rs index 2b6524a2e45e7..4ae1fcba65da9 100644 --- a/src/batch/src/executor/postgres_query.rs +++ b/src/batch/src/executor/postgres_query.rs @@ -37,6 +37,7 @@ pub struct PostgresQueryExecutor { database: String, query: String, identity: String, + chunk_size: usize, } impl Executor for PostgresQueryExecutor { @@ -115,6 +116,7 @@ impl PostgresQueryExecutor { database: String, query: String, identity: String, + chunk_size: usize, ) -> Self { Self { schema, @@ -125,6 +127,7 @@ impl PostgresQueryExecutor { database, query, identity, + chunk_size, } } @@ -151,7 +154,7 @@ impl PostgresQueryExecutor { .query_raw(&self.query, params) .await .context("postgres_query received error from remote server")?; - let mut builder = DataChunkBuilder::new(self.schema.data_types(), 1024); + let mut builder = DataChunkBuilder::new(self.schema.data_types(), self.chunk_size); tracing::debug!("postgres_query_executor: query executed, start deserializing rows"); // deserialize the rows #[for_await] @@ -191,6 +194,7 @@ impl BoxedExecutorBuilder for PostgresQueryExecutorBuilder { postgres_query_node.database.clone(), postgres_query_node.query.clone(), source.plan_node().get_identity().clone(), + source.context.get_config().developer.chunk_size, ))) } } From 0aa4fba62595314b726d1f3d241189d0246803fb Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 1 Nov 2024 10:06:10 +0800 Subject: [PATCH 46/48] safely parse port --- src/frontend/src/expr/table_function.rs | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index ef398141ac5d2..2d2aaec7a773f 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -405,13 +405,16 @@ impl TableFunction { { let schema = tokio::task::block_in_place(|| { RUNTIME.block_on(async { - let database_opts: mysql_async::Opts = mysql_async::OptsBuilder::default() - .ip_or_hostname(evaled_args[0].clone()) - .tcp_port(evaled_args[1].parse::().unwrap()) - .user(Some(evaled_args[2].clone())) - .pass(Some(evaled_args[3].clone())) - .db_name(Some(evaled_args[4].clone())) - .into(); + let database_opts: mysql_async::Opts = { + let port = evaled_args[1].parse::().context("failed to parse port")?; + mysql_async::OptsBuilder::default() + .ip_or_hostname(evaled_args[0].clone()) + .tcp_port(port) + .user(Some(evaled_args[2].clone())) + .pass(Some(evaled_args[3].clone())) + .db_name(Some(evaled_args[4].clone())) + .into(); + } let pool = mysql_async::Pool::new(database_opts); let mut conn = pool From f389b4d1b03c3c61cde0346e297b8db58cf79621 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 1 Nov 2024 10:18:40 +0800 Subject: [PATCH 47/48] fix typo + handling parse error --- src/frontend/src/expr/table_function.rs | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/frontend/src/expr/table_function.rs b/src/frontend/src/expr/table_function.rs index 2d2aaec7a773f..cee4188e75791 100644 --- a/src/frontend/src/expr/table_function.rs +++ b/src/frontend/src/expr/table_function.rs @@ -301,7 +301,7 @@ impl TableFunction { tokio::spawn(async move { if let Err(e) = connection.await { tracing::error!( - "postgres_query_executor: connection error: {:?}", + "mysql_query_executor: connection error: {:?}", e.as_report() ); } @@ -406,15 +406,17 @@ impl TableFunction { let schema = tokio::task::block_in_place(|| { RUNTIME.block_on(async { let database_opts: mysql_async::Opts = { - let port = evaled_args[1].parse::().context("failed to parse port")?; + let port = evaled_args[1] + .parse::() + .context("failed to parse port")?; mysql_async::OptsBuilder::default() .ip_or_hostname(evaled_args[0].clone()) .tcp_port(port) .user(Some(evaled_args[2].clone())) .pass(Some(evaled_args[3].clone())) .db_name(Some(evaled_args[4].clone())) - .into(); - } + .into() + }; let pool = mysql_async::Pool::new(database_opts); let mut conn = pool From e84ce4506ce8eecfab127fbdfc2aa69e529e6a63 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 1 Nov 2024 10:18:49 +0800 Subject: [PATCH 48/48] move conversion to connector --- Cargo.lock | 2 - src/batch/src/executor/mysql_query.rs | 2 +- src/common/Cargo.toml | 4 - src/common/src/lib.rs | 1 - src/common/src/mysql.rs | 191 -------------------------- src/connector/src/parser/mod.rs | 2 +- src/connector/src/parser/mysql.rs | 176 +++++++++++++++++++++++- 7 files changed, 177 insertions(+), 201 deletions(-) delete mode 100644 src/common/src/mysql.rs diff --git a/Cargo.lock b/Cargo.lock index 889caaab6f9f9..275024641e22e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10663,8 +10663,6 @@ dependencies = [ "madsim-tokio", "memcomparable", "more-asserts", - "mysql_async", - "mysql_common", "num-integer", "num-traits", "number_prefix", diff --git a/src/batch/src/executor/mysql_query.rs b/src/batch/src/executor/mysql_query.rs index 35c55eaf59497..721c9c5e55bf1 100644 --- a/src/batch/src/executor/mysql_query.rs +++ b/src/batch/src/executor/mysql_query.rs @@ -18,9 +18,9 @@ use futures_util::stream::StreamExt; use mysql_async; use mysql_async::prelude::*; use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::mysql::mysql_datum_to_rw_datum; use risingwave_common::row::OwnedRow; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; +use risingwave_connector::parser::mysql_datum_to_rw_datum; use risingwave_pb::batch_plan::plan_node::NodeBody; use crate::error::{BatchError, BatchExternalSystemError}; diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 7facfde900cc3..8b341fb621fe6 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -59,10 +59,6 @@ itoa = "1.0" jsonbb = { workspace = true } lru = { workspace = true } memcomparable = { version = "0.2", features = ["decimal"] } -mysql_async = { workspace = true } -mysql_common = { version = "0.32", default-features = false, features = [ - "chrono", -] } num-integer = "0.1" num-traits = "0.2" number_prefix = "0.4.0" diff --git a/src/common/src/lib.rs b/src/common/src/lib.rs index 86b2585ca9529..b44aba68b0985 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -80,7 +80,6 @@ pub use { risingwave_license as license, }; pub mod lru; -pub mod mysql; pub mod opts; pub mod range; pub mod row; diff --git a/src/common/src/mysql.rs b/src/common/src/mysql.rs deleted file mode 100644 index c3ed3d9903764..0000000000000 --- a/src/common/src/mysql.rs +++ /dev/null @@ -1,191 +0,0 @@ -// 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 anyhow::anyhow; -use chrono::NaiveDate; -use mysql_async::Row as MysqlRow; -use risingwave_common::types::{ - DataType, Date, Decimal, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, -}; -use rust_decimal::Decimal as RustDecimal; - -use crate::types::Datum; - -macro_rules! handle_data_type { - ($row:expr, $i:expr, $name:expr, $type:ty) => {{ - match $row.take_opt::, _>($i) { - None => bail!("no value found at column: {}, index: {}", $name, $i), - Some(Ok(val)) => Ok(val.map(|v| ScalarImpl::from(v))), - Some(Err(e)) => Err(anyhow::Error::new(e.clone()) - .context("failed to deserialize MySQL value into rust value") - .context(format!( - "column: {}, index: {}, rust_type: {}", - $name, - $i, - stringify!($type), - ))), - } - }}; - ($row:expr, $i:expr, $name:expr, $type:ty, $rw_type:ty) => {{ - match $row.take_opt::, _>($i) { - None => bail!("no value found at column: {}, index: {}", $name, $i), - Some(Ok(val)) => Ok(val.map(|v| ScalarImpl::from(<$rw_type>::from(v)))), - Some(Err(e)) => Err(anyhow::Error::new(e.clone()) - .context("failed to deserialize MySQL value into rust value") - .context(format!( - "column: {}, index: {}, rust_type: {}", - $name, - $i, - stringify!($ty), - ))), - } - }}; -} - -/// The decoding result can be interpreted as follows: -/// Ok(value) => The value was found and successfully decoded. -/// Err(error) => The value was found but could not be decoded, -/// either because it was not supported, -/// or there was an error during conversion. -pub fn mysql_datum_to_rw_datum( - mysql_row: &mut MysqlRow, - mysql_datum_index: usize, - column_name: &str, - rw_data_type: &DataType, -) -> Result { - match rw_data_type { - DataType::Boolean => { - // Bit(1) - match mysql_row.take_opt::>, _>(mysql_datum_index) { - None => bail!( - "no value found at column: {}, index: {}", - column_name, - mysql_datum_index - ), - Some(Ok(val)) => match val { - None => Ok(None), - Some(val) => match val.as_slice() { - [0] => Ok(Some(ScalarImpl::from(false))), - [1] => Ok(Some(ScalarImpl::from(true))), - _ => Err(anyhow!("invalid value for boolean: {:?}", val)), - }, - }, - Some(Err(e)) => Err(anyhow::Error::new(e.clone()) - .context("failed to deserialize MySQL value into rust value") - .context(format!( - "column: {}, index: {}, rust_type: Vec", - column_name, mysql_datum_index, - ))), - } - } - DataType::Int16 => { - handle_data_type!(mysql_row, mysql_datum_index, column_name, i16) - } - DataType::Int32 => { - handle_data_type!(mysql_row, mysql_datum_index, column_name, i32) - } - DataType::Int64 => { - handle_data_type!(mysql_row, mysql_datum_index, column_name, i64) - } - DataType::Float32 => { - handle_data_type!(mysql_row, mysql_datum_index, column_name, f32) - } - DataType::Float64 => { - handle_data_type!(mysql_row, mysql_datum_index, column_name, f64) - } - DataType::Decimal => { - handle_data_type!( - mysql_row, - mysql_datum_index, - column_name, - RustDecimal, - Decimal - ) - } - DataType::Varchar => { - handle_data_type!(mysql_row, mysql_datum_index, column_name, String) - } - DataType::Date => { - handle_data_type!(mysql_row, mysql_datum_index, column_name, NaiveDate, Date) - } - DataType::Time => { - handle_data_type!( - mysql_row, - mysql_datum_index, - column_name, - chrono::NaiveTime, - Time - ) - } - DataType::Timestamp => { - handle_data_type!( - mysql_row, - mysql_datum_index, - column_name, - chrono::NaiveDateTime, - Timestamp - ) - } - DataType::Timestamptz => { - match mysql_row.take_opt::, _>(mysql_datum_index) { - None => bail!( - "no value found at column: {}, index: {}", - column_name, - mysql_datum_index - ), - Some(Ok(val)) => Ok(val.map(|v| { - ScalarImpl::from(Timestamptz::from_micros(v.and_utc().timestamp_micros())) - })), - Some(Err(err)) => Err(anyhow::Error::new(err.clone()) - .context("failed to deserialize MySQL value into rust value") - .context(format!( - "column: {}, index: {}, rust_type: chrono::NaiveDateTime", - column_name, mysql_datum_index, - ))), - } - } - DataType::Bytea => match mysql_row.take_opt::>, _>(mysql_datum_index) { - None => bail!( - "no value found at column: {}, index: {}", - column_name, - mysql_datum_index - ), - Some(Ok(val)) => Ok(val.map(ScalarImpl::from)), - Some(Err(err)) => Err(anyhow::Error::new(err.clone()) - .context("failed to deserialize MySQL value into rust value") - .context(format!( - "column: {}, index: {}, rust_type: Vec", - column_name, mysql_datum_index, - ))), - }, - DataType::Jsonb => { - handle_data_type!( - mysql_row, - mysql_datum_index, - column_name, - serde_json::Value, - JsonbVal - ) - } - DataType::Interval - | DataType::Struct(_) - | DataType::List(_) - | DataType::Int256 - | DataType::Serial - | DataType::Map(_) => Err(anyhow!( - "unsupported data type: {}, set to null", - rw_data_type - )), - } -} diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 53621721ac25b..d8bd4a9dbcd56 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -44,7 +44,7 @@ use thiserror_ext::AsReport; use self::avro::AvroAccessBuilder; use self::bytes_parser::BytesAccessBuilder; -pub use self::mysql::mysql_row_to_owned_row; +pub use self::mysql::{mysql_datum_to_rw_datum, mysql_row_to_owned_row}; use self::plain_parser::PlainParser; pub use self::postgres::postgres_row_to_owned_row; use self::simd_json_parser::DebeziumJsonAccessBuilder; diff --git a/src/connector/src/parser/mysql.rs b/src/connector/src/parser/mysql.rs index 6aabe9d3ccafe..e9a8eeba70cb3 100644 --- a/src/connector/src/parser/mysql.rs +++ b/src/connector/src/parser/mysql.rs @@ -23,7 +23,181 @@ use thiserror_ext::AsReport; use crate::parser::util::log_error; static LOG_SUPPERSSER: LazyLock = LazyLock::new(LogSuppresser::default); -use risingwave_common::mysql::mysql_datum_to_rw_datum; +use anyhow::anyhow; +use chrono::NaiveDate; +use risingwave_common::bail; +use risingwave_common::types::{ + DataType, Date, Datum, Decimal, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, +}; +use rust_decimal::Decimal as RustDecimal; + +macro_rules! handle_data_type { + ($row:expr, $i:expr, $name:expr, $type:ty) => {{ + match $row.take_opt::, _>($i) { + None => bail!("no value found at column: {}, index: {}", $name, $i), + Some(Ok(val)) => Ok(val.map(|v| ScalarImpl::from(v))), + Some(Err(e)) => Err(anyhow::Error::new(e.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: {}", + $name, + $i, + stringify!($type), + ))), + } + }}; + ($row:expr, $i:expr, $name:expr, $type:ty, $rw_type:ty) => {{ + match $row.take_opt::, _>($i) { + None => bail!("no value found at column: {}, index: {}", $name, $i), + Some(Ok(val)) => Ok(val.map(|v| ScalarImpl::from(<$rw_type>::from(v)))), + Some(Err(e)) => Err(anyhow::Error::new(e.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: {}", + $name, + $i, + stringify!($ty), + ))), + } + }}; +} + +/// The decoding result can be interpreted as follows: +/// Ok(value) => The value was found and successfully decoded. +/// Err(error) => The value was found but could not be decoded, +/// either because it was not supported, +/// or there was an error during conversion. +pub fn mysql_datum_to_rw_datum( + mysql_row: &mut MysqlRow, + mysql_datum_index: usize, + column_name: &str, + rw_data_type: &DataType, +) -> Result { + match rw_data_type { + DataType::Boolean => { + // Bit(1) + match mysql_row.take_opt::>, _>(mysql_datum_index) { + None => bail!( + "no value found at column: {}, index: {}", + column_name, + mysql_datum_index + ), + Some(Ok(val)) => match val { + None => Ok(None), + Some(val) => match val.as_slice() { + [0] => Ok(Some(ScalarImpl::from(false))), + [1] => Ok(Some(ScalarImpl::from(true))), + _ => Err(anyhow!("invalid value for boolean: {:?}", val)), + }, + }, + Some(Err(e)) => Err(anyhow::Error::new(e.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: Vec", + column_name, mysql_datum_index, + ))), + } + } + DataType::Int16 => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, i16) + } + DataType::Int32 => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, i32) + } + DataType::Int64 => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, i64) + } + DataType::Float32 => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, f32) + } + DataType::Float64 => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, f64) + } + DataType::Decimal => { + handle_data_type!( + mysql_row, + mysql_datum_index, + column_name, + RustDecimal, + Decimal + ) + } + DataType::Varchar => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, String) + } + DataType::Date => { + handle_data_type!(mysql_row, mysql_datum_index, column_name, NaiveDate, Date) + } + DataType::Time => { + handle_data_type!( + mysql_row, + mysql_datum_index, + column_name, + chrono::NaiveTime, + Time + ) + } + DataType::Timestamp => { + handle_data_type!( + mysql_row, + mysql_datum_index, + column_name, + chrono::NaiveDateTime, + Timestamp + ) + } + DataType::Timestamptz => { + match mysql_row.take_opt::, _>(mysql_datum_index) { + None => bail!( + "no value found at column: {}, index: {}", + column_name, + mysql_datum_index + ), + Some(Ok(val)) => Ok(val.map(|v| { + ScalarImpl::from(Timestamptz::from_micros(v.and_utc().timestamp_micros())) + })), + Some(Err(err)) => Err(anyhow::Error::new(err.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: chrono::NaiveDateTime", + column_name, mysql_datum_index, + ))), + } + } + DataType::Bytea => match mysql_row.take_opt::>, _>(mysql_datum_index) { + None => bail!( + "no value found at column: {}, index: {}", + column_name, + mysql_datum_index + ), + Some(Ok(val)) => Ok(val.map(ScalarImpl::from)), + Some(Err(err)) => Err(anyhow::Error::new(err.clone()) + .context("failed to deserialize MySQL value into rust value") + .context(format!( + "column: {}, index: {}, rust_type: Vec", + column_name, mysql_datum_index, + ))), + }, + DataType::Jsonb => { + handle_data_type!( + mysql_row, + mysql_datum_index, + column_name, + serde_json::Value, + JsonbVal + ) + } + DataType::Interval + | DataType::Struct(_) + | DataType::List(_) + | DataType::Int256 + | DataType::Serial + | DataType::Map(_) => Err(anyhow!( + "unsupported data type: {}, set to null", + rw_data_type + )), + } +} pub fn mysql_row_to_owned_row(mysql_row: &mut MysqlRow, schema: &Schema) -> OwnedRow { let mut datums = vec![];