From 90fb4a3478ec90c1497732af3e75afea6d22ae17 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 26 Oct 2023 04:25:11 +0000 Subject: [PATCH 01/52] chore(deps): Bump comfy-table from 7.0.1 to 7.1.0 (#13049) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 19e79820fb5e9..3f7df00f7648d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1825,13 +1825,13 @@ dependencies = [ [[package]] name = "comfy-table" -version = "7.0.1" +version = "7.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ab77dbd8adecaf3f0db40581631b995f312a8a5ae3aa9993188bb8f23d83a5b" +checksum = "7c64043d6c7b7a4c58e39e7efccfdea7b93d885a795d0c054a69dbbf4dd52686" dependencies = [ - "crossterm 0.26.1", - "strum 0.24.1", - "strum_macros 0.24.3", + "crossterm 0.27.0", + "strum 0.25.0", + "strum_macros 0.25.2", "unicode-width", ] @@ -2140,17 +2140,14 @@ dependencies = [ [[package]] name = "crossterm" -version = "0.26.1" +version = "0.27.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a84cda67535339806297f1b331d6dd6320470d2a0fe65381e79ee9e156dd3d13" +checksum = "f476fe445d41c9e991fd07515a6f463074b782242ccf4a5b7b1d1012e70824df" dependencies = [ - "bitflags 1.3.2", + "bitflags 2.4.0", "crossterm_winapi", "libc", - "mio", "parking_lot 0.12.1", - "signal-hook", - "signal-hook-mio", "winapi", ] From 942e99dc74228659ff106589634ddc6196009191 Mon Sep 17 00:00:00 2001 From: Yufan Song <33971064+yufansong@users.noreply.github.com> Date: Wed, 25 Oct 2023 22:10:31 -0700 Subject: [PATCH 02/52] fix(nats-connector): change stream into optional string, add replace stream name logic (#13024) --- src/connector/src/common.rs | 13 ++++++++----- src/connector/src/source/nats/mod.rs | 3 +++ src/connector/src/source/nats/source/reader.rs | 6 +++++- 3 files changed, 16 insertions(+), 6 deletions(-) diff --git a/src/connector/src/common.rs b/src/connector/src/common.rs index 2af396f5c33b4..c9a55dfe15086 100644 --- a/src/connector/src/common.rs +++ b/src/connector/src/common.rs @@ -417,8 +417,6 @@ pub struct UpsertMessage<'a> { pub struct NatsCommon { #[serde(rename = "server_url")] pub server_url: String, - #[serde(rename = "stream")] - pub stream: String, #[serde(rename = "subject")] pub subject: String, #[serde(rename = "connect_mode")] @@ -505,14 +503,18 @@ impl NatsCommon { pub(crate) async fn build_consumer( &self, + stream: String, split_id: String, start_sequence: NatsOffset, ) -> anyhow::Result< async_nats::jetstream::consumer::Consumer, > { let context = self.build_context().await?; - let stream = self.build_or_get_stream(context.clone()).await?; - let subject_name = self.subject.replace(',', "-"); + let stream = self.build_or_get_stream(context.clone(), stream).await?; + let subject_name = self + .subject + .replace(',', "-") + .replace(['.', '>', '*', ' ', '\t'], "_"); let name = format!("risingwave-consumer-{}-{}", subject_name, split_id); let mut config = jetstream::consumer::pull::Config { ack_policy: jetstream::consumer::AckPolicy::None, @@ -545,10 +547,11 @@ impl NatsCommon { pub(crate) async fn build_or_get_stream( &self, jetstream: jetstream::Context, + stream: String, ) -> anyhow::Result { let subjects: Vec = self.subject.split(',').map(|s| s.to_string()).collect(); let mut config = jetstream::stream::Config { - name: self.stream.clone(), + name: stream, max_bytes: 1000000, subjects, ..Default::default() diff --git a/src/connector/src/source/nats/mod.rs b/src/connector/src/source/nats/mod.rs index 3e8cc57bc1da8..f209086693fd3 100644 --- a/src/connector/src/source/nats/mod.rs +++ b/src/connector/src/source/nats/mod.rs @@ -35,6 +35,9 @@ pub struct NatsProperties { #[serde(rename = "scan.startup.timestamp_millis")] pub start_time: Option, + + #[serde(rename = "stream")] + pub stream: String, } impl NatsProperties {} diff --git a/src/connector/src/source/nats/source/reader.rs b/src/connector/src/source/nats/source/reader.rs index 6e22748bcf468..7f9a5718f95d4 100644 --- a/src/connector/src/source/nats/source/reader.rs +++ b/src/connector/src/source/nats/source/reader.rs @@ -77,7 +77,11 @@ impl SplitReader for NatsSplitReader { let consumer = properties .common - .build_consumer(split_id.to_string(), start_position.clone()) + .build_consumer( + properties.stream.clone(), + split_id.to_string(), + start_position.clone(), + ) .await?; Ok(Self { consumer, From e2a36373fa1379667a70b60e83502627d9224c58 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Thu, 26 Oct 2023 13:56:26 +0800 Subject: [PATCH 03/52] =?UTF-8?q?fix(DBeaver):=20Add=20the=20=E2=80=98=5Fx?= =?UTF-8?q?xx=E2=80=99=20in=20RW=E2=80=98s=20pg=5Fcatalog.pg=5Ftype=20to?= =?UTF-8?q?=20avoid=20some=20DBeaver=20errors."=20(#13052)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- e2e_test/batch/catalog/issue_10177.slt.part | 49 ++++++++++++------- e2e_test/batch/catalog/pg_type.slt.part | 49 ++++++++++++------- .../tests/testdata/output/pg_catalog.yaml | 10 ++-- .../system_catalog/pg_catalog/pg_type.rs | 4 +- .../system_catalog/rw_catalog/rw_types.rs | 18 +++++-- 5 files changed, 84 insertions(+), 46 deletions(-) diff --git a/e2e_test/batch/catalog/issue_10177.slt.part b/e2e_test/batch/catalog/issue_10177.slt.part index 771d304d7a058..8f8c40aa75c19 100644 --- a/e2e_test/batch/catalog/issue_10177.slt.part +++ b/e2e_test/batch/catalog/issue_10177.slt.part @@ -43,20 +43,35 @@ ORDER BY CASE WHEN typtype = 'd' AND elemtyptype = 'a' THEN 6 -- Domains over arrays last END; ---- -pg_catalog 1043 varchar b f NULL -pg_catalog 1082 date b f NULL -pg_catalog 1083 time b f NULL -pg_catalog 1114 timestamp b f NULL -pg_catalog 1184 timestamptz b f NULL -pg_catalog 1186 interval b f NULL -pg_catalog 1301 rw_int256 b f NULL -pg_catalog 16 bool b f NULL -pg_catalog 17 bytea b f NULL -pg_catalog 1700 numeric b f NULL -pg_catalog 20 int8 b f NULL -pg_catalog 21 int2 b f NULL -pg_catalog 23 int4 b f NULL -pg_catalog 25 text b f NULL -pg_catalog 3802 jsonb b f NULL -pg_catalog 700 float4 b f NULL -pg_catalog 701 float8 b f NULL +pg_catalog 1000 _bool b f NULL +pg_catalog 1001 _bytea b f NULL +pg_catalog 1005 _int2 b f NULL +pg_catalog 1007 _int4 b f NULL +pg_catalog 1015 _varchar b f NULL +pg_catalog 1016 _int8 b f NULL +pg_catalog 1021 _float4 b f NULL +pg_catalog 1022 _float8 b f NULL +pg_catalog 1043 varchar b f NULL +pg_catalog 1082 date b f NULL +pg_catalog 1083 time b f NULL +pg_catalog 1114 timestamp b f NULL +pg_catalog 1115 _timestamp b f NULL +pg_catalog 1182 _date b f NULL +pg_catalog 1183 _time b f NULL +pg_catalog 1184 timestamptz b f NULL +pg_catalog 1185 _timestamptz b f NULL +pg_catalog 1186 interval b f NULL +pg_catalog 1187 _interval b f NULL +pg_catalog 1231 _numeric b f NULL +pg_catalog 1301 rw_int256 b f NULL +pg_catalog 16 bool b f NULL +pg_catalog 17 bytea b f NULL +pg_catalog 1700 numeric b f NULL +pg_catalog 20 int8 b f NULL +pg_catalog 21 int2 b f NULL +pg_catalog 23 int4 b f NULL +pg_catalog 25 text b f NULL +pg_catalog 3802 jsonb b f NULL +pg_catalog 3807 _jsonb b f NULL +pg_catalog 700 float4 b f NULL +pg_catalog 701 float8 b f NULL \ No newline at end of file diff --git a/e2e_test/batch/catalog/pg_type.slt.part b/e2e_test/batch/catalog/pg_type.slt.part index e35d21291bc14..29c510a394e61 100644 --- a/e2e_test/batch/catalog/pg_type.slt.part +++ b/e2e_test/batch/catalog/pg_type.slt.part @@ -1,20 +1,35 @@ query ITITT SELECT oid, typname, typelem, typnotnull, typtype, typinput FROM pg_catalog.pg_type order by oid; ---- -16 bool 0 f b boolin -17 bytea 0 f b byteain -20 int8 0 f b int8in -21 int2 0 f b int2in -23 int4 0 f b int4in -25 text 0 f b textin -700 float4 0 f b float4in -701 float8 0 f b float8in -1043 varchar 0 f b varcharin -1082 date 0 f b date_in -1083 time 0 f b time_in -1114 timestamp 0 f b timestamp_in -1184 timestamptz 0 f b timestamptz_in -1186 interval 0 f b interval_in -1301 rw_int256 0 f b rw_int256_in -1700 numeric 0 f b numeric_in -3802 jsonb 0 f b jsonb_in +16 bool 0 f b boolin +17 bytea 0 f b byteain +20 int8 0 f b int8in +21 int2 0 f b int2in +23 int4 0 f b int4in +25 text 0 f b textin +700 float4 0 f b float4in +701 float8 0 f b float8in +1000 _bool 16 f b array_in +1001 _bytea 17 f b array_in +1005 _int2 21 f b array_in +1007 _int4 23 f b array_in +1015 _varchar 1043 f b array_in +1016 _int8 20 f b array_in +1021 _float4 700 f b array_in +1022 _float8 701 f b array_in +1043 varchar 0 f b varcharin +1082 date 0 f b date_in +1083 time 0 f b time_in +1114 timestamp 0 f b timestamp_in +1115 _timestamp 1114 f b array_in +1182 _date 1082 f b array_in +1183 _time 1083 f b array_in +1184 timestamptz 0 f b timestamptz_in +1185 _timestamptz 1184 f b array_in +1186 interval 0 f b interval_in +1187 _interval 1186 f b array_in +1231 _numeric 1700 f b array_in +1301 rw_int256 0 f b rw_int256_in +1700 numeric 0 f b numeric_in +3802 jsonb 0 f b jsonb_in +3807 _jsonb 3802 f b array_in diff --git a/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml b/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml index 5c5a88fb472b6..5ae1827ad95a3 100644 --- a/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml +++ b/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml @@ -2,16 +2,16 @@ - sql: | select * from pg_catalog.pg_type logical_plan: |- - LogicalProject { exprs: [rw_types.id, rw_types.name, 0:Int32, 0:Int32, rw_types.input_oid, false:Boolean, 0:Int32, -1:Int32, 0:Int32, 0:Int32, rw_schemas.id, 'b':Varchar, 0:Int32, null:Varchar, null:Varchar, null:Int32] } + LogicalProject { exprs: [rw_types.id, rw_types.name, rw_types.typelem, rw_types.typarray, rw_types.input_oid, false:Boolean, 0:Int32, -1:Int32, 0:Int32, 0:Int32, rw_schemas.id, 'b':Varchar, 0:Int32, null:Varchar, null:Varchar, null:Int32] } └─LogicalShare { id: 3 } - └─LogicalProject { exprs: [rw_types.id, rw_types.name, 0:Int32, 0:Int32, rw_types.input_oid, false:Boolean, 0:Int32, -1:Int32, 0:Int32, 0:Int32, rw_schemas.id, 'b':Varchar, 0:Int32, null:Varchar, null:Varchar, null:Int32] } + └─LogicalProject { exprs: [rw_types.id, rw_types.name, rw_types.typelem, rw_types.typarray, rw_types.input_oid, false:Boolean, 0:Int32, -1:Int32, 0:Int32, 0:Int32, rw_schemas.id, 'b':Varchar, 0:Int32, null:Varchar, null:Varchar, null:Int32] } └─LogicalJoin { type: Inner, on: (rw_schemas.name = 'pg_catalog':Varchar), output: all } - ├─LogicalScan { table: rw_types, columns: [rw_types.id, rw_types.name, rw_types.input_oid] } + ├─LogicalScan { table: rw_types, columns: [rw_types.id, rw_types.name, rw_types.input_oid, rw_types.typelem, rw_types.typarray] } └─LogicalScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } batch_plan: |- - BatchProject { exprs: [rw_types.id, rw_types.name, 0:Int32, 0:Int32, rw_types.input_oid, false:Boolean, 0:Int32, -1:Int32, 0:Int32, 0:Int32, rw_schemas.id, 'b':Varchar, 0:Int32, null:Varchar, null:Varchar, null:Int32] } + BatchProject { exprs: [rw_types.id, rw_types.name, rw_types.typelem, rw_types.typarray, rw_types.input_oid, false:Boolean, 0:Int32, -1:Int32, 0:Int32, 0:Int32, rw_schemas.id, 'b':Varchar, 0:Int32, null:Varchar, null:Varchar, null:Int32] } └─BatchNestedLoopJoin { type: Inner, predicate: true, output: all } - ├─BatchScan { table: rw_types, columns: [rw_types.id, rw_types.name, rw_types.input_oid], distribution: Single } + ├─BatchScan { table: rw_types, columns: [rw_types.id, rw_types.name, rw_types.input_oid, rw_types.typelem, rw_types.typarray], distribution: Single } └─BatchProject { exprs: [rw_schemas.id] } └─BatchFilter { predicate: (rw_schemas.name = 'pg_catalog':Varchar) } └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs index af6a2968e3a8c..5b4993c2e8c9e 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_type.rs @@ -58,8 +58,8 @@ pub static PG_TYPE: LazyLock = LazyLock::new(|| BuiltinView { ], sql: "SELECT t.id AS oid, \ t.name AS typname, \ - 0 AS typelem, \ - 0 AS typarray, \ + t.typelem AS typelem, \ + t.typarray AS typarray, \ t.input_oid AS typinput, \ false AS typnotnull, \ 0 AS typbasetype, \ diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs index 02462c4cee303..9ce1d35784f54 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_types.rs @@ -27,16 +27,20 @@ macro_rules! impl_pg_type_data { ($( { $enum:ident | $oid:literal | $oid_array:literal | $name:ident | $input:ident | $len:literal } )*) => { &[ $( - ($oid, stringify!($name), stringify!($input)), + ($oid, stringify!($name), stringify!($input), 0, $oid_array), )* // Note: rw doesn't support `text` type, returning it is just a workaround to be compatible // with PostgreSQL. - (25, "text", "textin"), - (1301, "rw_int256", "rw_int256_in"), + (25, "text", "textin",0,1009), + (1301, "rw_int256", "rw_int256_in",0,0), + // Note: Here is only to avoid some components of psql from not being able to find relevant results, causing errors. We will not use it in the RW. + $( + ($oid_array, concat!("_", stringify!($name)), "array_in", $oid, 0), + )* ] } } -pub const RW_TYPE_DATA: &[(i32, &str, &str)] = for_all_base_types! { impl_pg_type_data }; +pub const RW_TYPE_DATA: &[(i32, &str, &str, i32, i32)] = for_all_base_types! { impl_pg_type_data }; /// `rw_types` stores all supported types in the database. pub static RW_TYPES: LazyLock = LazyLock::new(|| BuiltinTable { @@ -46,6 +50,8 @@ pub static RW_TYPES: LazyLock = LazyLock::new(|| BuiltinTable { (DataType::Int32, "id"), (DataType::Varchar, "name"), (DataType::Varchar, "input_oid"), + (DataType::Int32, "typelem"), + (DataType::Int32, "typarray"), ], pk: &[0], }); @@ -54,11 +60,13 @@ impl SysCatalogReaderImpl { pub fn read_rw_types(&self) -> Result> { Ok(RW_TYPE_DATA .iter() - .map(|(id, name, input)| { + .map(|(id, name, input, typelem, typarray)| { OwnedRow::new(vec![ Some(ScalarImpl::Int32(*id)), Some(ScalarImpl::Utf8(name.to_string().into())), Some(ScalarImpl::Utf8(input.to_string().into())), + Some(ScalarImpl::Int32(*typelem)), + Some(ScalarImpl::Int32(*typarray)), ]) }) .collect_vec()) From 71851d628602c4a7bab357781782198fefa511da Mon Sep 17 00:00:00 2001 From: Wallace Date: Thu, 26 Oct 2023 14:01:57 +0800 Subject: [PATCH 04/52] fix(compactor): collect metrics for fast compact runer (#13064) Signed-off-by: Little-Wallace --- .../hummock_test/src/compactor_tests.rs | 9 +- .../src/hummock/compactor/compactor_runner.rs | 4 +- .../compactor/fast_compactor_runner.rs | 83 +++++++++++++++++-- src/storage/src/hummock/utils.rs | 3 - 4 files changed, 79 insertions(+), 20 deletions(-) diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 50d739c5d1eb9..34b23a9b79774 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -1454,13 +1454,8 @@ pub(crate) mod tests { .await .unwrap(); let ret = ret1.into_iter().map(|sst| sst.sst_info).collect_vec(); - let fast_ret = fast_compact_runner - .run() - .await - .unwrap() - .into_iter() - .map(|sst| sst.sst_info) - .collect_vec(); + let (ssts, _) = fast_compact_runner.run().await.unwrap(); + let fast_ret = ssts.into_iter().map(|sst| sst.sst_info).collect_vec(); println!("ssts: {} vs {}", fast_ret.len(), ret.len()); let mut fast_tables = Vec::with_capacity(fast_ret.len()); let mut normal_tables = Vec::with_capacity(ret.len()); diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index a21016014d247..1925acbce7534 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -472,8 +472,8 @@ pub async fn compact( task_progress_guard.progress.clone(), ); match runner.run().await { - Ok(ssts) => { - output_ssts.push((0, ssts, CompactionStatistics::default())); + Ok((ssts, statistics)) => { + output_ssts.push((0, ssts, statistics)); } Err(e) => { task_status = TaskStatus::ExecuteFailed; diff --git a/src/storage/src/hummock/compactor/fast_compactor_runner.rs b/src/storage/src/hummock/compactor/fast_compactor_runner.rs index 6dcfb0e2392cf..c3184fc3e5f76 100644 --- a/src/storage/src/hummock/compactor/fast_compactor_runner.rs +++ b/src/storage/src/hummock/compactor/fast_compactor_runner.rs @@ -24,12 +24,15 @@ use bytes::Bytes; use itertools::Itertools; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::key_range::KeyRange; +use risingwave_hummock_sdk::table_stats::TableStats; use risingwave_hummock_sdk::{can_concat, HummockEpoch, LocalSstableInfo}; use risingwave_pb::hummock::{CompactTask, SstableInfo}; use crate::filter_key_extractor::FilterKeyExtractorImpl; use crate::hummock::compactor::task_progress::TaskProgress; -use crate::hummock::compactor::{Compactor, CompactorContext, RemoteBuilderFactory, TaskConfig}; +use crate::hummock::compactor::{ + CompactionStatistics, Compactor, CompactorContext, RemoteBuilderFactory, TaskConfig, +}; use crate::hummock::multi_builder::{CapacitySplitTableBuilder, TableBuilderFactory}; use crate::hummock::sstable_store::{BlockStream, SstableStoreRef}; use crate::hummock::value::HummockValue; @@ -280,7 +283,6 @@ pub struct CompactorRunner { >, compression_algorithm: CompressionAlgorithm, metrics: Arc, - task_progress: Arc, } impl CompactorRunner { @@ -343,17 +345,16 @@ impl CompactorRunner { )); Self { - executor: CompactTaskExecutor::new(sst_builder, task_config), + executor: CompactTaskExecutor::new(sst_builder, task_config, task_progress), left, right, task_id: task.task_id, metrics: context.compactor_metrics.clone(), compression_algorithm, - task_progress, } } - pub async fn run(mut self) -> HummockResult> { + pub async fn run(mut self) -> HummockResult<(Vec, CompactionStatistics)> { self.left.rewind().await?; self.right.rewind().await?; let mut skip_raw_block_count = 0; @@ -409,6 +410,7 @@ impl CompactorRunner { let largest_key = first.current_sstable().current_block_largest(); let block_len = block.len() as u64; + let block_key_count = meta.total_key_count; if self .executor @@ -419,6 +421,7 @@ impl CompactorRunner { skip_raw_block_size += block_len; skip_raw_block_count += 1; } + self.executor.may_report_process_key(block_key_count); self.executor.clear(); } if !first.current_sstable().is_valid() { @@ -462,6 +465,7 @@ impl CompactorRunner { sstable_iter.download_next_block().await?.unwrap(); let largest_key = sstable_iter.current_block_largest(); let block_len = block.len() as u64; + let block_key_count = block_meta.total_key_count; if self .executor .builder @@ -471,6 +475,7 @@ impl CompactorRunner { skip_raw_block_count += 1; skip_raw_block_size += block_len; } + self.executor.may_report_process_key(block_key_count); } rest_data.next_sstable().await?; } @@ -491,37 +496,61 @@ impl CompactorRunner { skip_raw_block_size * 100 / total_read_bytes, ); + let statistic = self.executor.take_statistics(); let outputs = self.executor.builder.finish().await?; let ssts = Compactor::report_progress( self.metrics.clone(), - Some(self.task_progress.clone()), + Some(self.executor.task_progress.clone()), outputs, false, ) .await?; let sst_infos = ssts.iter().map(|sst| sst.sst_info.clone()).collect_vec(); assert!(can_concat(&sst_infos)); - Ok(ssts) + Ok((ssts, statistic)) } } pub struct CompactTaskExecutor { last_key: FullKey>, + compaction_statistics: CompactionStatistics, + last_table_id: Option, + last_table_stats: TableStats, watermark_can_see_last_key: bool, builder: CapacitySplitTableBuilder, task_config: TaskConfig, + task_progress: Arc, last_key_is_delete: bool, + progress_key_num: u32, } impl CompactTaskExecutor { - pub fn new(builder: CapacitySplitTableBuilder, task_config: TaskConfig) -> Self { + pub fn new( + builder: CapacitySplitTableBuilder, + task_config: TaskConfig, + task_progress: Arc, + ) -> Self { Self { builder, task_config, last_key: FullKey::default(), watermark_can_see_last_key: false, last_key_is_delete: false, + compaction_statistics: CompactionStatistics::default(), + last_table_id: None, + last_table_stats: TableStats::default(), + progress_key_num: 0, + task_progress, + } + } + + fn take_statistics(&mut self) -> CompactionStatistics { + if let Some(last_table_id) = self.last_table_id.take() { + self.compaction_statistics + .delta_drop_stat + .insert(last_table_id, std::mem::take(&mut self.last_table_stats)); } + std::mem::take(&mut self.compaction_statistics) } fn clear(&mut self) { @@ -532,6 +561,17 @@ impl CompactTaskExecutor { self.last_key_is_delete = false; } + #[inline(always)] + fn may_report_process_key(&mut self, key_count: u32) { + const PROGRESS_KEY_INTERVAL: u32 = 100; + self.progress_key_num += key_count; + if self.progress_key_num > PROGRESS_KEY_INTERVAL { + self.task_progress + .inc_progress_key(self.progress_key_num as u64); + self.progress_key_num = 0; + } + } + pub async fn run( &mut self, iter: &mut BlockIterator, @@ -540,6 +580,9 @@ impl CompactTaskExecutor { while iter.is_valid() && iter.key().le(&target_key) { let is_new_user_key = !self.last_key.is_empty() && iter.key().user_key != self.last_key.user_key.as_ref(); + self.compaction_statistics.iter_total_key_counts += 1; + self.may_report_process_key(1); + let mut drop = false; let epoch = iter.key().epoch; let value = HummockValue::from_slice(iter.value()).unwrap(); @@ -562,7 +605,31 @@ impl CompactTaskExecutor { self.watermark_can_see_last_key = true; } + if self.last_table_id.map_or(true, |last_table_id| { + last_table_id != self.last_key.user_key.table_id.table_id + }) { + if let Some(last_table_id) = self.last_table_id.take() { + self.compaction_statistics + .delta_drop_stat + .insert(last_table_id, std::mem::take(&mut self.last_table_stats)); + } + self.last_table_id = Some(self.last_key.user_key.table_id.table_id); + } + if drop { + self.compaction_statistics.iter_drop_key_counts += 1; + + let should_count = match self.task_config.stats_target_table_ids.as_ref() { + Some(target_table_ids) => { + target_table_ids.contains(&self.last_key.user_key.table_id.table_id) + } + None => true, + }; + if should_count { + self.last_table_stats.total_key_count -= 1; + self.last_table_stats.total_key_size -= self.last_key.encoded_len() as i64; + self.last_table_stats.total_value_size -= value.encoded_len() as i64; + } iter.next(); continue; } diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 7ccb3fbf04790..6404d80bb265f 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -604,9 +604,6 @@ mod tests { use crate::hummock::utils::MemoryLimiter; - // This is a clippy bug, see https://github.com/rust-lang/rust-clippy/issues/11380. - // TODO: remove `allow` here after the issued is closed. - #[expect(clippy::needless_pass_by_ref_mut)] async fn assert_pending(future: &mut (impl Future + Unpin)) { for _ in 0..10 { assert!(poll_fn(|cx| Poll::Ready(future.poll_unpin(cx))) From 0f9e783d2ab5c02811e3c8254e6979cf63b46b7f Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Thu, 26 Oct 2023 14:22:04 +0800 Subject: [PATCH 05/52] refactor: always apply flow control on source/chain (#13057) --- src/stream/src/executor/flow_control.rs | 31 +++++++++++-------- src/stream/src/from_proto/chain.rs | 7 ++--- src/stream/src/from_proto/source/fs_fetch.rs | 6 ++-- .../src/from_proto/source/trad_source.rs | 7 ++--- 4 files changed, 24 insertions(+), 27 deletions(-) diff --git a/src/stream/src/executor/flow_control.rs b/src/stream/src/executor/flow_control.rs index 45e04717e2a9d..230359109fca0 100644 --- a/src/stream/src/executor/flow_control.rs +++ b/src/stream/src/executor/flow_control.rs @@ -32,12 +32,12 @@ use super::*; /// It is used to throttle problematic MVs that are consuming too much resources. pub struct FlowControlExecutor { input: BoxedExecutor, - rate_limit: u32, + rate_limit: Option, } impl FlowControlExecutor { #[allow(clippy::too_many_arguments)] - pub fn new(input: Box, rate_limit: u32) -> Self { + pub fn new(input: Box, rate_limit: Option) -> Self { #[cfg(madsim)] println!("FlowControlExecutor rate limiter is disabled in madsim as it will spawn system threads"); Self { input, rate_limit } @@ -45,9 +45,12 @@ impl FlowControlExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(self) { - let quota = Quota::per_second(NonZeroU32::new(self.rate_limit).unwrap()); - let clock = MonotonicClock; - let rate_limiter = RateLimiter::direct_with_clock(quota, &clock); + let get_rate_limiter = |rate_limit: u32| { + let quota = Quota::per_second(NonZeroU32::new(rate_limit).unwrap()); + let clock = MonotonicClock; + RateLimiter::direct_with_clock(quota, &clock) + }; + let rate_limiter = self.rate_limit.map(get_rate_limiter); #[for_await] for msg in self.input.execute() { let msg = msg?; @@ -55,14 +58,16 @@ impl FlowControlExecutor { Message::Chunk(chunk) => { #[cfg(not(madsim))] { - let result = rate_limiter - .until_n_ready(NonZeroU32::new(chunk.cardinality() as u32).unwrap()) - .await; - if let Err(InsufficientCapacity(n)) = result { - tracing::error!( - "Rate Limit {} smaller than chunk cardinality {n}", - self.rate_limit, - ); + if let Some(rate_limiter) = &rate_limiter { + let result = rate_limiter + .until_n_ready(NonZeroU32::new(chunk.cardinality() as u32).unwrap()) + .await; + if let Err(InsufficientCapacity(n)) = result { + tracing::error!( + "Rate Limit {:?} smaller than chunk cardinality {n}", + self.rate_limit, + ); + } } } yield Message::Chunk(chunk); diff --git a/src/stream/src/from_proto/chain.rs b/src/stream/src/from_proto/chain.rs index 81030526b82f3..ba48d2c5a25f1 100644 --- a/src/stream/src/from_proto/chain.rs +++ b/src/stream/src/from_proto/chain.rs @@ -178,10 +178,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { } ChainType::ChainUnspecified => unreachable!(), }; - if let Ok(rate_limit) = node.get_rate_limit() { - Ok(FlowControlExecutor::new(executor, *rate_limit).boxed()) - } else { - Ok(executor) - } + let rate_limit = node.get_rate_limit().cloned().ok(); + Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) } } diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index b6df84c8560e4..65923d2dced3a 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -112,9 +112,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { ) .boxed(); - if let Ok(rate_limit) = source.get_rate_limit() { - return Ok(FlowControlExecutor::new(executor, *rate_limit).boxed()); - } - Ok(executor) + let rate_limit = source.get_rate_limit().cloned().ok(); + Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) } } diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 3f0793595c7c5..b87ce5ff39dc7 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -207,11 +207,8 @@ impl ExecutorBuilder for SourceExecutorBuilder { } } }; - if let Ok(rate_limit) = source.get_rate_limit() { - Ok(FlowControlExecutor::new(executor, *rate_limit).boxed()) - } else { - Ok(executor) - } + let rate_limit = source.get_rate_limit().cloned().ok(); + Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) } else { // If there is no external stream source, then no data should be persisted. We pass a // `PanicStateStore` type here for indication. From 9d08bb28adbcaa95cf1f5df94c7f96acb95c2072 Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Thu, 26 Oct 2023 14:26:44 +0800 Subject: [PATCH 06/52] fix(sink): fix integer array type for remote sink (#12334) --- e2e_test/sink/remote/jdbc.check.pg.slt | 8 ++--- e2e_test/sink/remote/jdbc.load.slt | 14 +++++---- e2e_test/sink/remote/mysql_create_table.sql | 6 +++- .../sink/remote/mysql_expected_result_2.tsv | 6 ++-- e2e_test/sink/remote/pg_create_table.sql | 6 +++- .../connector/jdbc/PostgresDialect.java | 30 +++++++++++++++++-- src/jni_core/src/lib.rs | 6 ++-- 7 files changed, 56 insertions(+), 20 deletions(-) diff --git a/e2e_test/sink/remote/jdbc.check.pg.slt b/e2e_test/sink/remote/jdbc.check.pg.slt index 6293c44a5a444..7cf6faa7b0fb6 100644 --- a/e2e_test/sink/remote/jdbc.check.pg.slt +++ b/e2e_test/sink/remote/jdbc.check.pg.slt @@ -22,13 +22,13 @@ select * from t_remote_1 order by id; query III select * from biz.t_types order by id; ---- -1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} {"Value 1","Value 2"} {12.345,56.789} -2 Varcharvalue2 Textvalue2 234 567 890 23.45 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} {"Value 3","Value 4"} {43.21,65.432} -3 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} {"How're you?","\"hello\\ \\world\""} {12.345,56.789} +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} {"Value 1","Value 2"} {12.345,56.789} {1,2,3} {1,2,3} {1,2,3} {12.3,56.7} +2 Varcharvalue2 Textvalue2 234 567 890 23.45 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} {"Value 3","Value 4"} {43.21,65.432} {4,5,6} {4,5,6} {4,5,6} {43.2,65.4} +3 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} {"How're you?","\"hello\\ \\world\""} {12.345,56.789} {1,2,3} {1,2,3} {1,2,3} {43.2,65.4} query IT -select * from t_append_only order by v1, v2; +select * from t_append_only order by v1,v2; ---- 1 aaa 1 bbb diff --git a/e2e_test/sink/remote/jdbc.load.slt b/e2e_test/sink/remote/jdbc.load.slt index 70ad3f9a3a42b..a3bc63e48f7de 100644 --- a/e2e_test/sink/remote/jdbc.load.slt +++ b/e2e_test/sink/remote/jdbc.load.slt @@ -50,7 +50,11 @@ CREATE TABLE rw_typed_data ( interval_column INTERVAL, jsonb_column JSONB, array_column VARCHAR[], - array_column2 FLOAT[] + array_column2 FLOAT[], + array_column3 SMALLINT[], + array_column4 INTEGER[], + array_column5 BIGINT[], + array_column6 DOUBLE PRECISION[], ); statement ok @@ -196,10 +200,10 @@ INSERT INTO t_remote_1 VALUES (6, 'Varchar value 6', 'Text value 6', 789, 123, 456, 67.89, 34.56, 78.91, FALSE, '2023-05-27', '23:45:01', '2023-05-27 23:45:01', '2023-05-27 23:45:01', '2 years 3 months 4 days 5 hours 6 minutes 7 seconds', '{"key": "value6"}', E'\\xDEADBABE'); statement ok -INSERT INTO rw_typed_data (id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column, array_column, array_column2) VALUES - (1, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', ARRAY['Value 1', 'Value 2'], '{12.345,56.789}'), - (2, 'Varcharvalue2', 'Textvalue2', 234, 567, 890, 23.45, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}', ARRAY['Value 3', 'Value 4'], '{43.21,65.432}'), - (3, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', ARRAY['How''re you?', '"hello\ \world"'], ARRAY[12.345,56.789]); +INSERT INTO rw_typed_data (id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column, array_column, array_column2, array_column3, array_column4, array_column5, array_column6) VALUES + (1, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', ARRAY['Value 1', 'Value 2'], '{12.345,56.789}', '{1, 2, 3}', '{1, 2, 3}', '{1, 2, 3}', '{12.3,56.7}'), + (2, 'Varcharvalue2', 'Textvalue2', 234, 567, 890, 23.45, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}', ARRAY['Value 3', 'Value 4'], '{43.21,65.432}', '{4, 5, 6}', '{4, 5, 6}', '{4, 5, 6}', '{43.2,65.4}'), + (3, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', ARRAY['How''re you?', '"hello\ \world"'], ARRAY[12.345,56.789], ARRAY[1, 2, 3], ARRAY[1, 2, 3], ARRAY[1, 2, 3], ARRAY[43.2,65.4]); statement ok FLUSH; diff --git a/e2e_test/sink/remote/mysql_create_table.sql b/e2e_test/sink/remote/mysql_create_table.sql index 0cbe15f7dcb8e..9eab3d0f5e13a 100644 --- a/e2e_test/sink/remote/mysql_create_table.sql +++ b/e2e_test/sink/remote/mysql_create_table.sql @@ -47,5 +47,9 @@ CREATE TABLE t_types ( interval_column VARCHAR(100), jsonb_column JSON, array_column LONGTEXT, - array_column2 LONGTEXT + array_column2 LONGTEXT, + array_column3 LONGTEXT, + array_column4 LONGTEXT, + array_column5 LONGTEXT, + array_column6 LONGTEXT ); diff --git a/e2e_test/sink/remote/mysql_expected_result_2.tsv b/e2e_test/sink/remote/mysql_expected_result_2.tsv index 87ac3cb3bd123..061ee02d39d17 100644 --- a/e2e_test/sink/remote/mysql_expected_result_2.tsv +++ b/e2e_test/sink/remote/mysql_expected_result_2.tsv @@ -1,3 +1,3 @@ -1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 1 2023-05-22 12:34:56 2023-05-22 12:34:56 P0Y0M1DT0H0M0S {"key": "value"} Value 1,Value 2 12.345,56.789 -2 Varcharvalue2 Textvalue2 234 567 890 23.45 67.89 1.23 0 2023-05-23 23:45:01 2023-05-23 23:45:01 P0Y0M2DT0H0M0S {"key": "value2"} Value 3,Value 4 43.21,65.432 -3 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 1 2023-05-22 12:34:56 2023-05-22 12:34:56 P0Y0M1DT0H0M0S {"key": "value"} How're you?,"hello\ \world" 12.345,56.789 +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 1 2023-05-22 12:34:56 2023-05-22 12:34:56 P0Y0M1DT0H0M0S {"key": "value"} Value 1,Value 2 12.345,56.789 1,2,3 1,2,3 1,2,3 12.3,56.7 +2 Varcharvalue2 Textvalue2 234 567 890 23.45 67.89 1.23 0 2023-05-23 23:45:01 2023-05-23 23:45:01 P0Y0M2DT0H0M0S {"key": "value2"} Value 3,Value 4 43.21,65.432 4,5,6 4,5,6 4,5,6 43.2,65.4 +3 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 1 2023-05-22 12:34:56 2023-05-22 12:34:56 P0Y0M1DT0H0M0S {"key": "value"} How're you?,"hello\ \world" 12.345,56.789 1,2,3 1,2,3 1,2,3 43.2,65.4 diff --git a/e2e_test/sink/remote/pg_create_table.sql b/e2e_test/sink/remote/pg_create_table.sql index fd06aca93ce7b..3677dbb067131 100644 --- a/e2e_test/sink/remote/pg_create_table.sql +++ b/e2e_test/sink/remote/pg_create_table.sql @@ -73,7 +73,11 @@ CREATE TABLE biz.t_types ( interval_column INTERVAL, jsonb_column JSONB, array_column VARCHAR[], - array_column2 DECIMAL[] + array_column2 FLOAT[], + array_column3 SMALLINT[], + array_column4 INTEGER[], + array_column5 BIGINT[], + array_column6 DOUBLE PRECISION[] ); CREATE TABLE biz.t2 ( diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/jdbc/PostgresDialect.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/jdbc/PostgresDialect.java index 570e2beaf5a67..5bbdf1116f2c1 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/jdbc/PostgresDialect.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/jdbc/PostgresDialect.java @@ -16,9 +16,11 @@ import com.risingwave.connector.api.TableSchema; import com.risingwave.connector.api.sink.SinkRow; +import com.risingwave.proto.Data.DataType.TypeName; import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; +import java.util.HashMap; import java.util.List; import java.util.Optional; import java.util.stream.Collectors; @@ -33,6 +35,26 @@ public PostgresDialect(int[] columnSqlTypes) { this.columnSqlTypes = columnSqlTypes; } + private static final HashMap RW_TYPE_TO_JDBC_TYPE_NAME; + + static { + RW_TYPE_TO_JDBC_TYPE_NAME = new HashMap(); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.INT16, "int2"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.INT32, "int4"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.INT64, "int8"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.FLOAT, "float4"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.DOUBLE, "float8"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.BOOLEAN, "bool"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.VARCHAR, "varchar"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.DECIMAL, "numeric"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.TIME, "time"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.TIMESTAMP, "timestamp"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.INTERVAL, "varchar"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.DATE, "date"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.TIMESTAMPTZ, "timestamptz"); + RW_TYPE_TO_JDBC_TYPE_NAME.put(TypeName.JSONB, "varchar"); + } + @Override public SchemaTableName createSchemaTableName(String schemaName, String tableName) { if (schemaName == null || schemaName.isBlank()) { @@ -115,9 +137,11 @@ public void bindInsertIntoStatement( Object[] objArray = (Object[]) val; assert (column.getDataType().getFieldTypeCount() == 1); var fieldType = column.getDataType().getFieldType(0); - stmt.setArray( - placeholderIdx++, - conn.createArrayOf(fieldType.getTypeName().name(), objArray)); + var typeName = RW_TYPE_TO_JDBC_TYPE_NAME.get(fieldType.getTypeName()); + if (typeName == null) { + typeName = fieldType.getTypeName().name(); + } + stmt.setArray(placeholderIdx++, conn.createArrayOf(typeName, objArray)); break; case VARCHAR: // since VARCHAR column may sink to a UUID column, we get the target type diff --git a/src/jni_core/src/lib.rs b/src/jni_core/src/lib.rs index 4815cd7368370..c92ad2f146e6c 100644 --- a/src/jni_core/src/lib.rs +++ b/src/jni_core/src/lib.rs @@ -782,7 +782,7 @@ extern "system" fn Java_com_risingwave_java_binding_Binding_iteratorGetArrayValu let obj = env.call_static_method( &class, "valueOf", - "(S)Ljava.lang.Short;", + "(S)Ljava/lang/Short;", &[JValue::from(v as jshort)], )?; if let JValueOwned::Object(o) = obj { @@ -793,7 +793,7 @@ extern "system" fn Java_com_risingwave_java_binding_Binding_iteratorGetArrayValu let obj = env.call_static_method( &class, "valueOf", - "(I)Ljava.lang.Integer;", + "(I)Ljava/lang/Integer;", &[JValue::from(v as jint)], )?; if let JValueOwned::Object(o) = obj { @@ -804,7 +804,7 @@ extern "system" fn Java_com_risingwave_java_binding_Binding_iteratorGetArrayValu let obj = env.call_static_method( &class, "valueOf", - "(J)Ljava.lang.Long;", + "(J)Ljava/lang/Long;", &[JValue::from(v as jlong)], )?; if let JValueOwned::Object(o) = obj { From efdf3c9688a04232ebf5c7876220b965444e48db Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 26 Oct 2023 14:39:44 +0800 Subject: [PATCH 07/52] refactor(optimizer): type-safe plan base with compile-time convention check (#13000) Signed-off-by: Bugen Zhao --- Cargo.lock | 1 + src/frontend/Cargo.toml | 1 + src/frontend/src/handler/explain.rs | 1 + src/frontend/src/optimizer/mod.rs | 8 +- src/frontend/src/optimizer/plan_node/batch.rs | 8 + .../src/optimizer/plan_node/batch_delete.rs | 6 +- .../src/optimizer/plan_node/batch_exchange.rs | 5 +- .../src/optimizer/plan_node/batch_expand.rs | 3 +- .../src/optimizer/plan_node/batch_filter.rs | 3 +- .../optimizer/plan_node/batch_group_topn.rs | 3 +- .../src/optimizer/plan_node/batch_hash_agg.rs | 3 +- .../optimizer/plan_node/batch_hash_join.rs | 3 +- .../optimizer/plan_node/batch_hop_window.rs | 3 +- .../src/optimizer/plan_node/batch_insert.rs | 5 +- .../src/optimizer/plan_node/batch_limit.rs | 3 +- .../optimizer/plan_node/batch_lookup_join.rs | 3 +- .../plan_node/batch_nested_loop_join.rs | 3 +- .../optimizer/plan_node/batch_over_window.rs | 3 +- .../src/optimizer/plan_node/batch_project.rs | 4 +- .../optimizer/plan_node/batch_project_set.rs | 3 +- .../src/optimizer/plan_node/batch_seq_scan.rs | 5 +- .../optimizer/plan_node/batch_simple_agg.rs | 3 +- .../src/optimizer/plan_node/batch_sort.rs | 3 +- .../src/optimizer/plan_node/batch_sort_agg.rs | 3 +- .../src/optimizer/plan_node/batch_source.rs | 4 +- .../plan_node/batch_table_function.rs | 4 +- .../src/optimizer/plan_node/batch_topn.rs | 3 +- .../src/optimizer/plan_node/batch_union.rs | 3 +- .../src/optimizer/plan_node/batch_update.rs | 3 +- .../src/optimizer/plan_node/batch_values.rs | 4 +- .../src/optimizer/plan_node/generic/agg.rs | 14 +- .../plan_node/generic/dynamic_filter.rs | 4 +- .../src/optimizer/plan_node/generic/join.rs | 4 +- .../src/optimizer/plan_node/generic/mod.rs | 2 + .../src/optimizer/plan_node/logical_agg.rs | 8 +- .../src/optimizer/plan_node/logical_apply.rs | 6 +- .../src/optimizer/plan_node/logical_dedup.rs | 10 +- .../src/optimizer/plan_node/logical_delete.rs | 6 +- .../src/optimizer/plan_node/logical_except.rs | 6 +- .../src/optimizer/plan_node/logical_expand.rs | 6 +- .../src/optimizer/plan_node/logical_filter.rs | 6 +- .../optimizer/plan_node/logical_hop_window.rs | 7 +- .../src/optimizer/plan_node/logical_insert.rs | 6 +- .../optimizer/plan_node/logical_intersect.rs | 6 +- .../src/optimizer/plan_node/logical_join.rs | 12 +- .../src/optimizer/plan_node/logical_limit.rs | 6 +- .../optimizer/plan_node/logical_multi_join.rs | 4 +- .../src/optimizer/plan_node/logical_now.rs | 4 +- .../plan_node/logical_over_window.rs | 6 +- .../optimizer/plan_node/logical_project.rs | 6 +- .../plan_node/logical_project_set.rs | 7 +- .../src/optimizer/plan_node/logical_scan.rs | 4 +- .../src/optimizer/plan_node/logical_share.rs | 8 +- .../src/optimizer/plan_node/logical_source.rs | 8 +- .../plan_node/logical_table_function.rs | 4 +- .../src/optimizer/plan_node/logical_topn.rs | 10 +- .../src/optimizer/plan_node/logical_union.rs | 6 +- .../src/optimizer/plan_node/logical_update.rs | 6 +- .../src/optimizer/plan_node/logical_values.rs | 6 +- src/frontend/src/optimizer/plan_node/mod.rs | 184 ++++++---- .../src/optimizer/plan_node/plan_base.rs | 316 ++++++++++++------ .../src/optimizer/plan_node/stream.rs | 8 + .../src/optimizer/plan_node/stream_dedup.rs | 8 +- .../optimizer/plan_node/stream_delta_join.rs | 3 +- .../src/optimizer/plan_node/stream_dml.rs | 3 +- .../plan_node/stream_dynamic_filter.rs | 3 +- .../plan_node/stream_eowc_over_window.rs | 3 +- .../optimizer/plan_node/stream_exchange.rs | 5 +- .../src/optimizer/plan_node/stream_expand.rs | 3 +- .../src/optimizer/plan_node/stream_filter.rs | 3 +- .../optimizer/plan_node/stream_fs_fetch.rs | 4 +- .../optimizer/plan_node/stream_group_topn.rs | 3 +- .../optimizer/plan_node/stream_hash_agg.rs | 3 +- .../optimizer/plan_node/stream_hash_join.rs | 3 +- .../optimizer/plan_node/stream_hop_window.rs | 3 +- .../optimizer/plan_node/stream_materialize.rs | 3 +- .../src/optimizer/plan_node/stream_now.rs | 4 +- .../optimizer/plan_node/stream_over_window.rs | 3 +- .../src/optimizer/plan_node/stream_project.rs | 3 +- .../optimizer/plan_node/stream_project_set.rs | 3 +- .../optimizer/plan_node/stream_row_id_gen.rs | 3 +- .../src/optimizer/plan_node/stream_share.rs | 27 +- .../optimizer/plan_node/stream_simple_agg.rs | 5 +- .../src/optimizer/plan_node/stream_sink.rs | 9 +- .../src/optimizer/plan_node/stream_sort.rs | 5 +- .../src/optimizer/plan_node/stream_source.rs | 4 +- .../plan_node/stream_stateless_simple_agg.rs | 3 +- .../optimizer/plan_node/stream_table_scan.rs | 6 +- .../plan_node/stream_temporal_join.rs | 3 +- .../src/optimizer/plan_node/stream_topn.rs | 3 +- .../src/optimizer/plan_node/stream_union.rs | 3 +- .../src/optimizer/plan_node/stream_values.rs | 4 +- .../plan_node/stream_watermark_filter.rs | 9 +- .../src/optimizer/property/distribution.rs | 1 + src/frontend/src/optimizer/property/order.rs | 2 + src/frontend/src/scheduler/plan_fragmenter.rs | 2 +- 96 files changed, 622 insertions(+), 333 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 3f7df00f7648d..1c741e3a5ae67 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7491,6 +7491,7 @@ dependencies = [ "async-recursion", "async-trait", "auto_enums", + "auto_impl", "bk-tree", "bytes", "clap", diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 37f9f6326faea..bae582dd06e24 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -21,6 +21,7 @@ arrow-schema = { workspace = true } async-recursion = "1.0.5" async-trait = "0.1" auto_enums = { version = "0.8", features = ["futures03"] } +auto_impl = "1" bk-tree = "0.5.0" bytes = "1" clap = { version = "4", features = ["derive"] } diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index 5de7ec95b38bd..fe798f3fa2857 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -30,6 +30,7 @@ use super::create_table::{ use super::query::gen_batch_plan_by_statement; use super::RwPgResponse; use crate::handler::HandlerArgs; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{Convention, Explain}; use crate::optimizer::OptimizerContext; use crate::scheduler::worker_node_manager::WorkerNodeSelector; diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 0df387b0a53d5..4004748a2f4f9 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -17,6 +17,7 @@ use std::ops::DerefMut; pub mod plan_node; pub use plan_node::{Explain, PlanRef}; + pub mod property; mod delta_join_solver; @@ -46,10 +47,11 @@ use risingwave_connector::sink::catalog::SinkFormatDesc; use risingwave_pb::catalog::WatermarkDesc; use self::heuristic_optimizer::ApplyOrder; +use self::plan_node::generic::{self, PhysicalPlanRef}; use self::plan_node::{ - generic, stream_enforce_eowc_requirement, BatchProject, Convention, LogicalProject, - LogicalSource, StreamDml, StreamMaterialize, StreamProject, StreamRowIdGen, StreamSink, - StreamWatermarkFilter, ToStreamContext, + stream_enforce_eowc_requirement, BatchProject, Convention, LogicalProject, LogicalSource, + StreamDml, StreamMaterialize, StreamProject, StreamRowIdGen, StreamSink, StreamWatermarkFilter, + ToStreamContext, }; #[cfg(debug_assertions)] use self::plan_visitor::InputRefValidator; diff --git a/src/frontend/src/optimizer/plan_node/batch.rs b/src/frontend/src/optimizer/plan_node/batch.rs index 2cb2360b3e51d..5eeafab64f1cd 100644 --- a/src/frontend/src/optimizer/plan_node/batch.rs +++ b/src/frontend/src/optimizer/plan_node/batch.rs @@ -23,6 +23,14 @@ use crate::optimizer::property::Order; /// [`PhysicalPlanRef`]. /// /// [`GenericPlanRef`]: super::generic::GenericPlanRef +#[auto_impl::auto_impl(&)] pub trait BatchPlanRef: PhysicalPlanRef { fn order(&self) -> &Order; } + +/// Prelude for batch plan nodes. +pub mod prelude { + pub use super::super::generic::{GenericPlanRef, PhysicalPlanRef}; + pub use super::super::Batch; + pub use super::BatchPlanRef; +} diff --git a/src/frontend/src/optimizer/plan_node/batch_delete.rs b/src/frontend/src/optimizer/plan_node/batch_delete.rs index 85d22a46b450e..96ca967a2f745 100644 --- a/src/frontend/src/optimizer/plan_node/batch_delete.rs +++ b/src/frontend/src/optimizer/plan_node/batch_delete.rs @@ -16,24 +16,26 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::DeleteNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, }; +use crate::optimizer::plan_node::generic::PhysicalPlanRef; use crate::optimizer::plan_node::ToLocalBatch; use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchDelete` implements [`super::LogicalDelete`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchDelete { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Delete, } impl BatchDelete { pub fn new(core: generic::Delete) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = + let base = PlanBase::new_batch_with_core(&core, core.input.distribution().clone(), Order::any()); Self { base, core } } diff --git a/src/frontend/src/optimizer/plan_node/batch_exchange.rs b/src/frontend/src/optimizer/plan_node/batch_exchange.rs index 6477c7ec213e2..ec3a195e050bb 100644 --- a/src/frontend/src/optimizer/plan_node/batch_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/batch_exchange.rs @@ -17,8 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{ExchangeNode, MergeSortExchangeNode}; -use super::batch::BatchPlanRef; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::ToLocalBatch; @@ -28,7 +27,7 @@ use crate::optimizer::property::{Distribution, DistributionDisplay, Order, Order /// without changing its content. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchExchange { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, } diff --git a/src/frontend/src/optimizer/plan_node/batch_expand.rs b/src/frontend/src/optimizer/plan_node/batch_expand.rs index af4413b9e5152..7b31d59fd13e5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_expand.rs +++ b/src/frontend/src/optimizer/plan_node/batch_expand.rs @@ -18,6 +18,7 @@ use risingwave_pb::batch_plan::expand_node::Subset; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ExpandNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable}; use crate::optimizer::plan_node::{ @@ -28,7 +29,7 @@ use crate::optimizer::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index 4bff7cbfee3c0..c5c1430772750 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::FilterNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; @@ -25,7 +26,7 @@ use crate::utils::Condition; /// `BatchFilter` implements [`super::LogicalFilter`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs index 70ee8328623f5..1817c90442761 100644 --- a/src/frontend/src/optimizer/plan_node/batch_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_group_topn.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::GroupTopNNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, @@ -26,7 +27,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchGroupTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchGroupTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index b4ab3341ace29..58512a42f0240 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashAggNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ @@ -30,7 +31,7 @@ use crate::utils::{ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHashAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index bad586d4af1e4..d2135bb47b520 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -18,6 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashJoinNode; use risingwave_pb::plan_common::JoinType; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ @@ -35,7 +36,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// get output rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHashJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs index 2a4a27f9a0583..3cbfbb38372f4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hop_window.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HopWindowNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, @@ -29,7 +30,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// input rows #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, window_start_exprs: Vec, window_end_exprs: Vec, diff --git a/src/frontend/src/optimizer/plan_node/batch_insert.rs b/src/frontend/src/optimizer/plan_node/batch_insert.rs index aec05eee145b8..2d89b2a14b128 100644 --- a/src/frontend/src/optimizer/plan_node/batch_insert.rs +++ b/src/frontend/src/optimizer/plan_node/batch_insert.rs @@ -18,6 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::InsertNode; use risingwave_pb::plan_common::{DefaultColumns, IndexAndExpr}; +use super::batch::prelude::*; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; @@ -28,14 +29,14 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchInsert` implements [`super::LogicalInsert`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchInsert { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Insert, } impl BatchInsert { pub fn new(core: generic::Insert) -> Self { assert_eq!(core.input.distribution(), &Distribution::Single); - let base: PlanBase = + let base: PlanBase = PlanBase::new_batch_with_core(&core, core.input.distribution().clone(), Order::any()); BatchInsert { base, core } diff --git a/src/frontend/src/optimizer/plan_node/batch_limit.rs b/src/frontend/src/optimizer/plan_node/batch_limit.rs index 93b14d0198979..f1b9739e8ac2b 100644 --- a/src/frontend/src/optimizer/plan_node/batch_limit.rs +++ b/src/frontend/src/optimizer/plan_node/batch_limit.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::LimitNode; +use super::batch::prelude::*; use super::generic::PhysicalPlanRef; use super::utils::impl_distill_by_unit; use super::{ @@ -27,7 +28,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchLimit` implements [`super::LogicalLimit`] to fetch specified rows from input #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchLimit { - pub base: PlanBase, + pub base: PlanBase, core: generic::Limit, } diff --git a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs index 48f99668c3af7..b78bf314c1276 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -18,6 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::{DistributedLookupJoinNode, LocalLookupJoinNode}; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::ExprRewritable; @@ -33,7 +34,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchLookupJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs index 8980ad2f23f6d..fe789b47937f5 100644 --- a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::NestedLoopJoinNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchPb, ToDistributedBatch}; @@ -30,7 +31,7 @@ use crate::utils::ConditionDisplay; /// against all pairs of rows from inner & outer side within 2 layers of loops. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchNestedLoopJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, } diff --git a/src/frontend/src/optimizer/plan_node/batch_over_window.rs b/src/frontend/src/optimizer/plan_node/batch_over_window.rs index fb455758f331a..011de0dfb1459 100644 --- a/src/frontend/src/optimizer/plan_node/batch_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/batch_over_window.rs @@ -17,6 +17,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortOverWindowNode; +use super::batch::prelude::*; use super::batch::BatchPlanRef; use super::generic::PlanWindowFunction; use super::utils::impl_distill_by_unit; @@ -28,7 +29,7 @@ use crate::optimizer::property::{Order, RequiredDist}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 642683967c5c3..b32498d547dbf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -18,7 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectNode; use risingwave_pb::expr::ExprNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch, @@ -31,7 +31,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// rows #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, } diff --git a/src/frontend/src/optimizer/plan_node/batch_project_set.rs b/src/frontend/src/optimizer/plan_node/batch_project_set.rs index 5888df9d15889..5a355c0db22db 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project_set.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectSetNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable}; use crate::expr::ExprRewriter; @@ -28,7 +29,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, } diff --git a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs index 6834ed29353b9..7a2d97c266b36 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -24,8 +24,7 @@ use risingwave_pb::batch_plan::row_seq_scan_node::ChunkSize; use risingwave_pb::batch_plan::{RowSeqScanNode, SysRowSeqScanNode}; use risingwave_pb::plan_common::PbColumnDesc; -use super::batch::BatchPlanRef; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch}; use crate::catalog::ColumnId; @@ -36,7 +35,7 @@ use crate::optimizer::property::{Distribution, DistributionDisplay, Order}; /// `BatchSeqScan` implements [`super::LogicalScan`] to scan from a row-oriented table #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSeqScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, scan_ranges: Vec, } diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index bae8d70c2eedf..cbc8f0a95c1f8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; @@ -25,7 +26,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/batch_sort.rs b/src/frontend/src/optimizer/plan_node/batch_sort.rs index e7bff6d51d85b..c2e713c68138a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort.rs @@ -17,6 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortNode; +use super::batch::prelude::*; use super::batch::BatchPlanRef; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; @@ -27,7 +28,7 @@ use crate::optimizer::property::{Order, OrderDisplay}; /// collation required by user or parent plan node. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSort { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, } diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index 2252d4c0c0ee0..cd8e3b43ca8ec 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -17,6 +17,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; use risingwave_pb::expr::ExprNode; +use super::batch::prelude::*; use super::generic::{self, GenericPlanRef, PlanAggCall}; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchPb, ToDistributedBatch}; @@ -27,7 +28,7 @@ use crate::utils::{ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSortAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, input_order: Order, } diff --git a/src/frontend/src/optimizer/plan_node/batch_source.rs b/src/frontend/src/optimizer/plan_node/batch_source.rs index 9e2cd6006db0b..56f6399cd844e 100644 --- a/src/frontend/src/optimizer/plan_node/batch_source.rs +++ b/src/frontend/src/optimizer/plan_node/batch_source.rs @@ -19,7 +19,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SourceNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, column_names_pretty, Distill}; use super::{ generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch, ToLocalBatch, @@ -30,7 +30,7 @@ use crate::optimizer::property::{Distribution, Order}; /// [`BatchSource`] represents a table/connector source at the very beginning of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchSource { - pub base: PlanBase, + pub base: PlanBase, core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/batch_table_function.rs b/src/frontend/src/optimizer/plan_node/batch_table_function.rs index 0b9887cd4aaba..a249ac722cdcf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/batch_table_function.rs @@ -17,7 +17,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TableFunctionNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, ToDistributedBatch}; use crate::expr::ExprRewriter; @@ -27,7 +27,7 @@ use crate::optimizer::property::{Distribution, Order}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchTableFunction { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalTableFunction, } diff --git a/src/frontend/src/optimizer/plan_node/batch_topn.rs b/src/frontend/src/optimizer/plan_node/batch_topn.rs index b2eda24046d28..d508b0da53317 100644 --- a/src/frontend/src/optimizer/plan_node/batch_topn.rs +++ b/src/frontend/src/optimizer/plan_node/batch_topn.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::TopNNode; +use super::batch::prelude::*; use super::generic::TopNLimit; use super::utils::impl_distill_by_unit; use super::{ @@ -28,7 +29,7 @@ use crate::optimizer::property::{Order, RequiredDist}; /// `BatchTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/batch_union.rs b/src/frontend/src/optimizer/plan_node/batch_union.rs index c7c71111174c6..5f9e354970a65 100644 --- a/src/frontend/src/optimizer/plan_node/batch_union.rs +++ b/src/frontend/src/optimizer/plan_node/batch_union.rs @@ -16,6 +16,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UnionNode; +use super::batch::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, ToBatchPb, ToDistributedBatch}; use crate::optimizer::plan_node::{PlanBase, PlanTreeNode, ToLocalBatch}; @@ -24,7 +25,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchUnion` implements [`super::LogicalUnion`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index 20e4b8b6b966c..fdcc546f873a8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -18,6 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UpdateNode; +use super::batch::prelude::*; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ @@ -30,7 +31,7 @@ use crate::optimizer::property::{Distribution, Order, RequiredDist}; /// `BatchUpdate` implements [`super::LogicalUpdate`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchUpdate { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Update, } diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 9348cddba7422..00483f37256f4 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -18,7 +18,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::values_node::ExprTuple; use risingwave_pb::batch_plan::ValuesNode; -use super::generic::GenericPlanRef; +use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ ExprRewritable, LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchPb, @@ -30,7 +30,7 @@ use crate::optimizer::property::{Distribution, Order}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct BatchValues { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalValues, } diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index e0c7e339ee6a6..98109e695110f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -265,7 +265,7 @@ pub struct MaterializedInputState { impl Agg { pub fn infer_tables( &self, - me: &impl stream::StreamPlanRef, + me: impl stream::StreamPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> ( @@ -274,9 +274,9 @@ impl Agg { HashMap, ) { ( - self.infer_intermediate_state_table(me, vnode_col_idx, window_col_idx), - self.infer_stream_agg_state(me, vnode_col_idx, window_col_idx), - self.infer_distinct_dedup_tables(me, vnode_col_idx, window_col_idx), + self.infer_intermediate_state_table(&me, vnode_col_idx, window_col_idx), + self.infer_stream_agg_state(&me, vnode_col_idx, window_col_idx), + self.infer_distinct_dedup_tables(&me, vnode_col_idx, window_col_idx), ) } @@ -338,7 +338,7 @@ impl Agg { /// Infer `AggCallState`s for streaming agg. pub fn infer_stream_agg_state( &self, - me: &impl stream::StreamPlanRef, + me: impl stream::StreamPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> Vec { @@ -487,7 +487,7 @@ impl Agg { /// group key | state for AGG1 | state for AGG2 | ... pub fn infer_intermediate_state_table( &self, - me: &impl GenericPlanRef, + me: impl GenericPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> TableCatalog { @@ -550,7 +550,7 @@ impl Agg { /// group key | distinct key | count for AGG1(distinct x) | count for AGG2(distinct x) | ... pub fn infer_distinct_dedup_tables( &self, - me: &impl GenericPlanRef, + me: impl GenericPlanRef, vnode_col_idx: Option, window_col_idx: Option, ) -> HashMap { diff --git a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs index 85ffd922c43e7..edadd8b07ae3f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/generic/dynamic_filter.rs @@ -140,7 +140,7 @@ impl DynamicFilter { } pub fn infer_left_internal_table_catalog( - me: &impl stream::StreamPlanRef, + me: impl stream::StreamPlanRef, left_key_index: usize, ) -> TableCatalog { let schema = me.schema(); @@ -171,7 +171,7 @@ pub fn infer_left_internal_table_catalog( internal_table_catalog_builder.build(dist_keys, read_prefix_len_hint) } -pub fn infer_right_internal_table_catalog(input: &impl stream::StreamPlanRef) -> TableCatalog { +pub fn infer_right_internal_table_catalog(input: impl stream::StreamPlanRef) -> TableCatalog { let schema = input.schema(); // We require that the right table has distribution `Single` diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index 87c03cc14c8c9..2536cee984558 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -70,10 +70,10 @@ impl Join { } } -impl Join { +impl Join { /// Return stream hash join internal table catalog and degree table catalog. pub fn infer_internal_and_degree_table_catalog( - input: &PlanRef, + input: I, join_key_indices: Vec, dk_indices_in_jk: Vec, ) -> (TableCatalog, TableCatalog, Vec) { diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index aec59c90bcc4e..fe85e0919804e 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -84,6 +84,7 @@ macro_rules! impl_distill_unit_from_fields { } pub(super) use impl_distill_unit_from_fields; +#[auto_impl::auto_impl(&)] pub trait GenericPlanRef: Eq + Hash { fn id(&self) -> PlanNodeId; fn schema(&self) -> &Schema; @@ -92,6 +93,7 @@ pub trait GenericPlanRef: Eq + Hash { fn ctx(&self) -> OptimizerContextRef; } +#[auto_impl::auto_impl(&)] pub trait PhysicalPlanRef: GenericPlanRef { fn distribution(&self) -> &Distribution; } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 0ad9b828ead4b..0aed32abec40e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -22,7 +22,7 @@ use risingwave_expr::aggregate::{agg_kinds, AggKind}; use super::generic::{self, Agg, GenericPlanRef, PlanAggCall, ProjectBuilder}; use super::utils::impl_distill_by_unit; use super::{ - BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, PlanBase, PlanRef, + BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHashAgg, StreamProject, StreamSimpleAgg, StreamStatelessSimpleAgg, ToBatch, ToStream, }; @@ -48,7 +48,7 @@ use crate::utils::{ /// The output schema will first include the group key and then the aggregation calls. #[derive(Clone, Debug, PartialEq, Eq, Hash)] pub struct LogicalAgg { - pub base: PlanBase, + pub base: PlanBase, core: Agg, } @@ -182,6 +182,8 @@ impl LogicalAgg { /// Generates distributed stream plan. fn gen_dist_stream_agg_plan(&self, stream_input: PlanRef) -> Result { + use super::stream::prelude::*; + let input_dist = stream_input.distribution(); debug_assert!(*input_dist != Distribution::Broadcast); @@ -1137,6 +1139,8 @@ fn new_stream_hash_agg(core: Agg, vnode_col_idx: Option) -> Stre impl ToStream for LogicalAgg { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + use super::stream::prelude::*; + for agg_call in self.agg_calls() { if matches!(agg_call.agg_kind, agg_kinds::unimplemented_in_stream!()) { return Err(ErrorCode::NotImplemented( diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index b398ce7494f61..fa5576c61710b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -11,8 +11,8 @@ // 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::{Pretty, XmlNode}; use risingwave_common::catalog::Schema; use risingwave_common::error::{ErrorCode, Result, RwError}; @@ -23,7 +23,7 @@ use super::generic::{ }; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, LogicalJoin, LogicalProject, PlanBase, PlanRef, PlanTreeNodeBinary, + ColPrunable, Logical, LogicalJoin, LogicalProject, PlanBase, PlanRef, PlanTreeNodeBinary, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{CorrelatedId, Expr, ExprImpl, ExprRewriter, InputRef}; @@ -38,7 +38,7 @@ use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; /// left side. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalApply { - pub base: PlanBase, + pub base: PlanBase, left: PlanRef, right: PlanRef, on: Condition, diff --git a/src/frontend/src/optimizer/plan_node/logical_dedup.rs b/src/frontend/src/optimizer/plan_node/logical_dedup.rs index dd46f9af9be1d..7be8b64ae01af 100644 --- a/src/frontend/src/optimizer/plan_node/logical_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/logical_dedup.rs @@ -21,9 +21,9 @@ use super::generic::TopNLimit; use super::utils::impl_distill_by_unit; use super::{ gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, ColumnPruningContext, - ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, - PredicatePushdownContext, RewriteStreamContext, StreamDedup, StreamGroupTopN, ToBatch, - ToStream, ToStreamContext, + ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, + PredicatePushdown, PredicatePushdownContext, RewriteStreamContext, StreamDedup, + StreamGroupTopN, ToBatch, ToStream, ToStreamContext, }; use crate::optimizer::property::{Order, RequiredDist}; use crate::utils::Condition; @@ -32,7 +32,7 @@ use crate::utils::Condition; /// an `ORDER BY`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalDedup { - pub base: PlanBase, + pub base: PlanBase, core: generic::Dedup, } @@ -99,6 +99,8 @@ impl ToStream for LogicalDedup { } fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + use super::stream::prelude::*; + let input = self.input().to_stream(ctx)?; let input = RequiredDist::hash_shard(self.dedup_cols()) .enforce_if_not_satisfies(input, &Order::any())?; diff --git a/src/frontend/src/optimizer/plan_node/logical_delete.rs b/src/frontend/src/optimizer/plan_node/logical_delete.rs index 2cda2b782af7e..63f8c81b5991d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_delete.rs +++ b/src/frontend/src/optimizer/plan_node/logical_delete.rs @@ -17,8 +17,8 @@ use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchDelete, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchDelete, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::optimizer::plan_node::{ @@ -31,7 +31,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// It corresponds to the `DELETE` statements in SQL. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalDelete { - pub base: PlanBase, + pub base: PlanBase, core: generic::Delete, } diff --git a/src/frontend/src/optimizer/plan_node/logical_except.rs b/src/frontend/src/optimizer/plan_node/logical_except.rs index 11ff8b0210ee6..353cfe6583d5d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_except.rs +++ b/src/frontend/src/optimizer/plan_node/logical_except.rs @@ -17,7 +17,9 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; -use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; +use super::{ + ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, PlanTreeNode, PredicatePushdownContext, RewriteStreamContext, @@ -29,7 +31,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// matching rows from its other inputs. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalExcept { - pub base: PlanBase, + pub base: PlanBase, core: generic::Except, } diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index d1f3b666feef5..31209122e16a3 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -18,8 +18,8 @@ use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, ExprRewritable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchExpand, ColPrunable, ExprRewritable, Logical, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamExpand, ToBatch, ToStream, }; use crate::optimizer::plan_node::{ ColumnPruningContext, LogicalProject, PredicatePushdownContext, RewriteStreamContext, @@ -37,7 +37,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// is used to distinguish between different `subset`s in `column_subsets`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/logical_filter.rs b/src/frontend/src/optimizer/plan_node/logical_filter.rs index a62b91aac5277..a9ebd26d38219 100644 --- a/src/frontend/src/optimizer/plan_node/logical_filter.rs +++ b/src/frontend/src/optimizer/plan_node/logical_filter.rs @@ -21,8 +21,8 @@ use risingwave_common::types::DataType; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ - generic, ColPrunable, ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, - PredicatePushdown, ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{assert_input_ref, ExprImpl, ExprRewriter, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ @@ -37,7 +37,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If the condition allows nulls, then a null value is treated the same as false. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs index da2ec2138c3d1..a592337f7e26d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_hop_window.rs @@ -20,8 +20,9 @@ use risingwave_common::types::Interval; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, LogicalFilter, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchHopWindow, ColPrunable, ExprRewritable, Logical, + LogicalFilter, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHopWindow, + ToBatch, ToStream, }; use crate::expr::{ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ @@ -32,7 +33,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition}; /// `LogicalHopWindow` implements Hop Table Function. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, } diff --git a/src/frontend/src/optimizer/plan_node/logical_insert.rs b/src/frontend/src/optimizer/plan_node/logical_insert.rs index e93b77d79c1f2..f801affcc93f9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_insert.rs +++ b/src/frontend/src/optimizer/plan_node/logical_insert.rs @@ -19,8 +19,8 @@ use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ - gen_filter_and_pushdown, generic, BatchInsert, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchInsert, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::expr::{ExprImpl, ExprRewriter}; @@ -35,7 +35,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// statements, the input relation would be [`super::LogicalValues`]. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalInsert { - pub base: PlanBase, + pub base: PlanBase, core: generic::Insert, } diff --git a/src/frontend/src/optimizer/plan_node/logical_intersect.rs b/src/frontend/src/optimizer/plan_node/logical_intersect.rs index 2da89e01d8319..6e20cf87c6927 100644 --- a/src/frontend/src/optimizer/plan_node/logical_intersect.rs +++ b/src/frontend/src/optimizer/plan_node/logical_intersect.rs @@ -17,7 +17,9 @@ use risingwave_common::catalog::Schema; use risingwave_common::error::Result; use super::utils::impl_distill_by_unit; -use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; +use super::{ + ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, PlanTreeNode, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, @@ -28,7 +30,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If `all` is false, it needs to eliminate duplicates. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalIntersect { - pub base: PlanBase, + pub base: PlanBase, core: generic::Intersect, } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index a586af2f0bf42..a928481230d3c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -26,8 +26,8 @@ use super::generic::{ }; use super::utils::{childless_record, Distill}; use super::{ - generic, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, PredicatePushdown, - StreamHashJoin, StreamProject, ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeBinary, + PredicatePushdown, StreamHashJoin, StreamProject, ToBatch, ToStream, }; use crate::expr::{CollectInputRef, Expr, ExprImpl, ExprRewriter, ExprType, InputRef}; use crate::optimizer::plan_node::generic::DynamicFilter; @@ -49,7 +49,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, Condit /// right columns, dependent on the output indices provided. A repeat output index is illegal. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, } @@ -866,6 +866,8 @@ impl LogicalJoin { predicate: EqJoinPredicate, ctx: &mut ToStreamContext, ) -> Result { + use super::stream::prelude::*; + assert!(predicate.has_eq()); let mut right = self.right().to_stream_with_dist_required( &RequiredDist::shard_by_key(self.right().schema().len(), &predicate.right_eq_indexes()), @@ -1009,6 +1011,8 @@ impl LogicalJoin { predicate: EqJoinPredicate, ctx: &mut ToStreamContext, ) -> Result { + use super::stream::prelude::*; + assert!(predicate.has_eq()); let right = self.right(); @@ -1179,6 +1183,8 @@ impl LogicalJoin { predicate: Condition, ctx: &mut ToStreamContext, ) -> Result> { + use super::stream::prelude::*; + // If there is exactly one predicate, it is a comparison (<, <=, >, >=), and the // join is a `Inner` or `LeftSemi` join, we can convert the scalar subquery into a // `StreamDynamicFilter` diff --git a/src/frontend/src/optimizer/plan_node/logical_limit.rs b/src/frontend/src/optimizer/plan_node/logical_limit.rs index ea53c4a2d546c..f6678faf396a1 100644 --- a/src/frontend/src/optimizer/plan_node/logical_limit.rs +++ b/src/frontend/src/optimizer/plan_node/logical_limit.rs @@ -17,8 +17,8 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchLimit, ColPrunable, ExprRewritable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchLimit, ColPrunable, ExprRewritable, Logical, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::optimizer::plan_node::{ ColumnPruningContext, LogicalTopN, PredicatePushdownContext, RewriteStreamContext, @@ -30,7 +30,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalLimit` fetches up to `limit` rows from `offset` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalLimit { - pub base: PlanBase, + pub base: PlanBase, pub(super) core: generic::Limit, } diff --git a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs index 9b740abd7718e..819f84e963cfd 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -23,7 +23,7 @@ use risingwave_pb::plan_common::JoinType; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, ExprRewritable, LogicalFilter, LogicalJoin, LogicalProject, PlanBase, + ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalJoin, LogicalProject, PlanBase, PlanNodeType, PlanRef, PlanTreeNodeBinary, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; @@ -46,7 +46,7 @@ use crate::utils::{ /// expressed as 2-way `LogicalJoin`s. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalMultiJoin { - pub base: PlanBase, + pub base: PlanBase, inputs: Vec, on: Condition, output_indices: Vec, diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index 1d720db15b71a..c13a0c93a6e70 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -21,7 +21,7 @@ use risingwave_common::types::DataType; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, ColumnPruningContext, ExprRewritable, LogicalFilter, PlanBase, PlanRef, + ColPrunable, ColumnPruningContext, ExprRewritable, Logical, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, RewriteStreamContext, StreamNow, ToBatch, ToStream, ToStreamContext, }; use crate::optimizer::plan_node::utils::column_names_pretty; @@ -31,7 +31,7 @@ use crate::OptimizerContextRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalNow { - pub base: PlanBase, + pub base: PlanBase, } impl LogicalNow { diff --git a/src/frontend/src/optimizer/plan_node/logical_over_window.rs b/src/frontend/src/optimizer/plan_node/logical_over_window.rs index b1796ddc62752..a78a145ab1997 100644 --- a/src/frontend/src/optimizer/plan_node/logical_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/logical_over_window.rs @@ -23,7 +23,7 @@ use risingwave_expr::window_function::{Frame, FrameBound, WindowFuncKind}; use super::generic::{GenericPlanRef, OverWindow, PlanWindowFunction, ProjectBuilder}; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, BatchOverWindow, ColPrunable, ExprRewritable, LogicalProject, + gen_filter_and_pushdown, BatchOverWindow, ColPrunable, ExprRewritable, Logical, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamEowcOverWindow, StreamEowcSort, StreamOverWindow, ToBatch, ToStream, }; @@ -358,7 +358,7 @@ impl<'a> ExprVisitor for OverWindowProjectBuilder<'a> { /// The output schema is the input schema plus the window functions. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: OverWindow, } @@ -772,6 +772,8 @@ impl ToBatch for LogicalOverWindow { impl ToStream for LogicalOverWindow { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { + use super::stream::prelude::*; + if self .window_functions() .iter() diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index a96de7d91ecd5..59a5509ebcd70 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -19,8 +19,8 @@ use risingwave_common::error::Result; use super::utils::{childless_record, Distill}; use super::{ - gen_filter_and_pushdown, generic, BatchProject, ColPrunable, ExprRewritable, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamProject, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchProject, ColPrunable, ExprRewritable, Logical, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProject, ToBatch, ToStream, }; use crate::expr::{collect_input_refs, ExprImpl, ExprRewriter, InputRef}; use crate::optimizer::plan_node::generic::GenericPlanRef; @@ -33,7 +33,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition, Substi /// `LogicalProject` computes a set of expressions from its input relation. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, } diff --git a/src/frontend/src/optimizer/plan_node/logical_project_set.rs b/src/frontend/src/optimizer/plan_node/logical_project_set.rs index 4bf6b18cdabe3..fc894713c2f43 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -19,8 +19,9 @@ use risingwave_common::types::DataType; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchProjectSet, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchProjectSet, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamProjectSet, + ToBatch, ToStream, }; use crate::expr::{ collect_input_refs, Expr, ExprImpl, ExprRewriter, FunctionCall, InputRef, TableFunction, @@ -41,7 +42,7 @@ use crate::utils::{ColIndexMapping, Condition, Substitute}; /// column. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, } diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 07d2a6c7653e7..200302ae70f97 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -25,7 +25,7 @@ use risingwave_common::util::sort_util::ColumnOrder; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::{childless_record, Distill}; use super::{ - generic, BatchFilter, BatchProject, ColPrunable, ExprRewritable, PlanBase, PlanRef, + generic, BatchFilter, BatchProject, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, StreamTableScan, ToBatch, ToStream, }; use crate::catalog::{ColumnId, IndexCatalog}; @@ -42,7 +42,7 @@ use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; /// `LogicalScan` returns contents of a table or other equivalent object #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, } diff --git a/src/frontend/src/optimizer/plan_node/logical_share.rs b/src/frontend/src/optimizer/plan_node/logical_share.rs index d6b5711740a98..3e7193342fb67 100644 --- a/src/frontend/src/optimizer/plan_node/logical_share.rs +++ b/src/frontend/src/optimizer/plan_node/logical_share.rs @@ -20,8 +20,8 @@ use risingwave_common::error::Result; use super::utils::{childless_record, Distill}; use super::{ - generic, ColPrunable, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, - ToBatch, ToStream, + generic, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PlanTreeNodeUnary, + PredicatePushdown, ToBatch, ToStream, }; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ @@ -49,7 +49,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// ``` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalShare { - pub base: PlanBase, + pub base: PlanBase, core: generic::Share, } @@ -68,7 +68,7 @@ impl LogicalShare { LogicalShare::new(input).into() } - pub(super) fn pretty_fields<'a>(base: &PlanBase, name: &'a str) -> XmlNode<'a> { + pub(super) fn pretty_fields(base: impl GenericPlanRef, name: &str) -> XmlNode<'_> { childless_record(name, vec![("id", Pretty::debug(&base.id().0))]) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 45a5fbcb2240f..01166e74f1359 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -32,9 +32,9 @@ use super::generic::GenericPlanRef; use super::stream_watermark_filter::StreamWatermarkFilter; use super::utils::{childless_record, Distill}; use super::{ - generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, - PlanBase, PlanRef, PredicatePushdown, StreamProject, StreamRowIdGen, StreamSource, ToBatch, - ToStream, + generic, BatchProject, BatchSource, ColPrunable, ExprRewritable, Logical, LogicalFilter, + LogicalProject, PlanBase, PlanRef, PredicatePushdown, StreamProject, StreamRowIdGen, + StreamSource, ToBatch, ToStream, }; use crate::catalog::source_catalog::SourceCatalog; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, InputRef}; @@ -51,7 +51,7 @@ use crate::utils::{ColIndexMapping, Condition, IndexRewriter}; /// `LogicalSource` returns contents of a table or other equivalent object #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalSource { - pub base: PlanBase, + pub base: PlanBase, pub core: generic::Source, /// Expressions to output. This field presents and will be turned to a `Project` when diff --git a/src/frontend/src/optimizer/plan_node/logical_table_function.rs b/src/frontend/src/optimizer/plan_node/logical_table_function.rs index 15d510cc1c6fd..4553722dca328 100644 --- a/src/frontend/src/optimizer/plan_node/logical_table_function.rs +++ b/src/frontend/src/optimizer/plan_node/logical_table_function.rs @@ -19,7 +19,7 @@ use risingwave_common::types::DataType; use super::utils::{childless_record, Distill}; use super::{ - ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, PlanBase, PlanRef, + ColPrunable, ExprRewritable, Logical, LogicalFilter, LogicalProject, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, }; use crate::expr::{Expr, ExprRewriter, TableFunction}; @@ -35,7 +35,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// If the function returns a struct, it will be flattened into multiple columns. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalTableFunction { - pub base: PlanBase, + pub base: PlanBase, pub table_function: TableFunction, pub with_ordinality: bool, } diff --git a/src/frontend/src/optimizer/plan_node/logical_topn.rs b/src/frontend/src/optimizer/plan_node/logical_topn.rs index 39d97a56fe3a6..940714d7d4abb 100644 --- a/src/frontend/src/optimizer/plan_node/logical_topn.rs +++ b/src/frontend/src/optimizer/plan_node/logical_topn.rs @@ -20,9 +20,9 @@ use risingwave_common::util::sort_util::ColumnOrder; use super::generic::TopNLimit; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, ExprRewritable, PlanBase, - PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, ToBatch, - ToStream, + gen_filter_and_pushdown, generic, BatchGroupTopN, ColPrunable, ExprRewritable, Logical, + PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamGroupTopN, StreamProject, + ToBatch, ToStream, }; use crate::expr::{ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::{ @@ -36,7 +36,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, Condition}; /// `LogicalTopN` sorts the input data and fetches up to `limit` rows from `offset` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } @@ -107,6 +107,8 @@ impl LogicalTopN { } fn gen_dist_stream_top_n_plan(&self, stream_input: PlanRef) -> Result { + use super::stream::prelude::*; + let input_dist = stream_input.distribution().clone(); // if it is append only, for now we don't generate 2-phase rules diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index 1f02b026c0020..e108707e0b13a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -21,7 +21,9 @@ use risingwave_common::error::Result; use risingwave_common::types::{DataType, Scalar}; use super::utils::impl_distill_by_unit; -use super::{ColPrunable, ExprRewritable, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream}; +use super::{ + ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, ToStream, +}; use crate::expr::{ExprImpl, InputRef, Literal}; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::stream_union::StreamUnion; @@ -37,7 +39,7 @@ use crate::Explain; /// If `all` is false, it needs to eliminate duplicates. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/logical_update.rs b/src/frontend/src/optimizer/plan_node/logical_update.rs index 1dbe1d3d3c5c9..34c0939f13bc9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_update.rs +++ b/src/frontend/src/optimizer/plan_node/logical_update.rs @@ -18,8 +18,8 @@ use risingwave_common::error::Result; use super::generic::GenericPlanRef; use super::utils::impl_distill_by_unit; use super::{ - gen_filter_and_pushdown, generic, BatchUpdate, ColPrunable, ExprRewritable, LogicalProject, - PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, + gen_filter_and_pushdown, generic, BatchUpdate, ColPrunable, ExprRewritable, Logical, + LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, PredicatePushdown, ToBatch, ToStream, }; use crate::catalog::TableId; use crate::expr::{ExprImpl, ExprRewriter}; @@ -34,7 +34,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// It corresponds to the `UPDATE` statements in SQL. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalUpdate { - pub base: PlanBase, + pub base: PlanBase, core: generic::Update, } diff --git a/src/frontend/src/optimizer/plan_node/logical_values.rs b/src/frontend/src/optimizer/plan_node/logical_values.rs index e62c6400f2015..38867b3d9c223 100644 --- a/src/frontend/src/optimizer/plan_node/logical_values.rs +++ b/src/frontend/src/optimizer/plan_node/logical_values.rs @@ -24,8 +24,8 @@ use risingwave_common::types::{DataType, ScalarImpl}; use super::generic::GenericPlanRef; use super::utils::{childless_record, Distill}; use super::{ - BatchValues, ColPrunable, ExprRewritable, LogicalFilter, PlanBase, PlanRef, PredicatePushdown, - StreamValues, ToBatch, ToStream, + BatchValues, ColPrunable, ExprRewritable, Logical, LogicalFilter, PlanBase, PlanRef, + PredicatePushdown, StreamValues, ToBatch, ToStream, }; use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; use crate::optimizer::optimizer_context::OptimizerContextRef; @@ -38,7 +38,7 @@ use crate::utils::{ColIndexMapping, Condition}; /// `LogicalValues` builds rows according to a list of expressions #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct LogicalValues { - pub base: PlanBase, + pub base: PlanBase, rows: Arc<[Vec]>, } diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index f16ebfb0c792c..ec7777e42e737 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -51,11 +51,95 @@ use self::stream::StreamPlanRef; use self::utils::Distill; use super::property::{Distribution, FunctionalDependencySet, Order}; +/// A marker trait for different conventions, used for enforcing type safety. +/// +/// Implementors are [`Logical`], [`Batch`], and [`Stream`]. +pub trait ConventionMarker: 'static + Sized { + /// The extra fields in the [`PlanBase`] of this convention. + type Extra: 'static + Eq + Hash + Clone + Debug; + + /// Get the [`Convention`] enum value. + fn value() -> Convention; +} + +/// The marker for logical convention. +pub struct Logical; +impl ConventionMarker for Logical { + type Extra = plan_base::NoExtra; + + fn value() -> Convention { + Convention::Logical + } +} + +/// The marker for batch convention. +pub struct Batch; +impl ConventionMarker for Batch { + type Extra = plan_base::BatchExtra; + + fn value() -> Convention { + Convention::Batch + } +} + +/// The marker for stream convention. +pub struct Stream; +impl ConventionMarker for Stream { + type Extra = plan_base::StreamExtra; + + fn value() -> Convention { + Convention::Stream + } +} + +/// The trait for accessing the meta data and [`PlanBase`] for plan nodes. pub trait PlanNodeMeta { - fn node_type(&self) -> PlanNodeType; - fn plan_base(&self) -> &PlanBase; - fn convention(&self) -> Convention; + type Convention: ConventionMarker; + + const NODE_TYPE: PlanNodeType; + + /// Get the reference to the [`PlanBase`] with corresponding convention. + fn plan_base(&self) -> &PlanBase; + + /// Get the reference to the [`PlanBase`] with erased convention. + /// + /// This is mainly used for implementing [`AnyPlanNodeMeta`]. Callers should prefer + /// [`PlanNodeMeta::plan_base`] instead as it is more type-safe. + fn plan_base_ref(&self) -> PlanBaseRef<'_>; +} + +// Intentionally made private. +mod plan_node_meta { + use super::*; + + /// The object-safe version of [`PlanNodeMeta`], used as a super trait of [`PlanNode`]. + /// + /// Check [`PlanNodeMeta`] for more details. + pub trait AnyPlanNodeMeta { + fn node_type(&self) -> PlanNodeType; + fn plan_base(&self) -> PlanBaseRef<'_>; + fn convention(&self) -> Convention; + } + + /// Implement [`AnyPlanNodeMeta`] for all [`PlanNodeMeta`]. + impl

AnyPlanNodeMeta for P + where + P: PlanNodeMeta, + { + fn node_type(&self) -> PlanNodeType { + P::NODE_TYPE + } + + fn plan_base(&self) -> PlanBaseRef<'_> { + PlanNodeMeta::plan_base_ref(self) + } + + fn convention(&self) -> Convention { + P::Convention::value() + } + } } +use plan_node_meta::AnyPlanNodeMeta; /// The common trait over all plan nodes. Used by optimizer framework which will treat all node as /// `dyn PlanNode` @@ -77,7 +161,7 @@ pub trait PlanNode: + ToPb + ToLocalBatch + PredicatePushdown - + PlanNodeMeta + + AnyPlanNodeMeta { } @@ -194,7 +278,7 @@ pub trait VisitPlan: Visit { } } -#[derive(Debug, PartialEq)] +#[derive(Clone, Copy, Debug, PartialEq)] pub enum Convention { Logical, Batch, @@ -419,12 +503,13 @@ impl PlanTreeNode for PlanRef { } } -impl PlanNodeMeta for PlanRef { +/// Implement again for the `dyn` newtype wrapper. +impl AnyPlanNodeMeta for PlanRef { fn node_type(&self) -> PlanNodeType { self.0.node_type() } - fn plan_base(&self) -> &PlanBase { + fn plan_base(&self) -> PlanBaseRef<'_> { self.0.plan_base() } @@ -433,11 +518,9 @@ impl PlanNodeMeta for PlanRef { } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -impl

GenericPlanRef for P -where - P: PlanNodeMeta + Eq + Hash, -{ +/// Allow access to all fields defined in [`GenericPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. +impl GenericPlanRef for PlanRef { fn id(&self) -> PlanNodeId { self.plan_base().id() } @@ -459,23 +542,17 @@ where } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -// TODO: further constrain the convention to be `Stream` or `Batch`. -impl

PhysicalPlanRef for P -where - P: PlanNodeMeta + Eq + Hash, -{ +/// Allow access to all fields defined in [`PhysicalPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. +impl PhysicalPlanRef for PlanRef { fn distribution(&self) -> &Distribution { self.plan_base().distribution() } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -// TODO: further constrain the convention to be `Stream`. -impl

StreamPlanRef for P -where - P: PlanNodeMeta + Eq + Hash, -{ +/// Allow access to all fields defined in [`StreamPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. +impl StreamPlanRef for PlanRef { fn append_only(&self) -> bool { self.plan_base().append_only() } @@ -489,12 +566,9 @@ where } } -/// Implement for every type that provides [`PlanBase`] through [`PlanNodeMeta`]. -// TODO: further constrain the convention to be `Batch`. -impl

BatchPlanRef for P -where - P: PlanNodeMeta + Eq + Hash, -{ +/// Allow access to all fields defined in [`BatchPlanRef`] for the type-erased plan node. +// TODO: may also implement on `dyn PlanNode` directly. +impl BatchPlanRef for PlanRef { fn order(&self) -> &Order { self.plan_base().order() } @@ -553,6 +627,8 @@ pub(crate) fn pretty_config() -> PrettyConfig { } } +/// Directly implement methods for [`PlanNode`] to access the fields defined in [`GenericPlanRef`]. +// TODO: always require `GenericPlanRef` to make it more consistent. impl dyn PlanNode { pub fn id(&self) -> PlanNodeId { self.plan_base().id() @@ -570,36 +646,19 @@ impl dyn PlanNode { self.plan_base().stream_key() } - pub fn order(&self) -> &Order { - self.plan_base().order() - } - - // TODO: avoid no manual delegation - pub fn distribution(&self) -> &Distribution { - self.plan_base().distribution() - } - - pub fn append_only(&self) -> bool { - self.plan_base().append_only() - } - - pub fn emit_on_window_close(&self) -> bool { - self.plan_base().emit_on_window_close() - } - pub fn functional_dependency(&self) -> &FunctionalDependencySet { self.plan_base().functional_dependency() } +} - pub fn watermark_columns(&self) -> &FixedBitSet { - self.plan_base().watermark_columns() - } - +impl dyn PlanNode { /// Serialize the plan node and its children to a stream plan proto. /// /// Note that [`StreamTableScan`] has its own implementation of `to_stream_prost`. We have a /// hook inside to do some ad-hoc thing for [`StreamTableScan`]. pub fn to_stream_prost(&self, state: &mut BuildFragmentGraphState) -> StreamPlanPb { + use stream::prelude::*; + if let Some(stream_table_scan) = self.as_stream_table_scan() { return stream_table_scan.adhoc_to_stream_prost(state); } @@ -626,7 +685,7 @@ impl dyn PlanNode { .map(|x| *x as u32) .collect(), fields: self.schema().to_prost(), - append_only: self.append_only(), + append_only: self.plan_base().append_only(), } } @@ -1079,14 +1138,23 @@ macro_rules! impl_plan_node_meta { } $(impl PlanNodeMeta for [<$convention $name>] { - fn node_type(&self) -> PlanNodeType{ - PlanNodeType::[<$convention $name>] - } - fn plan_base(&self) -> &PlanBase { + type Convention = $convention; + const NODE_TYPE: PlanNodeType = PlanNodeType::[<$convention $name>]; + + fn plan_base(&self) -> &PlanBase<$convention> { &self.base } - fn convention(&self) -> Convention { - Convention::$convention + + fn plan_base_ref(&self) -> PlanBaseRef<'_> { + PlanBaseRef::$convention(&self.base) + } + } + + impl Deref for [<$convention $name>] { + type Target = PlanBase<$convention>; + + fn deref(&self) -> &Self::Target { + &self.base } })* } diff --git a/src/frontend/src/optimizer/plan_node/plan_base.rs b/src/frontend/src/optimizer/plan_node/plan_base.rs index 51b1aa5f41141..0d2e649379112 100644 --- a/src/frontend/src/optimizer/plan_node/plan_base.rs +++ b/src/frontend/src/optimizer/plan_node/plan_base.rs @@ -21,17 +21,36 @@ use super::*; use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::property::{Distribution, FunctionalDependencySet, Order}; -/// Common extra fields for physical plan nodes. +/// No extra fields for logical plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] -struct PhysicalCommonExtra { - /// The distribution property of the PlanNode's output, store an `Distribution::any()` here - /// will not affect correctness, but insert unnecessary exchange in plan - dist: Distribution, +pub struct NoExtra; + +// Make them public types in a private module to allow using them as public trait bounds, +// while still keeping them private to the super module. +mod physical_common { + use super::*; + + /// Common extra fields for physical plan nodes. + #[derive(Clone, Debug, PartialEq, Eq, Hash)] + pub struct PhysicalCommonExtra { + /// The distribution property of the PlanNode's output, store an `Distribution::any()` here + /// will not affect correctness, but insert unnecessary exchange in plan + pub dist: Distribution, + } + + /// A helper trait to reuse code for accessing the common physical fields of batch and stream + /// plan bases. + pub trait GetPhysicalCommon { + fn physical(&self) -> &PhysicalCommonExtra; + fn physical_mut(&mut self) -> &mut PhysicalCommonExtra; + } } +use physical_common::*; + /// Extra fields for stream plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] -struct StreamExtra { +pub struct StreamExtra { /// Common fields for physical plan nodes. physical: PhysicalCommonExtra, @@ -45,9 +64,19 @@ struct StreamExtra { watermark_columns: FixedBitSet, } +impl GetPhysicalCommon for StreamExtra { + fn physical(&self) -> &PhysicalCommonExtra { + &self.physical + } + + fn physical_mut(&mut self) -> &mut PhysicalCommonExtra { + &mut self.physical + } +} + /// Extra fields for batch plan nodes. #[derive(Clone, Debug, PartialEq, Eq, Hash)] -struct BatchExtra { +pub struct BatchExtra { /// Common fields for physical plan nodes. physical: PhysicalCommonExtra, @@ -56,55 +85,31 @@ struct BatchExtra { order: Order, } -/// Extra fields for physical plan nodes. -#[derive(Clone, Debug, PartialEq, Eq, Hash)] -enum PhysicalExtra { - Stream(StreamExtra), - Batch(BatchExtra), -} - -impl PhysicalExtra { - fn common(&self) -> &PhysicalCommonExtra { - match self { - PhysicalExtra::Stream(stream) => &stream.physical, - PhysicalExtra::Batch(batch) => &batch.physical, - } - } - - fn common_mut(&mut self) -> &mut PhysicalCommonExtra { - match self { - PhysicalExtra::Stream(stream) => &mut stream.physical, - PhysicalExtra::Batch(batch) => &mut batch.physical, - } - } - - fn stream(&self) -> &StreamExtra { - match self { - PhysicalExtra::Stream(extra) => extra, - _ => panic!("access stream properties from batch plan node"), - } +impl GetPhysicalCommon for BatchExtra { + fn physical(&self) -> &PhysicalCommonExtra { + &self.physical } - fn batch(&self) -> &BatchExtra { - match self { - PhysicalExtra::Batch(extra) => extra, - _ => panic!("access batch properties from stream plan node"), - } + fn physical_mut(&mut self) -> &mut PhysicalCommonExtra { + &mut self.physical } } -/// the common fields of all nodes, please make a field named `base` in -/// every planNode and correctly value it when construct the planNode. +/// The common fields of all plan nodes with different conventions. +/// +/// Please make a field named `base` in every planNode and correctly value +/// it when construct the planNode. /// /// All fields are intentionally made private and immutable, as they should /// normally be the same as the given [`GenericPlanNode`] when constructing. /// /// - To access them, use traits including [`GenericPlanRef`], -/// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`]. +/// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`] with +/// compile-time checks. /// - To mutate them, use methods like `new_*` or `clone_with_*`. -#[derive(Clone, Debug, Educe)] -#[educe(PartialEq, Eq, Hash)] -pub struct PlanBase { +#[derive(Educe)] +#[educe(PartialEq, Eq, Hash, Clone, Debug)] +pub struct PlanBase { // -- common fields -- #[educe(PartialEq(ignore), Hash(ignore))] id: PlanNodeId, @@ -113,29 +118,17 @@ pub struct PlanBase { schema: Schema, /// the pk indices of the PlanNode's output, a empty stream key vec means there is no stream key - // TODO: this is actually a logical and stream only property + // TODO: this is actually a logical and stream only property. + // - For logical nodes, this is `None` in most time expect for the phase after `logical_rewrite_for_stream`. + // - For stream nodes, this is always `Some`. stream_key: Option>, functional_dependency: FunctionalDependencySet, - /// Extra fields if the plan node is physical. - physical_extra: Option, + /// Extra fields for different conventions. + extra: C::Extra, } -impl PlanBase { - fn physical_extra(&self) -> &PhysicalExtra { - self.physical_extra - .as_ref() - .expect("access physical properties from logical plan node") - } - - fn physical_extra_mut(&mut self) -> &mut PhysicalExtra { - self.physical_extra - .as_mut() - .expect("access physical properties from logical plan node") - } -} - -impl generic::GenericPlanRef for PlanBase { +impl generic::GenericPlanRef for PlanBase { fn id(&self) -> PlanNodeId { self.id } @@ -157,33 +150,44 @@ impl generic::GenericPlanRef for PlanBase { } } -impl generic::PhysicalPlanRef for PlanBase { +impl generic::PhysicalPlanRef for PlanBase +where + C::Extra: GetPhysicalCommon, +{ fn distribution(&self) -> &Distribution { - &self.physical_extra().common().dist + &self.extra.physical().dist } } -impl stream::StreamPlanRef for PlanBase { +impl stream::StreamPlanRef for PlanBase { fn append_only(&self) -> bool { - self.physical_extra().stream().append_only + self.extra.append_only } fn emit_on_window_close(&self) -> bool { - self.physical_extra().stream().emit_on_window_close + self.extra.emit_on_window_close } fn watermark_columns(&self) -> &FixedBitSet { - &self.physical_extra().stream().watermark_columns + &self.extra.watermark_columns } } -impl batch::BatchPlanRef for PlanBase { +impl batch::BatchPlanRef for PlanBase { fn order(&self) -> &Order { - &self.physical_extra().batch().order + &self.extra.order + } +} + +impl PlanBase { + pub fn clone_with_new_plan_id(&self) -> Self { + let mut new = self.clone(); + new.id = self.ctx().next_plan_node_id(); + new } } -impl PlanBase { +impl PlanBase { pub fn new_logical( ctx: OptimizerContextRef, schema: Schema, @@ -197,7 +201,7 @@ impl PlanBase { schema, stream_key, functional_dependency, - physical_extra: None, + extra: NoExtra, } } @@ -209,7 +213,9 @@ impl PlanBase { core.functional_dependency(), ) } +} +impl PlanBase { pub fn new_stream( ctx: OptimizerContextRef, schema: Schema, @@ -228,14 +234,12 @@ impl PlanBase { schema, stream_key, functional_dependency, - physical_extra: Some(PhysicalExtra::Stream({ - StreamExtra { - physical: PhysicalCommonExtra { dist }, - append_only, - emit_on_window_close, - watermark_columns, - } - })), + extra: StreamExtra { + physical: PhysicalCommonExtra { dist }, + append_only, + emit_on_window_close, + watermark_columns, + }, } } @@ -257,7 +261,9 @@ impl PlanBase { watermark_columns, ) } +} +impl PlanBase { pub fn new_batch( ctx: OptimizerContextRef, schema: Schema, @@ -272,12 +278,10 @@ impl PlanBase { schema, stream_key: None, functional_dependency, - physical_extra: Some(PhysicalExtra::Batch({ - BatchExtra { - physical: PhysicalCommonExtra { dist }, - order, - } - })), + extra: BatchExtra { + physical: PhysicalCommonExtra { dist }, + order, + }, } } @@ -288,27 +292,149 @@ impl PlanBase { ) -> Self { Self::new_batch(core.ctx(), core.schema(), dist, order) } +} - pub fn clone_with_new_plan_id(&self) -> Self { - let mut new = self.clone(); - new.id = self.ctx().next_plan_node_id(); - new - } - +impl PlanBase +where + C::Extra: GetPhysicalCommon, +{ /// Clone the plan node with a new distribution. /// /// Panics if the plan node is not physical. pub fn clone_with_new_distribution(&self, dist: Distribution) -> Self { let mut new = self.clone(); - new.physical_extra_mut().common_mut().dist = dist; + new.extra.physical_mut().dist = dist; new } } // Mutators for testing only. #[cfg(test)] -impl PlanBase { +impl PlanBase { pub fn functional_dependency_mut(&mut self) -> &mut FunctionalDependencySet { &mut self.functional_dependency } } + +/// Reference to [`PlanBase`] with erased conventions. +/// +/// Used for accessing fields on a type-erased plan node. All traits of [`GenericPlanRef`], +/// [`PhysicalPlanRef`], [`StreamPlanRef`] and [`BatchPlanRef`] are implemented for this type, +/// so runtime checks are required when calling methods on it. +#[derive(PartialEq, Eq, Hash, Clone, Copy, Debug, enum_as_inner::EnumAsInner)] +pub enum PlanBaseRef<'a> { + Logical(&'a PlanBase), + Stream(&'a PlanBase), + Batch(&'a PlanBase), +} + +impl PlanBaseRef<'_> { + /// Get the convention of this plan base. + pub fn convention(self) -> Convention { + match self { + PlanBaseRef::Logical(_) => Convention::Logical, + PlanBaseRef::Stream(_) => Convention::Stream, + PlanBaseRef::Batch(_) => Convention::Batch, + } + } +} + +/// Dispatch a method call to the corresponding plan base type. +macro_rules! dispatch_plan_base { + ($self:ident, [$($convention:ident),+ $(,)?], $method:expr) => { + match $self { + $( + PlanBaseRef::$convention(plan) => $method(plan), + )+ + + #[allow(unreachable_patterns)] + _ => unreachable!("calling `{}` on a plan node of `{:?}`", stringify!($method), $self.convention()), + } + } +} + +/// Workaround for getters returning references. +/// +/// For example, callers writing `GenericPlanRef::schema(&foo.plan_base())` will lead to a +/// borrow checker error, as it borrows [`PlanBaseRef`] again, which is already a reference. +/// +/// As a workaround, we directly let the getters below take the ownership of [`PlanBaseRef`], +/// which is `Copy`. When callers write `foo.plan_base().schema()`, the compiler will prefer +/// these ones over the ones defined in traits like [`GenericPlanRef`]. +impl<'a> PlanBaseRef<'a> { + pub(super) fn schema(self) -> &'a Schema { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::schema) + } + + pub(super) fn stream_key(self) -> Option<&'a [usize]> { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::stream_key) + } + + pub(super) fn functional_dependency(self) -> &'a FunctionalDependencySet { + dispatch_plan_base!( + self, + [Logical, Stream, Batch], + GenericPlanRef::functional_dependency + ) + } + + pub(super) fn distribution(self) -> &'a Distribution { + dispatch_plan_base!(self, [Stream, Batch], PhysicalPlanRef::distribution) + } + + pub(super) fn watermark_columns(self) -> &'a FixedBitSet { + dispatch_plan_base!(self, [Stream], StreamPlanRef::watermark_columns) + } + + pub(super) fn order(self) -> &'a Order { + dispatch_plan_base!(self, [Batch], BatchPlanRef::order) + } +} + +impl GenericPlanRef for PlanBaseRef<'_> { + fn id(&self) -> PlanNodeId { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::id) + } + + fn schema(&self) -> &Schema { + (*self).schema() + } + + fn stream_key(&self) -> Option<&[usize]> { + (*self).stream_key() + } + + fn functional_dependency(&self) -> &FunctionalDependencySet { + (*self).functional_dependency() + } + + fn ctx(&self) -> OptimizerContextRef { + dispatch_plan_base!(self, [Logical, Stream, Batch], GenericPlanRef::ctx) + } +} + +impl PhysicalPlanRef for PlanBaseRef<'_> { + fn distribution(&self) -> &Distribution { + (*self).distribution() + } +} + +impl StreamPlanRef for PlanBaseRef<'_> { + fn append_only(&self) -> bool { + dispatch_plan_base!(self, [Stream], StreamPlanRef::append_only) + } + + fn emit_on_window_close(&self) -> bool { + dispatch_plan_base!(self, [Stream], StreamPlanRef::emit_on_window_close) + } + + fn watermark_columns(&self) -> &FixedBitSet { + (*self).watermark_columns() + } +} + +impl BatchPlanRef for PlanBaseRef<'_> { + fn order(&self) -> &Order { + (*self).order() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 866c62c2413a5..394a64b656ad3 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -24,8 +24,16 @@ use super::generic::PhysicalPlanRef; /// [`PhysicalPlanRef`]. /// /// [`GenericPlanRef`]: super::generic::GenericPlanRef +#[auto_impl::auto_impl(&)] pub trait StreamPlanRef: PhysicalPlanRef { fn append_only(&self) -> bool; fn emit_on_window_close(&self) -> bool; fn watermark_columns(&self) -> &FixedBitSet; } + +/// Prelude for stream plan nodes. +pub mod prelude { + pub use super::super::generic::{GenericPlanRef, PhysicalPlanRef}; + pub use super::super::Stream; + pub use super::StreamPlanRef; +} diff --git a/src/frontend/src/optimizer/plan_node/stream_dedup.rs b/src/frontend/src/optimizer/plan_node/stream_dedup.rs index 51b5e589e886e..af1e0cec77e41 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dedup.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dedup.rs @@ -17,17 +17,17 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::DedupNode; -use super::generic::{self, GenericPlanNode, GenericPlanRef, PhysicalPlanRef}; -use super::stream::StreamPlanRef; +use super::generic::GenericPlanNode; +use super::stream::prelude::*; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; -use super::{ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; +use super::{generic, ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::plan_node::PlanRef; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDedup { - pub base: PlanBase, + pub base: PlanBase, core: generic::Dedup, } diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs index bb18f9cffdf0f..49a2b97a97454 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -21,6 +21,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ArrangementInfo, DeltaIndexJoinNode}; use super::generic::{self, GenericPlanRef}; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode}; use crate::expr::{Expr, ExprRewriter}; @@ -34,7 +35,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// inputs to be indexes. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDeltaJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs index 9b000974786e4..07154463392f2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dml.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dml.rs @@ -17,6 +17,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{ColumnDesc, INITIAL_TABLE_VERSION_ID}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -24,7 +25,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDml { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, column_descs: Vec, } diff --git a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs index a4b74f37208e7..0ac631413ca3e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -18,6 +18,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::DynamicFilterNode; use super::generic::{DynamicFilter, GenericPlanRef}; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, column_names_pretty, watermark_pretty, Distill}; use super::{generic, ExprRewritable}; @@ -28,7 +29,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamDynamicFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::DynamicFilter, cleaned_by_watermark: bool, } diff --git a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs index d8c5a9635ce59..065fcfebf74b7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_eowc_over_window.rs @@ -19,6 +19,7 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, GenericPlanRef, PlanWindowFunction}; +use super::stream::prelude::*; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -26,7 +27,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamEowcOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index 99e6c3c5161a1..d9d28fce298ad 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -16,8 +16,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DispatchStrategy, DispatcherType, ExchangeNode}; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; -use super::stream::StreamPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, plan_node_name, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Distribution, DistributionDisplay}; @@ -27,7 +26,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// without changing its content. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamExchange { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, no_shuffle: bool, } diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index 5959b8d6be4d2..1085d38c06a18 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -17,6 +17,7 @@ use risingwave_pb::stream_plan::expand_node::Subset; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ExpandNode; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::Distribution; @@ -24,7 +25,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamExpand { - pub base: PlanBase, + pub base: PlanBase, core: generic::Expand, } diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 0f000e6b8c0db..8bba484a90e29 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -15,6 +15,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::FilterNode; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; @@ -25,7 +26,7 @@ use crate::utils::Condition; /// `StreamFilter` implements [`super::LogicalFilter`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamFilter { - pub base: PlanBase, + pub base: PlanBase, core: generic::Filter, } diff --git a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs index 95fd72e9f6aa0..2fc1aba8127fd 100644 --- a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs +++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs @@ -20,9 +20,9 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{PbStreamFsFetch, StreamFsFetchNode}; +use super::stream::prelude::*; use super::{PlanBase, PlanRef, PlanTreeNodeUnary}; use crate::catalog::source_catalog::SourceCatalog; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::utils::{childless_record, Distill}; use crate::optimizer::plan_node::{generic, ExprRewritable, StreamNode}; use crate::optimizer::property::Distribution; @@ -30,7 +30,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamFsFetch { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs index 3e8f3c00206c4..d0c3077f83286 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -17,6 +17,7 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{DistillUnit, GenericPlanRef, TopNLimit}; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{plan_node_name, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanTreeNodeUnary, StreamNode}; @@ -27,7 +28,7 @@ use crate::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamGroupTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, /// an optional column index which is the vnode of each row computed by the input's consistent /// hash distribution diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 55ab6b5906e59..6ef8849b1e142 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -19,6 +19,7 @@ use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, GenericPlanRef, PlanAggCall}; +use super::stream::prelude::*; use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; @@ -28,7 +29,7 @@ use crate::utils::{ColIndexMapping, ColIndexMappingRewriteExt, IndexSet}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHashAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, /// An optional column index which is the vnode of each row computed by the input's consistent diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index 9d9c41425c4b1..36aff15d96055 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -21,6 +21,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DeltaExpression, HashJoinNode, PbInequalityPair}; use super::generic::{GenericPlanRef, Join}; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, plan_node_name, watermark_pretty, Distill}; use super::{ @@ -38,7 +39,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// get output rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHashJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, /// The join condition must be equivalent to `logical.on`, but separated into equal and diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index e177be6942360..e2a55ff121949 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -18,6 +18,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::HopWindowNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -28,7 +29,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// [`StreamHopWindow`] represents a hop window table function. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamHopWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::HopWindow, window_start_exprs: Vec, window_end_exprs: Vec, diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index d5435e9beb397..fd34c47f7c19a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -24,6 +24,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::derive::derive_columns; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, Distill}; use super::{reorganize_elements_id, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -38,7 +39,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// Materializes a stream. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamMaterialize { - pub base: PlanBase, + pub base: PlanBase, /// Child of Materialize plan input: PlanRef, table: TableCatalog, diff --git a/src/frontend/src/optimizer/plan_node/stream_now.rs b/src/frontend/src/optimizer/plan_node/stream_now.rs index 91ebc344fa51d..36f9c3811516c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_now.rs +++ b/src/frontend/src/optimizer/plan_node/stream_now.rs @@ -19,7 +19,7 @@ use risingwave_common::types::DataType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::NowNode; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; +use super::stream::prelude::*; use super::utils::{childless_record, Distill, TableCatalogBuilder}; use super::{ExprRewritable, LogicalNow, PlanBase, StreamNode}; use crate::optimizer::plan_node::utils::column_names_pretty; @@ -29,7 +29,7 @@ use crate::OptimizerContextRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamNow { - pub base: PlanBase, + pub base: PlanBase, } impl StreamNow { diff --git a/src/frontend/src/optimizer/plan_node/stream_over_window.rs b/src/frontend/src/optimizer/plan_node/stream_over_window.rs index 5a2f9d98f1340..89469f63e5740 100644 --- a/src/frontend/src/optimizer/plan_node/stream_over_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_over_window.rs @@ -19,6 +19,7 @@ use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{GenericPlanNode, PlanWindowFunction}; +use super::stream::prelude::*; use super::utils::{impl_distill_by_unit, TableCatalogBuilder}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::plan_node::generic::GenericPlanRef; @@ -27,7 +28,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamOverWindow { - pub base: PlanBase, + pub base: PlanBase, core: generic::OverWindow, } diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index c0ff0d1cf2f43..da68260e8930b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -18,6 +18,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ProjectNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -29,7 +30,7 @@ use crate::utils::ColIndexMappingRewriteExt; /// rows. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamProject { - pub base: PlanBase, + pub base: PlanBase, core: generic::Project, /// All the watermark derivations, (input_column_index, output_column_index). And the /// derivation expression is the project's expression itself. diff --git a/src/frontend/src/optimizer/plan_node/stream_project_set.rs b/src/frontend/src/optimizer/plan_node/stream_project_set.rs index ba09d79c96c60..5fc922b215975 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -17,6 +17,7 @@ use itertools::Itertools; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::ProjectSetNode; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{try_derive_watermark, ExprRewriter, WatermarkDerivation}; @@ -25,7 +26,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamProjectSet { - pub base: PlanBase, + pub base: PlanBase, core: generic::ProjectSet, /// All the watermark derivations, (input_column_idx, expr_idx). And the /// derivation expression is the project_set's expression itself. diff --git a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs index 083cb877cd4d6..203bf8951493f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs @@ -15,6 +15,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::Distribution; @@ -22,7 +23,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamRowIdGen { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, row_id_index: usize, } diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index 3acf0b132805e..15690c047aa60 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -17,7 +17,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::PbStreamNode; use super::generic::GenericPlanRef; -use super::stream::StreamPlanRef; +use super::stream::prelude::*; use super::utils::Distill; use super::{generic, ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamExchange, StreamNode}; use crate::optimizer::plan_node::{LogicalShare, PlanBase, PlanTreeNode}; @@ -27,22 +27,25 @@ use crate::Explain; /// `StreamShare` will be translated into an `ExchangeNode` based on its distribution finally. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamShare { - pub base: PlanBase, + pub base: PlanBase, core: generic::Share, } impl StreamShare { pub fn new(core: generic::Share) -> Self { - let input = core.input.borrow().0.clone(); - let dist = input.distribution().clone(); - // Filter executor won't change the append-only behavior of the stream. - let base = PlanBase::new_stream_with_core( - &core, - dist, - input.append_only(), - input.emit_on_window_close(), - input.watermark_columns().clone(), - ); + let base = { + let input = core.input.borrow(); + let dist = input.distribution().clone(); + // Filter executor won't change the append-only behavior of the stream. + PlanBase::new_stream_with_core( + &core, + dist, + input.append_only(), + input.emit_on_window_close(), + input.watermark_columns().clone(), + ) + }; + StreamShare { base, core } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs index 92d96fdf21b08..06aa227d7e316 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -18,17 +18,16 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, PlanAggCall}; +use super::stream::prelude::*; use super::utils::{childless_record, plan_node_name, Distill}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; -use crate::optimizer::plan_node::generic::PhysicalPlanRef; -use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, /// The index of `count(*)` in `agg_calls`. diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 32e9fb487910c..cd48fda60e06b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -38,6 +38,7 @@ use tracing::info; use super::derive::{derive_columns, derive_pk}; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill, IndicesDisplay, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, StreamNode}; use crate::optimizer::plan_node::PlanTreeNodeUnary; @@ -50,7 +51,7 @@ const DOWNSTREAM_PK_KEY: &str = "primary_key"; /// [`StreamSink`] represents a table/connector sink at the very end of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSink { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, sink_desc: SinkDesc, } @@ -58,7 +59,11 @@ pub struct StreamSink { impl StreamSink { #[must_use] pub fn new(input: PlanRef, sink_desc: SinkDesc) -> Self { - let base = input.plan_base().clone_with_new_plan_id(); + let base = input + .plan_base() + .into_stream() + .expect("input should be stream plan") + .clone_with_new_plan_id(); Self { base, input, diff --git a/src/frontend/src/optimizer/plan_node/stream_sort.rs b/src/frontend/src/optimizer/plan_node/stream_sort.rs index 41a56a0fd5df2..2468fe52858e4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sort.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sort.rs @@ -20,8 +20,7 @@ use risingwave_common::catalog::FieldDisplay; use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use super::generic::{GenericPlanRef, PhysicalPlanRef}; -use super::stream::StreamPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -29,7 +28,7 @@ use crate::TableCatalog; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamEowcSort { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, sort_column_index: usize, diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index ae66cf568118b..51a764b53fab2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -20,10 +20,10 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::{PbStreamSource, SourceNode}; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, StreamNode}; use crate::catalog::source_catalog::SourceCatalog; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -31,7 +31,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// [`StreamSource`] represents a table/connector source at the very beginning of the graph. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSource { - pub base: PlanBase, + pub base: PlanBase, pub(crate) core: generic::Source, } diff --git a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs index 474582ec877c7..ca076132f3a55 100644 --- a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs @@ -17,6 +17,7 @@ use itertools::Itertools; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{self, PlanAggCall}; +use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; @@ -32,7 +33,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// by `StreamSimpleAgg`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamStatelessSimpleAgg { - pub base: PlanBase, + pub base: PlanBase, core: generic::Agg, } diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 965ca217a3369..882410e5c9010 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -24,13 +24,11 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::{ChainType, PbStreamNode}; -use super::generic::PhysicalPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanNodeId, PlanRef, StreamNode}; use crate::catalog::ColumnId; use crate::expr::{ExprRewriter, FunctionCall}; -use crate::optimizer::plan_node::generic::GenericPlanRef; -use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::plan_node::utils::{IndicesDisplay, TableCatalogBuilder}; use crate::optimizer::property::{Distribution, DistributionDisplay}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -41,7 +39,7 @@ use crate::{Explain, TableCatalog}; /// creation request. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTableScan { - pub base: PlanBase, + pub base: PlanBase, core: generic::Scan, batch_plan_id: PlanNodeId, chain_type: ChainType, diff --git a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs index 675dbeb9ab381..cb524f35bd329 100644 --- a/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_temporal_join.rs @@ -19,6 +19,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::TemporalJoinNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode}; @@ -33,7 +34,7 @@ use crate::utils::ColIndexMappingRewriteExt; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTemporalJoin { - pub base: PlanBase, + pub base: PlanBase, core: generic::Join, eq_join_predicate: EqJoinPredicate, } diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index 87890625f6be7..1f3618369d15e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -17,6 +17,7 @@ use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::generic::{DistillUnit, TopNLimit}; +use super::stream::prelude::*; use super::utils::{plan_node_name, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Distribution, Order}; @@ -25,7 +26,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamTopN` implements [`super::LogicalTopN`] to find the top N elements with a heap #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamTopN { - pub base: PlanBase, + pub base: PlanBase, core: generic::TopN, } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 6d6dca2d8dd02..425cdc6914564 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -20,6 +20,7 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::UnionNode; use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanRef}; @@ -30,7 +31,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamUnion` implements [`super::LogicalUnion`] #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamUnion { - pub base: PlanBase, + pub base: PlanBase, core: generic::Union, } diff --git a/src/frontend/src/optimizer/plan_node/stream_values.rs b/src/frontend/src/optimizer/plan_node/stream_values.rs index f8cc5db851159..3ae19a3112c5e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_values.rs +++ b/src/frontend/src/optimizer/plan_node/stream_values.rs @@ -18,7 +18,7 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::values_node::ExprTuple; use risingwave_pb::stream_plan::ValuesNode; -use super::generic::GenericPlanRef; +use super::stream::prelude::*; use super::utils::{childless_record, Distill}; use super::{ExprRewritable, LogicalValues, PlanBase, StreamNode}; use crate::expr::{Expr, ExprImpl}; @@ -28,7 +28,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamValues` implements `LogicalValues.to_stream()` #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamValues { - pub base: PlanBase, + pub base: PlanBase, logical: LogicalValues, } diff --git a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs index 066bc9a234ca5..45997be797c0f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_watermark_filter.rs @@ -21,6 +21,7 @@ use risingwave_common::util::sort_util::OrderType; use risingwave_pb::catalog::WatermarkDesc; use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use super::stream::prelude::*; use super::stream::StreamPlanRef; use super::utils::{childless_record, watermark_pretty, Distill, TableCatalogBuilder}; use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; @@ -30,7 +31,7 @@ use crate::{TableCatalog, WithOptions}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamWatermarkFilter { - pub base: PlanBase, + pub base: PlanBase, input: PlanRef, watermark_descs: Vec, } @@ -54,7 +55,11 @@ impl StreamWatermarkFilter { Self::with_base(base, input, watermark_descs) } - fn with_base(base: PlanBase, input: PlanRef, watermark_descs: Vec) -> Self { + fn with_base( + base: PlanBase, + input: PlanRef, + watermark_descs: Vec, + ) -> Self { Self { base, input, diff --git a/src/frontend/src/optimizer/property/distribution.rs b/src/frontend/src/optimizer/property/distribution.rs index 2df1d7ae00bc3..8ab78c0cd4279 100644 --- a/src/frontend/src/optimizer/property/distribution.rs +++ b/src/frontend/src/optimizer/property/distribution.rs @@ -47,6 +47,7 @@ use std::fmt; use std::fmt::Debug; use fixedbitset::FixedBitSet; +use generic::PhysicalPlanRef; use itertools::Itertools; use risingwave_common::catalog::{FieldDisplay, Schema, TableId}; use risingwave_common::error::Result; diff --git a/src/frontend/src/optimizer/property/order.rs b/src/frontend/src/optimizer/property/order.rs index 19ad7586e1c11..bf373e8629d7b 100644 --- a/src/frontend/src/optimizer/property/order.rs +++ b/src/frontend/src/optimizer/property/order.rs @@ -85,6 +85,8 @@ const ANY_ORDER: Order = Order { impl Order { pub fn enforce_if_not_satisfies(&self, plan: PlanRef) -> Result { + use crate::optimizer::plan_node::batch::prelude::*; + if !plan.order().satisfies(self) { Ok(self.enforce(plan)) } else { diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index cb20103b3e76f..cc23256f43687 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -42,7 +42,7 @@ use uuid::Uuid; use super::SchedulerError; use crate::catalog::catalog_service::CatalogReader; use crate::catalog::TableId; -use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; use crate::optimizer::plan_node::{PlanNodeId, PlanNodeType}; use crate::optimizer::property::Distribution; use crate::optimizer::PlanRef; From e74b32dab8229bd61af22a6250c32588510dea44 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Thu, 26 Oct 2023 16:11:58 +0800 Subject: [PATCH 08/52] feat: Add pulsar iceberg table reader (#12735) --- Cargo.lock | 3 +- Cargo.toml | 2 +- src/connector/src/aws_utils.rs | 10 +- src/connector/src/error.rs | 3 + src/connector/src/source/pulsar/mod.rs | 11 +- .../src/source/pulsar/source/reader.rs | 372 +++++++++++++++++- src/connector/src/source/pulsar/topic.rs | 9 + 7 files changed, 400 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 1c741e3a5ae67..2f2ee2991b10f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -3852,7 +3852,7 @@ dependencies = [ [[package]] name = "icelake" version = "0.0.10" -source = "git+https://github.com/icelake-io/icelake?rev=16dab0e36ab337e58ee8002d828def2d212fa116#16dab0e36ab337e58ee8002d828def2d212fa116" +source = "git+https://github.com/icelake-io/icelake?rev=186fde7663545d1d6a5856ce9fbbc541224eadfb#186fde7663545d1d6a5856ce9fbbc541224eadfb" dependencies = [ "anyhow", "apache-avro 0.15.0 (registry+https://github.com/rust-lang/crates.io-index)", @@ -3869,6 +3869,7 @@ dependencies = [ "bytes", "chrono", "csv", + "derive_builder", "enum-display", "faster-hex", "futures", diff --git a/Cargo.toml b/Cargo.toml index ac533e733f7a8..f0fbb8485d318 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -112,7 +112,7 @@ criterion = { version = "0.5", features = ["async_futures"] } tonic = { package = "madsim-tonic", version = "0.4.0" } tonic-build = { package = "madsim-tonic-build", version = "0.4.2" } prost = { version = "0.12" } -icelake = { git = "https://github.com/icelake-io/icelake", rev = "16dab0e36ab337e58ee8002d828def2d212fa116" } +icelake = { git = "https://github.com/icelake-io/icelake", rev = "186fde7663545d1d6a5856ce9fbbc541224eadfb" } arrow-array = "48" arrow-cast = "48" arrow-schema = "48" diff --git a/src/connector/src/aws_utils.rs b/src/connector/src/aws_utils.rs index 785a4396bacba..c62d9b9d6d14d 100644 --- a/src/connector/src/aws_utils.rs +++ b/src/connector/src/aws_utils.rs @@ -24,12 +24,16 @@ use url::Url; use crate::aws_auth::AwsAuthProps; +pub const REGION: &str = "region"; +pub const ACCESS_KEY: &str = "access_key"; +pub const SECRET_ACCESS: &str = "secret_access"; + pub const AWS_DEFAULT_CONFIG: [&str; 7] = [ - "region", + REGION, "arn", "profile", - "access_key", - "secret_access", + ACCESS_KEY, + SECRET_ACCESS, "session_token", "endpoint_url", ]; diff --git a/src/connector/src/error.rs b/src/connector/src/error.rs index 73cd27d5801e5..2cdfdd99ecfe0 100644 --- a/src/connector/src/error.rs +++ b/src/connector/src/error.rs @@ -35,6 +35,9 @@ pub enum ConnectorError { #[error("MySQL error: {0}")] MySql(#[from] mysql_async::Error), + #[error("Pulsar error: {0}")] + Pulsar(anyhow::Error), + #[error(transparent)] Internal(#[from] anyhow::Error), } diff --git a/src/connector/src/source/pulsar/mod.rs b/src/connector/src/source/pulsar/mod.rs index 544d1b7fb3ed3..c9db87fd3bb94 100644 --- a/src/connector/src/source/pulsar/mod.rs +++ b/src/connector/src/source/pulsar/mod.rs @@ -19,10 +19,11 @@ pub mod topic; pub use enumerator::*; use serde::Deserialize; +use serde_with::serde_as; pub use split::*; +use self::source::reader::PulsarSplitReader; use crate::common::PulsarCommon; -use crate::source::pulsar::source::reader::PulsarSplitReader; use crate::source::SourceProperties; pub const PULSAR_CONNECTOR: &str = "pulsar"; @@ -36,6 +37,7 @@ impl SourceProperties for PulsarProperties { } #[derive(Clone, Debug, Deserialize)] +#[serde_as] pub struct PulsarProperties { #[serde(rename = "scan.startup.mode", alias = "pulsar.scan.startup.mode")] pub scan_startup_mode: Option, @@ -45,4 +47,11 @@ pub struct PulsarProperties { #[serde(flatten)] pub common: PulsarCommon, + + #[serde(rename = "iceberg.enabled")] + #[serde_as(as = "Option")] + pub iceberg_loader_enabled: bool, + + #[serde(rename = "iceberg.bucket", default)] + pub iceberg_bucket: Option, } diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 85d85a8d18714..0568a8935932e 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -12,26 +12,92 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::time::{SystemTime, UNIX_EPOCH}; use anyhow::{anyhow, ensure, Result}; +use arrow_array::{Int32Array, Int64Array, RecordBatch}; use async_trait::async_trait; use futures::StreamExt; use futures_async_stream::try_stream; +use icelake::catalog::{load_catalog, CATALOG_NAME, CATALOG_TYPE}; +use icelake::io::FileScanStream; +use icelake::types::{Any, AnyValue, StructValueBuilder}; +use icelake::{Table, TableIdentifier}; use itertools::Itertools; use pulsar::consumer::InitialPosition; use pulsar::message::proto::MessageIdData; use pulsar::{Consumer, ConsumerBuilder, ConsumerOptions, Pulsar, SubType, TokioExecutor}; +use risingwave_common::array::{DataChunk, StreamChunk}; +use risingwave_common::catalog::ROWID_PREFIX; +use risingwave_common::error::RwError; +use crate::aws_utils::{ACCESS_KEY, REGION, SECRET_ACCESS}; +use crate::error::ConnectorError; use crate::parser::ParserConfig; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties}; use crate::source::{ into_chunk_stream, BoxSourceWithStateStream, Column, CommonSplitReader, SourceContextRef, - SourceMessage, SplitId, SplitMetaData, SplitReader, + SourceMessage, SplitId, SplitMetaData, SplitReader, StreamChunkWithState, }; -pub struct PulsarSplitReader { +pub enum PulsarSplitReader { + Broker(PulsarBrokerReader), + Iceberg(PulsarIcebergReader), +} + +#[async_trait] +impl SplitReader for PulsarSplitReader { + type Properties = PulsarProperties; + type Split = PulsarSplit; + + async fn new( + props: PulsarProperties, + splits: Vec, + parser_config: ParserConfig, + source_ctx: SourceContextRef, + _columns: Option>, + ) -> Result { + ensure!(splits.len() == 1, "only support single split"); + let split = splits.into_iter().next().unwrap(); + let topic = split.topic.to_string(); + + tracing::debug!("creating consumer for pulsar split topic {}", topic,); + + if props.iceberg_loader_enabled + && matches!(split.start_offset, PulsarEnumeratorOffset::Earliest) + && !topic.starts_with("non-persistent://") + { + tracing::debug!("Creating iceberg reader for pulsar split topic {}", topic); + Ok(Self::Iceberg(PulsarIcebergReader::new( + props, + split, + source_ctx, + parser_config, + ))) + } else { + Ok(Self::Broker( + PulsarBrokerReader::new(props, vec![split], parser_config, source_ctx, None) + .await?, + )) + } + } + + fn into_stream(self) -> BoxSourceWithStateStream { + match self { + Self::Broker(reader) => { + let (parser_config, source_context) = + (reader.parser_config.clone(), reader.source_ctx.clone()); + Box::pin(into_chunk_stream(reader, parser_config, source_context)) + } + Self::Iceberg(reader) => Box::pin(reader.into_stream()), + } + } +} + +/// This reader reads from pulsar broker +pub struct PulsarBrokerReader { pulsar: Pulsar, consumer: Consumer, TokioExecutor>, split: PulsarSplit, @@ -84,7 +150,7 @@ fn parse_message_id(id: &str) -> Result { } #[async_trait] -impl SplitReader for PulsarSplitReader { +impl SplitReader for PulsarBrokerReader { type Properties = PulsarProperties; type Split = PulsarSplit; @@ -173,7 +239,7 @@ impl SplitReader for PulsarSplitReader { } } -impl CommonSplitReader for PulsarSplitReader { +impl CommonSplitReader for PulsarBrokerReader { #[try_stream(ok = Vec, error = anyhow::Error)] async fn into_data_stream(self) { let max_chunk_size = self.source_ctx.source_ctrl_opts.chunk_size; @@ -188,3 +254,301 @@ impl CommonSplitReader for PulsarSplitReader { } } } + +const META_COLUMN_TOPIC: &str = "__topic"; +const META_COLUMN_KEY: &str = "__key"; +const META_COLUMN_LEDGER_ID: &str = "__ledgerId"; +const META_COLUMN_ENTRY_ID: &str = "__entryId"; +const META_COLUMN_BATCH_INDEX: &str = "__batchIndex"; +const META_COLUMN_PARTITION: &str = "__partition"; + +/// Read history data from iceberg table +pub struct PulsarIcebergReader { + props: PulsarProperties, + split: PulsarSplit, + source_ctx: SourceContextRef, + parser_config: ParserConfig, +} + +impl PulsarIcebergReader { + fn new( + props: PulsarProperties, + split: PulsarSplit, + source_ctx: SourceContextRef, + parser_config: ParserConfig, + ) -> Self { + Self { + props, + split, + source_ctx, + parser_config, + } + } + + async fn scan(&self) -> Result { + let table = self.create_iceberg_table().await?; + let schema = table.current_table_metadata().current_schema()?; + tracing::debug!("Created iceberg pulsar table, schema is: {:?}", schema,); + + let max_chunk_size = self.source_ctx.source_ctrl_opts.chunk_size; + + let partition_value = match &self.split.topic.partition_index { + Some(partition_id) => { + let (partition_type, partition_field) = match table.current_partition_type()? { + Any::Struct(s) => { + let field = s + .fields() + .iter() + .find(|f| f.name == META_COLUMN_PARTITION) + .ok_or_else(|| { + ConnectorError::Pulsar(anyhow!( + "Partition field not found in partition spec" + )) + })?; + (s.clone(), field.clone()) + } + _ => { + return Err(ConnectorError::Pulsar(anyhow!( + "Partition type is not struct in iceberg table: {}", + table.table_name() + )))?; + } + }; + + let mut partition_value_builder = StructValueBuilder::new(partition_type); + partition_value_builder.add_field( + partition_field.id, + Some(AnyValue::Primitive(icelake::types::PrimitiveValue::Int( + *partition_id, + ))), + )?; + Some(partition_value_builder.build()?) + } + None => None, + }; + + // TODO: Add partition + Ok(table + .new_scan_builder() + .with_partition_value(partition_value) + .with_batch_size(max_chunk_size) + .build()? + .scan(&table) + .await?) + } + + async fn create_iceberg_table(&self) -> Result { + let catalog = load_catalog(&self.build_iceberg_configs()?) + .await + .map_err(|e| ConnectorError::Pulsar(anyhow!("Unable to load iceberg catalog: {e}")))?; + + let table_id = + TableIdentifier::new(vec![self.split.topic.topic_str_without_partition()?]) + .map_err(|e| ConnectorError::Pulsar(anyhow!("Unable to parse table name: {e}")))?; + + let table = catalog + .load_table(&table_id) + .await + .map_err(|err| ConnectorError::Pulsar(anyhow!(err)))?; + + Ok(table) + } + + #[try_stream(ok = StreamChunkWithState, error = anyhow::Error)] + async fn as_stream_chunk_stream(&self) { + #[for_await] + for file_scan in self.scan().await? { + let file_scan = file_scan?; + + #[for_await] + for record_batch in file_scan.scan().await? { + let batch = record_batch?; + let msgs = self.convert_record_batch_to_source_with_state(&batch)?; + yield msgs; + } + } + } + + #[try_stream(ok = StreamChunkWithState, error = RwError)] + async fn into_stream(self) { + let (props, mut split, parser_config, source_ctx) = ( + self.props.clone(), + self.split.clone(), + self.parser_config.clone(), + self.source_ctx.clone(), + ); + tracing::info!("Starting to read pulsar message from iceberg"); + let mut last_msg_id = None; + + #[for_await] + for msg in self.as_stream_chunk_stream() { + let msg = + msg.inspect_err(|e| tracing::error!("Failed to read message from iceberg: {}", e))?; + last_msg_id = msg + .split_offset_mapping + .as_ref() + .and_then(|m| m.get(self.split.topic.to_string().as_str())) + .cloned(); + } + + tracing::info!("Finished reading pulsar message from iceberg"); + // We finished reading all the data from iceberg table, now we need to start from broker. + if let Some(msg_id) = last_msg_id { + tracing::info!("Last iceberg message id is {}", msg_id); + split.start_offset = PulsarEnumeratorOffset::MessageId(msg_id); + } + + tracing::info!( + "Switching from pulsar iceberg reader to broker reader with offset: {:?}", + split.start_offset + ); + let broker_reader = PulsarSplitReader::Broker( + PulsarBrokerReader::new(props, vec![split], parser_config, source_ctx, None).await?, + ); + + #[for_await] + for msg in broker_reader.into_stream() { + yield msg?; + } + } + + fn build_iceberg_configs(&self) -> Result> { + let mut iceberg_configs = HashMap::new(); + + let bucket = + self.props.iceberg_bucket.as_ref().ok_or_else(|| { + ConnectorError::Pulsar(anyhow!("Iceberg bucket is not configured")) + })?; + + iceberg_configs.insert(CATALOG_TYPE.to_string(), "storage".to_string()); + iceberg_configs.insert(CATALOG_NAME.to_string(), "pulsar".to_string()); + iceberg_configs.insert( + "iceberg.catalog.pulsar.warehouse".to_string(), + format!( + "s3://{}/{}/{}", + bucket, self.split.topic.tenant, self.split.topic.namespace, + ), + ); + + if let Some(s3_configs) = self.props.common.oauth.as_ref().map(|s| &s.s3_credentials) { + if let Some(region) = s3_configs.get(REGION) { + iceberg_configs.insert("iceberg.table.io.region".to_string(), region.to_string()); + } + + if let Some(access_key) = s3_configs.get(ACCESS_KEY) { + iceberg_configs.insert( + "iceberg.table.io.access_key_id".to_string(), + access_key.to_string(), + ); + } + + if let Some(secret_key) = s3_configs.get(SECRET_ACCESS) { + iceberg_configs.insert( + "iceberg.table.io.secret_access_key".to_string(), + secret_key.to_string(), + ); + } + } + + iceberg_configs.insert("iceberg.table.io.bucket".to_string(), bucket.to_string()); + iceberg_configs.insert( + "iceberg.table.io.root".to_string(), + format!( + "/{}/{}", + self.split.topic.tenant, self.split.topic.namespace + ), + ); + // #TODO + // Support load config file + iceberg_configs.insert( + "iceberg.table.io.disable_config_load".to_string(), + "true".to_string(), + ); + + Ok(iceberg_configs) + } + + // Converts arrow record batch to stream chunk. + fn convert_record_batch_to_source_with_state( + &self, + record_batch: &RecordBatch, + ) -> Result { + let mut offsets = Vec::with_capacity(record_batch.num_rows()); + + let ledger_id_array = record_batch + .column_by_name(META_COLUMN_LEDGER_ID) + .ok_or_else(|| ConnectorError::Pulsar(anyhow!("Ledger id not found in iceberg table")))? + .as_any() + .downcast_ref::() + .ok_or_else(|| { + ConnectorError::Pulsar(anyhow!("Ledger id is not i64 in iceberg table")) + })?; + + let entry_id_array = record_batch + .column_by_name(META_COLUMN_ENTRY_ID) + .ok_or_else(|| ConnectorError::Pulsar(anyhow!("Entry id not found in iceberg table")))? + .as_any() + .downcast_ref::() + .ok_or_else(|| { + ConnectorError::Pulsar(anyhow!("Entry id is not i64 in iceberg table")) + })?; + + let partition_array = record_batch + .column_by_name(META_COLUMN_PARTITION) + .map(|arr| { + arr.as_any().downcast_ref::().ok_or_else(|| { + ConnectorError::Pulsar(anyhow!("Partition is not i32 in iceberg table")) + }) + }) + .transpose()?; + + let batch_index_array = record_batch + .column_by_name(META_COLUMN_BATCH_INDEX) + .map(|arr| { + arr.as_any().downcast_ref::().ok_or_else(|| { + ConnectorError::Pulsar(anyhow!("Batch index is not i64 in iceberg table")) + }) + }) + .transpose()?; + + let field_indices = self + .parser_config + .common + .rw_columns + .iter() + .filter(|col| col.name != ROWID_PREFIX) + .map(|col| { + record_batch + .schema() + .index_of(col.name.as_str()) + .map_err(|e| anyhow!(e)) + }) + .collect::>>()?; + + for row in 0..record_batch.num_rows() { + let offset = format!( + "{}:{}:{}:{}", + ledger_id_array.value(row), + entry_id_array.value(row), + partition_array.map(|arr| arr.value(row)).unwrap_or(-1), + batch_index_array.map(|arr| arr.value(row)).unwrap_or(-1) + ); + + offsets.push(offset); + } + + let data_chunk = DataChunk::try_from(&record_batch.project(&field_indices)?)?; + + let stream_chunk = StreamChunk::from(data_chunk); + + let state = Some(HashMap::from([( + self.split.topic.to_string().into(), + offsets.last().unwrap().clone(), + )])); + + Ok(StreamChunkWithState { + chunk: stream_chunk, + split_offset_mapping: state, + }) + } +} diff --git a/src/connector/src/source/pulsar/topic.rs b/src/connector/src/source/pulsar/topic.rs index ab4b1f84750f3..e6fbc6dbe15de 100644 --- a/src/connector/src/source/pulsar/topic.rs +++ b/src/connector/src/source/pulsar/topic.rs @@ -73,6 +73,15 @@ impl Topic { partition_index: Some(partition), }) } + + pub fn topic_str_without_partition(&self) -> Result { + if self.topic.contains(PARTITIONED_TOPIC_SUFFIX) { + let parts: Vec<&str> = self.topic.split(PARTITIONED_TOPIC_SUFFIX).collect(); + Ok(parts[0].to_string()) + } else { + Ok(self.topic.clone()) + } + } } /// `get_partition_index` returns the partition index of the topic. From 41d2bbd3988b33350240a41742f82392e538d9ff Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 26 Oct 2023 16:27:21 +0800 Subject: [PATCH 09/52] refactor(sqlparser): rename `SourceSchemaV2`/`SinkSchema` to `ConnectorSchema` (#13065) --- .../src/handler/alter_table_column.rs | 10 +- src/frontend/src/handler/create_sink.rs | 8 +- src/frontend/src/handler/create_source.rs | 15 ++- src/frontend/src/handler/create_table.rs | 10 +- src/sqlparser/src/ast/statement.rs | 94 ++++--------------- src/sqlparser/tests/testdata/create.yaml | 6 +- 6 files changed, 44 insertions(+), 99 deletions(-) diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index 1a6d02b963e9d..18313e0458a04 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -22,7 +22,7 @@ use risingwave_pb::catalog::Table; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_sqlparser::ast::{ - AlterTableOperation, ColumnOption, Encode, ObjectName, SourceSchemaV2, Statement, + AlterTableOperation, ColumnOption, ConnectorSchema, Encode, ObjectName, Statement, }; use risingwave_sqlparser::parser::Parser; @@ -32,7 +32,7 @@ use super::{HandlerArgs, RwPgResponse}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::table_catalog::TableType; use crate::handler::create_table::gen_create_table_plan_with_source; -use crate::{build_graph, Binder, OptimizerContext, TableCatalog}; +use crate::{build_graph, Binder, OptimizerContext, TableCatalog, WithOptions}; /// Handle `ALTER TABLE [ADD|DROP] COLUMN` statements. The `operation` must be either `AddColumn` or /// `DropColumn`. @@ -262,12 +262,12 @@ pub async fn handle_alter_table_column( Ok(PgResponse::empty_result(StatementType::ALTER_TABLE)) } -fn schema_has_schema_registry(schema: &SourceSchemaV2) -> bool { +fn schema_has_schema_registry(schema: &ConnectorSchema) -> bool { match schema.row_encode { Encode::Avro | Encode::Protobuf => true, Encode::Json => { - let mut options = schema.gen_options().unwrap(); - matches!(get_json_schema_location(&mut options), Ok(Some(_))) + let mut options = WithOptions::try_from(schema.row_options()).unwrap(); + matches!(get_json_schema_location(options.inner_mut()), Ok(Some(_))) } _ => false, } diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index ddb1d697b856d..2ab987308a4f5 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -27,8 +27,8 @@ use risingwave_connector::sink::{ }; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ - CreateSink, CreateSinkStatement, EmitMode, Encode, Format, ObjectName, Query, Select, - SelectItem, SetExpr, SinkSchema, TableFactor, TableWithJoins, + ConnectorSchema, CreateSink, CreateSinkStatement, EmitMode, Encode, Format, ObjectName, Query, + Select, SelectItem, SetExpr, TableFactor, TableWithJoins, }; use super::create_mv::get_column_names; @@ -228,7 +228,7 @@ pub async fn handle_create_sink( /// Transforms the (format, encode, options) from sqlparser AST into an internal struct `SinkFormatDesc`. /// This is an analogy to (part of) [`crate::handler::create_source::try_bind_columns_from_source`] /// which transforms sqlparser AST `SourceSchemaV2` into `StreamSourceInfo`. -fn bind_sink_format_desc(value: SinkSchema) -> Result { +fn bind_sink_format_desc(value: ConnectorSchema) -> Result { use risingwave_connector::sink::catalog::{SinkEncode, SinkFormat}; use risingwave_sqlparser::ast::{Encode as E, Format as F}; @@ -288,7 +288,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock Result<()> { +pub fn validate_compatibility(connector: &str, format_desc: &ConnectorSchema) -> Result<()> { let compatible_formats = CONNECTORS_COMPATIBLE_FORMATS .get(connector) .ok_or_else(|| { diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 0c2398a608eb8..0ce3e32ed584e 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -15,7 +15,6 @@ use std::collections::{BTreeMap, HashMap}; use std::sync::LazyLock; -use anyhow::anyhow; use itertools::Itertools; use maplit::{convert_args, hashmap}; use pgwire::pg_response::{PgResponse, StatementType}; @@ -46,8 +45,8 @@ use risingwave_pb::catalog::{ }; use risingwave_pb::plan_common::{EncodeType, FormatType}; use risingwave_sqlparser::ast::{ - self, get_delimiter, AstString, AvroSchema, ColumnDef, ColumnOption, CreateSourceStatement, - DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceSchemaV2, SourceWatermark, + self, get_delimiter, AstString, AvroSchema, ColumnDef, ColumnOption, ConnectorSchema, + CreateSourceStatement, DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceWatermark, }; use super::RwPgResponse; @@ -278,7 +277,7 @@ fn get_name_strategy_or_default(name_strategy: Option) -> Result for more information. /// return `(columns, pk_names, source info)` pub(crate) async fn try_bind_columns_from_source( - source_schema: &SourceSchemaV2, + source_schema: &ConnectorSchema, sql_defined_pk_names: Vec, sql_defined_columns: &[ColumnDef], with_properties: &HashMap, @@ -290,7 +289,7 @@ pub(crate) async fn try_bind_columns_from_source( let sql_defined_pk = !sql_defined_pk_names.is_empty(); let sql_defined_schema = !sql_defined_columns.is_empty(); let is_kafka: bool = is_kafka_connector(with_properties); - let mut options = source_schema.gen_options().map_err(|e| anyhow!(e))?; + let mut options = WithOptions::try_from(source_schema.row_options())?.into_inner(); let get_key_message_name = |options: &mut BTreeMap| -> Option { consume_string_from_options(options, KEY_MESSAGE_NAME_KEY) @@ -904,7 +903,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock, ) -> Result<()> { let connector = get_connector(props) @@ -922,8 +921,8 @@ pub fn validate_compatibility( if connector != KAFKA_CONNECTOR { let res = match (&source_schema.format, &source_schema.row_encode) { (Format::Plain, Encode::Protobuf) | (Format::Plain, Encode::Avro) => { - let mut options = source_schema.gen_options().map_err(|e| anyhow!(e))?; - let (_, use_schema_registry) = get_schema_location(&mut options)?; + let mut options = WithOptions::try_from(source_schema.row_options())?; + let (_, use_schema_registry) = get_schema_location(options.inner_mut())?; use_schema_registry } (Format::Debezium, Encode::Avro) => true, diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index bb02797c21395..e412658cb712f 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -33,7 +33,7 @@ use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::{DefaultColumnDesc, GeneratedColumnDesc}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ - ColumnDef, ColumnOption, DataType as AstDataType, Format, ObjectName, SourceSchemaV2, + ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, Format, ObjectName, SourceWatermark, TableConstraint, }; @@ -436,7 +436,7 @@ pub(crate) async fn gen_create_table_plan_with_source( table_name: ObjectName, column_defs: Vec, constraints: Vec, - source_schema: SourceSchemaV2, + source_schema: ConnectorSchema, source_watermarks: Vec, mut col_id_gen: ColumnIdGenerator, append_only: bool, @@ -749,7 +749,7 @@ pub async fn handle_create_table( columns: Vec, constraints: Vec, if_not_exists: bool, - source_schema: Option, + source_schema: Option, source_watermarks: Vec, append_only: bool, notice: Option, @@ -826,8 +826,8 @@ pub async fn handle_create_table( pub fn check_create_table_with_source( with_options: &WithOptions, - source_schema: Option, -) -> Result> { + source_schema: Option, +) -> Result> { if with_options.inner().contains_key(UPSTREAM_SOURCE_KEY) { source_schema.as_ref().ok_or_else(|| { ErrorCode::InvalidInputSyntax("Please specify a source schema using FORMAT".to_owned()) diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index 58fb2d50c6287..3ff012c81b766 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -13,7 +13,6 @@ // limitations under the License. use core::fmt; -use std::collections::BTreeMap; use std::fmt::Write; use itertools::Itertools; @@ -105,7 +104,7 @@ pub enum SourceSchema { } impl SourceSchema { - pub fn into_source_schema_v2(self) -> SourceSchemaV2 { + pub fn into_source_schema_v2(self) -> ConnectorSchema { let (format, row_encode) = match self { SourceSchema::Protobuf(_) => (Format::Plain, Encode::Protobuf), SourceSchema::Json => (Format::Plain, Encode::Json), @@ -205,7 +204,7 @@ impl SourceSchema { _ => vec![], }; - SourceSchemaV2 { + ConnectorSchema { format, row_encode, row_options, @@ -336,7 +335,7 @@ impl Encode { #[derive(Debug, Clone, PartialEq, Eq, Hash)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -pub struct SourceSchemaV2 { +pub struct ConnectorSchema { pub format: Format, pub row_encode: Encode, pub row_options: Vec, @@ -346,7 +345,7 @@ pub struct SourceSchemaV2 { #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] pub enum CompatibleSourceSchema { RowFormat(SourceSchema), - V2(SourceSchemaV2), + V2(ConnectorSchema), } impl fmt::Display for CompatibleSourceSchema { @@ -363,7 +362,7 @@ impl fmt::Display for CompatibleSourceSchema { } impl CompatibleSourceSchema { - pub fn into_source_schema_v2(self) -> (SourceSchemaV2, Option) { + pub fn into_source_schema_v2(self) -> (ConnectorSchema, Option) { match self { CompatibleSourceSchema::RowFormat(inner) => ( inner.into_source_schema_v2(), @@ -373,29 +372,15 @@ impl CompatibleSourceSchema { } } -impl From for CompatibleSourceSchema { - fn from(value: SourceSchemaV2) -> Self { +impl From for CompatibleSourceSchema { + fn from(value: ConnectorSchema) -> Self { Self::V2(value) } } fn parse_source_schema(p: &mut Parser) -> Result { - if p.peek_nth_any_of_keywords(0, &[Keyword::FORMAT]) { - p.expect_keyword(Keyword::FORMAT)?; - let id = p.parse_identifier()?; - let s = id.value.to_ascii_uppercase(); - let format = Format::from_keyword(&s)?; - p.expect_keyword(Keyword::ENCODE)?; - let id = p.parse_identifier()?; - let s = id.value.to_ascii_uppercase(); - let row_encode = Encode::from_keyword(&s)?; - let row_options = p.parse_options()?; - - Ok(CompatibleSourceSchema::V2(SourceSchemaV2 { - format, - row_encode, - row_options, - })) + if let Some(schema_v2) = p.parse_schema()? { + Ok(CompatibleSourceSchema::V2(schema_v2)) } else if p.peek_nth_any_of_keywords(0, &[Keyword::ROW]) && p.peek_nth_any_of_keywords(1, &[Keyword::FORMAT]) { @@ -465,7 +450,7 @@ impl Parser { // row format for nexmark source must be native // default row format for datagen source is native if connector.contains("-cdc") { - let expected = SourceSchemaV2::debezium_json(); + let expected = ConnectorSchema::debezium_json(); if self.peek_source_schema_format() { let schema = parse_source_schema(self)?.into_source_schema_v2().0; if schema != expected { @@ -477,7 +462,7 @@ impl Parser { } Ok(expected.into()) } else if connector.contains("nexmark") { - let expected = SourceSchemaV2::native(); + let expected = ConnectorSchema::native(); if self.peek_source_schema_format() { let schema = parse_source_schema(self)?.into_source_schema_v2().0; if schema != expected { @@ -492,17 +477,15 @@ impl Parser { Ok(if self.peek_source_schema_format() { parse_source_schema(self)? } else { - SourceSchemaV2::native().into() + ConnectorSchema::native().into() }) } else { Ok(parse_source_schema(self)?) } } - /// Parse `FORMAT ... ENCODE ... (...)` in `CREATE SINK`. - /// - /// TODO: After [`SourceSchemaV2`] and [`SinkSchema`] merge, call this in [`parse_source_schema`]. - pub fn parse_schema(&mut self) -> Result, ParserError> { + /// Parse `FORMAT ... ENCODE ... (...)` in `CREATE SOURCE` and `CREATE SINK`. + pub fn parse_schema(&mut self) -> Result, ParserError> { if !self.parse_keyword(Keyword::FORMAT) { return Ok(None); } @@ -516,7 +499,7 @@ impl Parser { let row_encode = Encode::from_keyword(&s)?; let row_options = self.parse_options()?; - Ok(Some(SinkSchema { + Ok(Some(ConnectorSchema { format, row_encode, row_options, @@ -524,10 +507,10 @@ impl Parser { } } -impl SourceSchemaV2 { +impl ConnectorSchema { /// Create a new source schema with `Debezium` format and `Json` encoding. pub const fn debezium_json() -> Self { - SourceSchemaV2 { + ConnectorSchema { format: Format::Debezium, row_encode: Encode::Json, row_options: Vec::new(), @@ -536,35 +519,19 @@ impl SourceSchemaV2 { /// Create a new source schema with `Native` format and encoding. pub const fn native() -> Self { - SourceSchemaV2 { + ConnectorSchema { format: Format::Native, row_encode: Encode::Native, row_options: Vec::new(), } } - pub fn gen_options(&self) -> Result, ParserError> { - self.row_options - .iter() - .cloned() - .map(|x| match x.value { - Value::CstyleEscapedString(s) => Ok((x.name.real_value(), s.value)), - Value::SingleQuotedString(s) => Ok((x.name.real_value(), s)), - Value::Number(n) => Ok((x.name.real_value(), n)), - Value::Boolean(b) => Ok((x.name.real_value(), b.to_string())), - _ => Err(ParserError::ParserError( - "`row format options` only support single quoted string value and C style escaped string".to_owned(), - )), - }) - .try_collect() - } - pub fn row_options(&self) -> &[SqlOption] { self.row_options.as_ref() } } -impl fmt::Display for SourceSchemaV2 { +impl fmt::Display for ConnectorSchema { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { write!(f, "FORMAT {} ENCODE {}", self.format, self.row_encode)?; @@ -823,27 +790,6 @@ impl fmt::Display for CreateSink { } } -/// Same as [`SourceSchemaV2`]. Will be merged in a dedicated rename PR. -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] -pub struct SinkSchema { - pub format: Format, - pub row_encode: Encode, - pub row_options: Vec, -} - -impl fmt::Display for SinkSchema { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - write!(f, "FORMAT {} ENCODE {}", self.format, self.row_encode)?; - - if !self.row_options.is_empty() { - write!(f, " ({})", display_comma_separated(&self.row_options)) - } else { - Ok(()) - } - } -} - // sql_grammar!(CreateSinkStatement { // if_not_exists => [Keyword::IF, Keyword::NOT, Keyword::EXISTS], // sink_name: Ident, @@ -860,7 +806,7 @@ pub struct CreateSinkStatement { pub sink_from: CreateSink, pub columns: Vec, pub emit_mode: Option, - pub sink_schema: Option, + pub sink_schema: Option, } impl ParseTo for CreateSinkStatement { diff --git a/src/sqlparser/tests/testdata/create.yaml b/src/sqlparser/tests/testdata/create.yaml index 5509ccad53a04..cde40d8a75da1 100644 --- a/src/sqlparser/tests/testdata/create.yaml +++ b/src/sqlparser/tests/testdata/create.yaml @@ -21,13 +21,13 @@ formatted_sql: CREATE SOURCE src FORMAT PLAIN ENCODE JSON - input: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://') formatted_sql: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://') - formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(SourceSchemaV2 { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "location", quote_style: None }]), value: SingleQuotedString("file://") }] }), source_watermarks: [] } }' + formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(ConnectorSchema { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "location", quote_style: None }]), value: SingleQuotedString("file://") }] }), source_watermarks: [] } }' - input: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://') formatted_sql: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.servers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://') - formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(SourceSchemaV2 { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: SingleQuotedString("http://") }] }), source_watermarks: [] } }' + formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: true, columns: [], constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "servers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), source_schema: V2(ConnectorSchema { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: SingleQuotedString("http://") }] }), source_watermarks: [] } }' - input: CREATE SOURCE bid (auction INTEGER, bidder INTEGER, price INTEGER, WATERMARK FOR auction AS auction - 1, "date_time" TIMESTAMP) with (connector = 'nexmark', nexmark.table.type = 'Bid', nexmark.split.num = '12', nexmark.min.event.gap.in.ns = '0') formatted_sql: CREATE SOURCE bid (auction INT, bidder INT, price INT, "date_time" TIMESTAMP, WATERMARK FOR auction AS auction - 1) WITH (connector = 'nexmark', nexmark.table.type = 'Bid', nexmark.split.num = '12', nexmark.min.event.gap.in.ns = '0') FORMAT NATIVE ENCODE NATIVE - formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: false, columns: [ColumnDef { name: Ident { value: "auction", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "bidder", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "price", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "date_time", quote_style: Some(''"'') }, data_type: Some(Timestamp(false)), collation: None, options: [] }], constraints: [], source_name: ObjectName([Ident { value: "bid", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "connector", quote_style: None }]), value: SingleQuotedString("nexmark") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "table", quote_style: None }, Ident { value: "type", quote_style: None }]), value: SingleQuotedString("Bid") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "split", quote_style: None }, Ident { value: "num", quote_style: None }]), value: SingleQuotedString("12") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "min", quote_style: None }, Ident { value: "event", quote_style: None }, Ident { value: "gap", quote_style: None }, Ident { value: "in", quote_style: None }, Ident { value: "ns", quote_style: None }]), value: SingleQuotedString("0") }]), source_schema: V2(SourceSchemaV2 { format: Native, row_encode: Native, row_options: [] }), source_watermarks: [SourceWatermark { column: Ident { value: "auction", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "auction", quote_style: None }), op: Minus, right: Value(Number("1")) } }] } }' + formatted_ast: 'CreateSource { stmt: CreateSourceStatement { if_not_exists: false, columns: [ColumnDef { name: Ident { value: "auction", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "bidder", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "price", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "date_time", quote_style: Some(''"'') }, data_type: Some(Timestamp(false)), collation: None, options: [] }], constraints: [], source_name: ObjectName([Ident { value: "bid", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "connector", quote_style: None }]), value: SingleQuotedString("nexmark") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "table", quote_style: None }, Ident { value: "type", quote_style: None }]), value: SingleQuotedString("Bid") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "split", quote_style: None }, Ident { value: "num", quote_style: None }]), value: SingleQuotedString("12") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "min", quote_style: None }, Ident { value: "event", quote_style: None }, Ident { value: "gap", quote_style: None }, Ident { value: "in", quote_style: None }, Ident { value: "ns", quote_style: None }]), value: SingleQuotedString("0") }]), source_schema: V2(ConnectorSchema { format: Native, row_encode: Native, row_options: [] }), source_watermarks: [SourceWatermark { column: Ident { value: "auction", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "auction", quote_style: None }), op: Minus, right: Value(Number("1")) } }] } }' - input: CREATE TABLE T (v1 INT, v2 STRUCT) formatted_sql: CREATE TABLE T (v1 INT, v2 STRUCT) - input: CREATE TABLE T (v1 INT, v2 STRUCT>) From d6e5bec7192e851700bd1ff753c699f9ffa9d99a Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 26 Oct 2023 16:38:46 +0800 Subject: [PATCH 10/52] feat(sink): upsert avro with schema registry (#13007) --- ci/scripts/e2e-kafka-sink-test.sh | 8 ++ e2e_test/sink/kafka/avro.slt | 110 ++++++++++++++++++ e2e_test/sink/kafka/protobuf.slt | 9 ++ e2e_test/sink/kafka/register_schema.py | 48 ++++++++ src/connector/src/schema/avro.rs | 105 +++++++++++++++++ src/connector/src/schema/mod.rs | 4 + .../src/schema/schema_registry/client.rs | 18 ++- src/connector/src/sink/encoder/avro.rs | 90 +++++++++++--- src/connector/src/sink/encoder/mod.rs | 2 +- src/connector/src/sink/formatter/mod.rs | 44 ++++++- src/connector/src/sink/kafka.rs | 2 + src/connector/src/sink/kinesis.rs | 17 ++- src/connector/src/sink/pulsar.rs | 17 ++- src/connector/src/sink/redis.rs | 6 +- src/connector/src/test_data/all-types.avsc | 69 +++++++++++ src/frontend/src/handler/create_sink.rs | 2 +- 16 files changed, 515 insertions(+), 36 deletions(-) create mode 100644 e2e_test/sink/kafka/avro.slt create mode 100644 e2e_test/sink/kafka/register_schema.py create mode 100644 src/connector/src/schema/avro.rs create mode 100644 src/connector/src/test_data/all-types.avsc diff --git a/ci/scripts/e2e-kafka-sink-test.sh b/ci/scripts/e2e-kafka-sink-test.sh index 71a91f2d8fba9..d51482a912235 100755 --- a/ci/scripts/e2e-kafka-sink-test.sh +++ b/ci/scripts/e2e-kafka-sink-test.sh @@ -138,3 +138,11 @@ cp src/connector/src/test_data/proto_recursive/recursive.pb ./proto-recursive ./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-append-only-protobuf --create > /dev/null 2>&1 sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/protobuf.slt' ./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-append-only-protobuf --delete > /dev/null 2>&1 + +echo "testing avro" +python3 -m pip install requests confluent-kafka +python3 e2e_test/sink/kafka/register_schema.py 'http://message_queue:8081' 'test-rw-sink-upsert-avro-value' src/connector/src/test_data/all-types.avsc +python3 e2e_test/sink/kafka/register_schema.py 'http://message_queue:8081' 'test-rw-sink-upsert-avro-key' src/connector/src/test_data/all-types.avsc 'string_field,int32_field' +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert-avro --create > /dev/null 2>&1 +sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/avro.slt' +./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server message_queue:29092 --topic test-rw-sink-upsert-avro --delete > /dev/null 2>&1 diff --git a/e2e_test/sink/kafka/avro.slt b/e2e_test/sink/kafka/avro.slt new file mode 100644 index 0000000000000..e1b09e3608e37 --- /dev/null +++ b/e2e_test/sink/kafka/avro.slt @@ -0,0 +1,110 @@ +statement ok +create table from_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro', + properties.bootstrap.server = 'message_queue:29092') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081'); + +statement ok +create table into_kafka ( + bool_field bool, + string_field varchar, + bytes_field bytea, + float_field real, + double_field double precision, + int32_field int, + int64_field bigint, + record_field struct, + array_field int[][], + timestamp_micros_field timestamptz, + timestamp_millis_field timestamptz, + date_field date, + time_micros_field time, + time_millis_field time); + +statement ok +insert into into_kafka values + (true, 'Rising', 'a0', 3.5, 4.25, 22, 23, null, array[array[null, 3], null, array[7, null, 2]], '2006-01-02 15:04:05-07:00', null, null, '12:34:56.123456', null), + (false, 'Wave', 'ZDF', 1.5, null, 11, 12, row(null::int, 'foo'), null, null, '2006-01-02 15:04:05-07:00', '2021-04-01', null, '23:45:16.654321'); + +statement ok +flush; + +statement ok +create sink sink0 from into_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro', + properties.bootstrap.server = 'message_queue:29092', + primary_key = 'int32_field,string_field') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081'); + +sleep 2s + +query TTTRRIITTTTTTTT +select + bool_field, + string_field, + bytes_field, + float_field, + double_field, + int32_field, + int64_field, + record_field, + array_field, + timestamp_micros_field, + timestamp_millis_field, + date_field, + time_micros_field, + time_millis_field from from_kafka; +---- +t Rising \x6130 3.5 4.25 22 23 NULL {{NULL,3},NULL,{7,NULL,2}} 2006-01-02 22:04:05+00:00 NULL NULL 12:34:56.123456 NULL +f Wave \x5a4446 1.5 NULL 11 12 (NULL,foo) NULL NULL 2006-01-02 22:04:05+00:00 2021-04-01 NULL 23:45:16.654 + +statement error SchemaFetchError +create sink sink_err from into_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro-err', + properties.bootstrap.server = 'message_queue:29092', + primary_key = 'int32_field,string_field') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081'); + +statement error encode extra_column error: field not in avro +create sink sink_err as select 1 as extra_column, * from into_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro', + properties.bootstrap.server = 'message_queue:29092', + primary_key = 'int32_field,string_field') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081'); + +statement error unrecognized +create sink sink_err from into_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro', + properties.bootstrap.server = 'message_queue:29092', + primary_key = 'int32_field,string_field') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081', + schema.registry.name.strategy = 'typo'); + +statement error empty field key.message +create sink sink_err from into_kafka with ( + connector = 'kafka', + topic = 'test-rw-sink-upsert-avro', + properties.bootstrap.server = 'message_queue:29092', + primary_key = 'int32_field,string_field') +format upsert encode avro ( + schema.registry = 'http://message_queue:8081', + schema.registry.name.strategy = 'record_name_strategy'); + +statement ok +drop sink sink0; + +statement ok +drop table into_kafka; + +statement ok +drop table from_kafka; diff --git a/e2e_test/sink/kafka/protobuf.slt b/e2e_test/sink/kafka/protobuf.slt index 87ab884eddbde..2f827aeda9fc0 100644 --- a/e2e_test/sink/kafka/protobuf.slt +++ b/e2e_test/sink/kafka/protobuf.slt @@ -95,3 +95,12 @@ format plain encode protobuf ( force_append_only = true, schema.location = 's3:///risingwave/proto-recursive', message = 'recursive.AllTypes'); + +statement ok +drop sink sink0; + +statement ok +drop table into_kafka; + +statement ok +drop table from_kafka; diff --git a/e2e_test/sink/kafka/register_schema.py b/e2e_test/sink/kafka/register_schema.py new file mode 100644 index 0000000000000..2606e07bcb89b --- /dev/null +++ b/e2e_test/sink/kafka/register_schema.py @@ -0,0 +1,48 @@ +import sys +from confluent_kafka.schema_registry import SchemaRegistryClient, Schema + + +def main(): + url = sys.argv[1] + subject = sys.argv[2] + with open(sys.argv[3]) as f: + schema_str = f.read() + if 4 < len(sys.argv): + keys = sys.argv[4].split(',') + else: + keys = [] + + client = SchemaRegistryClient({"url": url}) + + if keys: + schema_str = select_keys(schema_str, keys) + else: + schema_str = remove_unsupported(schema_str) + schema = Schema(schema_str, 'AVRO') + client.register_schema(subject, schema) + + +def select_fields(schema_str, f): + import json + root = json.loads(schema_str) + if not isinstance(root, dict): + return schema_str + if root['type'] != 'record': + return schema_str + root['fields'] = f(root['fields']) + return json.dumps(root) + + +def remove_unsupported(schema_str): + return select_fields(schema_str, lambda fields: [f for f in fields if f['name'] not in {'unsupported', 'mon_day_sec_field'}]) + + +def select_keys(schema_str, keys): + def process(fields): + by_name = {f['name']: f for f in fields} + return [by_name[k] for k in keys] + return select_fields(schema_str, process) + + +if __name__ == '__main__': + main() diff --git a/src/connector/src/schema/avro.rs b/src/connector/src/schema/avro.rs new file mode 100644 index 0000000000000..fc12ba90f25aa --- /dev/null +++ b/src/connector/src/schema/avro.rs @@ -0,0 +1,105 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::BTreeMap; +use std::sync::Arc; + +use apache_avro::Schema as AvroSchema; +use risingwave_pb::catalog::PbSchemaRegistryNameStrategy; + +use super::schema_registry::{ + get_subject_by_strategy, handle_sr_list, name_strategy_from_str, Client, ConfluentSchema, + SchemaRegistryAuth, +}; +use super::{ + SchemaFetchError, KEY_MESSAGE_NAME_KEY, MESSAGE_NAME_KEY, NAME_STRATEGY_KEY, + SCHEMA_REGISTRY_KEY, +}; + +pub struct SchemaWithId { + pub schema: Arc, + pub id: i32, +} + +impl TryFrom for SchemaWithId { + type Error = SchemaFetchError; + + fn try_from(fetched: ConfluentSchema) -> Result { + let parsed = + AvroSchema::parse_str(&fetched.content).map_err(|e| SchemaFetchError(e.to_string()))?; + Ok(Self { + schema: Arc::new(parsed), + id: fetched.id, + }) + } +} + +/// Schema registry only +pub async fn fetch_schema( + format_options: &BTreeMap, + topic: &str, +) -> Result<(SchemaWithId, SchemaWithId), SchemaFetchError> { + let schema_location = format_options + .get(SCHEMA_REGISTRY_KEY) + .ok_or_else(|| SchemaFetchError(format!("{SCHEMA_REGISTRY_KEY} required")))? + .clone(); + let client_config = format_options.into(); + let name_strategy = format_options + .get(NAME_STRATEGY_KEY) + .map(|s| { + name_strategy_from_str(s) + .ok_or_else(|| SchemaFetchError(format!("unrecognized strategy {s}"))) + }) + .transpose()? + .unwrap_or_default(); + let key_record_name = format_options + .get(KEY_MESSAGE_NAME_KEY) + .map(std::ops::Deref::deref); + let val_record_name = format_options + .get(MESSAGE_NAME_KEY) + .map(std::ops::Deref::deref); + + let (key_schema, val_schema) = fetch_schema_inner( + &schema_location, + &client_config, + &name_strategy, + topic, + key_record_name, + val_record_name, + ) + .await + .map_err(|e| SchemaFetchError(e.to_string()))?; + + Ok((key_schema.try_into()?, val_schema.try_into()?)) +} + +async fn fetch_schema_inner( + schema_location: &str, + client_config: &SchemaRegistryAuth, + name_strategy: &PbSchemaRegistryNameStrategy, + topic: &str, + key_record_name: Option<&str>, + val_record_name: Option<&str>, +) -> Result<(ConfluentSchema, ConfluentSchema), risingwave_common::error::RwError> { + let urls = handle_sr_list(schema_location)?; + let client = Client::new(urls, client_config)?; + + let key_subject = get_subject_by_strategy(name_strategy, topic, key_record_name, true)?; + let key_schema = client.get_schema_by_subject(&key_subject).await?; + + let val_subject = get_subject_by_strategy(name_strategy, topic, val_record_name, false)?; + let val_schema = client.get_schema_by_subject(&val_subject).await?; + + Ok((key_schema, val_schema)) +} diff --git a/src/connector/src/schema/mod.rs b/src/connector/src/schema/mod.rs index 3c8694fc8f359..75a521a50ec55 100644 --- a/src/connector/src/schema/mod.rs +++ b/src/connector/src/schema/mod.rs @@ -12,11 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +pub mod avro; pub mod protobuf; pub mod schema_registry; const MESSAGE_NAME_KEY: &str = "message"; +const KEY_MESSAGE_NAME_KEY: &str = "key.message"; const SCHEMA_LOCATION_KEY: &str = "schema.location"; +const SCHEMA_REGISTRY_KEY: &str = "schema.registry"; +const NAME_STRATEGY_KEY: &str = "schema.registry.name.strategy"; #[derive(Debug)] pub struct SchemaFetchError(pub String); diff --git a/src/connector/src/schema/schema_registry/client.rs b/src/connector/src/schema/schema_registry/client.rs index 591478752d926..6d5778cd367c4 100644 --- a/src/connector/src/schema/schema_registry/client.rs +++ b/src/connector/src/schema/schema_registry/client.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::fmt::Debug; use std::sync::Arc; @@ -43,6 +43,18 @@ impl From<&HashMap> for SchemaRegistryAuth { } } +impl From<&BTreeMap> for SchemaRegistryAuth { + fn from(props: &BTreeMap) -> Self { + const SCHEMA_REGISTRY_USERNAME: &str = "schema.registry.username"; + const SCHEMA_REGISTRY_PASSWORD: &str = "schema.registry.password"; + + SchemaRegistryAuth { + username: props.get(SCHEMA_REGISTRY_USERNAME).cloned(), + password: props.get(SCHEMA_REGISTRY_PASSWORD).cloned(), + } + } +} + /// An client for communication with schema registry #[derive(Debug)] pub struct Client { @@ -123,7 +135,9 @@ impl Client { Err(RwError::from(ProtocolError(format!( "all request confluent registry all timeout, req path {:?}, urls {:?}, err: {:?}", - path, self.url, errs + path, + self.url, + errs.iter().map(|e| e.to_string()).collect_vec() )))) } diff --git a/src/connector/src/sink/encoder/avro.rs b/src/connector/src/sink/encoder/avro.rs index fc2db75eb4c38..7aff29dbb43b7 100644 --- a/src/connector/src/sink/encoder/avro.rs +++ b/src/connector/src/sink/encoder/avro.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use apache_avro::schema::Schema as AvroSchema; use apache_avro::types::{Record, Value}; -use apache_avro::Writer; use risingwave_common::catalog::Schema; use risingwave_common::row::Row; use risingwave_common::types::{DataType, DatumRef, ScalarRefImpl, StructType}; @@ -30,6 +29,28 @@ pub struct AvroEncoder { schema: Schema, col_indices: Option>, avro_schema: Arc, + header: AvroHeader, +} + +#[derive(Debug, Clone, Copy)] +pub enum AvroHeader { + None, + /// + /// + /// * C3 01 + /// * 8-byte little-endian CRC-64-AVRO fingerprint + SingleObject, + /// + /// + /// * 4F 62 6A 01 + /// * schema + /// * 16-byte random sync marker + ContainerFile, + /// + /// + /// * 00 + /// * 4-byte big-endian schema ID + ConfluentSchemaRegistry(i32), } impl AvroEncoder { @@ -37,6 +58,7 @@ impl AvroEncoder { schema: Schema, col_indices: Option>, avro_schema: Arc, + header: AvroHeader, ) -> SinkResult { match &col_indices { Some(col_indices) => validate_fields( @@ -59,12 +81,19 @@ impl AvroEncoder { schema, col_indices, avro_schema, + header, }) } } +pub struct AvroEncoded { + value: Value, + schema: Arc, + header: AvroHeader, +} + impl RowEncoder for AvroEncoder { - type Output = (Value, Arc); + type Output = AvroEncoded; fn schema(&self) -> &Schema { &self.schema @@ -86,16 +115,32 @@ impl RowEncoder for AvroEncoder { }), &self.avro_schema, )?; - Ok((record.into(), self.avro_schema.clone())) + Ok(AvroEncoded { + value: record.into(), + schema: self.avro_schema.clone(), + header: self.header, + }) } } -impl SerTo> for (Value, Arc) { +impl SerTo> for AvroEncoded { fn ser_to(self) -> SinkResult> { - let mut w = Writer::new(&self.1, Vec::new()); - w.append(self.0) - .and_then(|_| w.into_inner()) - .map_err(|e| crate::sink::SinkError::Encode(e.to_string())) + use bytes::BufMut as _; + + let AvroHeader::ConfluentSchemaRegistry(schema_id) = self.header else { + return Err(crate::sink::SinkError::Encode(format!( + "{:?} unsupported yet", + self.header + ))); + }; + let raw = apache_avro::to_avro_datum(&self.schema, self.value) + .map_err(|e| crate::sink::SinkError::Encode(e.to_string()))?; + let mut buf = Vec::with_capacity(1 + 4 + raw.len()); + buf.put_u8(0); + buf.put_i32(schema_id); + buf.put_slice(&raw); + + Ok(buf) } } @@ -616,8 +661,16 @@ mod tests { .unwrap(); let mut record = Record::new(&avro_schema).unwrap(); record.put("f0", Value::String("2".into())); - let res: SinkResult> = (Value::from(record), Arc::new(avro_schema)).ser_to(); - assert_eq!(res.unwrap_err().to_string(), "Encode error: Value does not match schema: Reason: Unsupported value-schema combination"); + let res: SinkResult> = AvroEncoded { + value: Value::from(record), + schema: Arc::new(avro_schema), + header: AvroHeader::ConfluentSchemaRegistry(42), + } + .ser_to(); + assert_eq!( + res.unwrap_err().to_string(), + "Encode error: Value does not match schema" + ); } #[test] @@ -634,6 +687,7 @@ mod tests { ) .unwrap(); let avro_schema = Arc::new(avro_schema); + let header = AvroHeader::None; let schema = Schema::new(vec![ Field::with_name(DataType::Int64, "opt"), @@ -643,10 +697,10 @@ mod tests { Some(ScalarImpl::Int64(31)), Some(ScalarImpl::Int32(15)), ]); - let encoder = AvroEncoder::new(schema, None, avro_schema.clone()).unwrap(); + let encoder = AvroEncoder::new(schema, None, avro_schema.clone(), header).unwrap(); let actual = encoder.encode(row).unwrap(); assert_eq!( - actual.0, + actual.value, Value::Record(vec![ ("req".into(), Value::Int(15)), ("opt".into(), Value::Union(1, Value::Long(31).into())), @@ -655,10 +709,10 @@ mod tests { let schema = Schema::new(vec![Field::with_name(DataType::Int32, "req")]); let row = OwnedRow::new(vec![Some(ScalarImpl::Int32(15))]); - let encoder = AvroEncoder::new(schema, None, avro_schema.clone()).unwrap(); + let encoder = AvroEncoder::new(schema, None, avro_schema.clone(), header).unwrap(); let actual = encoder.encode(row).unwrap(); assert_eq!( - actual.0, + actual.value, Value::Record(vec![ ("req".into(), Value::Int(15)), ("opt".into(), Value::Union(0, Value::Null.into())), @@ -666,7 +720,7 @@ mod tests { ); let schema = Schema::new(vec![Field::with_name(DataType::Int64, "opt")]); - let Err(err) = AvroEncoder::new(schema, None, avro_schema.clone()) else { + let Err(err) = AvroEncoder::new(schema, None, avro_schema.clone(), header) else { panic!() }; assert_eq!( @@ -679,7 +733,7 @@ mod tests { Field::with_name(DataType::Int32, "req"), Field::with_name(DataType::Varchar, "extra"), ]); - let Err(err) = AvroEncoder::new(schema, None, avro_schema.clone()) else { + let Err(err) = AvroEncoder::new(schema, None, avro_schema.clone(), header) else { panic!() }; assert_eq!( @@ -689,7 +743,7 @@ mod tests { let avro_schema = AvroSchema::parse_str(r#"["null", "long"]"#).unwrap(); let schema = Schema::new(vec![Field::with_name(DataType::Int64, "opt")]); - let Err(err) = AvroEncoder::new(schema, None, avro_schema.into()) else { + let Err(err) = AvroEncoder::new(schema, None, avro_schema.into(), header) else { panic!() }; assert_eq!( @@ -843,7 +897,7 @@ mod tests { /// The encoder is not using these buggy calls and is already tested above. #[test] fn test_encode_avro_lib_bug() { - use apache_avro::Reader; + use apache_avro::{Reader, Writer}; // a record with 2 optional int fields let avro_schema = AvroSchema::parse_str( diff --git a/src/connector/src/sink/encoder/mod.rs b/src/connector/src/sink/encoder/mod.rs index 83b2ab4f09df0..b55fd534d5eb3 100644 --- a/src/connector/src/sink/encoder/mod.rs +++ b/src/connector/src/sink/encoder/mod.rs @@ -25,7 +25,7 @@ mod json; mod proto; pub mod template; -pub use avro::AvroEncoder; +pub use avro::{AvroEncoder, AvroHeader}; pub use json::JsonEncoder; pub use proto::ProtoEncoder; diff --git a/src/connector/src/sink/formatter/mod.rs b/src/connector/src/sink/formatter/mod.rs index 17cb708292890..1e165268300fa 100644 --- a/src/connector/src/sink/formatter/mod.rs +++ b/src/connector/src/sink/formatter/mod.rs @@ -30,7 +30,9 @@ use super::catalog::{SinkEncode, SinkFormat, SinkFormatDesc}; use super::encoder::template::TemplateEncoder; use super::encoder::KafkaConnectParams; use super::redis::{KEY_FORMAT, VALUE_FORMAT}; -use crate::sink::encoder::{JsonEncoder, ProtoEncoder, TimestampHandlingMode}; +use crate::sink::encoder::{ + AvroEncoder, AvroHeader, JsonEncoder, ProtoEncoder, TimestampHandlingMode, +}; /// Transforms a `StreamChunk` into a sequence of key-value pairs according a specific format, /// for example append-only, upsert or debezium. @@ -67,6 +69,7 @@ pub enum SinkFormatterImpl { AppendOnlyJson(AppendOnlyFormatter), AppendOnlyProto(AppendOnlyFormatter), UpsertJson(UpsertFormatter), + UpsertAvro(UpsertFormatter), DebeziumJson(DebeziumJsonFormatter), AppendOnlyTemplate(AppendOnlyFormatter), UpsertTemplate(UpsertFormatter), @@ -79,6 +82,7 @@ impl SinkFormatterImpl { pk_indices: Vec, db_name: String, sink_from_name: String, + topic: &str, ) -> Result { let err_unsupported = || { Err(SinkError::Config(anyhow!( @@ -211,7 +215,27 @@ impl SinkFormatterImpl { val_encoder, ))) } - _ => err_unsupported(), + SinkEncode::Avro => { + let (key_schema, val_schema) = + crate::schema::avro::fetch_schema(&format_desc.options, topic) + .await + .map_err(|e| SinkError::Config(anyhow!("{e:?}")))?; + let key_encoder = AvroEncoder::new( + schema.clone(), + Some(pk_indices), + key_schema.schema, + AvroHeader::ConfluentSchemaRegistry(key_schema.id), + )?; + let val_encoder = AvroEncoder::new( + schema.clone(), + None, + val_schema.schema, + AvroHeader::ConfluentSchemaRegistry(val_schema.id), + )?; + let formatter = UpsertFormatter::new(key_encoder, val_encoder); + Ok(SinkFormatterImpl::UpsertAvro(formatter)) + } + SinkEncode::Protobuf => err_unsupported(), } } } @@ -225,6 +249,22 @@ macro_rules! dispatch_sink_formatter_impl { SinkFormatterImpl::AppendOnlyJson($name) => $body, SinkFormatterImpl::AppendOnlyProto($name) => $body, SinkFormatterImpl::UpsertJson($name) => $body, + SinkFormatterImpl::UpsertAvro($name) => $body, + SinkFormatterImpl::DebeziumJson($name) => $body, + SinkFormatterImpl::AppendOnlyTemplate($name) => $body, + SinkFormatterImpl::UpsertTemplate($name) => $body, + } + }; +} + +#[macro_export] +macro_rules! dispatch_sink_formatter_str_key_impl { + ($impl:expr, $name:ident, $body:expr) => { + match $impl { + SinkFormatterImpl::AppendOnlyJson($name) => $body, + SinkFormatterImpl::AppendOnlyProto($name) => $body, + SinkFormatterImpl::UpsertJson($name) => $body, + SinkFormatterImpl::UpsertAvro(_) => unreachable!(), SinkFormatterImpl::DebeziumJson($name) => $body, SinkFormatterImpl::AppendOnlyTemplate($name) => $body, SinkFormatterImpl::UpsertTemplate($name) => $body, diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index f77b2b0a88c36..07709f182dc47 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -312,6 +312,7 @@ impl Sink for KafkaSink { self.pk_indices.clone(), self.db_name.clone(), self.sink_from_name.clone(), + &self.config.common.topic, ) .await?; let max_delivery_buffer_size = (self @@ -343,6 +344,7 @@ impl Sink for KafkaSink { self.pk_indices.clone(), self.db_name.clone(), self.sink_from_name.clone(), + &self.config.common.topic, ) .await?; diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index 605edde3b1eb0..03e044ad37b91 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -29,7 +29,7 @@ use tokio_retry::Retry; use super::catalog::SinkFormatDesc; use super::SinkParam; use crate::common::KinesisCommon; -use crate::dispatch_sink_formatter_impl; +use crate::dispatch_sink_formatter_str_key_impl; use crate::sink::catalog::desc::SinkDesc; use crate::sink::formatter::SinkFormatterImpl; use crate::sink::log_store::DeliveryFutureManagerAddFuture; @@ -94,6 +94,7 @@ impl Sink for KinesisSink { self.pk_indices.clone(), self.db_name.clone(), self.sink_from_name.clone(), + &self.config.common.stream_name, ) .await?; @@ -161,9 +162,15 @@ impl KinesisSinkWriter { db_name: String, sink_from_name: String, ) -> Result { - let formatter = - SinkFormatterImpl::new(format_desc, schema, pk_indices, db_name, sink_from_name) - .await?; + let formatter = SinkFormatterImpl::new( + format_desc, + schema, + pk_indices, + db_name, + sink_from_name, + &config.common.stream_name, + ) + .await?; let client = config .common .build_client() @@ -228,7 +235,7 @@ impl AsyncTruncateSinkWriter for KinesisSinkWriter { chunk: StreamChunk, _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, ) -> Result<()> { - dispatch_sink_formatter_impl!( + dispatch_sink_formatter_str_key_impl!( &self.formatter, formatter, self.payload_writer.write_chunk(chunk, formatter).await diff --git a/src/connector/src/sink/pulsar.rs b/src/connector/src/sink/pulsar.rs index 9eb57c1ae0771..04da204ef79e7 100644 --- a/src/connector/src/sink/pulsar.rs +++ b/src/connector/src/sink/pulsar.rs @@ -36,7 +36,7 @@ use crate::sink::writer::{ AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, FormattedSink, }; use crate::sink::{DummySinkCommitCoordinator, Result}; -use crate::{deserialize_duration_from_string, dispatch_sink_formatter_impl}; +use crate::{deserialize_duration_from_string, dispatch_sink_formatter_str_key_impl}; pub const PULSAR_SINK: &str = "pulsar"; @@ -194,6 +194,7 @@ impl Sink for PulsarSink { self.downstream_pk.clone(), self.db_name.clone(), self.sink_from_name.clone(), + &self.config.common.topic, ) .await?; @@ -237,9 +238,15 @@ impl PulsarSinkWriter { db_name: String, sink_from_name: String, ) -> Result { - let formatter = - SinkFormatterImpl::new(format_desc, schema, downstream_pk, db_name, sink_from_name) - .await?; + let formatter = SinkFormatterImpl::new( + format_desc, + schema, + downstream_pk, + db_name, + sink_from_name, + &config.common.topic, + ) + .await?; let pulsar = config.common.build_client().await?; let producer = build_pulsar_producer(&pulsar, &config).await?; Ok(Self { @@ -322,7 +329,7 @@ impl AsyncTruncateSinkWriter for PulsarSinkWriter { chunk: StreamChunk, add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, ) -> Result<()> { - dispatch_sink_formatter_impl!(&self.formatter, formatter, { + dispatch_sink_formatter_str_key_impl!(&self.formatter, formatter, { let mut payload_writer = PulsarPayloadWriter { producer: &mut self.producer, add_future, diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index af3ec3b981620..910582b9662b7 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -28,7 +28,7 @@ use super::encoder::template::TemplateEncoder; use super::formatter::SinkFormatterImpl; use super::writer::FormattedSink; use super::{SinkError, SinkParam}; -use crate::dispatch_sink_formatter_impl; +use crate::dispatch_sink_formatter_str_key_impl; use crate::sink::log_store::DeliveryFutureManagerAddFuture; use crate::sink::writer::{ AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, @@ -224,6 +224,7 @@ impl RedisSinkWriter { pk_indices.clone(), db_name, sink_from_name, + "NO_TOPIC", ) .await?; @@ -248,6 +249,7 @@ impl RedisSinkWriter { pk_indices.clone(), "d1".to_string(), "t1".to_string(), + "NO_TOPIC", ) .await?; Ok(Self { @@ -266,7 +268,7 @@ impl AsyncTruncateSinkWriter for RedisSinkWriter { chunk: StreamChunk, _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, ) -> Result<()> { - dispatch_sink_formatter_impl!(&self.formatter, formatter, { + dispatch_sink_formatter_str_key_impl!(&self.formatter, formatter, { self.payload_writer.write_chunk(chunk, formatter).await }) } diff --git a/src/connector/src/test_data/all-types.avsc b/src/connector/src/test_data/all-types.avsc new file mode 100644 index 0000000000000..3fea69bbef4ca --- /dev/null +++ b/src/connector/src/test_data/all-types.avsc @@ -0,0 +1,69 @@ +{ + "type": "record", + "name": "AllTypes", + "fields": [ + {"name": "bool_field", "type": ["null", "boolean"]}, + {"name": "string_field", "type": ["null", "string"]}, + {"name": "bytes_field", "type": ["null", "bytes"]}, + {"name": "float_field", "type": ["null", "float"]}, + {"name": "double_field", "type": ["null", "double"]}, + {"name": "int32_field", "type": ["null", "int"]}, + {"name": "int64_field", "type": ["null", "long"]}, + {"name": "record_field", "type": ["null", { + "type": "record", + "name": "Nested", + "fields": [ + {"name": "id", "type": ["null", "int"]}, + {"name": "name", "type": ["null", "string"]} + ] + }]}, + {"name": "array_field", "type": ["null", { + "type": "array", + "items": ["null", { + "type": "array", + "items": ["null", "int"] + }] + }]}, + {"name": "timestamp_micros_field", "type": ["null", {"type": "long", "logicalType": "timestamp-micros"}]}, + {"name": "timestamp_millis_field", "type": ["null", {"type": "long", "logicalType": "timestamp-millis"}]}, + {"name": "date_field", "type": ["null", {"type": "int", "logicalType": "date"}]}, + {"name": "time_micros_field", "type": ["null", {"type": "long", "logicalType": "time-micros"}]}, + {"name": "time_millis_field", "type": ["null", {"type": "int", "logicalType": "time-millis"}]}, + {"name": "mon_day_sec_field", "type": ["null", { + "type": "fixed", + "name": "Duration", + "size": 12, + "logicalType": "duration" + }]}, + {"name": "unsupported", "type": ["null", { + "type": "record", + "name": "Unsupported", + "fields": [ + {"name": "enum_field", "type": ["null", { + "type": "enum", + "name": "Suit", + "symbols": ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"] + }]}, + {"name": "map_field", "type": ["null", { + "type": "map", + "values": ["null", "string"] + }]}, + {"name": "union_field", "type": ["null", "string", "double", "boolean"]}, + {"name": "fixed_field", "type": ["null", { + "type": "fixed", + "name": "Int256", + "size": 32 + }]}, + {"name": "decimal_field", "type": ["null", { + "type": "bytes", + "logicalType": "decimal", + "precision": 38, + "scale": 10 + }]}, + {"name": "uuid_field", "type": ["null", {"type": "string", "logicalType": "uuid"}]}, + {"name": "local_micros_field", "type": ["null", {"type": "long", "logicalType": "local-timestamp-micros"}]}, + {"name": "local_millis_field", "type": ["null", {"type": "long", "logicalType": "local-timestamp-millis"}]} + ] + }]} + ] +} diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 2ab987308a4f5..85d867d538e7f 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -269,7 +269,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock hashmap!( Format::Plain => vec![Encode::Json, Encode::Protobuf], - Format::Upsert => vec![Encode::Json], + Format::Upsert => vec![Encode::Json, Encode::Avro], Format::Debezium => vec![Encode::Json], ), KinesisSink::SINK_NAME => hashmap!( From b0f266b0a90c2a7a1c7dcd566d5fbc94f38751a2 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 26 Oct 2023 17:02:56 +0800 Subject: [PATCH 11/52] feat(mysql-cdc): deprecate `server.id` WITH option and generate it internally (#13031) --- Cargo.lock | 1 + ci/scripts/e2e-source-test.sh | 1 + .../cdc_inline/mysql/mysql_create_drop.slt | 133 ++++++++++++++++++ src/connector/src/macros.rs | 5 +- src/connector/src/source/base.rs | 2 +- src/connector/src/source/cdc/mod.rs | 11 ++ src/connector/src/source/cdc/source/reader.rs | 3 + src/connector/src/source/mod.rs | 2 +- src/meta/service/Cargo.toml | 1 + src/meta/service/src/ddl_service.rs | 29 ++++ src/source/src/connector_source.rs | 2 +- 11 files changed, 186 insertions(+), 4 deletions(-) create mode 100644 e2e_test/source/cdc_inline/mysql/mysql_create_drop.slt diff --git a/Cargo.lock b/Cargo.lock index 2f2ee2991b10f..dbce3dfae9f3b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7778,6 +7778,7 @@ dependencies = [ "itertools 0.11.0", "madsim-tokio", "madsim-tonic", + "rand", "regex", "risingwave_common", "risingwave_connector", diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 59618d24641aa..2edd099e31187 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -81,6 +81,7 @@ echo "waiting for connector node to start" wait_for_connector_node_start echo "--- inline cdc test" +export MYSQL_HOST=mysql MYSQL_TCP_PORT=3306 MYSQL_PWD=123456 sqllogictest -p 4566 -d dev './e2e_test/source/cdc_inline/**/*.slt' echo "--- mysql & postgres cdc validate test" diff --git a/e2e_test/source/cdc_inline/mysql/mysql_create_drop.slt b/e2e_test/source/cdc_inline/mysql/mysql_create_drop.slt new file mode 100644 index 0000000000000..071fe0ef2da83 --- /dev/null +++ b/e2e_test/source/cdc_inline/mysql/mysql_create_drop.slt @@ -0,0 +1,133 @@ +# create and drop CDC mysql tables concurrently + +control substitution on + +statement ok +ALTER SYSTEM SET max_concurrent_creating_streaming_jobs TO 4; + +system ok +mysql --protocol=tcp -u root -e " + DROP DATABASE IF EXISTS testdb1; CREATE DATABASE testdb1; + USE testdb1; + CREATE TABLE tt1 (v1 int primary key, v2 timestamp); + INSERT INTO tt1 VALUES (1, '2023-10-23 10:00:00'); + CREATE TABLE tt2 (v1 int primary key, v2 timestamp); + INSERT INTO tt2 VALUES (2, '2023-10-23 11:00:00'); + CREATE TABLE tt3 (v1 int primary key, v2 timestamp); + INSERT INTO tt3 VALUES (3, '2023-10-23 12:00:00'); + CREATE TABLE tt4 (v1 int primary key, v2 timestamp); + INSERT INTO tt4 VALUES (4, '2023-10-23 13:00:00'); + CREATE TABLE tt5 (v1 int primary key, v2 timestamp); + INSERT INTO tt5 VALUES (5, '2023-10-23 14:00:00');" + +statement ok +create table tt1 (v1 int, + v2 timestamptz, + PRIMARY KEY (v1) +) with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'testdb1', + table.name = 'tt1', +); + +statement ok +create table tt2 (v1 int, + v2 timestamptz, + PRIMARY KEY (v1) +) with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'testdb1', + table.name = 'tt2', +); + +statement ok +create table tt3 (v1 int, + v2 timestamptz, + PRIMARY KEY (v1) +) with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'testdb1', + table.name = 'tt3', +); + +statement ok +create table tt4 (v1 int, + v2 timestamptz, + PRIMARY KEY (v1) +) with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'testdb1', + table.name = 'tt4', +); + +statement ok +create table tt5 (v1 int, + v2 timestamptz, + PRIMARY KEY (v1) +) with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'testdb1', + table.name = 'tt5', +); + +sleep 3s + +query IT +select * from tt1; +---- +1 2023-10-23 10:00:00+00:00 + +query IT +select * from tt2; +---- +2 2023-10-23 11:00:00+00:00 + +query IT +select * from tt3; +---- +3 2023-10-23 12:00:00+00:00 + +query IT +select * from tt4; +---- +4 2023-10-23 13:00:00+00:00 + +query IT +select * from tt5; +---- +5 2023-10-23 14:00:00+00:00 + +statement ok +drop table tt1; + +statement ok +drop table tt2; + +statement ok +drop table tt3; + +statement ok +drop table tt4; + +statement ok +drop table tt5; diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 62a3cfdcd9682..fdc3ed8867297 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -234,12 +234,13 @@ macro_rules! impl_cdc_source_type { $( $cdc_source_type, )* + Unspecified, } impl From for CdcSourceType { fn from(value: PbSourceType) -> Self { match value { - PbSourceType::Unspecified => unreachable!(), + PbSourceType::Unspecified => CdcSourceType::Unspecified, $( PbSourceType::$cdc_source_type => CdcSourceType::$cdc_source_type, )* @@ -253,8 +254,10 @@ macro_rules! impl_cdc_source_type { $( CdcSourceType::$cdc_source_type => PbSourceType::$cdc_source_type, )* + CdcSourceType::Unspecified => PbSourceType::Unspecified, } } } + } } diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 6a8cd12ce9fac..49dc3b5d87119 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -51,7 +51,7 @@ use crate::{ const SPLIT_TYPE_FIELD: &str = "split_type"; const SPLIT_INFO_FIELD: &str = "split_info"; -const UPSTREAM_SOURCE_KEY: &str = "connector"; +pub const UPSTREAM_SOURCE_KEY: &str = "connector"; pub trait TryFromHashmap: Sized { fn try_from_hashmap(props: HashMap) -> Result; diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index 1d795a7141e84..d55273bf725db 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -42,6 +42,17 @@ pub trait CdcSourceTypeTrait: Send + Sync + Clone + 'static { for_all_classified_sources!(impl_cdc_source_type); +impl<'a> From<&'a str> for CdcSourceType { + fn from(name: &'a str) -> Self { + match name { + MYSQL_CDC_CONNECTOR => CdcSourceType::Mysql, + POSTGRES_CDC_CONNECTOR => CdcSourceType::Postgres, + CITUS_CDC_CONNECTOR => CdcSourceType::Citus, + _ => CdcSourceType::Unspecified, + } + } +} + #[derive(Clone, Debug, Default)] pub struct CdcProperties { /// Properties specified in the WITH clause by user diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index 7410834ce1daa..4d25d82c106c3 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -87,6 +87,9 @@ impl SplitReader for CdcSplitReader { parser_config, source_ctx, }), + CdcSourceType::Unspecified => { + unreachable!(); + } } } diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 869b7089ac271..c866ed6c3c223 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -24,7 +24,7 @@ pub mod monitor; pub mod nats; pub mod nexmark; pub mod pulsar; -pub use base::*; +pub use base::{UPSTREAM_SOURCE_KEY, *}; pub(crate) use common::*; pub use google_pubsub::GOOGLE_PUBSUB_CONNECTOR; pub use kafka::KAFKA_CONNECTOR; diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index 87b293f64a5e6..d1c08a642c8ca 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -19,6 +19,7 @@ async-trait = "0.1" either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } itertools = "0.11" +rand = "0.8" regex = "1" risingwave_common = { workspace = true } risingwave_connector = { workspace = true } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 6f08ebfb18d17..5f73ffb815117 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -16,9 +16,12 @@ use std::collections::HashMap; use std::sync::Arc; use anyhow::anyhow; +use rand::Rng; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::stream_graph_visitor::visit_fragment; use risingwave_connector::sink::catalog::SinkId; +use risingwave_connector::source::cdc::CdcSourceType; +use risingwave_connector::source::UPSTREAM_SOURCE_KEY; use risingwave_pb::catalog::connection::private_link_service::{ PbPrivateLinkProvider, PrivateLinkProvider, }; @@ -428,6 +431,16 @@ impl DdlService for DdlServiceImpl { // Generate source id. let source_id = self.gen_unique_id::<{ IdCategory::Table }>().await?; // TODO: Use source category fill_table_source(source, source_id, &mut mview, table_id, &mut fragment_graph); + + // Modify properties for cdc sources if needed + if let Some(connector) = source.properties.get(UPSTREAM_SOURCE_KEY) { + if matches!( + CdcSourceType::from(connector.as_str()), + CdcSourceType::Mysql + ) { + fill_cdc_mysql_server_id(&mut fragment_graph); + } + } } let mut stream_job = StreamingJob::Table(source, mview); @@ -827,3 +840,19 @@ fn fill_table_source( table.optional_associated_source_id = Some(OptionalAssociatedSourceId::AssociatedSourceId(source_id)); } + +// `server.id` (in the range from 1 to 2^32 - 1). This value MUST be unique across whole replication +// group (that is, different from any other server id being used by any master or slave) +fn fill_cdc_mysql_server_id(fragment_graph: &mut PbStreamFragmentGraph) { + for fragment in fragment_graph.fragments.values_mut() { + visit_fragment(fragment, |node_body| { + if let NodeBody::Source(source_node) = node_body { + let props = &mut source_node.source_inner.as_mut().unwrap().properties; + let rand_server_id = rand::thread_rng().gen_range(1..u32::MAX); + props + .entry("server.id".to_string()) + .or_insert(rand_server_id.to_string()); + } + }); + } +} diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 733ba6a8c4a83..31ee763d2a0b9 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -155,7 +155,7 @@ impl ConnectorSource { let to_reader_splits = splits.into_iter().map(|split| vec![split]); try_join_all(to_reader_splits.into_iter().map(|splits| { - tracing::debug!("spawning connector split reader for split {:?}", splits); + tracing::debug!(?splits, ?prop, "spawning connector split reader"); let props = prop.clone(); let data_gen_columns = data_gen_columns.clone(); let parser_config = parser_config.clone(); From de8d2172feb03a338f334ce257e7c014c5384a93 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 26 Oct 2023 18:23:14 +0800 Subject: [PATCH 12/52] fix(standalone): hide etcd pwd in logs (#13034) --- Cargo.lock | 7 +++++++ src/cmd_all/src/standalone.rs | 6 +++--- src/meta/node/Cargo.toml | 1 + src/meta/node/src/lib.rs | 9 +++++++-- 4 files changed, 18 insertions(+), 5 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index dbce3dfae9f3b..f03347d622208 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6645,6 +6645,12 @@ dependencies = [ "zstd-sys", ] +[[package]] +name = "redact" +version = "0.1.5" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0b97c0a6319ae55341eb213c8ef97002630a3a5bd6f287f0124d077121d3f2a5" + [[package]] name = "redis" version = "0.23.3" @@ -7753,6 +7759,7 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "prometheus-http-query", + "redact", "regex", "risingwave_common", "risingwave_common_heap_profiling", diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index 8ebe2c7112c49..15a46d789ff3d 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -195,7 +195,7 @@ mod test { // Test parsing into standalone-level opts. let raw_opts = " --compute-opts=--listen-addr 127.0.0.1:8000 --total-memory-bytes 34359738368 --parallelism 10 ---meta-opts=--advertise-addr 127.0.0.1:9999 --data-directory \"some path with spaces\" --listen-addr 127.0.0.1:8001 +--meta-opts=--advertise-addr 127.0.0.1:9999 --data-directory \"some path with spaces\" --listen-addr 127.0.0.1:8001 --etcd-password 1234 --frontend-opts=--config-path=src/config/original.toml --prometheus-listener-addr=127.0.0.1:1234 --config-path=src/config/test.toml @@ -203,7 +203,7 @@ mod test { let actual = StandaloneOpts::parse_from(raw_opts.lines()); let opts = StandaloneOpts { compute_opts: Some("--listen-addr 127.0.0.1:8000 --total-memory-bytes 34359738368 --parallelism 10".into()), - meta_opts: Some("--advertise-addr 127.0.0.1:9999 --data-directory \"some path with spaces\" --listen-addr 127.0.0.1:8001".into()), + meta_opts: Some("--advertise-addr 127.0.0.1:9999 --data-directory \"some path with spaces\" --listen-addr 127.0.0.1:8001 --etcd-password 1234".into()), frontend_opts: Some("--config-path=src/config/original.toml".into()), compactor_opts: None, prometheus_listener_addr: Some("127.0.0.1:1234".into()), @@ -228,7 +228,7 @@ mod test { etcd_endpoints: "", etcd_auth: false, etcd_username: "", - etcd_password: "", + etcd_password: [REDACTED alloc::string::String], sql_endpoint: None, dashboard_ui_path: None, prometheus_endpoint: None, diff --git a/src/meta/node/Cargo.toml b/src/meta/node/Cargo.toml index 84793a74591c8..dcfa053fdfc4b 100644 --- a/src/meta/node/Cargo.toml +++ b/src/meta/node/Cargo.toml @@ -21,6 +21,7 @@ etcd-client = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } itertools = "0.11" prometheus-http-query = "0.7" +redact = "0.1.5" regex = "1" risingwave_common = { workspace = true } risingwave_common_heap_profiling = { workspace = true } diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index bf1bddad2070f..4be2bba4039b2 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -17,10 +17,12 @@ #![cfg_attr(coverage, feature(coverage_attribute))] mod server; + use std::time::Duration; use clap::Parser; pub use error::{MetaError, MetaResult}; +use redact::Secret; use risingwave_common::config::OverrideConfig; use risingwave_common::util::resource_util; use risingwave_common::{GIT_SHA, RW_VERSION}; @@ -71,7 +73,7 @@ pub struct MetaNodeOpts { /// Password of etcd, required when --etcd-auth is enabled. #[clap(long, env = "RW_ETCD_PASSWORD", default_value = "")] - etcd_password: String, + etcd_password: Secret, /// Endpoint of the SQL service, make it non-option when SQL service is required. #[clap(long, env = "RW_SQL_ENDPOINT")] @@ -196,7 +198,10 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { .map(|x| x.to_string()) .collect(), credentials: match opts.etcd_auth { - true => Some((opts.etcd_username, opts.etcd_password)), + true => Some(( + opts.etcd_username, + opts.etcd_password.expose_secret().to_string(), + )), false => None, }, }, From 46d2ff9d9ba9a501b37bd15dd144333c4bf0913f Mon Sep 17 00:00:00 2001 From: August Date: Thu, 26 Oct 2023 18:26:16 +0800 Subject: [PATCH 13/52] feat: use bit flags for user privilege check (#13033) --- Cargo.lock | 21 +++ proto/user.proto | 1 + src/common/Cargo.toml | 1 + src/common/src/acl/mod.rs | 144 +++++++++++++++ src/common/src/lib.rs | 1 + .../src/catalog/system_catalog/mod.rs | 49 ++--- src/frontend/src/handler/alter_user.rs | 7 +- src/frontend/src/handler/create_index.rs | 5 +- src/frontend/src/handler/create_mv.rs | 4 +- src/frontend/src/handler/create_schema.rs | 5 +- src/frontend/src/handler/create_user.rs | 3 +- src/frontend/src/handler/drop_user.rs | 10 +- src/frontend/src/handler/handle_privilege.rs | 22 +-- src/frontend/src/handler/privilege.rs | 51 +++--- src/frontend/src/session.rs | 14 +- src/frontend/src/test_utils.rs | 35 +--- src/frontend/src/user/mod.rs | 1 + src/frontend/src/user/user_catalog.rs | 171 ++++++++++++++++++ src/frontend/src/user/user_manager.rs | 17 +- src/frontend/src/user/user_privilege.rs | 90 ++++----- 20 files changed, 465 insertions(+), 187 deletions(-) create mode 100644 src/common/src/acl/mod.rs create mode 100644 src/frontend/src/user/user_catalog.rs diff --git a/Cargo.lock b/Cargo.lock index f03347d622208..4e969a94fc134 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2735,6 +2735,26 @@ dependencies = [ "syn 2.0.37", ] +[[package]] +name = "enumflags2" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5998b4f30320c9d93aed72f63af821bfdac50465b75428fce77b48ec482c3939" +dependencies = [ + "enumflags2_derive", +] + +[[package]] +name = "enumflags2_derive" +version = "0.7.8" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f95e2801cd355d4a1a3e3953ce6ee5ae9603a5c833455343a8bfe3f44d418246" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.37", +] + [[package]] name = "env_logger" version = "0.10.0" @@ -7068,6 +7088,7 @@ dependencies = [ "educe", "either", "enum-as-inner", + "enumflags2", "ethnum", "fixedbitset", "fs-err", diff --git a/proto/user.proto b/proto/user.proto index 7468147cad271..c998f66d15133 100644 --- a/proto/user.proto +++ b/proto/user.proto @@ -43,6 +43,7 @@ message GrantPrivilege { DELETE = 4; CREATE = 5; CONNECT = 6; + USAGE = 7; } message ActionWithGrantOption { diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 168ba836d4c1b..f44c0c9ba8a5d 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -38,6 +38,7 @@ easy-ext = "1" educe = "0.4" either = "1" enum-as-inner = "0.6" +enumflags2 = { version = "0.7.8" } ethnum = { version = "1", features = ["serde"] } fixedbitset = { version = "0.4", features = ["std"] } fs-err = "2" diff --git a/src/common/src/acl/mod.rs b/src/common/src/acl/mod.rs new file mode 100644 index 0000000000000..929577437571d --- /dev/null +++ b/src/common/src/acl/mod.rs @@ -0,0 +1,144 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//! `Acl` defines all grantable privileges. + +use std::convert::Into; +use std::fmt::Formatter; +use std::sync::LazyLock; + +use enumflags2::{bitflags, make_bitflags, BitFlags}; +use parse_display::Display; +use risingwave_pb::user::grant_privilege::PbAction; + +#[bitflags] +#[repr(u64)] +#[derive(Clone, Copy, Debug, Display, Eq, PartialEq)] +pub enum AclMode { + #[display("a")] + Insert = 1 << 0, // formerly known as "append". + #[display("r")] + Select = 1 << 1, // formerly known as "read". + #[display("w")] + Update = 1 << 2, // formerly known as "write". + #[display("d")] + Delete = 1 << 3, + #[display("D")] + Truncate = 1 << 4, // super-delete, as it were + #[display("x")] + References = 1 << 5, + #[display("t")] + Trigger = 1 << 6, + #[display("X")] + Execute = 1 << 7, // For functions + #[display("U")] + Usage = 1 << 8, // For various object types + #[display("C")] + Create = 1 << 9, // For namespaces and databases + #[display("T")] + CreateTemp = 1 << 10, // For databases + #[display("c")] + Connect = 1 << 11, // For databases + #[display("s")] + Set = 1 << 12, // For configuration parameters + #[display("A")] + AlterSystem = 1 << 13, // For configuration parameters + #[display("m")] + Maintain = 1 << 14, // For relations +} + +impl From for AclMode { + fn from(action: PbAction) -> Self { + match action { + PbAction::Unspecified => unreachable!(), + PbAction::Select => AclMode::Select, + PbAction::Insert => AclMode::Insert, + PbAction::Update => AclMode::Update, + PbAction::Delete => AclMode::Delete, + PbAction::Create => AclMode::Create, + PbAction::Connect => AclMode::Connect, + PbAction::Usage => AclMode::Usage, + } + } +} + +impl From for PbAction { + fn from(val: AclMode) -> Self { + match val { + AclMode::Select => PbAction::Select, + AclMode::Insert => PbAction::Insert, + AclMode::Update => PbAction::Update, + AclMode::Delete => PbAction::Delete, + AclMode::Create => PbAction::Create, + AclMode::Connect => PbAction::Connect, + AclMode::Usage => PbAction::Usage, + _ => unreachable!(), + } + } +} + +/// `AclModeSet` defines a set of `AclMode`s. +#[derive(Clone, Debug)] +pub struct AclModeSet { + pub modes: BitFlags, +} + +pub static ALL_AVAILABLE_DATABASE_MODES: LazyLock = + LazyLock::new(|| make_bitflags!(AclMode::{Create | Connect}).into()); +pub static ALL_AVAILABLE_SCHEMA_MODES: LazyLock = + LazyLock::new(|| make_bitflags!(AclMode::{Create | Usage}).into()); +pub static ALL_AVAILABLE_TABLE_MODES: LazyLock = + LazyLock::new(|| make_bitflags!(AclMode::{Select | Insert | Update | Delete}).into()); +pub static ALL_AVAILABLE_SOURCE_MODES: LazyLock = LazyLock::new(AclModeSet::readonly); +pub static ALL_AVAILABLE_MVIEW_MODES: LazyLock = LazyLock::new(AclModeSet::readonly); +pub static ALL_AVAILABLE_VIEW_MODES: LazyLock = LazyLock::new(AclModeSet::readonly); +pub static ALL_AVAILABLE_SINK_MODES: LazyLock = LazyLock::new(AclModeSet::empty); +pub static ALL_AVAILABLE_FUNCTION_MODES: LazyLock = + LazyLock::new(|| BitFlags::from(AclMode::Execute).into()); +pub static ALL_AVAILABLE_CONNECTION_MODES: LazyLock = + LazyLock::new(|| BitFlags::from(AclMode::Usage).into()); + +impl AclModeSet { + pub fn empty() -> Self { + Self { + modes: BitFlags::empty(), + } + } + + pub fn readonly() -> Self { + Self { + modes: BitFlags::from(AclMode::Select), + } + } + + pub fn has_mode(&self, mode: AclMode) -> bool { + self.modes.contains(mode) + } + + pub fn iter(&self) -> impl Iterator + '_ { + self.modes.iter() + } +} + +impl From> for AclModeSet { + fn from(modes: BitFlags) -> Self { + Self { modes } + } +} + +impl std::fmt::Display for AclModeSet { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "{}", self.modes) + } +} diff --git a/src/common/src/lib.rs b/src/common/src/lib.rs index fbcd3854fa572..408c8823d397f 100644 --- a/src/common/src/lib.rs +++ b/src/common/src/lib.rs @@ -70,6 +70,7 @@ pub mod system_param; pub mod telemetry; pub mod transaction; +pub mod acl; pub mod metrics; pub mod test_utils; pub mod types; diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index d64db79b8ced1..656960bffb906 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -21,6 +21,7 @@ use std::sync::{Arc, LazyLock}; use async_trait::async_trait; use itertools::Itertools; +use risingwave_common::acl::AclMode; use risingwave_common::catalog::{ ColumnCatalog, ColumnDesc, Field, SysCatalogReader, TableDesc, TableId, DEFAULT_SUPER_USER_ID, NON_RESERVED_SYS_CATALOG_ID, @@ -28,8 +29,7 @@ use risingwave_common::catalog::{ use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::DataType; -use risingwave_pb::user::grant_privilege::{Action, Object}; -use risingwave_pb::user::UserInfo; +use risingwave_pb::user::grant_privilege::Object; use crate::catalog::catalog_service::CatalogReader; use crate::catalog::system_catalog::information_schema::*; @@ -39,6 +39,7 @@ use crate::catalog::view_catalog::ViewCatalog; use crate::meta_client::FrontendMetaClient; use crate::scheduler::worker_node_manager::WorkerNodeManagerRef; use crate::session::AuthContext; +use crate::user::user_catalog::UserCatalog; use crate::user::user_privilege::available_prost_privilege; use crate::user::user_service::UserInfoReader; use crate::user::UserId; @@ -212,17 +213,17 @@ fn infer_dummy_view_sql(columns: &[SystemCatalogColumnsDef<'_>]) -> String { fn get_acl_items( object: &Object, for_dml_table: bool, - users: &Vec, + users: &Vec, username_map: &HashMap, ) -> String { let mut res = String::from("{"); let mut empty_flag = true; let super_privilege = available_prost_privilege(object.clone(), for_dml_table); for user in users { - let privileges = if user.get_is_super() { + let privileges = if user.is_super { vec![&super_privilege] } else { - user.get_grant_privileges() + user.grant_privileges .iter() .filter(|&privilege| privilege.object.as_ref().unwrap() == object) .collect_vec() @@ -233,43 +234,29 @@ fn get_acl_items( let mut grantor_map = HashMap::new(); privileges.iter().for_each(|&privilege| { privilege.action_with_opts.iter().for_each(|ao| { - grantor_map.entry(ao.granted_by).or_insert_with(Vec::new); grantor_map - .get_mut(&ao.granted_by) - .unwrap() - .push((ao.action, ao.with_grant_option)); + .entry(ao.granted_by) + .or_insert_with(Vec::new) + .push((ao.get_action().unwrap(), ao.with_grant_option)); }) }); - for key in grantor_map.keys() { + for (granted_by, actions) in grantor_map { if empty_flag { empty_flag = false; } else { res.push(','); } - res.push_str(user.get_name()); + res.push_str(&user.name); res.push('='); - grantor_map - .get(key) - .unwrap() - .iter() - .for_each(|(action, option)| { - let str = match Action::try_from(*action).unwrap() { - Action::Select => "r", - Action::Insert => "a", - Action::Update => "w", - Action::Delete => "d", - Action::Create => "C", - Action::Connect => "c", - _ => unreachable!(), - }; - res.push_str(str); - if *option { - res.push('*'); - } - }); + for (action, option) in actions { + res.push_str(&AclMode::from(action).to_string()); + if option { + res.push('*'); + } + } res.push('/'); // should be able to query grantor's name - res.push_str(username_map.get(key).as_ref().unwrap()); + res.push_str(username_map.get(&granted_by).unwrap()); } } res.push('}'); diff --git a/src/frontend/src/handler/alter_user.rs b/src/frontend/src/handler/alter_user.rs index 5c91df3888b71..0d83c3ae867d5 100644 --- a/src/frontend/src/handler/alter_user.rs +++ b/src/frontend/src/handler/alter_user.rs @@ -24,11 +24,12 @@ use crate::binder::Binder; use crate::catalog::CatalogError; use crate::handler::HandlerArgs; use crate::user::user_authentication::encrypted_password; +use crate::user::user_catalog::UserCatalog; fn alter_prost_user_info( mut user_info: UserInfo, options: &UserOptions, - session_user: &UserInfo, + session_user: &UserCatalog, ) -> Result<(UserInfo, Vec)> { if !session_user.is_super { let require_super = user_info.is_super @@ -116,7 +117,7 @@ fn alter_prost_user_info( fn alter_rename_prost_user_info( mut user_info: UserInfo, new_name: ObjectName, - session_user: &UserInfo, + session_user: &UserCatalog, ) -> Result<(UserInfo, Vec)> { if session_user.id == user_info.id { return Err(InternalError("session user cannot be renamed".to_string()).into()); @@ -153,7 +154,7 @@ pub async fn handle_alter_user( let old_info = user_reader .get_user_by_name(&user_name) .ok_or(CatalogError::NotFound("user", user_name))? - .clone(); + .to_prost(); let session_user = user_reader .get_user_by_name(session.user_name()) diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index a5a002d3b3d79..bed35eadec9ae 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -18,12 +18,13 @@ use std::rc::Rc; use fixedbitset::FixedBitSet; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::acl::AclMode; use risingwave_common::catalog::{IndexId, TableDesc, TableId}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_pb::catalog::{PbIndex, PbStreamJobStatus, PbTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; -use risingwave_pb::user::grant_privilege::{Action, Object}; +use risingwave_pb::user::grant_privilege::Object; use risingwave_sqlparser::ast; use risingwave_sqlparser::ast::{Ident, ObjectName, OrderByExpr}; @@ -74,7 +75,7 @@ pub(crate) fn gen_create_index_plan( session.check_privileges(&[ObjectCheckItem::new( table.owner, - Action::Select, + AclMode::Select, Object::TableId(table.id.table_id), )])?; diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 053ba5aa30f19..75474ca576dd8 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -14,10 +14,10 @@ use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::acl::AclMode; use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::catalog::{CreateType, PbTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; -use risingwave_pb::user::grant_privilege::Action; use risingwave_sqlparser::ast::{EmitMode, Ident, ObjectName, Query}; use super::privilege::resolve_relation_privileges; @@ -66,7 +66,7 @@ pub(super) fn get_column_names( } if let Some(relation) = &select.from { let mut check_items = Vec::new(); - resolve_relation_privileges(relation, Action::Select, &mut check_items); + resolve_relation_privileges(relation, AclMode::Select, &mut check_items); session.check_privileges(&check_items)?; } } diff --git a/src/frontend/src/handler/create_schema.rs b/src/frontend/src/handler/create_schema.rs index 12970193fe33e..962d59178bf0c 100644 --- a/src/frontend/src/handler/create_schema.rs +++ b/src/frontend/src/handler/create_schema.rs @@ -13,9 +13,10 @@ // limitations under the License. use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::acl::AclMode; use risingwave_common::catalog::RESERVED_PG_SCHEMA_PREFIX; use risingwave_common::error::{ErrorCode, Result}; -use risingwave_pb::user::grant_privilege::{Action, Object}; +use risingwave_pb::user::grant_privilege::Object; use risingwave_sqlparser::ast::ObjectName; use super::RwPgResponse; @@ -63,7 +64,7 @@ pub async fn handle_create_schema( session.check_privileges(&[ObjectCheckItem::new( db_owner, - Action::Create, + AclMode::Create, Object::DatabaseId(db_id), )])?; diff --git a/src/frontend/src/handler/create_user.rs b/src/frontend/src/handler/create_user.rs index a4f8eaecc405e..8659e1b647c33 100644 --- a/src/frontend/src/handler/create_user.rs +++ b/src/frontend/src/handler/create_user.rs @@ -24,11 +24,12 @@ use crate::binder::Binder; use crate::catalog::{CatalogError, DatabaseId}; use crate::handler::HandlerArgs; use crate::user::user_authentication::encrypted_password; +use crate::user::user_catalog::UserCatalog; fn make_prost_user_info( user_name: String, options: &UserOptions, - session_user: &UserInfo, + session_user: &UserCatalog, database_id: DatabaseId, ) -> Result { if !session_user.is_super { diff --git a/src/frontend/src/handler/drop_user.rs b/src/frontend/src/handler/drop_user.rs index 11232faa64a95..5b0ae7d55596c 100644 --- a/src/frontend/src/handler/drop_user.rs +++ b/src/frontend/src/handler/drop_user.rs @@ -34,14 +34,14 @@ pub async fn handle_drop_user( let user_name = Binder::resolve_user_name(user_name)?; let user_info_reader = session.env().user_info_reader(); - let user = user_info_reader + let user_id = user_info_reader .read_guard() .get_user_by_name(&user_name) - .cloned(); - match user { - Some(user) => { + .map(|u| u.id); + match user_id { + Some(user_id) => { let user_info_writer = session.user_info_writer()?; - user_info_writer.drop_user(user.id).await?; + user_info_writer.drop_user(user_id).await?; } None => { return if if_exists { diff --git a/src/frontend/src/handler/handle_privilege.rs b/src/frontend/src/handler/handle_privilege.rs index 2f166e9fa1824..07b87fa3bc3d2 100644 --- a/src/frontend/src/handler/handle_privilege.rs +++ b/src/frontend/src/handler/handle_privilege.rs @@ -39,7 +39,10 @@ fn make_prost_privilege( let reader = catalog_reader.read_guard(); let actions = match privileges { Privileges::All { .. } => available_privilege_actions(&objects)?, - Privileges::Actions(actions) => actions, + Privileges::Actions(actions) => actions + .into_iter() + .map(|action| get_prost_action(&action)) + .collect(), }; let mut grant_objs = vec![]; match objects { @@ -147,14 +150,11 @@ fn make_prost_privilege( } }; let action_with_opts = actions - .iter() - .map(|action| { - let prost_action = get_prost_action(action); - ActionWithGrantOption { - action: prost_action as i32, - granted_by: session.user_id(), - ..Default::default() - } + .into_iter() + .map(|action| ActionWithGrantOption { + action: action as i32, + granted_by: session.user_id(), + ..Default::default() }) .collect::>(); @@ -318,12 +318,12 @@ mod tests { PbGrantPrivilege { action_with_opts: vec![ ActionWithGrantOption { - action: Action::Connect as i32, + action: Action::Create as i32, with_grant_option: true, granted_by: DEFAULT_SUPER_USER_ID, }, ActionWithGrantOption { - action: Action::Create as i32, + action: Action::Connect as i32, with_grant_option: true, granted_by: DEFAULT_SUPER_USER_ID, } diff --git a/src/frontend/src/handler/privilege.rs b/src/frontend/src/handler/privilege.rs index dcd8696b9670f..24227a46ff2a6 100644 --- a/src/frontend/src/handler/privilege.rs +++ b/src/frontend/src/handler/privilege.rs @@ -12,9 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::acl::AclMode; use risingwave_common::error::ErrorCode::PermissionDenied; use risingwave_common::error::Result; -use risingwave_pb::user::grant_privilege::{PbAction, PbObject}; +use risingwave_pb::user::grant_privilege::PbObject; use crate::binder::{BoundQuery, BoundStatement, Relation}; use crate::catalog::OwnedByUserCatalog; @@ -24,15 +25,16 @@ use crate::user::UserId; #[derive(Debug)] pub struct ObjectCheckItem { owner: UserId, - action: PbAction, + mode: AclMode, + // todo: change it to object id. object: PbObject, } impl ObjectCheckItem { - pub fn new(owner: UserId, action: PbAction, object: PbObject) -> Self { + pub fn new(owner: UserId, mode: AclMode, object: PbObject) -> Self { Self { owner, - action, + mode, object, } } @@ -41,14 +43,14 @@ impl ObjectCheckItem { /// resolve privileges in `relation` pub(crate) fn resolve_relation_privileges( relation: &Relation, - action: PbAction, + mode: AclMode, objects: &mut Vec, ) { match relation { Relation::Source(source) => { let item = ObjectCheckItem { owner: source.catalog.owner, - action, + mode, object: PbObject::SourceId(source.catalog.id), }; objects.push(item); @@ -56,7 +58,7 @@ pub(crate) fn resolve_relation_privileges( Relation::BaseTable(table) => { let item = ObjectCheckItem { owner: table.table_catalog.owner, - action, + mode, object: PbObject::TableId(table.table_id.table_id), }; objects.push(item); @@ -64,16 +66,16 @@ pub(crate) fn resolve_relation_privileges( Relation::Subquery(query) => { if let crate::binder::BoundSetExpr::Select(select) = &query.query.body { if let Some(sub_relation) = &select.from { - resolve_relation_privileges(sub_relation, action, objects); + resolve_relation_privileges(sub_relation, mode, objects); } } } Relation::Join(join) => { - resolve_relation_privileges(&join.left, action, objects); - resolve_relation_privileges(&join.right, action, objects); + resolve_relation_privileges(&join.left, mode, objects); + resolve_relation_privileges(&join.right, mode, objects); } Relation::WindowTableFunction(table) => { - resolve_relation_privileges(&table.input, action, objects) + resolve_relation_privileges(&table.input, mode, objects) } _ => {} }; @@ -86,20 +88,20 @@ pub(crate) fn resolve_privileges(stmt: &BoundStatement) -> Vec BoundStatement::Insert(ref insert) => { let object = ObjectCheckItem { owner: insert.owner, - action: PbAction::Insert, + mode: AclMode::Insert, object: PbObject::TableId(insert.table_id.table_id), }; objects.push(object); if let crate::binder::BoundSetExpr::Select(select) = &insert.source.body { if let Some(sub_relation) = &select.from { - resolve_relation_privileges(sub_relation, PbAction::Select, &mut objects); + resolve_relation_privileges(sub_relation, AclMode::Select, &mut objects); } } } BoundStatement::Delete(ref delete) => { let object = ObjectCheckItem { owner: delete.owner, - action: PbAction::Delete, + mode: AclMode::Delete, object: PbObject::TableId(delete.table_id.table_id), }; objects.push(object); @@ -107,7 +109,7 @@ pub(crate) fn resolve_privileges(stmt: &BoundStatement) -> Vec BoundStatement::Update(ref update) => { let object = ObjectCheckItem { owner: update.owner, - action: PbAction::Update, + mode: AclMode::Update, object: PbObject::TableId(update.table_id.table_id), }; objects.push(object); @@ -122,7 +124,7 @@ pub(crate) fn resolve_query_privileges(query: &BoundQuery) -> Vec Result<()> { let mut lock = self.user_info.write(); let id = update_user.get_id(); - let old_name = lock.get_user_name_by_id(id).unwrap(); - let mut user_info = lock.get_user_by_name(&old_name).unwrap().clone(); + let Some(old_name) = lock.get_user_name_by_id(id) else { + return Ok(()); + }; + let mut user_info = lock.get_user_by_name(&old_name).unwrap().to_prost(); update_fields.into_iter().for_each(|field| match field { UpdateField::Super => user_info.is_super = update_user.is_super, UpdateField::Login => user_info.can_login = update_user.can_login, @@ -679,7 +681,7 @@ impl UserInfoWriter for MockUserInfoWriter { .collect::>(); for user_id in users { if let Some(u) = self.user_info.write().get_user_mut(user_id) { - u.grant_privileges.extend(privileges.clone()); + u.extend_privileges(privileges.clone()); } } Ok(()) @@ -698,32 +700,7 @@ impl UserInfoWriter for MockUserInfoWriter { ) -> Result<()> { for user_id in users { if let Some(u) = self.user_info.write().get_user_mut(user_id) { - u.grant_privileges.iter_mut().for_each(|p| { - for rp in &privileges { - if rp.object != p.object { - continue; - } - if revoke_grant_option { - for ao in &mut p.action_with_opts { - if rp - .action_with_opts - .iter() - .any(|rao| rao.action == ao.action) - { - ao.with_grant_option = false; - } - } - } else { - p.action_with_opts.retain(|po| { - rp.action_with_opts - .iter() - .all(|rao| rao.action != po.action) - }); - } - } - }); - u.grant_privileges - .retain(|p| !p.action_with_opts.is_empty()); + u.revoke_privileges(privileges.clone(), revoke_grant_option); } } Ok(()) diff --git a/src/frontend/src/user/mod.rs b/src/frontend/src/user/mod.rs index c0e23f2a69d92..6dfec976ac7f8 100644 --- a/src/frontend/src/user/mod.rs +++ b/src/frontend/src/user/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. pub(crate) mod user_authentication; +pub(crate) mod user_catalog; pub(crate) mod user_manager; pub mod user_privilege; pub(crate) mod user_service; diff --git a/src/frontend/src/user/user_catalog.rs b/src/frontend/src/user/user_catalog.rs new file mode 100644 index 0000000000000..e1b5f85d446b1 --- /dev/null +++ b/src/frontend/src/user/user_catalog.rs @@ -0,0 +1,171 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::hash_map::Entry; +use std::collections::HashMap; + +use risingwave_common::acl::{AclMode, AclModeSet}; +use risingwave_pb::user::grant_privilege::{Object as GrantObject, Object}; +use risingwave_pb::user::{PbAuthInfo, PbGrantPrivilege, PbUserInfo}; + +use crate::catalog::{DatabaseId, SchemaId}; +use crate::user::UserId; + +/// `UserCatalog` is responsible for managing user's information. +#[derive(Clone, Debug)] +pub struct UserCatalog { + pub id: UserId, + pub name: String, + pub is_super: bool, + pub can_create_db: bool, + pub can_create_user: bool, + pub can_login: bool, + pub auth_info: Option, + pub grant_privileges: Vec, + + // User owned acl mode set, group by object id. + // TODO: merge it after we fully migrate to sql-backend. + pub database_acls: HashMap, + pub schema_acls: HashMap, + pub object_acls: HashMap, +} + +impl From for UserCatalog { + fn from(user: PbUserInfo) -> Self { + let mut user_catalog = Self { + id: user.id, + name: user.name, + is_super: user.is_super, + can_create_db: user.can_create_db, + can_create_user: user.can_create_user, + can_login: user.can_login, + auth_info: user.auth_info, + grant_privileges: user.grant_privileges, + database_acls: Default::default(), + schema_acls: Default::default(), + object_acls: Default::default(), + }; + user_catalog.refresh_acl_modes(); + + user_catalog + } +} + +impl UserCatalog { + pub fn to_prost(&self) -> PbUserInfo { + PbUserInfo { + id: self.id, + name: self.name.clone(), + is_super: self.is_super, + can_create_db: self.can_create_db, + can_create_user: self.can_create_user, + can_login: self.can_login, + auth_info: self.auth_info.clone(), + grant_privileges: self.grant_privileges.clone(), + } + } + + fn get_acl_entry(&mut self, object: GrantObject) -> Entry<'_, u32, AclModeSet> { + match object { + Object::DatabaseId(id) => self.database_acls.entry(id), + Object::SchemaId(id) => self.schema_acls.entry(id), + Object::TableId(id) => self.object_acls.entry(id), + Object::SourceId(id) => self.object_acls.entry(id), + Object::SinkId(id) => self.object_acls.entry(id), + Object::ViewId(id) => self.object_acls.entry(id), + Object::FunctionId(_) => { + unreachable!("grant privilege on function is not supported yet.") + } + _ => unreachable!(""), + } + } + + fn get_acl(&self, object: &GrantObject) -> Option<&AclModeSet> { + match object { + Object::DatabaseId(id) => self.database_acls.get(id), + Object::SchemaId(id) => self.schema_acls.get(id), + Object::TableId(id) => self.object_acls.get(id), + Object::SourceId(id) => self.object_acls.get(id), + Object::SinkId(id) => self.object_acls.get(id), + Object::ViewId(id) => self.object_acls.get(id), + Object::FunctionId(_) => { + unreachable!("grant privilege on function is not supported yet.") + } + _ => unreachable!("unexpected object type."), + } + } + + fn refresh_acl_modes(&mut self) { + self.database_acls.clear(); + self.schema_acls.clear(); + self.object_acls.clear(); + let privileges = self.grant_privileges.clone(); + for privilege in privileges { + let entry = self + .get_acl_entry(privilege.object.unwrap()) + .or_insert(AclModeSet::empty()); + for awo in privilege.action_with_opts { + entry + .modes + .insert::(awo.get_action().unwrap().into()); + } + } + } + + // Only for test, used in `MockUserInfoWriter`. + pub fn extend_privileges(&mut self, privileges: Vec) { + self.grant_privileges.extend(privileges); + self.refresh_acl_modes(); + } + + // Only for test, used in `MockUserInfoWriter`. + pub fn revoke_privileges( + &mut self, + privileges: Vec, + revoke_grant_option: bool, + ) { + self.grant_privileges.iter_mut().for_each(|p| { + for rp in &privileges { + if rp.object != p.object { + continue; + } + if revoke_grant_option { + for ao in &mut p.action_with_opts { + if rp + .action_with_opts + .iter() + .any(|rao| rao.action == ao.action) + { + ao.with_grant_option = false; + } + } + } else { + p.action_with_opts.retain(|po| { + rp.action_with_opts + .iter() + .all(|rao| rao.action != po.action) + }); + } + } + }); + self.grant_privileges + .retain(|p| !p.action_with_opts.is_empty()); + self.refresh_acl_modes(); + } + + pub fn check_privilege(&self, object: &GrantObject, mode: AclMode) -> bool { + self.get_acl(object) + .map_or(false, |acl_set| acl_set.has_mode(mode)) + } +} diff --git a/src/frontend/src/user/user_manager.rs b/src/frontend/src/user/user_manager.rs index 3620eef51114a..d42c764c0b5ed 100644 --- a/src/frontend/src/user/user_manager.rs +++ b/src/frontend/src/user/user_manager.rs @@ -17,12 +17,13 @@ use std::collections::HashMap; use itertools::Itertools; use risingwave_pb::user::{GrantPrivilege, UserInfo}; +use crate::user::user_catalog::UserCatalog; use crate::user::{UserId, UserInfoVersion}; /// `UserInfoManager` is responsible for managing users. pub struct UserInfoManager { version: UserInfoVersion, - user_by_name: HashMap, + user_by_name: HashMap, user_name_by_id: HashMap, } @@ -38,16 +39,16 @@ impl Default for UserInfoManager { } impl UserInfoManager { - pub fn get_user_mut(&mut self, id: UserId) -> Option<&mut UserInfo> { + pub fn get_user_mut(&mut self, id: UserId) -> Option<&mut UserCatalog> { let name = self.user_name_by_id.get(&id)?; self.user_by_name.get_mut(name) } - pub fn get_all_users(&self) -> Vec { + pub fn get_all_users(&self) -> Vec { self.user_by_name.values().cloned().collect_vec() } - pub fn get_user_by_name(&self, user_name: &str) -> Option<&UserInfo> { + pub fn get_user_by_name(&self, user_name: &str) -> Option<&UserCatalog> { self.user_by_name.get(user_name) } @@ -63,7 +64,7 @@ impl UserInfoManager { let id = user_info.id; let name = user_info.name.clone(); self.user_by_name - .try_insert(name.clone(), user_info) + .try_insert(name.clone(), user_info.into()) .unwrap(); self.user_name_by_id.try_insert(id, name).unwrap(); } @@ -78,9 +79,11 @@ impl UserInfoManager { let name = user_info.name.clone(); if let Some(old_name) = self.get_user_name_by_id(id) { self.user_by_name.remove(&old_name); - self.user_by_name.insert(name.clone(), user_info); + self.user_by_name.insert(name.clone(), user_info.into()); } else { - self.user_by_name.insert(name.clone(), user_info).unwrap(); + self.user_by_name + .insert(name.clone(), user_info.into()) + .unwrap(); } self.user_name_by_id.insert(id, name).unwrap(); } diff --git a/src/frontend/src/user/user_privilege.rs b/src/frontend/src/user/user_privilege.rs index 69ea6ce6830f0..104f1b28aa5be 100644 --- a/src/frontend/src/user/user_privilege.rs +++ b/src/frontend/src/user/user_privilege.rs @@ -13,54 +13,23 @@ // limitations under the License. use itertools::Itertools; +use risingwave_common::acl; +use risingwave_common::acl::{AclMode, AclModeSet}; use risingwave_common::catalog::DEFAULT_SUPER_USER_ID; use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::user::grant_privilege::{ActionWithGrantOption, PbAction, PbObject}; use risingwave_pb::user::PbGrantPrivilege; use risingwave_sqlparser::ast::{Action, GrantObjects, Privileges}; -// TODO: add user_privilege mod under user manager and move check and expand logic there, and bitmap -// impl for privilege check. -static AVAILABLE_ACTION_ON_DATABASE: &[Action] = &[Action::Connect, Action::Create]; -static AVAILABLE_ACTION_ON_SCHEMA: &[Action] = &[Action::Create]; - -static AVAILABLE_ACTION_ON_TABLE: &[Action] = &[ - Action::Select { columns: None }, - Action::Update { columns: None }, - Action::Insert { columns: None }, - Action::Delete, -]; -static AVAILABLE_ACTION_ON_MVIEW: &[Action] = &[Action::Select { columns: None }]; -static AVAILABLE_ACTION_ON_VIEW: &[Action] = AVAILABLE_ACTION_ON_MVIEW; -static AVAILABLE_ACTION_ON_SOURCE: &[Action] = AVAILABLE_ACTION_ON_MVIEW; -static AVAILABLE_ACTION_ON_SINK: &[Action] = &[]; -static AVAILABLE_ACTION_ON_FUNCTION: &[Action] = &[]; - pub fn check_privilege_type(privilege: &Privileges, objects: &GrantObjects) -> Result<()> { match privilege { Privileges::All { .. } => Ok(()), Privileges::Actions(actions) => { - let valid = match objects { - GrantObjects::Databases(_) => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_DATABASE.contains(action)), - GrantObjects::Schemas(_) => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_SCHEMA.contains(action)), - GrantObjects::Sources(_) | GrantObjects::AllSourcesInSchema { .. } => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_SOURCE.contains(action)), - GrantObjects::Mviews(_) | GrantObjects::AllMviewsInSchema { .. } => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_MVIEW.contains(action)), - GrantObjects::Tables(_) | GrantObjects::AllTablesInSchema { .. } => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_TABLE.contains(action)), - GrantObjects::Sinks(_) => actions - .iter() - .all(|action| AVAILABLE_ACTION_ON_SINK.contains(action)), - GrantObjects::Sequences(_) | GrantObjects::AllSequencesInSchema { .. } => true, - }; + let acl_sets = get_all_available_modes(objects)?; + let valid = actions + .iter() + .map(get_prost_action) + .all(|action| acl_sets.has_mode(action.into())); if !valid { return Err(ErrorCode::BindError( "Invalid privilege type for the given object.".to_string(), @@ -73,25 +42,31 @@ pub fn check_privilege_type(privilege: &Privileges, objects: &GrantObjects) -> R } } -pub fn available_privilege_actions(objects: &GrantObjects) -> Result> { - match objects { - GrantObjects::Databases(_) => Ok(AVAILABLE_ACTION_ON_DATABASE.to_vec()), - GrantObjects::Schemas(_) => Ok(AVAILABLE_ACTION_ON_SCHEMA.to_vec()), +fn get_all_available_modes(object: &GrantObjects) -> Result<&AclModeSet> { + match object { + GrantObjects::Databases(_) => Ok(&acl::ALL_AVAILABLE_DATABASE_MODES), + GrantObjects::Schemas(_) => Ok(&acl::ALL_AVAILABLE_SCHEMA_MODES), GrantObjects::Sources(_) | GrantObjects::AllSourcesInSchema { .. } => { - Ok(AVAILABLE_ACTION_ON_SOURCE.to_vec()) + Ok(&acl::ALL_AVAILABLE_SOURCE_MODES) } GrantObjects::Mviews(_) | GrantObjects::AllMviewsInSchema { .. } => { - Ok(AVAILABLE_ACTION_ON_MVIEW.to_vec()) + Ok(&acl::ALL_AVAILABLE_MVIEW_MODES) } GrantObjects::Tables(_) | GrantObjects::AllTablesInSchema { .. } => { - Ok(AVAILABLE_ACTION_ON_TABLE.to_vec()) + Ok(&acl::ALL_AVAILABLE_TABLE_MODES) } + GrantObjects::Sinks(_) => Ok(&acl::ALL_AVAILABLE_SINK_MODES), _ => Err( ErrorCode::BindError("Invalid privilege type for the given object.".to_string()).into(), ), } } +pub fn available_privilege_actions(objects: &GrantObjects) -> Result> { + let acl_sets = get_all_available_modes(objects)?; + Ok(acl_sets.iter().map(Into::into).collect_vec()) +} + #[inline(always)] pub fn get_prost_action(action: &Action) -> PbAction { match action { @@ -101,31 +76,32 @@ pub fn get_prost_action(action: &Action) -> PbAction { Action::Delete { .. } => PbAction::Delete, Action::Connect => PbAction::Connect, Action::Create => PbAction::Create, + Action::Usage => PbAction::Usage, _ => unreachable!(), } } pub fn available_prost_privilege(object: PbObject, for_dml_table: bool) -> PbGrantPrivilege { - let actions = match object { - PbObject::DatabaseId(_) => AVAILABLE_ACTION_ON_DATABASE.to_vec(), - PbObject::SchemaId(_) => AVAILABLE_ACTION_ON_SCHEMA.to_vec(), - PbObject::SourceId(_) => AVAILABLE_ACTION_ON_SOURCE.to_vec(), + let acl_set = match object { + PbObject::DatabaseId(_) => &acl::ALL_AVAILABLE_DATABASE_MODES, + PbObject::SchemaId(_) => &acl::ALL_AVAILABLE_SCHEMA_MODES, + PbObject::SourceId(_) => &acl::ALL_AVAILABLE_SOURCE_MODES, PbObject::TableId(_) => { if for_dml_table { - AVAILABLE_ACTION_ON_TABLE.to_vec() + &acl::ALL_AVAILABLE_TABLE_MODES } else { - AVAILABLE_ACTION_ON_MVIEW.to_vec() + &acl::ALL_AVAILABLE_MVIEW_MODES } } - PbObject::ViewId(_) => AVAILABLE_ACTION_ON_VIEW.to_vec(), - PbObject::SinkId(_) => AVAILABLE_ACTION_ON_SINK.to_vec(), - PbObject::FunctionId(_) => AVAILABLE_ACTION_ON_FUNCTION.to_vec(), + PbObject::ViewId(_) => &acl::ALL_AVAILABLE_VIEW_MODES, + PbObject::SinkId(_) => &acl::ALL_AVAILABLE_SINK_MODES, + PbObject::FunctionId(_) => &acl::ALL_AVAILABLE_FUNCTION_MODES, _ => unreachable!("Invalid object type"), }; - let actions = actions + let actions = acl_set .iter() - .map(|action| ActionWithGrantOption { - action: get_prost_action(action) as i32, + .map(|mode| ActionWithGrantOption { + action: >::into(mode) as i32, with_grant_option: false, granted_by: DEFAULT_SUPER_USER_ID, }) From bb100a743b593e1c95527a4fe9728c8f43df0be7 Mon Sep 17 00:00:00 2001 From: August Date: Thu, 26 Oct 2023 20:35:54 +0800 Subject: [PATCH 14/52] feat: implement comment function in catalog controller (#13084) --- .../migration/src/m20230908_072257_init.rs | 2 + src/meta/model_v2/src/table.rs | 1 + src/meta/src/controller/catalog.rs | 66 ++++++++++++++++++- 3 files changed, 66 insertions(+), 3 deletions(-) diff --git a/src/meta/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model_v2/migration/src/m20230908_072257_init.rs index c9559bd6feda2..59a30149a1158 100644 --- a/src/meta/model_v2/migration/src/m20230908_072257_init.rs +++ b/src/meta/model_v2/migration/src/m20230908_072257_init.rs @@ -472,6 +472,7 @@ impl MigrationTrait for Migration { ) .col(ColumnDef::new(Table::JobStatus).string().not_null()) .col(ColumnDef::new(Table::CreateType).string().not_null()) + .col(ColumnDef::new(Table::Description).string()) .col(ColumnDef::new(Table::Version).json().not_null()) .foreign_key( &mut ForeignKey::create() @@ -887,6 +888,7 @@ enum Table { CleanedByWatermark, JobStatus, CreateType, + Description, Version, } diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model_v2/src/table.rs index a335f41023442..78602110788a6 100644 --- a/src/meta/model_v2/src/table.rs +++ b/src/meta/model_v2/src/table.rs @@ -94,6 +94,7 @@ pub struct Model { pub cleaned_by_watermark: bool, pub job_status: JobStatus, pub create_type: CreateType, + pub description: Option, pub version: TableVersion, } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index daaa9b684850c..9cf2ac987cbc9 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -14,6 +14,7 @@ use std::iter; +use anyhow::anyhow; use itertools::Itertools; use risingwave_common::bail; use risingwave_common::catalog::{DEFAULT_SCHEMA_NAME, SYSTEM_SCHEMAS}; @@ -21,11 +22,12 @@ use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::{ connection, database, function, index, object, object_dependency, schema, sink, source, table, - view, ConnectionId, DatabaseId, FunctionId, ObjectId, PrivateLinkService, SchemaId, SourceId, - TableId, UserId, + view, ColumnCatalogArray, ConnectionId, DatabaseId, FunctionId, ObjectId, PrivateLinkService, + SchemaId, SourceId, TableId, UserId, }; use risingwave_pb::catalog::{ - PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, + PbComment, PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, + PbView, }; use risingwave_pb::meta::relation::PbRelationInfo; use risingwave_pb::meta::subscribe_response::{ @@ -490,6 +492,64 @@ impl CatalogController { Ok(version) } + pub async fn comment_on(&self, comment: PbComment) -> MetaResult { + let inner = self.inner.write().await; + let txn = inner.db.begin().await?; + ensure_object_id(ObjectType::Database, comment.database_id, &txn).await?; + ensure_object_id(ObjectType::Schema, comment.schema_id, &txn).await?; + let table_obj = Object::find_by_id(comment.table_id) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("table", comment.table_id))?; + + let table = if let Some(col_idx) = comment.column_index { + let mut columns: ColumnCatalogArray = Table::find_by_id(comment.table_id) + .select_only() + .column(table::Column::Columns) + .into_tuple() + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("table", comment.table_id))?; + let column = columns + .0 + .get_mut(col_idx as usize) + .ok_or_else(|| MetaError::catalog_id_not_found("column", col_idx))?; + let column_desc = column.column_desc.as_mut().ok_or_else(|| { + anyhow!( + "column desc at index {} for table id {} not found", + col_idx, + comment.table_id + ) + })?; + column_desc.description = comment.description; + table::ActiveModel { + table_id: ActiveValue::Set(comment.table_id), + columns: ActiveValue::Set(columns), + ..Default::default() + } + .update(&txn) + .await? + } else { + table::ActiveModel { + table_id: ActiveValue::Set(comment.table_id), + description: ActiveValue::Set(comment.description), + ..Default::default() + } + .update(&txn) + .await? + }; + txn.commit().await?; + + let version = self + .notify_frontend_relation_info( + NotificationOperation::Update, + PbRelationInfo::Table(ObjectModel(table, table_obj).into()), + ) + .await; + + Ok(version) + } + pub async fn drop_relation( &self, object_type: ObjectType, From 8eedf8d0862a005d68a06ca22ef2cc13f4561d91 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Thu, 26 Oct 2023 22:28:15 +0800 Subject: [PATCH 15/52] fix(object_store): fix a corner case with retry (#13078) --- Cargo.lock | 1 + e2e_test/sink/kafka/avro.slt | 2 +- e2e_test/sink/kafka/protobuf.slt | 2 +- src/object_store/Cargo.toml | 1 + src/object_store/src/object/s3.rs | 48 ++++++++++++++++++++++++------- 5 files changed, 41 insertions(+), 13 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 4e969a94fc134..a2a8941e5905a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7832,6 +7832,7 @@ dependencies = [ "aws-smithy-types", "bytes", "crc32fast", + "either", "fail", "futures", "hyper", diff --git a/e2e_test/sink/kafka/avro.slt b/e2e_test/sink/kafka/avro.slt index e1b09e3608e37..a30d8b70fd4ba 100644 --- a/e2e_test/sink/kafka/avro.slt +++ b/e2e_test/sink/kafka/avro.slt @@ -57,7 +57,7 @@ select timestamp_millis_field, date_field, time_micros_field, - time_millis_field from from_kafka; + time_millis_field from from_kafka order by string_field; ---- t Rising \x6130 3.5 4.25 22 23 NULL {{NULL,3},NULL,{7,NULL,2}} 2006-01-02 22:04:05+00:00 NULL NULL 12:34:56.123456 NULL f Wave \x5a4446 1.5 NULL 11 12 (NULL,foo) NULL NULL 2006-01-02 22:04:05+00:00 2021-04-01 NULL 23:45:16.654 diff --git a/e2e_test/sink/kafka/protobuf.slt b/e2e_test/sink/kafka/protobuf.slt index 2f827aeda9fc0..c6ccb2ac24416 100644 --- a/e2e_test/sink/kafka/protobuf.slt +++ b/e2e_test/sink/kafka/protobuf.slt @@ -61,7 +61,7 @@ select nested_message_field, repeated_int_field, timestamp_field, - oneof_int32 from from_kafka; + oneof_int32 from from_kafka order by string_field; ---- t Rising \x6130 3.5 4.25 22 23 24 0 26 27 (1,) {4,0,4} (1136239445,0) 42 f Wave \x5a4446 1.5 0 11 12 13 14 15 16 (4,foo) {} (0,0) 0 diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index 88f73ca1d3c7d..f117c272a9afc 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -18,6 +18,7 @@ aws-smithy-http = { workspace = true } aws-smithy-types = { workspace = true } bytes = { version = "1", features = ["serde"] } crc32fast = "1.3.2" +either = "1" fail = "0.5" futures = { version = "0.3", default-features = false, features = ["alloc"] } hyper = "0.14" diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 69e7f3687fdeb..89f9aa5a053d5 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -24,7 +24,7 @@ use aws_sdk_s3::operation::get_object::builders::GetObjectFluentBuilder; use aws_sdk_s3::operation::get_object::GetObjectError; use aws_sdk_s3::operation::list_objects_v2::ListObjectsV2Error; use aws_sdk_s3::operation::upload_part::UploadPartOutput; -use aws_sdk_s3::primitives::ByteStream; +use aws_sdk_s3::primitives::{ByteStream, ByteStreamError}; use aws_sdk_s3::types::{ AbortIncompleteMultipartUpload, BucketLifecycleConfiguration, CompletedMultipartUpload, CompletedPart, Delete, ExpirationStatus, LifecycleRule, LifecycleRuleFilter, ObjectIdentifier, @@ -34,6 +34,7 @@ use aws_smithy_client::http_connector::{ConnectorSettings, HttpConnector}; use aws_smithy_http::body::SdkBody; use aws_smithy_http::result::SdkError; use aws_smithy_types::retry::RetryConfig; +use either::Either; use fail::fail_point; use futures::future::{try_join_all, BoxFuture, FutureExt}; use futures::{stream, Stream}; @@ -354,11 +355,19 @@ impl ObjectStore for S3ObjectStore { ))); // retry if occurs AWS EC2 HTTP timeout error. - let resp = tokio_retry::RetryIf::spawn( + let val = tokio_retry::RetryIf::spawn( self.config.get_retry_strategy(), || async { match self.obj_store_request(path, range.clone()).send().await { - Ok(resp) => Ok(resp), + Ok(resp) => { + let val = resp + .body + .collect() + .await + .map_err(either::Right)? + .into_bytes(); + Ok(val) + } Err(err) => { if let SdkError::DispatchFailure(e) = &err && e.is_timeout() @@ -369,7 +378,7 @@ impl ObjectStore for S3ObjectStore { .inc(); } - Err(err) + Err(either::Left(err)) } } }, @@ -377,8 +386,6 @@ impl ObjectStore for S3ObjectStore { ) .await?; - let val = resp.body.collect().await?.into_bytes(); - if let Some(len) = range.len() && len != val.len() { return Err(ObjectError::internal(format!( "mismatched size: expected {}, found {} when reading {} at {:?}", @@ -445,7 +452,7 @@ impl ObjectStore for S3ObjectStore { .inc(); } - Err(err) + Err(either::Left(err)) } } }, @@ -768,13 +775,23 @@ impl S3ObjectStore { } #[inline(always)] - fn should_retry(err: &SdkError) -> bool { - if let SdkError::DispatchFailure(e) = err { - if e.is_timeout() { - tracing::warn!(target: "http_timeout_retry", "{:?} occurs, trying to retry S3 get_object request.", e); + fn should_retry(err: &Either, ByteStreamError>) -> bool { + match err { + Either::Left(err) => { + if let SdkError::DispatchFailure(e) = err { + if e.is_timeout() { + tracing::warn!(target: "http_timeout_retry", "{:?} occurs, trying to retry S3 get_object request.", e); + return true; + } + } + } + Either::Right(_) => { + // Unfortunately `ErrorKind` of `ByteStreamError` is not accessible. + // Always returns true and relies on req_retry_max_attempts to avoid infinite loop. return true; } } + false } } @@ -914,6 +931,15 @@ impl Stream for S3ObjectIter { } } +impl From, ByteStreamError>> for ObjectError { + fn from(e: Either, ByteStreamError>) -> Self { + match e { + Either::Left(e) => e.into(), + Either::Right(e) => e.into(), + } + } +} + #[cfg(test)] #[cfg(not(madsim))] mod tests { From a27e876cdb20692fcac2862130ac6a1c0454368c Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Thu, 26 Oct 2023 11:18:21 -0400 Subject: [PATCH 16/52] fix(stream): fix the minput's indicies when it's for distinct call (#13015) Signed-off-by: Richard Chien Co-authored-by: Richard Chien Co-authored-by: Bugen Zhao --- e2e_test/streaming/bug_fixes/issue_12140.slt | 75 +++++++++++++ proto/stream_plan.proto | 12 +++ .../tests/testdata/input/agg.yaml | 6 ++ .../tests/testdata/output/agg.yaml | 102 ++++++++++++++++++ .../src/optimizer/plan_node/generic/agg.rs | 19 +++- .../optimizer/plan_node/stream_hash_agg.rs | 1 + .../optimizer/plan_node/stream_simple_agg.rs | 1 + .../plan_node/stream_stateless_simple_agg.rs | 1 + src/stream/src/executor/agg_common.rs | 3 + .../src/executor/aggregation/agg_group.rs | 5 + .../src/executor/aggregation/agg_state.rs | 3 + src/stream/src/executor/aggregation/minput.rs | 38 ++++++- src/stream/src/executor/hash_agg.rs | 7 ++ src/stream/src/executor/simple_agg.rs | 33 +++--- src/stream/src/executor/test_utils.rs | 5 + src/stream/src/from_proto/hash_agg.rs | 2 + src/stream/src/from_proto/simple_agg.rs | 2 + 17 files changed, 296 insertions(+), 19 deletions(-) create mode 100644 e2e_test/streaming/bug_fixes/issue_12140.slt diff --git a/e2e_test/streaming/bug_fixes/issue_12140.slt b/e2e_test/streaming/bug_fixes/issue_12140.slt new file mode 100644 index 0000000000000..2240762868832 --- /dev/null +++ b/e2e_test/streaming/bug_fixes/issue_12140.slt @@ -0,0 +1,75 @@ +# https://github.com/risingwavelabs/risingwave/issues/12140 + +statement ok +CREATE TABLE t (c3 INT, c9 CHARACTER VARYING); + +statement ok +INSERT INTO t VALUES (1, 'interesting'), (2, 'interesting'), (3, 'interesting'), (4, 'IwfwuseZmg'), (5, 'ZuT4aIQVhA'); + +statement ok +CREATE MATERIALIZED VIEW mv AS +SELECT + first_value(DISTINCT t.c9 ORDER BY t.c9 ASC) +FROM + t; + +statement ok +DELETE FROM t WHERE c3 = 1; + +statement ok +DELETE FROM t WHERE c3 = 2; + +statement ok +DELETE FROM t WHERE c3 = 3; + +statement ok +drop materialized view mv; + +statement ok +drop table t; + +statement ok +CREATE TABLE t (c3 INT, c9 CHARACTER VARYING); + +statement ok +INSERT INTO t VALUES (1, 'interesting'), (2, 'interesting'), (3, 'interesting'), (1, 'boring'), (2, 'boring'), (3, 'boring'), (1, 'exciting'), (2, 'exciting'), (3, 'exciting'), (4, 'IwfwuseZmg'), (5, 'ZuT4aIQVhA'); + +statement ok +CREATE MATERIALIZED VIEW mv AS +SELECT + first_value(DISTINCT t.c9 ORDER BY t.c9 ASC), last_value(distinct c3 order by c3 asc) +FROM + t; + +statement ok +DELETE FROM t WHERE c3 = 1 and c9 = 'interesting'; + +statement ok +DELETE FROM t WHERE c3 = 2 and c9 = 'interesting'; + +statement ok +DELETE FROM t WHERE c3 = 3 and c9 = 'interesting'; + +statement ok +DELETE FROM t WHERE c3 = 1 and c9 = 'boring'; + +statement ok +DELETE FROM t WHERE c3 = 1 and c9 = 'exciting'; + +statement ok +DELETE FROM t WHERE c3 = 2 and c9 = 'boring'; + +statement ok +DELETE FROM t WHERE c3 = 2 and c9 = 'exciting'; + +statement ok +DELETE FROM t WHERE c3 = 3 and c9 = 'boring'; + +statement ok +DELETE FROM t WHERE c3 = 3 and c9 = 'exciting'; + +statement ok +drop materialized view mv; + +statement ok +drop table t; \ No newline at end of file diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 683a43ef6e9be..97f0c72937cda 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -268,6 +268,16 @@ message AggCallState { reserved "table_state"; } +enum AggNodeVersion { + AGG_NODE_VERSION_UNSPECIFIED = 0; + + // https://github.com/risingwavelabs/risingwave/issues/12140#issuecomment-1776289808 + AGG_NODE_VERSION_ISSUE_12140 = 1; + + // Used for test only. + AGG_NODE_VERSION_MAX = 2147483647; +} + message SimpleAggNode { repeated expr.AggCall agg_calls = 1; // Only used for stateless simple agg. @@ -279,6 +289,7 @@ message SimpleAggNode { bool is_append_only = 5; map distinct_dedup_tables = 6; uint32 row_count_index = 7; + AggNodeVersion version = 8; } message HashAggNode { @@ -292,6 +303,7 @@ message HashAggNode { map distinct_dedup_tables = 6; uint32 row_count_index = 7; bool emit_on_window_close = 8; + AggNodeVersion version = 9; } message TopNNode { diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 99aa94ff773b9..2b69b5a53145d 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -912,6 +912,12 @@ expected_outputs: - batch_plan - stream_plan + - stream_dist_plan # check the state table schema +- sql: | + create table t (x int, y int); + select first_value(distinct x order by x asc) from t; + expected_outputs: + - stream_dist_plan # check the state table schema - sql: | create table t (x int, y int); select last_value(x order by y desc nulls last) from t; diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index baa77dc79c89b..058fa2a794d1e 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1613,6 +1613,108 @@ └─StreamSimpleAgg { aggs: [first_value(t.x order_by(t.y ASC)), count] } └─StreamExchange { dist: Single } └─StreamTableScan { table: t, columns: [t.x, t.y, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [first_value], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamProject { exprs: [first_value(t.x order_by(t.y ASC))] } + └── StreamSimpleAgg { aggs: [first_value(t.x order_by(t.y ASC)), count] } + ├── intermediate state table: 1 + ├── state tables: [ 0 ] + ├── distinct tables: [] + └── StreamExchange Single from 1 + + Fragment 1 + Chain { table: t, columns: [t.x, t.y, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + ├── state table: 2 + ├── Upstream + └── BatchPlanNode + + Table 0 + ├── columns: [ t_y, t__row_id, t_x ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 1 + ├── columns: [ first_value(t_x order_by(t_y ASC)), count ] + ├── primary key: [] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 2 + ├── columns: [ vnode, _row_id, t_backfill_finished, t_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 4294967294 + ├── columns: [ first_value ] + ├── primary key: [] + ├── value indices: [ 0 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + +- sql: | + create table t (x int, y int); + select first_value(distinct x order by x asc) from t; + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [first_value], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamProject { exprs: [first_value(distinct t.x order_by(t.x ASC))] } + └── StreamSimpleAgg { aggs: [first_value(distinct t.x order_by(t.x ASC)), count] } + ├── intermediate state table: 1 + ├── state tables: [ 0 ] + ├── distinct tables: [ (distinct key: t.x, table id: 2) ] + └── StreamExchange Single from 1 + + Fragment 1 + Chain { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + ├── state table: 3 + ├── Upstream + └── BatchPlanNode + + Table 0 + ├── columns: [ t_x ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 1 + ├── columns: [ first_value(distinct t_x order_by(t_x ASC)), count ] + ├── primary key: [] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 2 + ├── columns: [ t_x, count_for_agg_call_0 ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 1 + + Table 3 + ├── columns: [ vnode, _row_id, t_backfill_finished, t_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 4294967294 + ├── columns: [ first_value ] + ├── primary key: [] + ├── value indices: [ 0 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + - sql: | create table t (x int, y int); select last_value(x order by y desc nulls last) from t; diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 98109e695110f..c7a0ea4f39389 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::collections::{BTreeMap, BTreeSet, HashMap}; -use std::fmt; +use std::{fmt, vec}; use fixedbitset::FixedBitSet; use itertools::{Either, Itertools}; @@ -348,6 +348,7 @@ impl Agg { let in_dist_key = self.input.distribution().dist_column_indices().to_vec(); let gen_materialized_input_state = |sort_keys: Vec<(OrderType, usize)>, + extra_keys: Vec, include_keys: Vec| -> MaterializedInputState { let (mut table_builder, mut included_upstream_indices, mut column_mapping) = @@ -375,7 +376,7 @@ impl Agg { for (order_type, idx) in sort_keys { add_column(idx, Some(order_type), true, &mut table_builder); } - for &idx in &in_pks { + for idx in extra_keys { add_column(idx, Some(OrderType::ascending()), true, &mut table_builder); } for idx in include_keys { @@ -458,6 +459,17 @@ impl Agg { _ => unreachable!(), } }; + + // columns to ensure each row unique + let extra_keys = if agg_call.distinct { + // if distinct, use distinct keys as extra keys + let distinct_key = agg_call.inputs[0].index; + vec![distinct_key] + } else { + // if not distinct, use primary keys as extra keys + in_pks.clone() + }; + // other columns that should be contained in state table let include_keys = match agg_call.agg_kind { AggKind::FirstValue @@ -470,7 +482,8 @@ impl Agg { } _ => vec![], }; - let state = gen_materialized_input_state(sort_keys, include_keys); + + let state = gen_materialized_input_state(sort_keys, extra_keys, include_keys); AggCallState::MaterializedInput(Box::new(state)) } agg_kinds::rewritten!() => { diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 6ef8849b1e142..cb181677c6aaa 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -217,6 +217,7 @@ impl StreamNode for StreamHashAgg { .collect(), row_count_index: self.row_count_idx as u32, emit_on_window_close: self.base.emit_on_window_close(), + version: PbAggNodeVersion::Issue12140 as _, }) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs index 06aa227d7e316..ff590e9695bb9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -128,6 +128,7 @@ impl StreamNode for StreamSimpleAgg { }) .collect(), row_count_index: self.row_count_idx as u32, + version: PbAggNodeVersion::Issue12140 as _, }) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs index ca076132f3a55..e9db33b13b626 100644 --- a/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_stateless_simple_agg.rs @@ -100,6 +100,7 @@ impl StreamNode for StreamStatelessSimpleAgg { intermediate_state_table: None, is_append_only: self.input().append_only(), distinct_dedup_tables: Default::default(), + version: AggNodeVersion::Issue12140 as _, }) } } diff --git a/src/stream/src/executor/agg_common.rs b/src/stream/src/executor/agg_common.rs index fbaa80c3fbeb7..d1ea23068d430 100644 --- a/src/stream/src/executor/agg_common.rs +++ b/src/stream/src/executor/agg_common.rs @@ -16,6 +16,7 @@ use std::collections::HashMap; use std::sync::Arc; use risingwave_expr::aggregate::AggCall; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::aggregation::AggStateStorage; @@ -27,6 +28,8 @@ use crate::task::AtomicU64Ref; /// Arguments needed to construct an `XxxAggExecutor`. pub struct AggExecutorArgs { + pub version: PbAggNodeVersion, + // basic pub input: Box, pub actor_ctx: ActorContextRef, diff --git a/src/stream/src/executor/aggregation/agg_group.rs b/src/stream/src/executor/aggregation/agg_group.rs index d854969120919..d0e97cd4783e9 100644 --- a/src/stream/src/executor/aggregation/agg_group.rs +++ b/src/stream/src/executor/aggregation/agg_group.rs @@ -25,6 +25,7 @@ use risingwave_common::must_match; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::{AggCall, BoxedAggregateFunction}; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::agg_state::{AggState, AggStateStorage}; @@ -192,6 +193,7 @@ impl AggGroup { /// For [`crate::executor::SimpleAggExecutor`], the `group_key` should be `None`. #[allow(clippy::too_many_arguments)] pub async fn create( + version: PbAggNodeVersion, group_key: Option, agg_calls: &[AggCall], agg_funcs: &[BoxedAggregateFunction], @@ -212,6 +214,7 @@ impl AggGroup { let mut states = Vec::with_capacity(agg_calls.len()); for (idx, (agg_call, agg_func)) in agg_calls.iter().zip_eq_fast(agg_funcs).enumerate() { let state = AggState::create( + version, agg_call, agg_func, &storages[idx], @@ -242,6 +245,7 @@ impl AggGroup { /// Create a group from encoded states for EOWC. The previous output is set to `None`. #[allow(clippy::too_many_arguments)] pub fn create_eowc( + version: PbAggNodeVersion, group_key: Option, agg_calls: &[AggCall], agg_funcs: &[BoxedAggregateFunction], @@ -255,6 +259,7 @@ impl AggGroup { let mut states = Vec::with_capacity(agg_calls.len()); for (idx, (agg_call, agg_func)) in agg_calls.iter().zip_eq_fast(agg_funcs).enumerate() { let state = AggState::create( + version, agg_call, agg_func, &storages[idx], diff --git a/src/stream/src/executor/aggregation/agg_state.rs b/src/stream/src/executor/aggregation/agg_state.rs index 0c1932c58831c..a0413ed4491d2 100644 --- a/src/stream/src/executor/aggregation/agg_state.rs +++ b/src/stream/src/executor/aggregation/agg_state.rs @@ -19,6 +19,7 @@ use risingwave_common::estimate_size::EstimateSize; use risingwave_common::must_match; use risingwave_common::types::Datum; use risingwave_expr::aggregate::{AggCall, AggregateState, BoxedAggregateFunction}; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::minput::MaterializedInputState; @@ -65,6 +66,7 @@ impl AggState { /// Create an [`AggState`] from a given [`AggCall`]. #[allow(clippy::too_many_arguments)] pub fn create( + version: PbAggNodeVersion, agg_call: &AggCall, agg_func: &BoxedAggregateFunction, storage: &AggStateStorage, @@ -83,6 +85,7 @@ impl AggState { } AggStateStorage::MaterializedInput { mapping, .. } => { Self::MaterializedInput(Box::new(MaterializedInputState::new( + version, agg_call, pk_indices, mapping, diff --git a/src/stream/src/executor/aggregation/minput.rs b/src/stream/src/executor/aggregation/minput.rs index 1329f08eb6d99..2392e53d7d49a 100644 --- a/src/stream/src/executor/aggregation/minput.rs +++ b/src/stream/src/executor/aggregation/minput.rs @@ -25,6 +25,7 @@ use risingwave_common::types::Datum; use risingwave_common::util::row_serde::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; use risingwave_expr::aggregate::{AggCall, AggKind, BoxedAggregateFunction}; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; @@ -68,6 +69,7 @@ pub struct MaterializedInputState { impl MaterializedInputState { /// Create an instance from [`AggCall`]. pub fn new( + version: PbAggNodeVersion, agg_call: &AggCall, pk_indices: &PkIndices, col_mapping: &StateTableColumnMapping, @@ -102,9 +104,26 @@ impl MaterializedInputState { .unzip() }; - let pk_len = pk_indices.len(); - order_col_indices.extend(pk_indices.iter()); - order_types.extend(itertools::repeat_n(OrderType::ascending(), pk_len)); + if agg_call.distinct { + if version < PbAggNodeVersion::Issue12140 { + panic!( + "RisingWave versions before issue #12140 is resolved has critical bug, you must re-create current MV to ensure correctness." + ); + } + + // If distinct, we need to materialize input with the distinct keys + // As we only support single-column distinct for now, we use the + // `agg_call.args.val_indices()[0]` as the distinct key. + if !order_col_indices.contains(&agg_call.args.val_indices()[0]) { + order_col_indices.push(agg_call.args.val_indices()[0]); + order_types.push(OrderType::ascending()); + } + } else { + // If not distinct, we need to materialize input with the primary keys + let pk_len = pk_indices.len(); + order_col_indices.extend(pk_indices.iter()); + order_types.extend(itertools::repeat_n(OrderType::ascending(), pk_len)); + } // map argument columns to state table column indices let state_table_arg_col_indices = arg_col_indices @@ -255,6 +274,7 @@ mod tests { use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::sort_util::OrderType; use risingwave_expr::aggregate::{build_append_only, AggCall}; + use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::StateStore; @@ -327,6 +347,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -379,6 +400,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -420,6 +442,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -472,6 +495,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -529,6 +553,7 @@ mod tests { table_2.init_epoch(epoch); let mut state_1 = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call_1, &input_pk_indices, &mapping_1, @@ -538,6 +563,7 @@ mod tests { .unwrap(); let mut state_2 = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call_2, &input_pk_indices, &mapping_2, @@ -621,6 +647,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -672,6 +699,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -715,6 +743,7 @@ mod tests { table.init_epoch(epoch); let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -814,6 +843,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -921,6 +951,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, @@ -1000,6 +1031,7 @@ mod tests { .await; let mut state = MaterializedInputState::new( + PbAggNodeVersion::Max, &agg_call, &input_pk_indices, &mapping, diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index cb62e8d8f94aa..8d02cc328fa43 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -29,6 +29,7 @@ use risingwave_common::types::ScalarImpl; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::{build_retractable, AggCall, BoxedAggregateFunction}; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::agg_common::{AggExecutorArgs, HashAggExecutorExtraArgs}; @@ -81,6 +82,9 @@ pub struct HashAggExecutor { struct ExecutorInner { _phantom: PhantomData, + /// Version of aggregation executors. + version: PbAggNodeVersion, + actor_ctx: ActorContextRef, info: ExecutorInfo, @@ -233,6 +237,7 @@ impl HashAggExecutor { input: args.input, inner: ExecutorInner { _phantom: PhantomData, + version: args.version, actor_ctx: args.actor_ctx, info: ExecutorInfo { schema, @@ -318,6 +323,7 @@ impl HashAggExecutor { // Create `AggGroup` for the current group if not exists. This will // restore agg states from the intermediate state table. let agg_group = AggGroup::create( + this.version, Some(GroupKey::new( key.deserialize(group_key_types)?, Some(this.group_key_table_pk_projection.clone()), @@ -466,6 +472,7 @@ impl HashAggExecutor { let states = row.into_iter().skip(this.group_key_indices.len()).collect(); let mut agg_group = AggGroup::create_eowc( + this.version, Some(GroupKey::new( group_key, Some(this.group_key_table_pk_projection.clone()), diff --git a/src/stream/src/executor/simple_agg.rs b/src/stream/src/executor/simple_agg.rs index 6e88241f48433..92730218ca148 100644 --- a/src/stream/src/executor/simple_agg.rs +++ b/src/stream/src/executor/simple_agg.rs @@ -18,6 +18,7 @@ use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::{build_retractable, AggCall, BoxedAggregateFunction}; +use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use super::agg_common::{AggExecutorArgs, SimpleAggExecutorExtraArgs}; @@ -52,6 +53,9 @@ pub struct SimpleAggExecutor { } struct ExecutorInner { + /// Version of aggregation executors. + version: PbAggNodeVersion, + actor_ctx: ActorContextRef, info: ExecutorInfo, @@ -135,6 +139,7 @@ impl SimpleAggExecutor { Ok(Self { input: args.input, inner: ExecutorInner { + version: args.version, actor_ctx: args.actor_ctx, info: ExecutorInfo { schema, @@ -257,9 +262,23 @@ impl SimpleAggExecutor { table.init_epoch(barrier.epoch); }); + let mut distinct_dedup = DistinctDeduplicater::new( + &this.agg_calls, + &this.watermark_epoch, + &this.distinct_dedup_tables, + this.actor_ctx.id, + this.metrics.clone(), + ); + distinct_dedup.dedup_caches_mut().for_each(|cache| { + cache.update_epoch(barrier.epoch.curr); + }); + + yield Message::Barrier(barrier); + let mut vars = ExecutionVars { // This will fetch previous agg states from the intermediate state table. agg_group: AggGroup::create( + this.version, None, &this.agg_calls, &this.agg_funcs, @@ -271,22 +290,10 @@ impl SimpleAggExecutor { &this.input_schema, ) .await?, - distinct_dedup: DistinctDeduplicater::new( - &this.agg_calls, - &this.watermark_epoch, - &this.distinct_dedup_tables, - this.actor_ctx.id, - this.metrics.clone(), - ), + distinct_dedup, state_changed: false, }; - vars.distinct_dedup.dedup_caches_mut().for_each(|cache| { - cache.update_epoch(barrier.epoch.curr); - }); - - yield Message::Barrier(barrier); - #[for_await] for msg in input { let msg = msg?; diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index 13a9237cf0159..2ae461e62351c 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -280,6 +280,7 @@ pub mod agg_executor { use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; use risingwave_expr::aggregate::{AggCall, AggKind}; + use risingwave_pb::stream_plan::PbAggNodeVersion; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; @@ -444,6 +445,8 @@ pub mod agg_executor { .await; HashAggExecutor::::new(AggExecutorArgs { + version: PbAggNodeVersion::Max, + input, actor_ctx: ActorContext::create(123), pk_indices, @@ -507,6 +510,8 @@ pub mod agg_executor { .await; SimpleAggExecutor::new(AggExecutorArgs { + version: PbAggNodeVersion::Max, + input, actor_ctx, pk_indices, diff --git a/src/stream/src/from_proto/hash_agg.rs b/src/stream/src/from_proto/hash_agg.rs index a369f8124ebfb..faf8a1f7fdad1 100644 --- a/src/stream/src/from_proto/hash_agg.rs +++ b/src/stream/src/from_proto/hash_agg.rs @@ -97,6 +97,8 @@ impl ExecutorBuilder for HashAggExecutorBuilder { HashAggExecutorDispatcherArgs { args: AggExecutorArgs { + version: node.version(), + input, actor_ctx: params.actor_context, pk_indices: params.pk_indices, diff --git a/src/stream/src/from_proto/simple_agg.rs b/src/stream/src/from_proto/simple_agg.rs index 5423e4fd2043f..fdd6d877b99ed 100644 --- a/src/stream/src/from_proto/simple_agg.rs +++ b/src/stream/src/from_proto/simple_agg.rs @@ -58,6 +58,8 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder { .await; Ok(SimpleAggExecutor::new(AggExecutorArgs { + version: node.version(), + input, actor_ctx: params.actor_context, pk_indices: params.pk_indices, From 4a8b0cb8d0da71963c7b449898c342a6c0353e89 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 26 Oct 2023 23:30:04 +0800 Subject: [PATCH 17/52] feat(expr): `substring`/`substr` for `bytea` (#13088) --- e2e_test/batch/functions/substr.slt.part | 10 +++++++++ src/expr/impl/src/scalar/substr.rs | 26 ++++++++++++++++++++++-- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/e2e_test/batch/functions/substr.slt.part b/e2e_test/batch/functions/substr.slt.part index 4f87043da3e86..22a495a3f8a87 100644 --- a/e2e_test/batch/functions/substr.slt.part +++ b/e2e_test/batch/functions/substr.slt.part @@ -27,3 +27,13 @@ query T select substr('a', 2147483646, 1); ---- (empty) + +query T +select substr('abcde'::bytea, 2, 7); +---- +\x62636465 + +query T +select substr('abcde'::bytea, -2, 5); +---- +\x6162 diff --git a/src/expr/impl/src/scalar/substr.rs b/src/expr/impl/src/scalar/substr.rs index dc2829c5d8a52..2b65575c91c19 100644 --- a/src/expr/impl/src/scalar/substr.rs +++ b/src/expr/impl/src/scalar/substr.rs @@ -28,8 +28,14 @@ pub fn substr_start(s: &str, start: i32, writer: &mut impl Write) -> Result<()> Ok(()) } -#[function("substr(varchar, int4, int4) -> varchar")] -pub fn substr_start_for(s: &str, start: i32, count: i32, writer: &mut impl Write) -> Result<()> { +#[function("substr(bytea, int4) -> bytea")] +pub fn substr_start_bytea(s: &[u8], start: i32) -> Box<[u8]> { + let skip = start.saturating_sub(1).max(0) as usize; + + s.iter().copied().skip(skip).collect() +} + +fn convert_args(start: i32, count: i32) -> Result<(usize, usize)> { if count < 0 { return Err(ExprError::InvalidParam { name: "length", @@ -44,6 +50,15 @@ pub fn substr_start_for(s: &str, start: i32, count: i32, writer: &mut impl Write count.saturating_add(start.saturating_sub(1)).max(0) as usize }; + // The returned args may still go out of bounds. + // So `skip` and `take` on iterator is safer than `[skip..(skip+take)]` + Ok((skip, take)) +} + +#[function("substr(varchar, int4, int4) -> varchar")] +pub fn substr_start_for(s: &str, start: i32, count: i32, writer: &mut impl Write) -> Result<()> { + let (skip, take) = convert_args(start, count)?; + let substr = s.chars().skip(skip).take(take); for char in substr { writer.write_char(char).unwrap(); @@ -52,6 +67,13 @@ pub fn substr_start_for(s: &str, start: i32, count: i32, writer: &mut impl Write Ok(()) } +#[function("substr(bytea, int4, int4) -> bytea")] +pub fn substr_start_for_bytea(s: &[u8], start: i32, count: i32) -> Result> { + let (skip, take) = convert_args(start, count)?; + + Ok(s.iter().copied().skip(skip).take(take).collect()) +} + #[cfg(test)] mod tests { use super::*; From da6626da5245c363614069ca60f8e7af4d5c6f65 Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Fri, 27 Oct 2023 01:12:49 +0800 Subject: [PATCH 18/52] chore(deps): bump rustix 0.37.23 to 0.37.26 reported by dependabot alerts (#13092) Signed-off-by: TennyZhuang Co-authored-by: xxchan --- Cargo.lock | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a2a8941e5905a..b303658d3c57f 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -480,7 +480,7 @@ dependencies = [ "log", "parking", "polling", - "rustix 0.37.23", + "rustix 0.37.26", "slab", "socket2 0.4.9", "waker-fn", @@ -8384,9 +8384,9 @@ dependencies = [ [[package]] name = "rustix" -version = "0.37.23" +version = "0.37.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4d69718bf81c6127a49dc64e44a742e8bb9213c0ff8869a22c308f84c1d4ab06" +checksum = "84f3f8f960ed3b5a59055428714943298bf3fa2d4a1d53135084e0544829d995" dependencies = [ "bitflags 1.3.2", "errno", From 23b69c47290a29267694a1ea362b848faaa792d6 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Fri, 27 Oct 2023 11:15:47 +0800 Subject: [PATCH 19/52] feat(expr): support `int8send` and `int8recv` (#13077) --- .../batch/functions/pgwire_send_recv.slt.part | 40 +++++++++++++++++++ proto/expr.proto | 2 + src/expr/impl/src/scalar/cast.rs | 14 +++++++ src/frontend/src/binder/expr/function.rs | 13 ++++++ src/frontend/src/expr/pure.rs | 2 + src/tests/sqlsmith/src/validation.rs | 5 +++ 6 files changed, 76 insertions(+) create mode 100644 e2e_test/batch/functions/pgwire_send_recv.slt.part diff --git a/e2e_test/batch/functions/pgwire_send_recv.slt.part b/e2e_test/batch/functions/pgwire_send_recv.slt.part new file mode 100644 index 0000000000000..47fea3a4fb19b --- /dev/null +++ b/e2e_test/batch/functions/pgwire_send_recv.slt.part @@ -0,0 +1,40 @@ +query TT +select int8send(2147483647); +---- +\x000000007fffffff + +query I +select int8recv(int8send(76)); +---- +76 + +query I +select int8recv(' a'::bytea); +---- +2314885530818453601 + +statement error could not convert slice to array +select int8recv('a'::bytea); + +query I +select int8recv(decode(substr(md5(''), 3, 16), 'hex')); +---- +2129315932054619369 + +query I +select int8recv(substr(decode(md5(''), 'hex'), 2, 8)); +---- +2129315932054619369 + +query I +select int8recv(substr(sha256(''), 2, 8)); +---- +-5709365202766785382 + +statement error decode +select int8recv(to_hex(2129315932054619369)); + +query I +select int8recv(decode(to_hex(2129315932054619369), 'hex')); +---- +2129315932054619369 diff --git a/proto/expr.proto b/proto/expr.proto index 2f252d67c8400..9c6f3598032f9 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -173,6 +173,8 @@ message ExprNode { LEFT = 317; RIGHT = 318; FORMAT = 319; + PGWIRE_SEND = 320; + PGWIRE_RECV = 321; // Unary operators NEG = 401; diff --git a/src/expr/impl/src/scalar/cast.rs b/src/expr/impl/src/scalar/cast.rs index c173c76c330c5..f22b643bb9a09 100644 --- a/src/expr/impl/src/scalar/cast.rs +++ b/src/expr/impl/src/scalar/cast.rs @@ -47,6 +47,14 @@ where .map_err(|err: ::Err| ExprError::Parse(err.to_string().into())) } +// TODO: introduce `FromBinary` and support all types +#[function("pgwire_recv(bytea) -> int8")] +pub fn pgwire_recv(elem: &[u8]) -> Result { + let fixed_length = + <[u8; 8]>::try_from(elem).map_err(|e| ExprError::Parse(e.to_string().into()))?; + Ok(i64::from_be_bytes(fixed_length)) +} + #[function("cast(int2) -> int256")] #[function("cast(int4) -> int256")] #[function("cast(int8) -> int256")] @@ -156,6 +164,12 @@ pub fn general_to_text(elem: impl ToText, mut writer: &mut impl Write) { elem.write(&mut writer).unwrap(); } +// TODO: use `ToBinary` and support all types +#[function("pgwire_send(int8) -> bytea")] +fn pgwire_send(elem: i64) -> Box<[u8]> { + elem.to_be_bytes().into() +} + #[function("cast(boolean) -> varchar")] pub fn bool_to_varchar(input: bool, writer: &mut impl Write) { writer diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 18438b28c0a98..0b8457bc1cb82 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -802,6 +802,19 @@ impl Binder { ("sha512", raw_call(ExprType::Sha512)), ("left", raw_call(ExprType::Left)), ("right", raw_call(ExprType::Right)), + ("int8send", raw_call(ExprType::PgwireSend)), + ("int8recv", guard_by_len(1, raw(|_binder, mut inputs| { + // Similar to `cast` from string, return type is set explicitly rather than inferred. + let hint = if !inputs[0].is_untyped() && inputs[0].return_type() == DataType::Varchar { + " Consider `decode` or cast." + } else { + "" + }; + inputs[0].cast_implicit_mut(DataType::Bytea).map_err(|e| { + ErrorCode::BindError(format!("{e} in `recv`.{hint}")) + })?; + Ok(FunctionCall::new_unchecked(ExprType::PgwireRecv, inputs, DataType::Int64).into()) + }))), // array ("array_cat", raw_call(ExprType::ArrayCat)), ("array_append", raw_call(ExprType::ArrayAppend)), diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index e5d698c2ce172..470e1efc6aba0 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -200,6 +200,8 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::ArrayPositions | expr_node::Type::StringToArray | expr_node::Type::Format + | expr_node::Type::PgwireSend + | expr_node::Type::PgwireRecv | expr_node::Type::ArrayTransform | expr_node::Type::Greatest | expr_node::Type::Least => diff --git a/src/tests/sqlsmith/src/validation.rs b/src/tests/sqlsmith/src/validation.rs index 7c88320b8d4ca..5195444d079f5 100644 --- a/src/tests/sqlsmith/src/validation.rs +++ b/src/tests/sqlsmith/src/validation.rs @@ -30,6 +30,10 @@ fn is_numeric_out_of_range_err(db_error: &str) -> bool { || db_error.contains("Casting to u32 out of range") } +fn is_parse_err(db_error: &str) -> bool { + db_error.contains("Parse error") +} + /// Skip queries with unimplemented features fn is_unimplemented_error(db_error: &str) -> bool { db_error.contains("not yet implemented") @@ -106,6 +110,7 @@ pub fn is_neg_exp_error(db_error: &str) -> bool { pub fn is_permissible_error(db_error: &str) -> bool { is_numeric_out_of_range_err(db_error) || is_zero_err(db_error) + || is_parse_err(db_error) || is_unimplemented_error(db_error) || not_unique_error(db_error) || is_window_error(db_error) From b2eeca1176de9ba4c87f2040b44c02f125241c0e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 27 Oct 2023 12:25:45 +0800 Subject: [PATCH 20/52] chore(deps): Bump ethnum from 1.4.0 to 1.5.0 (#13096) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index b303658d3c57f..a928849d0e03c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2832,9 +2832,9 @@ dependencies = [ [[package]] name = "ethnum" -version = "1.4.0" +version = "1.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6c8ff382b2fa527fb7fb06eeebfc5bbb3f17e3cc6b9d70b006c41daa8824adac" +checksum = "b90ca2580b73ab6a1f724b76ca11ab632df820fd6040c336200d2c1df7b3c82c" dependencies = [ "serde", ] From 00df63562dea12faddd4a2ccd87fe7ae08cf9ba7 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 27 Oct 2023 12:26:35 +0800 Subject: [PATCH 21/52] chore: disable tests by default for some heavy lib/bins without UTs (#12950) --- src/cmd/Cargo.toml | 8 ++++++++ src/cmd_all/Cargo.toml | 1 + src/meta/node/Cargo.toml | 3 +++ 3 files changed, 12 insertions(+) diff --git a/src/cmd/Cargo.toml b/src/cmd/Cargo.toml index 8ba72d6a24af4..de69cc3574509 100644 --- a/src/cmd/Cargo.toml +++ b/src/cmd/Cargo.toml @@ -49,25 +49,33 @@ workspace-hack = { path = "../workspace-hack" } [target.'cfg(enable_task_local_alloc)'.dependencies] task_stats_alloc = { path = "../utils/task_stats_alloc" } +[lib] +test = false + [[bin]] name = "frontend" path = "src/bin/frontend_node.rs" +test = false [[bin]] name = "meta-node" path = "src/bin/meta_node.rs" +test = false [[bin]] name = "compute-node" path = "src/bin/compute_node.rs" +test = false [[bin]] name = "compactor" path = "src/bin/compactor.rs" +test = false [[bin]] name = "risectl" path = "src/bin/ctl.rs" +test = false [lints] workspace = true diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index 9a4b34c094196..e5aa9e3c68d04 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -69,6 +69,7 @@ task_stats_alloc = { path = "../utils/task_stats_alloc" } [[bin]] name = "risingwave" path = "src/bin/risingwave.rs" +test = false [lints] workspace = true diff --git a/src/meta/node/Cargo.toml b/src/meta/node/Cargo.toml index dcfa053fdfc4b..e7597c0d041f3 100644 --- a/src/meta/node/Cargo.toml +++ b/src/meta/node/Cargo.toml @@ -54,5 +54,8 @@ workspace-hack = { path = "../../workspace-hack" } [dev-dependencies] +[lib] +test = false + [lints] workspace = true From 6b7f863e51601f1098caf7abcada94eef1fc70de Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 27 Oct 2023 13:34:34 +0800 Subject: [PATCH 22/52] fix(binder): only table-in-out functions can have subquery parameters (#13098) --- .../tests/testdata/input/subquery.yaml | 10 ++++++++++ .../tests/testdata/output/subquery.yaml | 13 +++++++++++++ .../src/binder/relation/table_function.rs | 16 +++++++++++++++- 3 files changed, 38 insertions(+), 1 deletion(-) diff --git a/src/frontend/planner_test/tests/testdata/input/subquery.yaml b/src/frontend/planner_test/tests/testdata/input/subquery.yaml index b1d58c5e422ed..47785f0234271 100644 --- a/src/frontend/planner_test/tests/testdata/input/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/input/subquery.yaml @@ -291,3 +291,13 @@ expected_outputs: - batch_plan - stream_plan +- name: Only table-in-out functions can have subquery parameters. + sql: | + SELECT * FROM generate_series(1, (select 1)); + expected_outputs: + - binder_error +- name: While this one is allowed. + sql: | + SELECT generate_series(1, (select 1)); + expected_outputs: + - batch_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index e07e84e040929..914d9b764c5e1 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -864,3 +864,16 @@ └─StreamHopWindow { time_col: t1.ts, slide: 00:10:00, size: 00:30:00, output: all } └─StreamFilter { predicate: IsNotNull(t1.ts) } └─StreamTableScan { table: t1, columns: [t1.k, t1.ts], pk: [t1.k], dist: UpstreamHashShard(t1.k) } +- name: Only table-in-out functions can have subquery parameters. + sql: | + SELECT * FROM generate_series(1, (select 1)); + binder_error: 'Invalid input syntax: Only table-in-out functions can have subquery parameters, generate_series only accepts constant parameters' +- name: While this one is allowed. + sql: | + SELECT generate_series(1, (select 1)); + batch_plan: |- + BatchProject { exprs: [GenerateSeries(1:Int32, $0)] } + └─BatchProjectSet { select_list: [GenerateSeries(1:Int32, $0)] } + └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } + ├─BatchValues { rows: [[]] } + └─BatchValues { rows: [[1:Int32]] } diff --git a/src/frontend/src/binder/relation/table_function.rs b/src/frontend/src/binder/relation/table_function.rs index 988ea0561a860..032791bfab30c 100644 --- a/src/frontend/src/binder/relation/table_function.rs +++ b/src/frontend/src/binder/relation/table_function.rs @@ -28,7 +28,7 @@ use crate::binder::bind_context::Clause; use crate::catalog::system_catalog::pg_catalog::{ PG_GET_KEYWORDS_FUNC_NAME, PG_KEYWORDS_TABLE_NAME, }; -use crate::expr::Expr; +use crate::expr::{Expr, ExprImpl}; impl Binder { /// Binds a table function AST, which is a function call in a relation position. @@ -125,6 +125,20 @@ impl Binder { self.pop_context()?; let func = func?; + if let ExprImpl::TableFunction(func) = &func { + if func + .args + .iter() + .any(|arg| matches!(arg, ExprImpl::Subquery(_))) + { + // Same error reports as DuckDB. + return Err(ErrorCode::InvalidInputSyntax( + format!("Only table-in-out functions can have subquery parameters, {} only accepts constant parameters", func.name()), + ) + .into()); + } + } + // bool indicates if the field is hidden let mut columns = if let DataType::Struct(s) = func.return_type() { // If the table function returns a struct, it will be flattened into multiple columns. From abf3a787a74a5576ddcee0c5e5169134f5f6e18f Mon Sep 17 00:00:00 2001 From: TennyZhuang Date: Fri, 27 Oct 2023 15:02:24 +0800 Subject: [PATCH 23/52] chore(deps): bump madsim-tokio to 0.2.24 (#13091) Signed-off-by: TennyZhuang --- Cargo.lock | 4 ++-- src/tests/e2e_extended_mode/Cargo.toml | 2 +- src/tests/simulation/Cargo.toml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a928849d0e03c..8f6acdbeae3d1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4604,9 +4604,9 @@ dependencies = [ [[package]] name = "madsim-tokio" -version = "0.2.23" +version = "0.2.24" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3b6df81cfc5c561311fa2e85ed52e8c6f3990b700751be914797addf984e72d" +checksum = "e5611fd0eb96867dd03a9fd2494d4c1bb126f413519673195065b6ea011e8c68" dependencies = [ "madsim", "spin 0.9.8", diff --git a/src/tests/e2e_extended_mode/Cargo.toml b/src/tests/e2e_extended_mode/Cargo.toml index ea83c5069c774..56012a64a25dc 100644 --- a/src/tests/e2e_extended_mode/Cargo.toml +++ b/src/tests/e2e_extended_mode/Cargo.toml @@ -19,7 +19,7 @@ chrono = { version = "0.4", features = ['serde'] } clap = { version = "4", features = ["derive"] } pg_interval = "0.4" rust_decimal ={ version = "1.32", features = ["db-postgres"] } -tokio = { version = "0.2.23", package = "madsim-tokio", features = ["rt", "macros","rt-multi-thread"] } +tokio = { version = "0.2.24", package = "madsim-tokio", features = ["rt", "macros","rt-multi-thread"] } tokio-postgres = { version = "0.7", features = ["with-chrono-0_4"] } tracing = "0.1" tracing-subscriber = "0.3.17" diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index b81be65edae42..734eccadd4c16 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -48,7 +48,7 @@ serde_json = "1.0.107" sqllogictest = "0.17.0" tempfile = "3" tikv-jemallocator = { workspace = true } -tokio = { version = "0.2.23", package = "madsim-tokio" } +tokio = { version = "0.2.24", package = "madsim-tokio" } tokio-postgres = "0.7" tokio-stream = "0.1" tracing = "0.1" From a1c47613434e94ba461e172c56c46436db5c78a2 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 27 Oct 2023 15:22:06 +0800 Subject: [PATCH 24/52] perf(connector): add integration benchmark for nexmark parsing (#13073) Signed-off-by: Bugen Zhao --- Cargo.lock | 1 + src/connector/Cargo.toml | 5 + src/connector/benches/nexmark_integration.rs | 138 +++++++++++++++++++ 3 files changed, 144 insertions(+) create mode 100644 src/connector/benches/nexmark_integration.rs diff --git a/Cargo.lock b/Cargo.lock index 8f6acdbeae3d1..0b7d0573bf22a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7373,6 +7373,7 @@ dependencies = [ "tonic 0.9.2", "tracing", "tracing-futures", + "tracing-subscriber", "tracing-test", "url", "urlencoding", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 87d2a0bdef689..8331adb8dc20a 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -139,6 +139,7 @@ workspace-hack = { path = "../workspace-hack" } criterion = { workspace = true, features = ["async_tokio", "async"] } rand = "0.8" tempfile = "3" +tracing-subscriber = "0.3" tracing-test = "0.2" [build-dependencies] @@ -148,5 +149,9 @@ prost-build = "0.12" name = "parser" harness = false +[[bench]] +name = "nexmark_integration" +harness = false + [lints] workspace = true diff --git a/src/connector/benches/nexmark_integration.rs b/src/connector/benches/nexmark_integration.rs new file mode 100644 index 0000000000000..4429db26de58a --- /dev/null +++ b/src/connector/benches/nexmark_integration.rs @@ -0,0 +1,138 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#![feature(lazy_cell)] + +use std::sync::LazyLock; + +use criterion::{criterion_group, criterion_main, BatchSize, Criterion}; +use futures::{FutureExt, StreamExt, TryStreamExt}; +use itertools::Itertools; +use risingwave_common::catalog::ColumnId; +use risingwave_common::types::DataType; +use risingwave_connector::parser::{ + ByteStreamSourceParser, JsonParser, SourceParserIntoStreamExt, SpecificParserConfig, +}; +use risingwave_connector::source::{ + BoxSourceStream, BoxSourceWithStateStream, SourceColumnDesc, SourceMessage, SourceMeta, + StreamChunkWithState, +}; +use tracing::Level; +use tracing_subscriber::prelude::*; + +static BATCH: LazyLock> = LazyLock::new(make_batch); + +fn make_batch() -> Vec { + let mut generator = nexmark::EventGenerator::default() + .with_type_filter(nexmark::event::EventType::Bid) + .map(|e| match e { + nexmark::event::Event::Bid(bid) => bid, // extract the bid event + _ => unreachable!(), + }) + .enumerate(); + + let message_base = SourceMessage { + split_id: "default".into(), + key: None, + payload: None, // to be filled + offset: "".into(), // to be filled + meta: SourceMeta::Empty, + }; + + generator + .by_ref() + .take(1024) + .map(|(i, e)| { + let payload = serde_json::to_vec(&e).unwrap(); + SourceMessage { + payload: Some(payload), + offset: i.to_string(), + ..message_base.clone() + } + }) + .collect_vec() +} + +fn make_data_stream() -> BoxSourceStream { + futures::future::ready(Ok(BATCH.clone())) + .into_stream() + .boxed() +} + +fn make_parser() -> impl ByteStreamSourceParser { + let columns = [ + ("auction", DataType::Int64), + ("bidder", DataType::Int64), + ("price", DataType::Int64), + ("channel", DataType::Varchar), + ("url", DataType::Varchar), + ("date_time", DataType::Timestamp), + ("extra", DataType::Varchar), + ] + .into_iter() + .enumerate() + .map(|(i, (n, t))| SourceColumnDesc::simple(n, t, ColumnId::new(i as _))) + .collect_vec(); + + let props = SpecificParserConfig::DEFAULT_PLAIN_JSON; + + JsonParser::new(props, columns, Default::default()).unwrap() +} + +fn make_stream_iter() -> impl Iterator { + let mut stream: BoxSourceWithStateStream = + make_parser().into_stream(make_data_stream()).boxed(); + + std::iter::from_fn(move || { + stream + .try_next() + .now_or_never() // there's actually no yield point + .unwrap() + .unwrap() + .unwrap() + .into() + }) +} + +fn bench(c: &mut Criterion) { + c.bench_function("parse_nexmark", |b| { + b.iter_batched( + make_stream_iter, + |mut iter| iter.next().unwrap(), + BatchSize::SmallInput, + ) + }); + + c.bench_function("parse_nexmark_with_tracing", |b| { + // Note: `From for Dispatch` has global side effects. Moving this out of `bench_function` + // does not work. Why? + let dispatch: tracing::dispatcher::Dispatch = tracing_subscriber::registry() + .with( + tracing_subscriber::fmt::layer().with_filter( + tracing_subscriber::filter::Targets::new() + .with_target("risingwave_connector", Level::INFO), + ), + ) + .into(); + + b.iter_batched( + make_stream_iter, + |mut iter| tracing::dispatcher::with_default(&dispatch, || iter.next().unwrap()), + BatchSize::SmallInput, + ) + }); +} + +criterion_group!(benches, bench); +criterion_main!(benches); From dd3d59a563c05299c4ee0f0c97adecbacb12ffed Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Fri, 27 Oct 2023 02:30:46 -0500 Subject: [PATCH 25/52] fix: column index mapping bug of `LogicalExpand::rewrite_with_input` (#13090) Signed-off-by: Richard Chien --- e2e_test/streaming/bug_fixes/issue_12620.slt | 44 +++++++ .../tests/testdata/output/agg.yaml | 24 ++-- .../tests/testdata/output/grouping_sets.yaml | 116 +++++++++--------- .../tests/testdata/output/nexmark.yaml | 58 ++++----- .../tests/testdata/output/nexmark_source.yaml | 88 ++++++------- .../testdata/output/nexmark_watermark.yaml | 16 +-- .../tests/testdata/output/subquery.yaml | 12 +- .../src/optimizer/plan_node/generic/expand.rs | 10 +- .../src/optimizer/plan_node/logical_expand.rs | 32 +++-- .../rule/grouping_sets_to_expand_rule.rs | 60 +++++---- 10 files changed, 267 insertions(+), 193 deletions(-) create mode 100644 e2e_test/streaming/bug_fixes/issue_12620.slt diff --git a/e2e_test/streaming/bug_fixes/issue_12620.slt b/e2e_test/streaming/bug_fixes/issue_12620.slt new file mode 100644 index 0000000000000..773c1b4cb3999 --- /dev/null +++ b/e2e_test/streaming/bug_fixes/issue_12620.slt @@ -0,0 +1,44 @@ +# https://github.com/risingwavelabs/risingwave/issues/12620 +# TL;DR Wrong output column change mapping when the upstream node of `LogicalExpand` produces output column mapping with `target_size` different from its output schema len. + +statement ok +SET RW_IMPLICIT_FLUSH TO TRUE; + +statement ok +CREATE TABLE part (p_partkey INT, p_name CHARACTER VARYING, p_mfgr CHARACTER VARYING, p_brand CHARACTER VARYING, p_type CHARACTER VARYING, p_size INT, p_container CHARACTER VARYING, p_retailprice NUMERIC, p_comment CHARACTER VARYING, PRIMARY KEY (p_partkey)); + +statement ok +CREATE TABLE auction (id BIGINT, item_name CHARACTER VARYING, description CHARACTER VARYING, initial_bid BIGINT, reserve BIGINT, date_time TIMESTAMP, expires TIMESTAMP, seller BIGINT, category BIGINT, extra CHARACTER VARYING, PRIMARY KEY (id)); + +statement ok +CREATE TABLE bid (auction BIGINT, bidder BIGINT, price BIGINT, channel CHARACTER VARYING, url CHARACTER VARYING, date_time TIMESTAMP, extra CHARACTER VARYING); + +statement ok +INSERT INTO part VALUES ((INT '-372542577'), 'WXiQlOLMp1', 'yKNCWQ0fr8', 'TXCuU8SZ9L', 'Ik5RMnntap', (INT '46'), 'DVRuRz6NSI', (473), 'D7ETlwOsRB'), ((INT '283'), 'uaAegjucZx', 'l4eE5J18ha', 'AKJLSon3O4', 'qR5xXnyLqX', (INT '641'), 's02i2IJJmv', (773), 'SF2ZkgFV1g'), ((INT '0'), 'IljPetC1kn', 'kXvT8PhcHU', 'DAbCiE5jiW', '453K1NO3vy', (INT '459'), '4LUnWwenGM', (-2147483648), 'ikB8QJpui2'), ((INT '442'), 'Ldzht8vDwx', 'BHb2KTtoKy', 'IlVMnhYr78', 'XVmgb3fne4', (INT '670'), 'X0uO2UHsFq', (456), 'voMOgn4hhG'), ((INT '85'), 'V29bZx7wFI', 'g3odozaztT', '493QsFCf1b', 'l4zX3nVo8N', (INT '650'), '9uW1PApXVs', (502793074), 'DxT9u5dEd9'), ((INT '884'), 'efwWgTdl1W', '0RYmAf9Wm8', 'cgCyHvRA87', 'kdlMfd6bbh', (INT '489'), 'KRdbaj41Yw', (-2147483648), 'iIojZe1E69'), ((INT '423'), 'aWekmXGDc7', 'pwit8GkYVe', 'JnufnOYvvO', '25dRBbPnmZ', (INT '533'), '2smOxLCuzV', (148), '42Vz4U7A5x'), ((INT '619'), 'SJqpnJSR18', 'PKwb7KYys9', '3pBaJAUfNT', 'yMCyiWtWo7', (INT '262'), 'LUdT4IqN1e', (0), 'ueOY3OSO5L'), ((INT '5'), '8ThDmKNPyj', 'hnYM2em4qw', 'VKurRm6ZAx', 'AswksUxdFJ', (INT '2147483647'), '85kU6KEMgl', (-2147483648), 'uGHrMLok1i'), ((INT '266'), '7TeqxZmdPk', 'wgvfw4IBdq', '3UPC1JuFC3', 'gqwzeByBOg', (INT '976'), 'xamfNkoAPz', (756), 'CaXSH4v0GC'), ((INT '1'), 'UPlQDjWqyb', '2cW34p9QZv', 'Lru9qYty99', 'kwRkYmcTlB', (INT '278'), 'hGPx2Ii39U', (713), 'TernQP9rbd'), ((INT '1'), '3HF5ukJslH', 'bbYsGByiwx', 'bJJYit3EQR', 'Ddq1ww3wd9', (INT '299'), 'oxYVxaTxiH', (-1119572313), 'CmosqGAbWX'), ((INT '-2147483648'), 'c95b6pjIDr', 'ILW0SgyAuT', 'UhbPvdDFax', 'cRMrGDrwsw', (INT '422'), 'Npn53n9MMp', (786), '2BIJ6AVHSL'), ((INT '426'), 'GmlCkwhI9q', '8rfpHCssLd', 'YgXEgZb4ND', 'hXFkcIkXjj', (INT '805'), 'k15qpaaDo7', (574), 'xpM4jIP3pg'), ((INT '132'), 'VLzbKI1ceN', 'AKnFr2eMYH', '0lPSyCcKWp', 'lmWzN32rJt', (INT '499'), 'Uoagt929TQ', (228), '6YmRhlxI4T'), ((INT '-2147483648'), 'fP4qVHos6M', 'c5s2jk1Et7', 'YXBGHV6u3M', 'DNjkaP7i4S', (INT '229'), '2GMyZUnrg2', (819), '11Bzeabscr'), ((INT '148'), 'QjEnL2qdFI', 'dAqMERXzuH', 'ZlUmzvf3zi', 's79Gxkm9yB', (INT '769196242'), 'bxsxOgabxJ', (811), 'CiEC9W9P58'), ((INT '344'), 'YWlcktntPN', 'HseXn1e5Lt', 'XXKCrIqr2s', 'Q7wbX7xDub', (INT '-2147483648'), '6o7BqBeUl9', (225), 'U7VdSSy6mv'), ((INT '924'), 'Rg63tbGVaR', 'JBl8vdj8Xq', 'pqhM1S7oIJ', '4mDXD62HAC', (INT '451'), '5A3eEo6BcA', (270), 'OR0p8NThcq'), ((INT '38'), '3bEQ57mUNw', 'vWTweCQapH', 'aarswhPcD9', '4RNUjdsqZy', (INT '689'), 'mm5v15G8Qb', (-1647149109), 'nLgD2YQ89g'), ((INT '208'), 'FPBIn2AAVl', '30KU8YB2pD', 'y6mZc794By', 'gJvZkuJ1e0', (INT '577'), 'cs2HwjzFNn', (573), 'oIBAe7mzBb'), ((INT '705'), '42PXi4EIm9', 'xit4z4WS22', 'NJ49gayobx', 'KJBVb97SwJ', (INT '884'), 'oDegSs9ayt', (14), 'DoLCX5UDsP'), ((INT '2147483647'), 'Q7z1b4smto', 'dFcTeBhzhi', 'qOvsbslcb6', 'VNxwEkGAOl', (INT '843'), 'FdPV1JtgSF', (403), 'BDhgEcsOGu'), ((INT '-2127900759'), 'L3ijaL5RUe', 'NyNhv1ojkM', 'ipNFeXaA3P', 'R1DhzWuPug', (INT '1981673467'), 'v48aQqtGzS', (1214914702), '01bVIF5IQv'), ((INT '-2147483648'), 'CUJZ4WZ2Ij', 'i4vDzSPBqy', '94uTeWChpz', 'kl0ieHX8zG', (INT '-2147483648'), 'bxW7Xv794x', (301), 'KHsRJuK9FX'), ((INT '107'), '1I18jPGlak', '9ZBwzTpZkT', 'V18f8vV9AC', 'NAVOfnZkPt', (INT '642'), '2xhKydI4El', (885), 'UZ69oVskjj'), ((INT '403'), 'AWZ71Jesc7', 'lf0FmtgW3T', 'Eh0f1GNZOd', 'B9uBbu9KFl', (INT '0'), 'W9cebQWomv', (-2147483648), 'gL6V07EJr5'), ((INT '834'), 'TNuCgnB9Py', 'mMlOsZ4egS', 'zlw0O76KV0', 'FH0eRCFyod', (INT '15'), '9w6T3a4hPp', (471022515), 'mUoxBmufcU'), ((INT '-1969538307'), 'aCTBcvqcA6', 'Tc3Uvtxuep', 'tk4lPfBPte', 'GCOlsJUcuY', (INT '1'), 'jYq1m0Gx1n', (0), 'im5SlwgMsr'), ((INT '143994250'), 'bHF0JGOxn9', 'hksln1WlhA', 'v2wrlggmOI', '0eLKUQeyuA', (INT '2147483647'), 'p5p7XP8DgY', (1), '2kaz3S3R2P'), ((INT '292'), 'VnOslgy80a', 'wZWozdunIM', 'NdnIipzxiA', 'kXP7V7pFk2', (INT '357'), 'zZtkKLWZi8', (140), 'n8cGFaBFwa'), ((INT '0'), 'W2QaamrDft', 'k3peUOmACD', 'HkJmTNMGWj', 'QlSyltYPmz', (INT '731'), 'HPBI4eTU1b', (187), 'rnaVSKtoXq'), ((INT '807'), '23FXXDzNBN', 'Bg2jsODOif', '3MJU4XaOfR', 'PIFicPtRpC', (INT '509'), 'hFJIDT74ub', (0), 'RaGuNrLLlm'), ((INT '848'), 'ZGVf7UVGlo', 'xWw7QymjG1', '43HsuBlG6s', 'QsBYqNJiFR', (INT '372'), 'aRPjfoUdkf', (255), 'aVFa7KkdoB'), ((INT '314'), 'NGwtzLXkcG', 'ckky8h3CvC', 'gXfMLQ2YkU', 'nPmcSXGPvW', (INT '773'), 'eUDD0ySJVw', (720837291), 'TcKb8ZQ2il'), ((INT '1'), 'y1jnuqi2of', 'RKR5dDwW9E', '9NrpmSlslV', 'ntu1jQgq5x', (INT '723'), 'IQu99zHsT1', (824), 'R90ekO0IKy'), ((INT '930'), 'hCiDr0ULIt', 'qhpNj3QDyd', 'Isx4gns8x5', 'pX9uBsXGLw', (INT '141'), 'pVHDWZAgun', (88), 'EjvPVNA1uh'), ((INT '307'), '38o9jpMv2b', 'bS3dWRdV7W', 'WzRTWnXlet', 'BNjNyHmUD0', (INT '934'), 'CMiFL883Mf', (0), 'Sz27XALbEo'), ((INT '738'), 'WnzZwCeXby', '5Xt7jR8nuL', 'iTPZFjEEAW', '1oHd9rw6DX', (INT '982'), 'n2aouYmUIo', (269), 'kDob18ZZB3'), ((INT '1316881894'), 'tp8tIz7BQ6', 'rA4QgXhb7O', 'YZj9arjRA5', 'xooINp89lE', (INT '609'), 'Y1UcVEGiRT', (368), 'LmQoLtciKd'), ((INT '961'), 'ogxY2lnuJS', 'MVUB4i4jPE', '62HgUqnkUI', 'VStwGHkEVD', (INT '-755643310'), 'S5k5YRfTYk', (159), 'wtN5SFArpk'), ((INT '0'), 'POVouVKeCj', '6LKQQPsUV0', 'jBGaFxFXKk', 'yqqdAZNsur', (INT '558'), 'w0jpZaMVAY', (512), 'ZeR80RdMDR'), ((INT '845'), '4Kz6wEMD4T', 'O9CogTu0E1', 'Jw8DZ4s79z', 'jCo3vW1wSK', (INT '769'), 'gAAz6v5mzo', (355), 'SSQdwrm73O'), ((INT '341'), 'lqMKAK5QiU', 'SNjGAFghl7', 'YN7R1rmboI', 'O7KoZ3tUni', (INT '581'), 'a9abrBNyki', (1368558480), 'Ooylm8IP03'), ((INT '1'), 'Vm0zZCVtmp', 'jg7MCkQ2C7', 'y6c1wv72fy', 'EeStdfxTx5', (INT '824'), 'q6gmiJLv63', (267), 'LOmGu3mMLh'), ((INT '138'), '85xi9y0aDx', '2qigYb9ww1', '9gaxdIKqUG', 'u6TbKzJshc', (INT '862'), 'yskfxejoN9', (2147483647), '7SHdikz8m3'), ((INT '609'), 'JvKATqgK1c', 'dJZ0ycuLk6', 'ExG5k65mPz', 'Ia96WreFET', (INT '88'), 'kTanOvQ0eX', (1), 'oyuglhfVSv'), ((INT '164870842'), 'Vu3OvugIdb', 'H1lK83nax4', 'TxGLP7NPSR', 'eswEPfqtuO', (INT '500'), 'RGJIkWiPGa', (753), 'BgfwJmskcQ'), ((INT '283'), '4xSJHTXbeF', '6sRGymfLW6', 'IdpNP49wxN', '8aJN0rfnWu', (INT '-1266364401'), '5uJi3maiCU', (752), 'VJc6DIOPJq'), ((INT '30'), 'mERGHSSEPh', 'fETrr0B74o', 'oQH5NMCc6C', 'ClwsdPG1Kq', (INT '0'), 'uKoJ6lplmc', (442), 'IJ0LGth58W'); + +statement ok +INSERT INTO auction VALUES ((BIGINT '38'), '28MsHtRX48', 'QVaTtpjz4g', (BIGINT '752'), (BIGINT '263'), TIMESTAMP '2023-10-03 20:31:30', TIMESTAMP '2023-10-03 19:33:02', (BIGINT '9223372036854775807'), (BIGINT '530'), 'cnpF74PLNl'), ((BIGINT '362'), 'BMAf4cqZ8v', 'wmOzhDt92h', (BIGINT '1'), (BIGINT '0'), TIMESTAMP '2023-10-03 20:32:53', TIMESTAMP '2023-10-03 20:32:25', (BIGINT '0'), (BIGINT '734'), 'Tbh6Sra9ea'), ((BIGINT '-9223372036854775808'), 'hSCQdaWP9J', 'udtQlkbgSG', (BIGINT '926'), (BIGINT '5814458343827052937'), TIMESTAMP '2023-10-03 20:33:02', TIMESTAMP '2023-10-03 19:33:02', (BIGINT '13'), (BIGINT '494'), 'CtcUKsfCeC'), ((BIGINT '8156629396116441350'), 'LKYgym1Lwc', 'TAd0sSC5EP', (BIGINT '9223372036854775807'), (BIGINT '697'), TIMESTAMP '2023-10-03 20:32:07', TIMESTAMP '2023-10-03 20:32:56', (BIGINT '4590950017041454046'), (BIGINT '128'), 'QBGHZco54p'), ((BIGINT '5012206546319524841'), 'egim3xKFXV', 'pmcMgHA4fX', (BIGINT '582'), (BIGINT '297'), TIMESTAMP '2023-10-03 20:31:51', TIMESTAMP '2023-10-03 20:31:52', (BIGINT '-9223372036854775808'), (BIGINT '617'), '6DqznGXUnx'), ((BIGINT '0'), 'zK6XwyUJZS', 'biVMTGD53g', (BIGINT '9223372036854775807'), (BIGINT '584'), TIMESTAMP '2023-10-03 20:31:50', TIMESTAMP '2023-10-02 20:33:02', (BIGINT '-9223372036854775808'), (BIGINT '-9223372036854775808'), '4WQyuz2fgX'), ((BIGINT '967'), '6tE8ZHhoSd', '7g9k473Xwl', (BIGINT '339'), (BIGINT '492'), TIMESTAMP '2023-09-26 20:33:02', TIMESTAMP '2023-10-03 20:32:02', (BIGINT '289'), (BIGINT '726'), 'isVoibkwyJ'), ((BIGINT '764'), 'qiGzac9F0o', '630RP6tNTl', (BIGINT '0'), (BIGINT '956'), TIMESTAMP '2023-10-03 20:31:26', TIMESTAMP '2023-10-03 20:31:33', (BIGINT '1'), (BIGINT '673'), 'OlJwjW11za'), ((BIGINT '60'), 'gOUBEnrjrv', '9SnSFeSlPO', (BIGINT '-8927887404158173337'), (BIGINT '766'), TIMESTAMP '2023-10-03 20:31:52', TIMESTAMP '2023-10-03 20:31:47', (BIGINT '578'), (BIGINT '988'), 'lPBtbGcnrl'), ((BIGINT '9223372036854775807'), 'pIekmZH4Dr', 'uzftaW955T', (BIGINT '788'), (BIGINT '264'), TIMESTAMP '2023-10-03 20:32:02', TIMESTAMP '2023-10-03 20:33:02', (BIGINT '482'), (BIGINT '112'), '4Wbf8YabwS'), ((BIGINT '910'), 'QsztDtt9zV', 'iVuhaPemov', (BIGINT '101'), (BIGINT '59'), TIMESTAMP '2023-10-03 20:33:01', TIMESTAMP '2023-10-03 20:32:55', (BIGINT '-8650673193240613193'), (BIGINT '613'), 'dygczVyXtf'), ((BIGINT '7562023557758454047'), 'i22lF2OVPQ', 'CqvlmwePm4', (BIGINT '128'), (BIGINT '980'), TIMESTAMP '2023-10-03 20:31:54', TIMESTAMP '2023-10-03 20:31:58', (BIGINT '720'), (BIGINT '0'), 'ZXnLCMiveV'), ((BIGINT '767'), 'yni20cGkzN', '66NUAc46Af', (BIGINT '800'), (BIGINT '746'), TIMESTAMP '2023-10-03 20:33:01', TIMESTAMP '2023-10-03 20:32:19', (BIGINT '-7323534369770906352'), (BIGINT '677'), '3dX3E2h87b'), ((BIGINT '653'), 'btEDjs0dEG', 'OkddWFYKOp', (BIGINT '904'), (BIGINT '1'), TIMESTAMP '2023-10-03 20:32:43', TIMESTAMP '2023-10-03 20:31:53', (BIGINT '-9223372036854775808'), (BIGINT '818'), 'L3fm1SOhhW'), ((BIGINT '979'), 'FuJYIgEst6', '0pW4Y1YBgK', (BIGINT '143'), (BIGINT '362'), TIMESTAMP '2023-10-03 20:31:49', TIMESTAMP '2023-10-03 20:31:49', (BIGINT '459'), (BIGINT '-5439985925919238580'), '8NjZ40jSkI'), ((BIGINT '546'), 'JFO93TCSWL', 'vW2cZRWGBE', (BIGINT '52'), (BIGINT '370'), TIMESTAMP '2023-10-03 20:32:54', TIMESTAMP '2023-10-03 20:32:02', (BIGINT '698'), (BIGINT '450'), 'c4tjXPs6Qz'), ((BIGINT '738'), 'rSpxvirEh7', 'iPrzw4geL5', (BIGINT '4859950609254809211'), (BIGINT '993'), TIMESTAMP '2023-10-03 20:32:02', TIMESTAMP '2023-10-03 20:32:43', (BIGINT '793'), (BIGINT '47'), 'vNHyGqC9Al'), ((BIGINT '-6320544476683661240'), 'f0iorSZOAW', 'SQBdyCLDtg', (BIGINT '644'), (BIGINT '906'), TIMESTAMP '2023-10-03 20:32:44', TIMESTAMP '2023-10-03 20:32:31', (BIGINT '0'), (BIGINT '678'), '0KI0i89Vq3'), ((BIGINT '417'), 'Yb6zlck0fQ', 'FssfdReg2d', (BIGINT '555'), (BIGINT '794'), TIMESTAMP '2023-10-03 20:32:02', TIMESTAMP '2023-10-03 20:32:52', (BIGINT '303'), (BIGINT '776'), 'v4z6MSX7AK'), ((BIGINT '706'), 'gIAdyoSoE7', 'qz9MzwsvC1', (BIGINT '402'), (BIGINT '7742953920353791929'), TIMESTAMP '2023-10-03 20:32:35', TIMESTAMP '2023-10-03 20:31:32', (BIGINT '9223372036854775807'), (BIGINT '9223372036854775807'), '5u65hHy2UZ'), ((BIGINT '725'), 'xNXgZKiYye', 'KvFEHdbI6j', (BIGINT '113'), (BIGINT '-4139256451612021380'), TIMESTAMP '2023-10-03 20:31:56', TIMESTAMP '2023-09-26 20:33:02', (BIGINT '92'), (BIGINT '179'), 'tD8wI2EYuy'), ((BIGINT '997'), '6q0QWJDoG0', '7qr43POJxq', (BIGINT '33'), (BIGINT '928'), TIMESTAMP '2023-10-03 20:32:06', TIMESTAMP '2023-10-03 20:32:40', (BIGINT '428'), (BIGINT '780'), 'nxR4XsI36J'), ((BIGINT '940'), '2uTMU5hFoP', 'hK7K1iAaBS', (BIGINT '78'), (BIGINT '429'), TIMESTAMP '2023-10-03 20:32:34', TIMESTAMP '2023-10-03 20:32:26', (BIGINT '4690983236096974460'), (BIGINT '996'), '5w0ok5m0xG'), ((BIGINT '264'), 'aJAmBmnJ57', 'YaESFufm0a', (BIGINT '241'), (BIGINT '709'), TIMESTAMP '2023-09-26 20:33:02', TIMESTAMP '2023-10-03 20:32:49', (BIGINT '407'), (BIGINT '495'), 'iMF4Q3jJhn'), ((BIGINT '775'), 'e75NdejMcW', 'T3czrClXev', (BIGINT '199'), (BIGINT '390'), TIMESTAMP '2023-10-03 20:31:29', TIMESTAMP '2023-10-03 20:31:25', (BIGINT '121'), (BIGINT '18'), '12HQvJFtPh'), ((BIGINT '9223372036854775807'), 'XdOR53LrWq', 'dRCEu20mTN', (BIGINT '534'), (BIGINT '915'), TIMESTAMP '2023-10-03 20:32:27', TIMESTAMP '2023-10-03 20:33:02', (BIGINT '0'), (BIGINT '190'), '9xZkZqjic3'), ((BIGINT '0'), '7fL5dQIfkX', 'zXfWw9TRsk', (BIGINT '9223372036854775807'), (BIGINT '337'), TIMESTAMP '2023-10-03 20:33:02', TIMESTAMP '2023-10-03 20:32:56', (BIGINT '-9223372036854775808'), (BIGINT '21'), 'uzLlx2o8Ak'), ((BIGINT '1814607164874825427'), 'o9k9eMHOmf', 'LNjQlM1vPr', (BIGINT '532'), (BIGINT '631'), TIMESTAMP '2023-10-03 20:31:51', TIMESTAMP '2023-10-02 20:33:02', (BIGINT '631'), (BIGINT '816'), '1TtALRP5uL'), ((BIGINT '616'), 'dRO9H0opT4', '1cHEFrvVzJ', (BIGINT '-7054838417642316298'), (BIGINT '887'), TIMESTAMP '2023-10-03 20:31:48', TIMESTAMP '2023-10-03 20:32:28', (BIGINT '638'), (BIGINT '666'), 's9wtVK5YGg'), ((BIGINT '580'), 'CvtaikrIsv', '15hQunGUFR', (BIGINT '108'), (BIGINT '-9223372036854775808'), TIMESTAMP '2023-10-03 20:32:41', TIMESTAMP '2023-09-26 20:33:02', (BIGINT '1'), (BIGINT '687'), '90GqB3CCHL'), ((BIGINT '9223372036854775807'), 'A5SPgGWh8d', 'DYcliKIm78', (BIGINT '945'), (BIGINT '826'), TIMESTAMP '2023-10-03 20:31:55', TIMESTAMP '2023-10-03 20:33:02', (BIGINT '-9223372036854775808'), (BIGINT '960'), 'zVOk2IsG0p'), ((BIGINT '716'), 'Ntat20sFQD', '2hyNxu7Cwe', (BIGINT '600'), (BIGINT '514'), TIMESTAMP '2023-10-03 20:31:45', TIMESTAMP '2023-10-03 20:32:02', (BIGINT '214'), (BIGINT '493'), 'AVEqVLuBVL'), ((BIGINT '9223372036854775807'), 'H7cAGNpMpH', 'P94WIuE1mh', (BIGINT '286'), (BIGINT '532'), TIMESTAMP '2023-10-03 20:33:01', TIMESTAMP '2023-10-03 20:31:30', (BIGINT '352'), (BIGINT '948'), 'uHQhjqFOMc'), ((BIGINT '764'), 'oV5zkZgpjh', 'bYjs0xnPBB', (BIGINT '280'), (BIGINT '427'), TIMESTAMP '2023-10-02 20:33:02', TIMESTAMP '2023-10-03 20:32:02', (BIGINT '969'), (BIGINT '638'), 'DoAu7mcihA'), ((BIGINT '241'), '8gAg5ZAZm0', 'jlsFEHElzy', (BIGINT '767'), (BIGINT '608'), TIMESTAMP '2023-10-03 20:33:01', TIMESTAMP '2023-10-03 20:33:02', (BIGINT '959'), (BIGINT '806'), '3PelZVzKFt'), ((BIGINT '-9223372036854775808'), 'jsY1nNsDsa', 'ezDBcoNF8n', (BIGINT '999'), (BIGINT '879'), TIMESTAMP '2023-10-03 20:32:41', TIMESTAMP '2023-10-03 20:32:35', (BIGINT '1479482111856095930'), (BIGINT '430'), 'ydqcn9npbx'), ((BIGINT '910527018521782926'), 'JsbGF3kYOM', '8CVBixXa9h', (BIGINT '673'), (BIGINT '841'), TIMESTAMP '2023-10-03 20:32:10', TIMESTAMP '2023-09-26 20:33:02', (BIGINT '763'), (BIGINT '958'), 'J609X0WX2U'), ((BIGINT '771'), 'TLqjnM9s4z', 'GSwIt0kdQd', (BIGINT '353'), (BIGINT '-5017954508784194721'), TIMESTAMP '2023-10-03 20:33:02', TIMESTAMP '2023-10-03 20:32:24', (BIGINT '820'), (BIGINT '-3313022488966569990'), 'ti7i3BtYcB'), ((BIGINT '275'), 'wICQnlmpQ8', 'RKbSrYCQIT', (BIGINT '914'), (BIGINT '425'), TIMESTAMP '2023-10-03 20:32:33', TIMESTAMP '2023-10-03 20:31:51', (BIGINT '-7804905732576356809'), (BIGINT '-1725860857621923054'), '3Ue6lkIZG2'), ((BIGINT '219'), 'lEayiFoO9o', 'OfRDAXJmGb', (BIGINT '604'), (BIGINT '82'), TIMESTAMP '2023-10-03 20:32:18', TIMESTAMP '2023-10-03 20:33:01', (BIGINT '-9113972164284914643'), (BIGINT '703'), 'Uu3OoWDOcd'), ((BIGINT '817'), 'wRKoCISTlg', 'JHiPMclVuJ', (BIGINT '1'), (BIGINT '89'), TIMESTAMP '2023-10-03 20:31:29', TIMESTAMP '2023-10-02 20:33:02', (BIGINT '564'), (BIGINT '-9223372036854775808'), '9ntXInZnwx'), ((BIGINT '29'), 'u92Q3mU0uE', 'w3BIeZSS1U', (BIGINT '957'), (BIGINT '364'), TIMESTAMP '2023-10-03 20:33:02', TIMESTAMP '2023-10-03 20:32:23', (BIGINT '-9223372036854775808'), (BIGINT '0'), 'Hmd5DWIvpK'), ((BIGINT '-5814494892331957830'), 'YFwxbhaxqb', 'T6ai1ZiQhG', (BIGINT '51'), (BIGINT '507'), TIMESTAMP '2023-10-03 20:32:36', TIMESTAMP '2023-10-03 20:31:50', (BIGINT '645'), (BIGINT '606'), 'vkS90fXF9p'), ((BIGINT '106'), 'D8MVriPvPO', '37GG155Ohn', (BIGINT '-2087101760633867781'), (BIGINT '847'), TIMESTAMP '2023-10-03 20:31:25', TIMESTAMP '2023-10-03 20:32:28', (BIGINT '1'), (BIGINT '-3638389483963085576'), 'kWKIhn0NZa'), ((BIGINT '727'), 'VvQ3Wvn7sS', 'gwM9Tr1Ona', (BIGINT '906'), (BIGINT '741'), TIMESTAMP '2023-10-03 20:32:20', TIMESTAMP '2023-10-03 20:31:49', (BIGINT '469'), (BIGINT '766'), '7xkWMKmUEp'), ((BIGINT '-9223372036854775808'), '38gyOwa0h6', 'M5uslwjzdk', (BIGINT '916'), (BIGINT '16'), TIMESTAMP '2023-10-03 20:33:02', TIMESTAMP '2023-10-03 20:32:02', (BIGINT '-9223372036854775808'), (BIGINT '459'), 'GOBXZkRfeK'), ((BIGINT '859040557973202211'), '5AbnCjFRv3', 'WWfJ6lBRqS', (BIGINT '0'), (BIGINT '1'), TIMESTAMP '2023-10-03 20:32:10', TIMESTAMP '2023-10-03 20:31:42', (BIGINT '936'), (BIGINT '415'), '8pfR47KQHe'), ((BIGINT '410'), 'lmrUV2nT2b', 'DY7GjRvnyg', (BIGINT '9223372036854775807'), (BIGINT '569'), TIMESTAMP '2023-10-03 20:33:02', TIMESTAMP '2023-10-03 20:33:01', (BIGINT '265'), (BIGINT '107'), '9awmVZsOtY'), ((BIGINT '399'), 'qCEyQCsSD7', 'qdCJcEM0vV', (BIGINT '277'), (BIGINT '9223372036854775807'), TIMESTAMP '2023-10-03 20:32:52', TIMESTAMP '2023-10-03 20:31:58', (BIGINT '534'), (BIGINT '879'), 'JcUEeJYF5y'), ((BIGINT '95'), 'dXsmAf1SAw', '7SrFVJMN31', (BIGINT '758'), (BIGINT '258'), TIMESTAMP '2023-10-03 20:33:02', TIMESTAMP '2023-10-03 20:31:31', (BIGINT '-220234528822828794'), (BIGINT '765'), 'wiQmoHUPMt'); + +statement ok +INSERT INTO bid VALUES ((BIGINT '480'), (BIGINT '294'), (BIGINT '0'), 'OEpfcZkR0h', '2AfUjqRjem', TIMESTAMP '2023-10-03 20:32:43', 'gXJfm8YhJS'), ((BIGINT '138'), (BIGINT '328'), (BIGINT '6190494391463723767'), 'nR5KBsa0rp', 'MSJ6rKHKly', TIMESTAMP '2023-10-03 20:31:29', '8XaJAXmoGX'), ((BIGINT '40'), (BIGINT '-9223372036854775808'), (BIGINT '816'), '4rFxg76s59', 'ykJEzodyo8', TIMESTAMP '2023-10-03 20:32:30', 'fHqSzwAt1f'), ((BIGINT '780'), (BIGINT '-9223372036854775808'), (BIGINT '1'), 'cIxwxQwehj', 'YdL5bSIML3', TIMESTAMP '2023-10-03 20:33:02', 'sGoHrWFw4z'), ((BIGINT '-9223372036854775808'), (BIGINT '884'), (BIGINT '887'), 'GW06e1u5By', 'GImZjVfvlB', TIMESTAMP '2023-10-03 20:32:27', 'Pirc18rQOA'), ((BIGINT '640'), (BIGINT '389'), (BIGINT '8038613313277906394'), '4tgpPeslEv', 'bISM0rdlTg', TIMESTAMP '2023-10-03 20:32:11', 'Wus7JeeNXK'), ((BIGINT '0'), (BIGINT '1'), (BIGINT '804'), 'IOIkW0RHkr', 'hFGLaNt7M6', TIMESTAMP '2023-10-03 20:32:20', 'tTe7PjscQS'), ((BIGINT '360'), (BIGINT '819'), (BIGINT '9223372036854775807'), 'MTZ91Pdlnp', 'Xi1txBjLkA', TIMESTAMP '2023-09-26 20:33:02', 'oz1OnXX5OT'), ((BIGINT '903'), (BIGINT '23'), (BIGINT '289'), 'OxhCIFjDMJ', '5fnCSFGzE5', TIMESTAMP '2023-10-03 20:32:02', '5kAufkBCSd'), ((BIGINT '271'), (BIGINT '252'), (BIGINT '699'), '2QrcWh2wE2', 'YdQChIIixp', TIMESTAMP '2023-10-03 20:32:40', 'XgohUXIXLn'), ((BIGINT '258'), (BIGINT '896'), (BIGINT '2032144812801576832'), '9q5iwNGsZ5', '9LtqSgZXaM', TIMESTAMP '2023-10-03 20:32:55', '7JqsWBYvAU'), ((BIGINT '73'), (BIGINT '827'), (BIGINT '216'), 'fMAfjSztE3', 'YYhgtGCDge', TIMESTAMP '2023-10-03 19:33:02', 'joW1Pydxqt'), ((BIGINT '-9223372036854775808'), (BIGINT '-9223372036854775808'), (BIGINT '731'), 'YZgTAPbPHk', 'GENaJSuIRu', TIMESTAMP '2023-10-03 20:32:15', 'FjQxWNy6sE'), ((BIGINT '534'), (BIGINT '-9223372036854775808'), (BIGINT '417'), 'M64KzZVBiM', 'jjMPib7WqH', TIMESTAMP '2023-10-03 20:32:03', '5H20NiYmzS'), ((BIGINT '543'), (BIGINT '492146098246613262'), (BIGINT '86'), 'GeRFvtlbP5', '2n7pc86xAL', TIMESTAMP '2023-10-03 20:32:49', 'ohI5G0brhC'), ((BIGINT '172'), (BIGINT '9223372036854775807'), (BIGINT '1'), 'EFEuy3jE8V', 'FxHILDEt2v', TIMESTAMP '2023-10-03 20:32:03', 'YA9F8P6LeO'), ((BIGINT '858'), (BIGINT '175'), (BIGINT '344'), 'DUqiocFkjR', 'rg5T4cMruK', TIMESTAMP '2023-10-03 20:32:34', 'pq65Kf6iZI'), ((BIGINT '52'), (BIGINT '-6811232900748935635'), (BIGINT '158'), 'v7mbWFM2FQ', 'Upv1CKDZAe', TIMESTAMP '2023-10-03 20:33:01', 'OakYz0a3t3'), ((BIGINT '653'), (BIGINT '635'), (BIGINT '7570630020121374916'), 'sd3cpW3tBR', '10oqwqiZD9', TIMESTAMP '2023-10-03 20:32:51', 'gAqygv65R4'), ((BIGINT '-1018295940439782764'), (BIGINT '502'), (BIGINT '885'), 'MBKZNojED3', '5dEVApa0VQ', TIMESTAMP '2023-10-03 20:31:54', 'npoSDLaADQ'), ((BIGINT '76'), (BIGINT '6217304740600455919'), (BIGINT '408'), 'loKGMW8N3J', '4YGOoz9GiG', TIMESTAMP '2023-10-03 20:32:06', 'VmQ3begoKk'), ((BIGINT '118'), (BIGINT '187'), (BIGINT '115'), 'DoTDc6WOzm', 'Ebs5WXlcnV', TIMESTAMP '2023-10-03 20:32:09', 'E5Uh1rqtZk'), ((BIGINT '1'), (BIGINT '225'), (BIGINT '7'), '6soFd54NZC', 'sgLTHkhJpI', TIMESTAMP '2023-10-03 20:32:02', 'XZZ4oTOzF5'), ((BIGINT '390'), (BIGINT '678'), (BIGINT '151'), 'xLf78hDs5s', 'avs3Jh2K86', TIMESTAMP '2023-10-03 20:32:56', 'AuhRn64tVQ'), ((BIGINT '0'), (BIGINT '972'), (BIGINT '19'), 'XYGl23IUV6', 'zVnssjuus2', TIMESTAMP '2023-10-03 20:31:49', 'Zzp33H9dJd'), ((BIGINT '547'), (BIGINT '89'), (BIGINT '696'), 'o9nrxLhqit', 'IyHi3SDxRC', TIMESTAMP '2023-10-03 20:33:01', 'hpfS2PyU6P'), ((BIGINT '858'), (BIGINT '288'), (BIGINT '-9223372036854775808'), 'OmWXimSDLM', '5qdirkoFu3', TIMESTAMP '2023-10-03 20:31:46', 'vDSiAZB9cw'), ((BIGINT '661'), (BIGINT '1'), (BIGINT '871'), 'ao9lFGTL3D', 'eAQN6VPFBv', TIMESTAMP '2023-10-03 20:32:44', 'RrCeBqA1nB'), ((BIGINT '974'), (BIGINT '164'), (BIGINT '760'), 'anxFf5qHkM', 'Urj87HaVoW', TIMESTAMP '2023-10-03 20:32:31', 'AI5clJ7Mvm'), ((BIGINT '118'), (BIGINT '787'), (BIGINT '235'), 'YZRhur7haK', 'C2IhpwuqBV', TIMESTAMP '2023-10-03 20:33:02', 'PHqf1WlKHq'), ((BIGINT '240'), (BIGINT '9223372036854775807'), (BIGINT '930'), 'NjDjoAk9V6', 'Z45wdBASsX', TIMESTAMP '2023-10-02 20:33:02', 'SjloqMXqze'), ((BIGINT '887'), (BIGINT '275'), (BIGINT '448'), 'zKFb52hNzb', 'GdMQZ5JRg6', TIMESTAMP '2023-10-03 20:32:46', 't17xud8f7N'), ((BIGINT '263'), (BIGINT '-2565329144012062115'), (BIGINT '0'), 'KgsycFoQz2', 'oSo5kQpEZx', TIMESTAMP '2023-10-03 19:33:02', '6PcpL07wAW'), ((BIGINT '245'), (BIGINT '0'), (BIGINT '-9223372036854775808'), 'eKBadXQiKS', 'UFkSfXifee', TIMESTAMP '2023-10-03 20:32:36', 'nIf5uBhJhX'), ((BIGINT '822'), (BIGINT '9223372036854775807'), (BIGINT '129'), 'pbT5sXYFb5', 'O5zIivAra2', TIMESTAMP '2023-10-03 20:32:37', 'AAHBKS6iLM'), ((BIGINT '200'), (BIGINT '-9223372036854775808'), (BIGINT '-3406295100039320832'), 'KS1mjvueiN', '2CkJcTSLn0', TIMESTAMP '2023-10-03 20:32:30', 'qfEbqJl7hp'), ((BIGINT '654'), (BIGINT '80'), (BIGINT '585'), 'aDV8MDjNP1', '06ZUtssow3', TIMESTAMP '2023-10-03 20:31:52', '7Z9Xw6tjnX'), ((BIGINT '1992384720676355545'), (BIGINT '609628433314068580'), (BIGINT '121'), '8YgoJciU4q', 'yYdCrIaMG1', TIMESTAMP '2023-10-03 20:32:06', 'kwb7JQHSfA'), ((BIGINT '264'), (BIGINT '0'), (BIGINT '7121019654275208159'), 'RU2kTHh21X', '7qH0XUArTg', TIMESTAMP '2023-10-03 20:32:27', 'CMjvPzUz5h'), ((BIGINT '539'), (BIGINT '962'), (BIGINT '213'), 'bRrqWotQYa', 'Pg3IYtr2v7', TIMESTAMP '2023-09-26 20:33:02', 'AaqnpKV9hw'), ((BIGINT '9223372036854775807'), (BIGINT '205'), (BIGINT '-7297979838675035456'), 'Jz4wQc32VQ', 'cYH3iT7Ugy', TIMESTAMP '2023-10-03 20:31:49', 'HeqBU0aUvZ'), ((BIGINT '310'), (BIGINT '321'), (BIGINT '315'), 'Ue3zdm8bce', '92U5KOjhJX', TIMESTAMP '2023-10-03 20:33:01', 'xQPzWy15h0'), ((BIGINT '-9223372036854775808'), (BIGINT '214'), (BIGINT '275'), 'zzBlZegGoR', 'hWNpjwpRRV', TIMESTAMP '2023-10-03 20:31:44', '72UfK0FszD'), ((BIGINT '-737992005867829895'), (BIGINT '247'), (BIGINT '0'), 'VXV9ITd1PI', 'wBOSdHwghB', TIMESTAMP '2023-10-03 20:32:14', 'u9Qpne5SJF'), ((BIGINT '610'), (BIGINT '246'), (BIGINT '5784670931058355539'), 'gLTPMbKmMF', 'aLuZuZdnuz', TIMESTAMP '2023-10-03 20:31:42', '9qmUkFWWW0'), ((BIGINT '207'), (BIGINT '-9223372036854775808'), (BIGINT '9223372036854775807'), '242dCkThjf', 'D8sSvChwNe', TIMESTAMP '2023-10-03 20:32:16', 'QF2wVcuMHU'), ((BIGINT '762'), (BIGINT '252'), (BIGINT '574'), 'iYjrPM7DBC', 'DKiulwA99J', TIMESTAMP '2023-10-03 20:32:48', 'c2507XTVza'), ((BIGINT '183'), (BIGINT '866'), (BIGINT '551'), '9fVEd6lDtC', '1mOPDVw2kk', TIMESTAMP '2023-10-03 20:32:51', 'IozEEl7DwL'), ((BIGINT '169'), (BIGINT '573'), (BIGINT '7487214739806778793'), 'KCz35Suwgk', 'ZZXtQ8sG0B', TIMESTAMP '2023-10-03 20:31:42', 'nNsUek849P'), ((BIGINT '199'), (BIGINT '735'), (BIGINT '993'), '3IuFqAyVHX', 'Y1UFLz2wJU', TIMESTAMP '2023-10-03 20:31:50', 'to4fdxg0NA'); + +statement ok +CREATE MATERIALIZED VIEW m9 AS SELECT TIMESTAMP '2023-10-03 20:32:03' AS col_0, (tumble_0.price + (SMALLINT '726')) AS col_1, tumble_0.price AS col_2 FROM tumble(bid, bid.date_time, INTERVAL '10') AS tumble_0 GROUP BY tumble_0.price HAVING true; + +statement ok +CREATE MATERIALIZED VIEW stream_query AS SELECT ((INT '850') % t_0.p_size) AS col_0, hop_1.id AS col_1, TIME '20:34:09' AS col_2, last_value(DISTINCT hop_1.item_name ORDER BY hop_1.item_name ASC NULLS FIRST) AS col_3 FROM (part AS t_0 FULL JOIN hop(auction, auction.expires, INTERVAL '51', INTERVAL '816') AS hop_1 ON t_0.p_type = hop_1.item_name) FULL JOIN m9 AS t_2 ON hop_1.initial_bid = t_2.col_1 AND true AND hop_1.reserve = t_2.col_2 AND hop_1.date_time = t_2.col_0 AND hop_1.expires = t_2.col_0 AND hop_1.id = t_2.col_2 GROUP BY GROUPING SETS ((t_0.p_mfgr, hop_1.id, t_0.p_size, t_0.p_partkey, hop_1.category, hop_1.reserve), (t_0.p_size), (t_0.p_container, t_0.p_retailprice, t_0.p_brand), (t_2.col_1, t_0.p_comment, t_0.p_name)) HAVING true; + +statement ok +DROP MATERIALIZED VIEW stream_query; + +statement ok +DROP MATERIALIZED VIEW m9; + +statement ok +DROP TABLE bid; + +statement ok +DROP TABLE auction; + +statement ok +DROP TABLE part; diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 058fa2a794d1e..be8e1e3a4a1b1 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -804,8 +804,8 @@ create table t(a int, b int, c int); select a, count(distinct b) as distinct_b_num, sum(distinct c) filter(where c < 100) as distinct_c_sum from t group by a; optimized_logical_plan_for_batch: |- - LogicalAgg { group_key: [t.a], aggs: [count(t.b) filter((flag = 0:Int64)), sum(t.c) filter((count filter((t.c < 100:Int32)) > 0:Int64) AND (flag = 1:Int64))] } - └─LogicalAgg { group_key: [t.a, t.b, t.c, flag], aggs: [count filter((t.c < 100:Int32))] } + LogicalAgg { group_key: [t.a_expanded], aggs: [count(t.b_expanded) filter((flag = 0:Int64)), sum(t.c_expanded) filter((count filter((t.c < 100:Int32)) > 0:Int64) AND (flag = 1:Int64))] } + └─LogicalAgg { group_key: [t.a_expanded, t.b_expanded, t.c_expanded, flag], aggs: [count filter((t.c < 100:Int32))] } └─LogicalExpand { column_subsets: [[t.a, t.b], [t.a, t.c]] } └─LogicalScan { table: t, columns: [t.a, t.b, t.c] } - name: single distinct agg and non-disintct agg @@ -834,16 +834,16 @@ create table t(a int, b int, c int); select a, count(distinct b) as distinct_b_num, count(distinct c) as distinct_c_sum, sum(c) as sum_c from t group by a; optimized_logical_plan_for_batch: |- - LogicalAgg { group_key: [t.a], aggs: [count(t.b) filter((flag = 1:Int64)), count(t.c) filter((flag = 0:Int64)), sum(sum(t.c)) filter((flag = 0:Int64))] } - └─LogicalAgg { group_key: [t.a, t.b, t.c, flag], aggs: [sum(t.c)] } + LogicalAgg { group_key: [t.a_expanded], aggs: [count(t.b_expanded) filter((flag = 1:Int64)), count(t.c_expanded) filter((flag = 0:Int64)), sum(sum(t.c_expanded)) filter((flag = 0:Int64))] } + └─LogicalAgg { group_key: [t.a_expanded, t.b_expanded, t.c_expanded, flag], aggs: [sum(t.c_expanded)] } └─LogicalExpand { column_subsets: [[t.a, t.c], [t.a, t.b]] } └─LogicalScan { table: t, columns: [t.a, t.b, t.c] } batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [t.a], aggs: [count(t.b) filter((flag = 1:Int64)), count(t.c) filter((flag = 0:Int64)), sum(sum(t.c)) filter((flag = 0:Int64))] } - └─BatchExchange { order: [], dist: HashShard(t.a) } - └─BatchHashAgg { group_key: [t.a, t.b, t.c, flag], aggs: [sum(t.c)] } - └─BatchExchange { order: [], dist: HashShard(t.a, t.b, t.c, flag) } + └─BatchHashAgg { group_key: [t.a_expanded], aggs: [count(t.b_expanded) filter((flag = 1:Int64)), count(t.c_expanded) filter((flag = 0:Int64)), sum(sum(t.c_expanded)) filter((flag = 0:Int64))] } + └─BatchExchange { order: [], dist: HashShard(t.a_expanded) } + └─BatchHashAgg { group_key: [t.a_expanded, t.b_expanded, t.c_expanded, flag], aggs: [sum(t.c_expanded)] } + └─BatchExchange { order: [], dist: HashShard(t.a_expanded, t.b_expanded, t.c_expanded, flag) } └─BatchExpand { column_subsets: [[t.a, t.c], [t.a, t.b]] } └─BatchScan { table: t, columns: [t.a, t.b, t.c], distribution: SomeShard } stream_plan: |- @@ -1039,13 +1039,13 @@ create table t(x int, y int); select count(distinct x), sum(distinct y) from t; optimized_logical_plan_for_batch: |- - LogicalAgg { aggs: [count(t.x) filter((flag = 0:Int64)), sum(t.y) filter((flag = 1:Int64))] } - └─LogicalAgg { group_key: [t.x, t.y, flag], aggs: [] } + LogicalAgg { aggs: [count(t.x_expanded) filter((flag = 0:Int64)), sum(t.y_expanded) filter((flag = 1:Int64))] } + └─LogicalAgg { group_key: [t.x_expanded, t.y_expanded, flag], aggs: [] } └─LogicalExpand { column_subsets: [[t.x], [t.y]] } └─LogicalScan { table: t, columns: [t.x, t.y] } optimized_logical_plan_for_stream: |- - LogicalAgg { aggs: [count(t.x) filter((flag = 0:Int64)), sum(t.y) filter((flag = 1:Int64))] } - └─LogicalAgg { group_key: [t.x, t.y, flag], aggs: [] } + LogicalAgg { aggs: [count(t.x_expanded) filter((flag = 0:Int64)), sum(t.y_expanded) filter((flag = 1:Int64))] } + └─LogicalAgg { group_key: [t.x_expanded, t.y_expanded, flag], aggs: [] } └─LogicalExpand { column_subsets: [[t.x], [t.y]] } └─LogicalScan { table: t, columns: [t.x, t.y, t._row_id] } with_config_map: diff --git a/src/frontend/planner_test/tests/testdata/output/grouping_sets.yaml b/src/frontend/planner_test/tests/testdata/output/grouping_sets.yaml index ca4eb9f4f04ad..95700da01095e 100644 --- a/src/frontend/planner_test/tests/testdata/output/grouping_sets.yaml +++ b/src/frontend/planner_test/tests/testdata/output/grouping_sets.yaml @@ -5,16 +5,16 @@ SELECT brand, size, sum(sales) FROM items_sold GROUP BY GROUPING SETS ((brand), (size), ()); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales)] } - └─BatchHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales)] } - └─BatchExchange { order: [], dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─BatchProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales)] } + └─BatchHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales)] } + └─BatchExchange { order: [], dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─BatchExpand { column_subsets: [[items_sold.brand], [items_sold.size], []] } └─BatchScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [brand, size, sum, flag(hidden)], stream_key: [brand, size, flag], pk_columns: [brand, size, flag], pk_conflict: NoCheck } - └─StreamProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales), flag] } - └─StreamHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales), count] } - └─StreamExchange { dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─StreamProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales), flag] } + └─StreamHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales), count] } + └─StreamExchange { dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─StreamExpand { column_subsets: [[items_sold.brand], [items_sold.size], []] } └─StreamTableScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales, items_sold._row_id], pk: [items_sold._row_id], dist: UpstreamHashShard(items_sold._row_id) } - name: with distinct @@ -23,18 +23,18 @@ SELECT brand, size, sum(distinct sales) FROM items_sold GROUP BY GROUPING SETS ((brand), (size), ()); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales)] } - └─BatchHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales)] } - └─BatchExchange { order: [], dist: HashShard(items_sold.brand, items_sold.size, flag) } - └─BatchHashAgg { group_key: [items_sold.brand, items_sold.size, items_sold.sales, flag], aggs: [] } - └─BatchExchange { order: [], dist: HashShard(items_sold.brand, items_sold.size, items_sold.sales, flag) } + └─BatchProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales)] } + └─BatchHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales)] } + └─BatchExchange { order: [], dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } + └─BatchHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, items_sold.sales, flag], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, items_sold.sales, flag) } └─BatchExpand { column_subsets: [[items_sold.brand], [items_sold.size], []] } └─BatchScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [brand, size, sum, flag(hidden)], stream_key: [brand, size, flag], pk_columns: [brand, size, flag], pk_conflict: NoCheck } - └─StreamProject { exprs: [items_sold.brand, items_sold.size, sum(distinct items_sold.sales), flag] } - └─StreamHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(distinct items_sold.sales), count] } - └─StreamExchange { dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─StreamProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(distinct items_sold.sales), flag] } + └─StreamHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(distinct items_sold.sales), count] } + └─StreamExchange { dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─StreamExpand { column_subsets: [[items_sold.brand], [items_sold.size], []] } └─StreamTableScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales, items_sold._row_id], pk: [items_sold._row_id], dist: UpstreamHashShard(items_sold._row_id) } - name: column pruning @@ -43,17 +43,17 @@ SELECT brand, size, sum(sales) FROM items_sold GROUP BY GROUPING SETS ((size), (brand), ()); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales)] } - └─BatchHashAgg { group_key: [items_sold.size, items_sold.brand, flag], aggs: [sum(items_sold.sales)] } - └─BatchExchange { order: [], dist: HashShard(items_sold.size, items_sold.brand, flag) } + └─BatchProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales)] } + └─BatchHashAgg { group_key: [items_sold.size_expanded, items_sold.brand_expanded, flag], aggs: [sum(items_sold.sales)] } + └─BatchExchange { order: [], dist: HashShard(items_sold.size_expanded, items_sold.brand_expanded, flag) } └─BatchExpand { column_subsets: [[items_sold.size], [items_sold.brand], []] } └─BatchProject { exprs: [items_sold.size, items_sold.brand, items_sold.sales] } └─BatchScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [brand, size, sum, flag(hidden)], stream_key: [size, brand, flag], pk_columns: [size, brand, flag], pk_conflict: NoCheck } - └─StreamProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales), flag] } - └─StreamHashAgg { group_key: [items_sold.size, items_sold.brand, flag], aggs: [sum(items_sold.sales), count] } - └─StreamExchange { dist: HashShard(items_sold.size, items_sold.brand, flag) } + └─StreamProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales), flag] } + └─StreamHashAgg { group_key: [items_sold.size_expanded, items_sold.brand_expanded, flag], aggs: [sum(items_sold.sales), count] } + └─StreamExchange { dist: HashShard(items_sold.size_expanded, items_sold.brand_expanded, flag) } └─StreamExpand { column_subsets: [[items_sold.size], [items_sold.brand], []] } └─StreamProject { exprs: [items_sold.size, items_sold.brand, items_sold.sales, items_sold._row_id] } └─StreamTableScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales, items_sold._row_id], pk: [items_sold._row_id], dist: UpstreamHashShard(items_sold._row_id) } @@ -63,18 +63,18 @@ SELECT brand, size, sum(sales), grouping(brand) g1, grouping(size) g2, grouping(brand,size) g3, count(distinct sales) FROM items_sold GROUP BY GROUPING SETS ((brand), (size), ()); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [items_sold.brand, items_sold.size, sum(sum(items_sold.sales)), Case((0:Int64 = flag), 0:Int32, (1:Int64 = flag), 1:Int32, (2:Int64 = flag), 1:Int32) as $expr1, Case((0:Int64 = flag), 1:Int32, (1:Int64 = flag), 0:Int32, (2:Int64 = flag), 1:Int32) as $expr2, Case((0:Int64 = flag), 1:Int32, (1:Int64 = flag), 2:Int32, (2:Int64 = flag), 3:Int32) as $expr3, count(items_sold.sales)] } - └─BatchHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(sum(items_sold.sales)), count(items_sold.sales)] } - └─BatchExchange { order: [], dist: HashShard(items_sold.brand, items_sold.size, flag) } - └─BatchHashAgg { group_key: [items_sold.brand, items_sold.size, items_sold.sales, flag], aggs: [sum(items_sold.sales)] } - └─BatchExchange { order: [], dist: HashShard(items_sold.brand, items_sold.size, items_sold.sales, flag) } + └─BatchProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(sum(items_sold.sales)), Case((0:Int64 = flag), 0:Int32, (1:Int64 = flag), 1:Int32, (2:Int64 = flag), 1:Int32) as $expr1, Case((0:Int64 = flag), 1:Int32, (1:Int64 = flag), 0:Int32, (2:Int64 = flag), 1:Int32) as $expr2, Case((0:Int64 = flag), 1:Int32, (1:Int64 = flag), 2:Int32, (2:Int64 = flag), 3:Int32) as $expr3, count(items_sold.sales)] } + └─BatchHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(sum(items_sold.sales)), count(items_sold.sales)] } + └─BatchExchange { order: [], dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } + └─BatchHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, items_sold.sales, flag], aggs: [sum(items_sold.sales)] } + └─BatchExchange { order: [], dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, items_sold.sales, flag) } └─BatchExpand { column_subsets: [[items_sold.brand], [items_sold.size], []] } └─BatchScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [brand, size, sum, g1, g2, g3, count, flag(hidden)], stream_key: [brand, size, flag], pk_columns: [brand, size, flag], pk_conflict: NoCheck } - └─StreamProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales), Case((0:Int64 = flag), 0:Int32, (1:Int64 = flag), 1:Int32, (2:Int64 = flag), 1:Int32) as $expr1, Case((0:Int64 = flag), 1:Int32, (1:Int64 = flag), 0:Int32, (2:Int64 = flag), 1:Int32) as $expr2, Case((0:Int64 = flag), 1:Int32, (1:Int64 = flag), 2:Int32, (2:Int64 = flag), 3:Int32) as $expr3, count(distinct items_sold.sales), flag] } - └─StreamHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales), count(distinct items_sold.sales), count] } - └─StreamExchange { dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─StreamProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales), Case((0:Int64 = flag), 0:Int32, (1:Int64 = flag), 1:Int32, (2:Int64 = flag), 1:Int32) as $expr1, Case((0:Int64 = flag), 1:Int32, (1:Int64 = flag), 0:Int32, (2:Int64 = flag), 1:Int32) as $expr2, Case((0:Int64 = flag), 1:Int32, (1:Int64 = flag), 2:Int32, (2:Int64 = flag), 3:Int32) as $expr3, count(distinct items_sold.sales), flag] } + └─StreamHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales), count(distinct items_sold.sales), count] } + └─StreamExchange { dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─StreamExpand { column_subsets: [[items_sold.brand], [items_sold.size], []] } └─StreamTableScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales, items_sold._row_id], pk: [items_sold._row_id], dist: UpstreamHashShard(items_sold._row_id) } - name: too many arguments for grouping error @@ -95,16 +95,16 @@ SELECT brand, size, sum(sales) FROM items_sold GROUP BY ROLLUP(brand, size); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales)] } - └─BatchHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales)] } - └─BatchExchange { order: [], dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─BatchProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales)] } + └─BatchHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales)] } + └─BatchExchange { order: [], dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─BatchExpand { column_subsets: [[], [items_sold.brand], [items_sold.brand, items_sold.size]] } └─BatchScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [brand, size, sum, flag(hidden)], stream_key: [brand, size, flag], pk_columns: [brand, size, flag], pk_conflict: NoCheck } - └─StreamProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales), flag] } - └─StreamHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales), count] } - └─StreamExchange { dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─StreamProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales), flag] } + └─StreamHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales), count] } + └─StreamExchange { dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─StreamExpand { column_subsets: [[], [items_sold.brand], [items_sold.brand, items_sold.size]] } └─StreamTableScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales, items_sold._row_id], pk: [items_sold._row_id], dist: UpstreamHashShard(items_sold._row_id) } - name: rollup2 @@ -113,16 +113,16 @@ SELECT brand, size, sum(sales) FROM items_sold GROUP BY ROLLUP((brand, size)); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales)] } - └─BatchHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales)] } - └─BatchExchange { order: [], dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─BatchProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales)] } + └─BatchHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales)] } + └─BatchExchange { order: [], dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─BatchExpand { column_subsets: [[], [items_sold.brand, items_sold.size]] } └─BatchScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [brand, size, sum, flag(hidden)], stream_key: [brand, size, flag], pk_columns: [brand, size, flag], pk_conflict: NoCheck } - └─StreamProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales), flag] } - └─StreamHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales), count] } - └─StreamExchange { dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─StreamProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales), flag] } + └─StreamHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales), count] } + └─StreamExchange { dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─StreamExpand { column_subsets: [[], [items_sold.brand, items_sold.size]] } └─StreamTableScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales, items_sold._row_id], pk: [items_sold._row_id], dist: UpstreamHashShard(items_sold._row_id) } - name: cube1 @@ -131,16 +131,16 @@ SELECT brand, size, sum(sales) FROM items_sold GROUP BY CUBE(brand, size); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales)] } - └─BatchHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales)] } - └─BatchExchange { order: [], dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─BatchProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales)] } + └─BatchHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales)] } + └─BatchExchange { order: [], dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─BatchExpand { column_subsets: [[], [items_sold.brand], [items_sold.size], [items_sold.brand, items_sold.size]] } └─BatchScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [brand, size, sum, flag(hidden)], stream_key: [brand, size, flag], pk_columns: [brand, size, flag], pk_conflict: NoCheck } - └─StreamProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales), flag] } - └─StreamHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales), count] } - └─StreamExchange { dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─StreamProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales), flag] } + └─StreamHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales), count] } + └─StreamExchange { dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─StreamExpand { column_subsets: [[], [items_sold.brand], [items_sold.size], [items_sold.brand, items_sold.size]] } └─StreamTableScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales, items_sold._row_id], pk: [items_sold._row_id], dist: UpstreamHashShard(items_sold._row_id) } - name: cube2 @@ -149,16 +149,16 @@ SELECT brand, size, sum(sales) FROM items_sold GROUP BY CUBE(brand, size, size); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales)] } - └─BatchHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales)] } - └─BatchExchange { order: [], dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─BatchProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales)] } + └─BatchHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales)] } + └─BatchExchange { order: [], dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─BatchExpand { column_subsets: [[], [items_sold.brand], [items_sold.size], [items_sold.size], [items_sold.brand, items_sold.size], [items_sold.brand, items_sold.size], [items_sold.size], [items_sold.brand, items_sold.size]] } └─BatchScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [brand, size, sum, flag(hidden)], stream_key: [brand, size, flag], pk_columns: [brand, size, flag], pk_conflict: NoCheck } - └─StreamProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales), flag] } - └─StreamHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales), count] } - └─StreamExchange { dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─StreamProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales), flag] } + └─StreamHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales), count] } + └─StreamExchange { dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─StreamExpand { column_subsets: [[], [items_sold.brand], [items_sold.size], [items_sold.size], [items_sold.brand, items_sold.size], [items_sold.brand, items_sold.size], [items_sold.size], [items_sold.brand, items_sold.size]] } └─StreamTableScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales, items_sold._row_id], pk: [items_sold._row_id], dist: UpstreamHashShard(items_sold._row_id) } - name: cube3 @@ -167,16 +167,16 @@ SELECT brand, size, sum(sales) FROM items_sold GROUP BY CUBE((brand, size), size); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales)] } - └─BatchHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales)] } - └─BatchExchange { order: [], dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─BatchProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales)] } + └─BatchHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales)] } + └─BatchExchange { order: [], dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─BatchExpand { column_subsets: [[], [items_sold.brand, items_sold.size], [items_sold.size], [items_sold.brand, items_sold.size]] } └─BatchScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [brand, size, sum, flag(hidden)], stream_key: [brand, size, flag], pk_columns: [brand, size, flag], pk_conflict: NoCheck } - └─StreamProject { exprs: [items_sold.brand, items_sold.size, sum(items_sold.sales), flag] } - └─StreamHashAgg { group_key: [items_sold.brand, items_sold.size, flag], aggs: [sum(items_sold.sales), count] } - └─StreamExchange { dist: HashShard(items_sold.brand, items_sold.size, flag) } + └─StreamProject { exprs: [items_sold.brand_expanded, items_sold.size_expanded, sum(items_sold.sales), flag] } + └─StreamHashAgg { group_key: [items_sold.brand_expanded, items_sold.size_expanded, flag], aggs: [sum(items_sold.sales), count] } + └─StreamExchange { dist: HashShard(items_sold.brand_expanded, items_sold.size_expanded, flag) } └─StreamExpand { column_subsets: [[], [items_sold.brand, items_sold.size], [items_sold.size], [items_sold.brand, items_sold.size]] } └─StreamTableScan { table: items_sold, columns: [items_sold.brand, items_sold.size, items_sold.sales, items_sold._row_id], pk: [items_sold._row_id], dist: UpstreamHashShard(items_sold._row_id) } - name: only one set in grouping sets diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index 8d452bf45bc36..637b7b3ddecc7 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -1258,10 +1258,10 @@ GROUP BY to_char(date_time, 'yyyy-MM-dd'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [$expr1], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─BatchExchange { order: [], dist: HashShard($expr1) } - └─BatchHashAgg { group_key: [$expr1, bid.bidder, bid.auction, flag], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } - └─BatchExchange { order: [], dist: HashShard($expr1, bid.bidder, bid.auction, flag) } + └─BatchHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder_expanded) filter((flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction_expanded) filter((flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─BatchExchange { order: [], dist: HashShard($expr1_expanded) } + └─BatchHashAgg { group_key: [$expr1_expanded, bid.bidder_expanded, bid.auction_expanded, flag], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + └─BatchExchange { order: [], dist: HashShard($expr1_expanded, bid.bidder_expanded, bid.auction_expanded, flag) } └─BatchExpand { column_subsets: [[$expr1], [$expr1, bid.bidder], [$expr1, bid.auction]] } └─BatchProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, bid.price, bid.bidder, bid.auction] } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time], distribution: SomeShard } @@ -1323,26 +1323,26 @@ GROUP BY to_char(date_time, 'yyyy-MM-dd'); stream_plan: |- StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr1, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamHashAgg { group_key: [$expr1], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamHashAgg [append_only] { group_key: [$expr1, bid.bidder, bid.auction, flag], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard($expr1, bid.bidder, bid.auction, flag) } + └─StreamProject { exprs: [$expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder_expanded) filter((flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction_expanded) filter((flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder_expanded) filter((flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction_expanded) filter((flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamExchange { dist: HashShard($expr1_expanded) } + └─StreamHashAgg [append_only] { group_key: [$expr1_expanded, bid.bidder_expanded, bid.auction_expanded, flag], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr1_expanded, bid.bidder_expanded, bid.auction_expanded, flag) } └─StreamExpand { column_subsets: [[$expr1], [$expr1, bid.bidder], [$expr1, bid.auction]] } └─StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, bid.price, bid.bidder, bid.auction, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } { materialized table: 4294967294 } - └── StreamProject { exprs: [$expr1, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └── StreamHashAgg { group_key: [$expr1], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └── StreamProject { exprs: [$expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder_expanded) filter((flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction_expanded) filter((flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └── StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder_expanded) filter((flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction_expanded) filter((flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } ├── intermediate state table: 0 ├── state tables: [] ├── distinct tables: [] └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamHashAgg [append_only] { group_key: [$expr1, bid.bidder, bid.auction, flag], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } { intermediate state table: 1, state tables: [], distinct tables: [] } + StreamHashAgg [append_only] { group_key: [$expr1_expanded, bid.bidder_expanded, bid.auction_expanded, flag], aggs: [count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } { intermediate state table: 1, state tables: [], distinct tables: [] } └── StreamExchange Hash([0, 2, 3, 10]) from 2 Fragment 2 @@ -1353,13 +1353,13 @@ └── BatchPlanNode Table 0 - ├── columns: [ $expr1, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder) filter((flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction) filter((flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ $expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder_expanded) filter((flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction_expanded) filter((flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr1, bid_bidder, bid_auction, flag, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 1 { columns: [ $expr1_expanded, bid_bidder_expanded, bid_auction_expanded, flag, count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } Table 2 { columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } @@ -1391,10 +1391,10 @@ GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [bid.channel, $expr1], aggs: [max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─BatchExchange { order: [], dist: HashShard(bid.channel, $expr1) } - └─BatchHashAgg { group_key: [bid.channel, $expr1, bid.bidder, bid.auction, flag], aggs: [max($expr2), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } - └─BatchExchange { order: [], dist: HashShard(bid.channel, $expr1, bid.bidder, bid.auction, flag) } + └─BatchHashAgg { group_key: [bid.channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder_expanded) filter((flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction_expanded) filter((flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─BatchExchange { order: [], dist: HashShard(bid.channel_expanded, $expr1_expanded) } + └─BatchHashAgg { group_key: [bid.channel_expanded, $expr1_expanded, bid.bidder_expanded, bid.auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + └─BatchExchange { order: [], dist: HashShard(bid.channel_expanded, $expr1_expanded, bid.bidder_expanded, bid.auction_expanded, flag) } └─BatchExpand { column_subsets: [[bid.channel, $expr1, $expr2], [bid.channel, $expr1, bid.bidder], [bid.channel, $expr1, bid.auction]] } └─BatchProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr2, bid.price, bid.bidder, bid.auction] } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time], distribution: SomeShard } @@ -1458,26 +1458,26 @@ GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); stream_plan: |- StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } - └─StreamProject { exprs: [bid.channel, $expr1, max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamHashAgg { group_key: [bid.channel, $expr1], aggs: [max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } - └─StreamExchange { dist: HashShard(bid.channel, $expr1) } - └─StreamHashAgg [append_only] { group_key: [bid.channel, $expr1, bid.bidder, bid.auction, flag], aggs: [max($expr2), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard(bid.channel, $expr1, bid.bidder, bid.auction, flag) } + └─StreamProject { exprs: [bid.channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder_expanded) filter((flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction_expanded) filter((flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [bid.channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder_expanded) filter((flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction_expanded) filter((flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamExchange { dist: HashShard(bid.channel_expanded, $expr1_expanded) } + └─StreamHashAgg [append_only] { group_key: [bid.channel_expanded, $expr1_expanded, bid.bidder_expanded, bid.auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard(bid.channel_expanded, $expr1_expanded, bid.bidder_expanded, bid.auction_expanded, flag) } └─StreamExpand { column_subsets: [[bid.channel, $expr1, $expr2], [bid.channel, $expr1, bid.bidder], [bid.channel, $expr1, bid.auction]] } └─StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr2, bid.price, bid.bidder, bid.auction, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } { materialized table: 4294967294 } - └── StreamProject { exprs: [bid.channel, $expr1, max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └── StreamHashAgg { group_key: [bid.channel, $expr1], aggs: [max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └── StreamProject { exprs: [bid.channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder_expanded) filter((flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction_expanded) filter((flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └── StreamHashAgg { group_key: [bid.channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder_expanded) filter((flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction_expanded) filter((flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction_expanded) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } ├── intermediate state table: 1 ├── state tables: [ 0 ] ├── distinct tables: [] └── StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamHashAgg [append_only] { group_key: [bid.channel, $expr1, bid.bidder, bid.auction, flag], aggs: [max($expr2), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } { intermediate state table: 2, state tables: [], distinct tables: [] } + StreamHashAgg [append_only] { group_key: [bid.channel_expanded, $expr1_expanded, bid.bidder_expanded, bid.auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } { intermediate state table: 2, state tables: [], distinct tables: [] } └── StreamExchange Hash([0, 1, 4, 5, 14]) from 2 Fragment 2 @@ -1487,16 +1487,16 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ bid_channel, $expr1, max($expr2), bid_bidder, bid_auction, flag ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 0 { columns: [ bid_channel_expanded, $expr1_expanded, max($expr2_expanded), bid_bidder_expanded, bid_auction_expanded, flag ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 1 - ├── columns: [ bid_channel, $expr1, max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder) filter((flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction) filter((flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ bid_channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder_expanded) filter((flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction_expanded) filter((flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction_expanded) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 2 { columns: [ bid_channel, $expr1, bid_bidder, bid_auction, flag, max($expr2), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } + Table 2 { columns: [ bid_channel_expanded, $expr1_expanded, bid_bidder_expanded, bid_auction_expanded, flag, max($expr2_expanded), count, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } Table 3 { columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml index 31be64b2c480a..e5643da203e06 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -1056,10 +1056,10 @@ GROUP BY to_char(date_time, 'yyyy-MM-dd'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [$expr1], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─BatchExchange { order: [], dist: HashShard($expr1) } - └─BatchHashAgg { group_key: [$expr1, bidder, auction, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - └─BatchExchange { order: [], dist: HashShard($expr1, bidder, auction, flag) } + └─BatchHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─BatchExchange { order: [], dist: HashShard($expr1_expanded) } + └─BatchHashAgg { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─BatchExchange { order: [], dist: HashShard($expr1_expanded, bidder_expanded, auction_expanded, flag) } └─BatchExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } └─BatchProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction] } └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } @@ -1121,11 +1121,11 @@ GROUP BY to_char(date_time, 'yyyy-MM-dd'); stream_plan: |- StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr1, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamHashAgg { group_key: [$expr1], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamHashAgg [append_only] { group_key: [$expr1, bidder, auction, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard($expr1, bidder, auction, flag) } + └─StreamProject { exprs: [$expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamExchange { dist: HashShard($expr1_expanded) } + └─StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr1_expanded, bidder_expanded, auction_expanded, flag) } └─StreamExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } @@ -1133,15 +1133,15 @@ stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } { materialized table: 4294967294 } - └── StreamProject { exprs: [$expr1, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └── StreamHashAgg { group_key: [$expr1], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └── StreamProject { exprs: [$expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └── StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } ├── intermediate state table: 0 ├── state tables: [] ├── distinct tables: [] └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamHashAgg [append_only] { group_key: [$expr1, bidder, auction, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { intermediate state table: 1, state tables: [], distinct tables: [] } + StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { intermediate state table: 1, state tables: [], distinct tables: [] } └── StreamExchange Hash([0, 2, 3, 10]) from 2 Fragment 2 @@ -1151,13 +1151,13 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 2 } Table 0 - ├── columns: [ $expr1, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ $expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 1 { columns: [ $expr1, bidder, auction, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 1 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } Table 2 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } @@ -1187,13 +1187,13 @@ GROUP BY to_char(date_time, 'yyyy-MM-dd'); stream_plan: |- StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr1, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder) filter((flag = 1:Int64))), sum0(count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction) filter((flag = 2:Int64))), sum0(count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)))] } - └─StreamHashAgg { group_key: [$expr1], aggs: [sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder) filter((flag = 1:Int64))), sum0(count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction) filter((flag = 2:Int64))), sum0(count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count] } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamHashAgg { group_key: [$expr1, $expr2], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } - └─StreamProject { exprs: [$expr1, bidder, auction, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), Vnode($expr1, bidder, auction, flag) as $expr2] } - └─StreamHashAgg [append_only] { group_key: [$expr1, bidder, auction, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard($expr1, bidder, auction, flag) } + └─StreamProject { exprs: [$expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)))] } + └─StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count] } + └─StreamExchange { dist: HashShard($expr1_expanded) } + └─StreamHashAgg { group_key: [$expr1_expanded, $expr2], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamProject { exprs: [$expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), Vnode($expr1_expanded, bidder_expanded, auction_expanded, flag) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr1_expanded, bidder_expanded, auction_expanded, flag) } └─StreamExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } @@ -1201,17 +1201,17 @@ stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } { materialized table: 4294967294 } - └── StreamProject { exprs: [$expr1, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder) filter((flag = 1:Int64))), sum0(count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction) filter((flag = 2:Int64))), sum0(count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)))] } - └── StreamHashAgg { group_key: [$expr1], aggs: [sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder) filter((flag = 1:Int64))), sum0(count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction) filter((flag = 2:Int64))), sum0(count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count] } + └── StreamProject { exprs: [$expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)))] } + └── StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count] } ├── intermediate state table: 0 ├── state tables: [] ├── distinct tables: [] └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamHashAgg { group_key: [$expr1, $expr2], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } { intermediate state table: 1, state tables: [], distinct tables: [] } - └── StreamProject { exprs: [$expr1, bidder, auction, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), Vnode($expr1, bidder, auction, flag) as $expr2] } - └── StreamHashAgg [append_only] { group_key: [$expr1, bidder, auction, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { intermediate state table: 2, state tables: [], distinct tables: [] } + StreamHashAgg { group_key: [$expr1_expanded, $expr2], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } { intermediate state table: 1, state tables: [], distinct tables: [] } + └── StreamProject { exprs: [$expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), Vnode($expr1_expanded, bidder_expanded, auction_expanded, flag) as $expr2] } + └── StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { intermediate state table: 2, state tables: [], distinct tables: [] } └── StreamExchange Hash([0, 2, 3, 10]) from 2 Fragment 2 @@ -1221,21 +1221,21 @@ └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 3 } Table 0 - ├── columns: [ $expr1, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder) filter((flag = 1:Int64))), sum0(count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction) filter((flag = 2:Int64))), sum0(count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count ] + ├── columns: [ $expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 1 - ├── columns: [ $expr1, $expr2, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ $expr1_expanded, $expr2, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] ├── distribution key: [] ├── read pk prefix len hint: 2 └── vnode column idx: 1 - Table 2 { columns: [ $expr1, bidder, auction, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + Table 2 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } Table 3 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } @@ -1268,10 +1268,10 @@ GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [channel, $expr1], aggs: [max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─BatchExchange { order: [], dist: HashShard(channel, $expr1) } - └─BatchHashAgg { group_key: [channel, $expr1, bidder, auction, flag], aggs: [max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - └─BatchExchange { order: [], dist: HashShard(channel, $expr1, bidder, auction, flag) } + └─BatchHashAgg { group_key: [channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─BatchExchange { order: [], dist: HashShard(channel_expanded, $expr1_expanded) } + └─BatchHashAgg { group_key: [channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─BatchExchange { order: [], dist: HashShard(channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag) } └─BatchExpand { column_subsets: [[channel, $expr1, $expr2], [channel, $expr1, bidder], [channel, $expr1, auction]] } └─BatchProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction] } └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } @@ -1335,11 +1335,11 @@ GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); stream_plan: |- StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } - └─StreamProject { exprs: [channel, $expr1, max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamHashAgg { group_key: [channel, $expr1], aggs: [max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } - └─StreamExchange { dist: HashShard(channel, $expr1) } - └─StreamHashAgg [append_only] { group_key: [channel, $expr1, bidder, auction, flag], aggs: [max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard(channel, $expr1, bidder, auction, flag) } + └─StreamProject { exprs: [channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamExchange { dist: HashShard(channel_expanded, $expr1_expanded) } + └─StreamHashAgg [append_only] { group_key: [channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard(channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag) } └─StreamExpand { column_subsets: [[channel, $expr1, $expr2], [channel, $expr1, bidder], [channel, $expr1, auction]] } └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } @@ -1347,15 +1347,15 @@ stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } { materialized table: 4294967294 } - └── StreamProject { exprs: [channel, $expr1, max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └── StreamHashAgg { group_key: [channel, $expr1], aggs: [max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └── StreamProject { exprs: [channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └── StreamHashAgg { group_key: [channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } ├── intermediate state table: 1 ├── state tables: [ 0 ] ├── distinct tables: [] └── StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamHashAgg [append_only] { group_key: [channel, $expr1, bidder, auction, flag], aggs: [max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { intermediate state table: 2, state tables: [], distinct tables: [] } + StreamHashAgg [append_only] { group_key: [channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { intermediate state table: 2, state tables: [], distinct tables: [] } └── StreamExchange Hash([0, 1, 4, 5, 14]) from 2 Fragment 2 @@ -1364,16 +1364,16 @@ └── StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { source state table: 3 } - Table 0 { columns: [ channel, $expr1, max($expr2), bidder, auction, flag ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 0 { columns: [ channel_expanded, $expr1_expanded, max($expr2_expanded), bidder_expanded, auction_expanded, flag ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 1 - ├── columns: [ channel, $expr1, max(max($expr2)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] + ├── columns: [ channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count ] ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0, 1 ] └── read pk prefix len hint: 2 - Table 2 { columns: [ channel, $expr1, bidder, auction, flag, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } + Table 2 { columns: [ channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag, max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)) ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } Table 3 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml index 39adc39a16653..827eaf3d37e80 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -1291,10 +1291,10 @@ GROUP BY to_char(date_time, 'yyyy-MM-dd'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [$expr2], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter(($expr3 < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter(($expr3 >= 1000000:Int32))) filter((flag = 0:Int64)), count($expr4) filter((flag = 1:Int64)), count($expr4) filter((count filter(($expr3 < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr4) filter((count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr4) filter((count filter(($expr3 >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr5) filter((flag = 2:Int64)), count($expr5) filter((count filter(($expr3 < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count($expr5) filter((count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count($expr5) filter((count filter(($expr3 >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─BatchExchange { order: [], dist: HashShard($expr2) } - └─BatchHashAgg { group_key: [$expr2, $expr4, $expr5, flag], aggs: [count, count filter(($expr3 < 10000:Int32)), count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count filter(($expr3 >= 1000000:Int32))] } - └─BatchExchange { order: [], dist: HashShard($expr2, $expr4, $expr5, flag) } + └─BatchHashAgg { group_key: [$expr2_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter(($expr3 < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter(($expr3 >= 1000000:Int32))) filter((flag = 0:Int64)), count($expr4_expanded) filter((flag = 1:Int64)), count($expr4_expanded) filter((count filter(($expr3 < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr4_expanded) filter((count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr4_expanded) filter((count filter(($expr3 >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr5_expanded) filter((flag = 2:Int64)), count($expr5_expanded) filter((count filter(($expr3 < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count($expr5_expanded) filter((count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count($expr5_expanded) filter((count filter(($expr3 >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─BatchExchange { order: [], dist: HashShard($expr2_expanded) } + └─BatchHashAgg { group_key: [$expr2_expanded, $expr4_expanded, $expr5_expanded, flag], aggs: [count, count filter(($expr3 < 10000:Int32)), count filter(($expr3 >= 10000:Int32) AND ($expr3 < 1000000:Int32)), count filter(($expr3 >= 1000000:Int32))] } + └─BatchExchange { order: [], dist: HashShard($expr2_expanded, $expr4_expanded, $expr5_expanded, flag) } └─BatchExpand { column_subsets: [[$expr2], [$expr2, $expr4], [$expr2, $expr5]] } └─BatchProject { exprs: [ToChar($expr1, 'yyyy-MM-dd':Varchar) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 1:Int32) as $expr4, Field(bid, 0:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } @@ -1371,10 +1371,10 @@ GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [$expr2, $expr3], aggs: [max(max($expr4)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter(($expr5 < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter(($expr5 >= 1000000:Int32))) filter((flag = 0:Int64)), count($expr6) filter((flag = 1:Int64)), count($expr6) filter((count filter(($expr5 < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr6) filter((count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr6) filter((count filter(($expr5 >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr7) filter((flag = 2:Int64)), count($expr7) filter((count filter(($expr5 < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count($expr7) filter((count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count($expr7) filter((count filter(($expr5 >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─BatchExchange { order: [], dist: HashShard($expr2, $expr3) } - └─BatchHashAgg { group_key: [$expr2, $expr3, $expr6, $expr7, flag], aggs: [max($expr4), count, count filter(($expr5 < 10000:Int32)), count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count filter(($expr5 >= 1000000:Int32))] } - └─BatchExchange { order: [], dist: HashShard($expr2, $expr3, $expr6, $expr7, flag) } + └─BatchHashAgg { group_key: [$expr2_expanded, $expr3_expanded], aggs: [max(max($expr4_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter(($expr5 < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter(($expr5 >= 1000000:Int32))) filter((flag = 0:Int64)), count($expr6_expanded) filter((flag = 1:Int64)), count($expr6_expanded) filter((count filter(($expr5 < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr6_expanded) filter((count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr6_expanded) filter((count filter(($expr5 >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count($expr7_expanded) filter((flag = 2:Int64)), count($expr7_expanded) filter((count filter(($expr5 < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count($expr7_expanded) filter((count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count($expr7_expanded) filter((count filter(($expr5 >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─BatchExchange { order: [], dist: HashShard($expr2_expanded, $expr3_expanded) } + └─BatchHashAgg { group_key: [$expr2_expanded, $expr3_expanded, $expr6_expanded, $expr7_expanded, flag], aggs: [max($expr4_expanded), count, count filter(($expr5 < 10000:Int32)), count filter(($expr5 >= 10000:Int32) AND ($expr5 < 1000000:Int32)), count filter(($expr5 >= 1000000:Int32))] } + └─BatchExchange { order: [], dist: HashShard($expr2_expanded, $expr3_expanded, $expr6_expanded, $expr7_expanded, flag) } └─BatchExpand { column_subsets: [[$expr2, $expr3, $expr4], [$expr2, $expr3, $expr6], [$expr2, $expr3, $expr7]] } └─BatchProject { exprs: [Field(bid, 3:Int32) as $expr2, ToChar($expr1, 'yyyy-MM-dd':Varchar) as $expr3, ToChar($expr1, 'HH:mm':Varchar) as $expr4, Field(bid, 2:Int32) as $expr5, Field(bid, 1:Int32) as $expr6, Field(bid, 0:Int32) as $expr7] } └─BatchFilter { predicate: (event_type = 2:Int32) } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 914d9b764c5e1..69b6e21592c0a 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -803,15 +803,15 @@ select * from integers where 2 in (select count(distinct k) + count(distinct v) from rows where correlated_col = integers.correlated_col); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col] } + └─BatchHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col_expanded AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col] } ├─BatchExchange { order: [], dist: HashShard(integers.correlated_col) } │ └─BatchProject { exprs: [integers.i, integers.correlated_col, 2:Int64] } │ └─BatchScan { table: integers, columns: [integers.i, integers.correlated_col], distribution: SomeShard } - └─BatchProject { exprs: [integers.correlated_col, (count(rows.k) filter((flag = 0:Int64)) + count(rows.v) filter((flag = 1:Int64))) as $expr1] } - └─BatchHashAgg { group_key: [integers.correlated_col], aggs: [count(rows.k) filter((flag = 0:Int64)), count(rows.v) filter((flag = 1:Int64))] } - └─BatchExchange { order: [], dist: HashShard(integers.correlated_col) } - └─BatchHashAgg { group_key: [integers.correlated_col, rows.k, rows.v, flag], aggs: [] } - └─BatchExchange { order: [], dist: HashShard(integers.correlated_col, rows.k, rows.v, flag) } + └─BatchProject { exprs: [integers.correlated_col_expanded, (count(rows.k_expanded) filter((flag = 0:Int64)) + count(rows.v_expanded) filter((flag = 1:Int64))) as $expr1] } + └─BatchHashAgg { group_key: [integers.correlated_col_expanded], aggs: [count(rows.k_expanded) filter((flag = 0:Int64)), count(rows.v_expanded) filter((flag = 1:Int64))] } + └─BatchExchange { order: [], dist: HashShard(integers.correlated_col_expanded) } + └─BatchHashAgg { group_key: [integers.correlated_col_expanded, rows.k_expanded, rows.v_expanded, flag], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(integers.correlated_col_expanded, rows.k_expanded, rows.v_expanded, flag) } └─BatchExpand { column_subsets: [[integers.correlated_col, rows.k], [integers.correlated_col, rows.v]] } └─BatchHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, 1:Int32] } ├─BatchHashAgg { group_key: [integers.correlated_col], aggs: [] } diff --git a/src/frontend/src/optimizer/plan_node/generic/expand.rs b/src/frontend/src/optimizer/plan_node/generic/expand.rs index ba7c52aa814ef..d346677ac2d62 100644 --- a/src/frontend/src/optimizer/plan_node/generic/expand.rs +++ b/src/frontend/src/optimizer/plan_node/generic/expand.rs @@ -51,8 +51,14 @@ impl Expand { impl GenericPlanNode for Expand { fn schema(&self) -> Schema { - let mut fields = self.input.schema().clone().into_fields(); - fields.extend(fields.clone()); + let mut fields = self + .input + .schema() + .fields() + .iter() + .map(|f| Field::with_name(f.data_type(), format!("{}_expanded", f.name))) + .collect::>(); + fields.extend(self.input.schema().fields().iter().cloned()); fields.push(Field::with_name(DataType::Int64, "flag")); Schema::new(fields) } diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index 31209122e16a3..054621ef967bf 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -94,14 +94,30 @@ impl PlanTreeNodeUnary for LogicalExpand { .collect_vec() }) .collect_vec(); - let (mut mapping, new_input_col_num) = input_col_change.into_parts(); - mapping.extend({ - mapping - .iter() - .map(|p| p.map(|i| i + new_input_col_num)) - .collect_vec() - }); - mapping.push(Some(2 * new_input_col_num)); + + let old_out_len = self.schema().len(); + let old_in_len = self.input().schema().len(); + let new_in_len = input.schema().len(); + assert_eq!( + old_out_len, + old_in_len * 2 + 1 // expanded input cols + real input cols + flag + ); + + let mut mapping = Vec::with_capacity(old_out_len); + // map the expanded input columns + for i in 0..old_in_len { + mapping.push(input_col_change.try_map(i)); + } + // map the real input columns + for i in 0..old_in_len { + mapping.push( + input_col_change + .try_map(i) + .map(|x| x + new_in_len /* # of new expanded input cols */), + ); + } + // map the flag column + mapping.push(Some(2 * new_in_len)); let expand = Self::new(input, column_subsets); let output_col_num = expand.schema().len(); diff --git a/src/frontend/src/optimizer/rule/grouping_sets_to_expand_rule.rs b/src/frontend/src/optimizer/rule/grouping_sets_to_expand_rule.rs index 344241441f81c..24cc4fd09b49d 100644 --- a/src/frontend/src/optimizer/rule/grouping_sets_to_expand_rule.rs +++ b/src/frontend/src/optimizer/rule/grouping_sets_to_expand_rule.rs @@ -75,10 +75,11 @@ impl Rule for GroupingSetsToExpandRule { return None; } let agg = Self::prune_column_for_agg(agg); - let (agg_calls, mut group_keys, grouping_sets, input, enable_two_phase) = agg.decompose(); + let (old_agg_calls, old_group_keys, grouping_sets, input, enable_two_phase) = + agg.decompose(); - let flag_col_idx = group_keys.len(); - let input_schema_len = input.schema().len(); + let old_input_schema_len = input.schema().len(); + let flag_col_idx = old_group_keys.len(); let column_subset = grouping_sets .iter() @@ -86,17 +87,23 @@ impl Rule for GroupingSetsToExpandRule { .collect_vec(); let expand = LogicalExpand::create(input, column_subset.clone()); - // Add the expand flag. - group_keys.extend(std::iter::once(expand.schema().len() - 1)); + let new_group_keys = { + let mut k = old_group_keys.clone(); + // Add the expand flag. + k.extend(std::iter::once(expand.schema().len() - 1)); + k + }; + // Map from old input ref to expanded input (`LogicalExpand` prepends the same number of fields + // as expanded ones with NULLs before the real input fields). let mut input_col_change = - ColIndexMapping::with_shift_offset(input_schema_len, input_schema_len as isize); + ColIndexMapping::with_shift_offset(old_input_schema_len, old_input_schema_len as isize); // Grouping agg calls need to be transformed into a project expression, and other agg calls // need to shift their `input_ref`. - let mut project_exprs = vec![]; + let mut project_agg_call_exprs = vec![]; let mut new_agg_calls = vec![]; - for mut agg_call in agg_calls { + for agg_call in old_agg_calls { // Deal with grouping agg call for grouping sets. if agg_call.agg_kind == AggKind::Grouping { let mut grouping_values = vec![]; @@ -140,34 +147,35 @@ impl Rule for GroupingSetsToExpandRule { FunctionCall::new_unchecked(ExprType::Case, case_inputs, DataType::Int32) .into(), ); - project_exprs.push(case_expr); + project_agg_call_exprs.push(case_expr); } else { + let mut new_agg_call = agg_call; // Shift agg_call to the original input columns - agg_call.inputs.iter_mut().for_each(|i| { - *i = InputRef::new(input_col_change.map(i.index()), i.return_type()) + new_agg_call.inputs.iter_mut().for_each(|i| { + let new_i = input_col_change.map(i.index()); + assert_eq!(expand.schema()[new_i].data_type(), i.return_type()); + *i = InputRef::new(new_i, i.return_type()); }); - agg_call.order_by.iter_mut().for_each(|o| { + new_agg_call.order_by.iter_mut().for_each(|o| { o.column_index = input_col_change.map(o.column_index); }); - agg_call.filter = agg_call.filter.rewrite_expr(&mut input_col_change); - let agg_return_type = agg_call.return_type.clone(); - new_agg_calls.push(agg_call); - project_exprs.push(ExprImpl::InputRef( - InputRef::new(group_keys.len() + new_agg_calls.len() - 1, agg_return_type) - .into(), + new_agg_call.filter = new_agg_call.filter.rewrite_expr(&mut input_col_change); + project_agg_call_exprs.push(ExprImpl::InputRef( + InputRef::new( + new_group_keys.len() + new_agg_calls.len(), + new_agg_call.return_type.clone(), + ) + .into(), )); + new_agg_calls.push(new_agg_call); } } let new_agg = - Agg::new(new_agg_calls, group_keys, expand).with_enable_two_phase(enable_two_phase); - let project_exprs = (0..flag_col_idx) - .map(|i| { - ExprImpl::InputRef( - InputRef::new(i, new_agg.schema().fields()[i].data_type.clone()).into(), - ) - }) - .chain(project_exprs) + Agg::new(new_agg_calls, new_group_keys, expand).with_enable_two_phase(enable_two_phase); + let project_exprs = (0..old_group_keys.len()) + .map(|i| ExprImpl::InputRef(InputRef::new(i, new_agg.schema()[i].data_type()).into())) + .chain(project_agg_call_exprs) .collect(); let project = LogicalProject::new(new_agg.into(), project_exprs); From 12b35356ead0c634b5816880fcc9be7b0acbf9f7 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Fri, 27 Oct 2023 15:36:29 +0800 Subject: [PATCH 26/52] feat(metrics): remove metrics labels after being collected (#13080) --- src/common/src/metrics/guarded_metrics.rs | 159 +++++++++++++++------- 1 file changed, 109 insertions(+), 50 deletions(-) diff --git a/src/common/src/metrics/guarded_metrics.rs b/src/common/src/metrics/guarded_metrics.rs index 7cc166aa5998b..1256a1cb439e6 100644 --- a/src/common/src/metrics/guarded_metrics.rs +++ b/src/common/src/metrics/guarded_metrics.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::any::type_name; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::fmt::{Debug, Formatter}; use std::ops::Deref; use std::sync::Arc; @@ -21,9 +21,10 @@ use std::sync::Arc; use itertools::Itertools; use parking_lot::Mutex; use prometheus::core::{ - Atomic, AtomicI64, AtomicU64, Collector, GenericCounter, GenericCounterVec, GenericGauge, - GenericGaugeVec, MetricVec, MetricVecBuilder, + Atomic, AtomicF64, AtomicI64, AtomicU64, Collector, Desc, GenericCounter, GenericCounterVec, + GenericGauge, GenericGaugeVec, MetricVec, MetricVecBuilder, }; +use prometheus::proto::MetricFamily; use prometheus::{Histogram, HistogramVec}; use tracing::warn; @@ -53,11 +54,27 @@ macro_rules! register_guarded_histogram_vec_with_registry { } }}; ($HOPTS:expr, $LABELS_NAMES:expr, $REGISTRY:expr $(,)?) => {{ - let result = - prometheus::register_histogram_vec_with_registry!($HOPTS, $LABELS_NAMES, $REGISTRY); - result.map(|inner| { + let inner = prometheus::HistogramVec::new($HOPTS, $LABELS_NAMES); + inner.and_then(|inner| { let inner = $crate::metrics::__extract_histogram_builder(inner); - $crate::metrics::LabelGuardedHistogramVec::new(inner, { $LABELS_NAMES }) + let label_guarded = + $crate::metrics::LabelGuardedHistogramVec::new(inner, { $LABELS_NAMES }); + let result = ($REGISTRY).register(Box::new(label_guarded.clone())); + result.map(move |()| label_guarded) + }) + }}; +} + +#[macro_export] +macro_rules! register_guarded_gauge_vec_with_registry { + ($NAME:expr, $HELP:expr, $LABELS_NAMES:expr, $REGISTRY:expr $(,)?) => {{ + let inner = prometheus::GaugeVec::new(prometheus::opts!($NAME, $HELP), $LABELS_NAMES); + inner.and_then(|inner| { + let inner = $crate::metrics::__extract_gauge_builder(inner); + let label_guarded = + $crate::metrics::LabelGuardedGaugeVec::new(inner, { $LABELS_NAMES }); + let result = ($REGISTRY).register(Box::new(label_guarded.clone())); + result.map(move |()| label_guarded) }) }}; } @@ -65,14 +82,13 @@ macro_rules! register_guarded_histogram_vec_with_registry { #[macro_export] macro_rules! register_guarded_int_gauge_vec_with_registry { ($NAME:expr, $HELP:expr, $LABELS_NAMES:expr, $REGISTRY:expr $(,)?) => {{ - let result = prometheus::register_int_gauge_vec_with_registry!( - prometheus::opts!($NAME, $HELP), - $LABELS_NAMES, - $REGISTRY - ); - result.map(|inner| { + let inner = prometheus::IntGaugeVec::new(prometheus::opts!($NAME, $HELP), $LABELS_NAMES); + inner.and_then(|inner| { let inner = $crate::metrics::__extract_gauge_builder(inner); - $crate::metrics::LabelGuardedIntGaugeVec::new(inner, { $LABELS_NAMES }) + let label_guarded = + $crate::metrics::LabelGuardedIntGaugeVec::new(inner, { $LABELS_NAMES }); + let result = ($REGISTRY).register(Box::new(label_guarded.clone())); + result.map(move |()| label_guarded) }) }}; } @@ -80,14 +96,13 @@ macro_rules! register_guarded_int_gauge_vec_with_registry { #[macro_export] macro_rules! register_guarded_int_counter_vec_with_registry { ($NAME:expr, $HELP:expr, $LABELS_NAMES:expr, $REGISTRY:expr $(,)?) => {{ - let result = prometheus::register_int_counter_vec_with_registry!( - prometheus::opts!($NAME, $HELP), - $LABELS_NAMES, - $REGISTRY - ); - result.map(|inner| { + let inner = prometheus::IntCounterVec::new(prometheus::opts!($NAME, $HELP), $LABELS_NAMES); + inner.and_then(|inner| { let inner = $crate::metrics::__extract_counter_builder(inner); - $crate::metrics::LabelGuardedIntCounterVec::new(inner, { $LABELS_NAMES }) + let label_guarded = + $crate::metrics::LabelGuardedIntCounterVec::new(inner, { $LABELS_NAMES }); + let result = ($REGISTRY).register(Box::new(label_guarded.clone())); + result.map(move |()| label_guarded) }) }}; } @@ -101,11 +116,14 @@ pub type LabelGuardedIntCounterVec = LabelGuardedMetricVec, N>; pub type LabelGuardedIntGaugeVec = LabelGuardedMetricVec, N>; +pub type LabelGuardedGaugeVec = + LabelGuardedMetricVec, N>; pub type LabelGuardedHistogram = LabelGuardedMetric; pub type LabelGuardedIntCounter = LabelGuardedMetric, N>; pub type LabelGuardedIntGauge = LabelGuardedMetric, N>; +pub type LabelGuardedGauge = LabelGuardedMetric, N>; fn gen_test_label() -> [&'static str; N] { const TEST_LABELS: [&str; 5] = ["test1", "test2", "test3", "test4", "test5"]; @@ -116,10 +134,16 @@ fn gen_test_label() -> [&'static str; N] { .unwrap() } +#[derive(Default)] +struct LabelGuardedMetricsInfo { + labeled_metrics_count: HashMap<[String; N], usize>, + uncollected_removed_labels: HashSet<[String; N]>, +} + #[derive(Clone)] pub struct LabelGuardedMetricVec { inner: MetricVec, - labeled_metrics_count: Arc>>, + info: Arc>>, labels: [&'static str; N], } @@ -131,26 +155,51 @@ impl Debug for LabelGuardedMetricVec } } +impl Collector for LabelGuardedMetricVec { + fn desc(&self) -> Vec<&Desc> { + self.inner.desc() + } + + fn collect(&self) -> Vec { + let mut guard = self.info.lock(); + let ret = self.inner.collect(); + for labels in guard.uncollected_removed_labels.drain() { + if let Err(e) = self + .inner + .remove_label_values(&labels.each_ref().map(|s| s.as_str())) + { + warn!( + "err when delete metrics of {:?} of labels {:?}. Err {:?}", + self.inner.desc().first().expect("should have desc").fq_name, + self.labels, + e, + ); + } + } + ret + } +} + impl LabelGuardedMetricVec { pub fn new(inner: MetricVec, labels: &[&'static str; N]) -> Self { Self { inner, - labeled_metrics_count: Default::default(), + info: Default::default(), labels: *labels, } } pub fn with_label_values(&self, labels: &[&str; N]) -> LabelGuardedMetric { - let mut count_guard = self.labeled_metrics_count.lock(); + let mut guard = self.info.lock(); let label_string = labels.map(|str| str.to_string()); - *count_guard.entry(label_string).or_insert(0) += 1; + guard.uncollected_removed_labels.remove(&label_string); + *guard.labeled_metrics_count.entry(label_string).or_insert(0) += 1; let inner = self.inner.with_label_values(labels); LabelGuardedMetric { inner: Arc::new(LabelGuardedMetricInner { inner, labels: labels.map(|str| str.to_string()), - vec: self.inner.clone(), - labeled_metrics_count: self.labeled_metrics_count.clone(), + info: self.info.clone(), }), } } @@ -187,6 +236,14 @@ impl LabelGuardedIntGaugeVec { } } +impl LabelGuardedGaugeVec { + pub fn test_gauge_vec() -> Self { + let registry = prometheus::Registry::new(); + register_guarded_gauge_vec_with_registry!("test", "test", &gen_test_label::(), ®istry) + .unwrap() + } +} + impl LabelGuardedHistogramVec { pub fn test_histogram_vec() -> Self { let registry = prometheus::Registry::new(); @@ -204,30 +261,22 @@ impl LabelGuardedHistogramVec { struct LabelGuardedMetricInner { inner: T::M, labels: [String; N], - vec: MetricVec, - labeled_metrics_count: Arc>>, + info: Arc>>, } impl Drop for LabelGuardedMetricInner { fn drop(&mut self) { - let mut count_guard = self.labeled_metrics_count.lock(); - let count = count_guard.get_mut(&self.labels).expect( + let mut guard = self.info.lock(); + let count = guard.labeled_metrics_count.get_mut(&self.labels).expect( "should exist because the current existing dropping one means the count is not zero", ); *count -= 1; if *count == 0 { - count_guard.remove(&self.labels).expect("should exist"); - if let Err(e) = self - .vec - .remove_label_values(&self.labels.each_ref().map(|s| s.as_str())) - { - warn!( - "err when delete metrics of {:?} of labels {:?}. Err {:?}", - self.vec.desc().first().expect("should have desc").fq_name, - self.labels, - e, - ); - } + guard + .labeled_metrics_count + .remove(&self.labels) + .expect("should exist"); + guard.uncollected_removed_labels.insert(self.labels.clone()); } } } @@ -269,6 +318,12 @@ impl LabelGuardedIntGauge { } } +impl LabelGuardedGauge { + pub fn test_gauge() -> Self { + LabelGuardedGaugeVec::::test_gauge_vec().with_test_label() + } +} + #[cfg(test)] mod tests { use prometheus::core::Collector; @@ -279,19 +334,23 @@ mod tests { fn test_label_guarded_metrics_drop() { let vec = LabelGuardedIntCounterVec::<3>::test_int_counter_vec(); let m1_1 = vec.with_label_values(&["1", "2", "3"]); - assert_eq!(1, vec.inner.collect().pop().unwrap().get_metric().len()); + assert_eq!(1, vec.collect().pop().unwrap().get_metric().len()); let m1_2 = vec.with_label_values(&["1", "2", "3"]); let m1_3 = m1_2.clone(); - assert_eq!(1, vec.inner.collect().pop().unwrap().get_metric().len()); + assert_eq!(1, vec.collect().pop().unwrap().get_metric().len()); let m2 = vec.with_label_values(&["2", "2", "3"]); - assert_eq!(2, vec.inner.collect().pop().unwrap().get_metric().len()); + assert_eq!(2, vec.collect().pop().unwrap().get_metric().len()); drop(m1_3); - assert_eq!(2, vec.inner.collect().pop().unwrap().get_metric().len()); + assert_eq!(2, vec.collect().pop().unwrap().get_metric().len()); + assert_eq!(2, vec.collect().pop().unwrap().get_metric().len()); drop(m2); - assert_eq!(1, vec.inner.collect().pop().unwrap().get_metric().len()); + assert_eq!(2, vec.collect().pop().unwrap().get_metric().len()); + assert_eq!(1, vec.collect().pop().unwrap().get_metric().len()); drop(m1_1); - assert_eq!(1, vec.inner.collect().pop().unwrap().get_metric().len()); + assert_eq!(1, vec.collect().pop().unwrap().get_metric().len()); + assert_eq!(1, vec.collect().pop().unwrap().get_metric().len()); drop(m1_2); - assert_eq!(0, vec.inner.collect().pop().unwrap().get_metric().len()); + assert_eq!(1, vec.collect().pop().unwrap().get_metric().len()); + assert_eq!(0, vec.collect().pop().unwrap().get_metric().len()); } } From 03f285b72044682ffcf23cb0a2d1deb36d3876f2 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Fri, 27 Oct 2023 16:15:17 +0800 Subject: [PATCH 27/52] feat(expr): add `jsonb_pretty` function (#13050) Signed-off-by: Runji Wang --- proto/expr.proto | 1 + src/common/src/types/jsonb.rs | 38 +++++++++++++++-------- src/expr/impl/src/scalar/jsonb_info.rs | 21 +++++++++++++ src/frontend/src/binder/expr/function.rs | 1 + src/frontend/src/expr/pure.rs | 1 + src/risedevtool/src/bin/risedev-docslt.rs | 7 +++-- src/tests/regress/data/sql/jsonb.sql | 6 ++-- 7 files changed, 56 insertions(+), 19 deletions(-) diff --git a/proto/expr.proto b/proto/expr.proto index 9c6f3598032f9..2fcc905c4b2ab 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -220,6 +220,7 @@ message ExprNode { JSONB_ARRAY_LENGTH = 603; IS_JSON = 604; JSONB_CAT = 605; + JSONB_PRETTY = 607; // Non-pure functions below (> 1000) // ------------------------ diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 590b693e47891..be708ac9013a8 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -107,19 +107,6 @@ impl Ord for JsonbRef<'_> { impl crate::types::to_text::ToText for JsonbRef<'_> { fn write(&self, f: &mut W) -> std::fmt::Result { - struct FmtToIoUnchecked(F); - impl std::io::Write for FmtToIoUnchecked { - fn write(&mut self, buf: &[u8]) -> std::io::Result { - let s = unsafe { std::str::from_utf8_unchecked(buf) }; - self.0.write_str(s).map_err(|_| std::io::ErrorKind::Other)?; - Ok(buf.len()) - } - - fn flush(&mut self) -> std::io::Result<()> { - Ok(()) - } - } - // Use custom [`ToTextFormatter`] to serialize. If we are okay with the default, this can be // just `write!(f, "{}", self.0)` use serde::Serialize as _; @@ -412,6 +399,16 @@ impl<'a> JsonbRef<'a> { .ok_or_else(|| format!("cannot deconstruct a jsonb {}", self.type_name()))?; Ok(object.iter().map(|(k, v)| (k, Self(v)))) } + + /// Pretty print the jsonb value to the given writer, with 4 spaces indentation. + pub fn pretty(self, f: &mut impl std::fmt::Write) -> std::fmt::Result { + use serde::Serialize; + use serde_json::ser::{PrettyFormatter, Serializer}; + + let mut ser = + Serializer::with_formatter(FmtToIoUnchecked(f), PrettyFormatter::with_indent(b" ")); + self.0.serialize(&mut ser).map_err(|_| std::fmt::Error) + } } /// A custom implementation for [`serde_json::ser::Formatter`] to match PostgreSQL, which adds extra @@ -448,3 +445,18 @@ impl serde_json::ser::Formatter for ToTextFormatter { writer.write_all(b": ") } } + +/// A wrapper of [`std::fmt::Write`] to implement [`std::io::Write`]. +struct FmtToIoUnchecked(F); + +impl std::io::Write for FmtToIoUnchecked { + fn write(&mut self, buf: &[u8]) -> std::io::Result { + let s = unsafe { std::str::from_utf8_unchecked(buf) }; + self.0.write_str(s).map_err(|_| std::io::ErrorKind::Other)?; + Ok(buf.len()) + } + + fn flush(&mut self) -> std::io::Result<()> { + Ok(()) + } +} diff --git a/src/expr/impl/src/scalar/jsonb_info.rs b/src/expr/impl/src/scalar/jsonb_info.rs index 28acf7762a9f8..be6a5e1af293d 100644 --- a/src/expr/impl/src/scalar/jsonb_info.rs +++ b/src/expr/impl/src/scalar/jsonb_info.rs @@ -52,3 +52,24 @@ pub fn is_json_type(s: &str, t: &str) -> bool { } }) } + +/// Converts the given JSON value to pretty-printed, indented text. +/// +/// # Examples +// TODO: enable docslt after sqllogictest supports multiline output +/// ```text +/// query T +/// select jsonb_pretty('[{"f1":1,"f2":null}, 2]'); +/// ---- +/// [ +/// { +/// "f1": 1, +/// "f2": null +/// }, +/// 2 +/// ] +/// ``` +#[function("jsonb_pretty(jsonb) -> varchar")] +pub fn jsonb_pretty(v: JsonbRef<'_>, writer: &mut impl Write) { + v.pretty(writer).unwrap() +} diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 0b8457bc1cb82..cc2519fb4371f 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -877,6 +877,7 @@ impl Binder { ("jsonb_array_element_text", raw_call(ExprType::JsonbAccessStr)), ("jsonb_typeof", raw_call(ExprType::JsonbTypeof)), ("jsonb_array_length", raw_call(ExprType::JsonbArrayLength)), + ("jsonb_pretty", raw_call(ExprType::JsonbPretty)), // Functions that return a constant value ("pi", pi()), // greatest and least diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 470e1efc6aba0..42813ca07b354 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -177,6 +177,7 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::JsonbAccessStr | expr_node::Type::JsonbTypeof | expr_node::Type::JsonbArrayLength + | expr_node::Type::JsonbPretty | expr_node::Type::IsJson | expr_node::Type::Sind | expr_node::Type::Cosd diff --git a/src/risedevtool/src/bin/risedev-docslt.rs b/src/risedevtool/src/bin/risedev-docslt.rs index 6a76ed1035959..0f5307422f3d6 100644 --- a/src/risedevtool/src/bin/risedev-docslt.rs +++ b/src/risedevtool/src/bin/risedev-docslt.rs @@ -53,11 +53,12 @@ fn extract_slt(filepath: &Path) -> Vec { if !(line.starts_with("///") || line.starts_with("//!")) { panic!("expect /// or //! at {}:{}", filepath.display(), i + 1); } - line = line[3..].trim(); - if line == "```" { + line = &line[3..]; + if line.trim() == "```" { break; } - content += line; + // strip one leading space + content += line.strip_prefix(' ').unwrap_or(line); content += "\n"; } blocks.push(SltBlock { diff --git a/src/tests/regress/data/sql/jsonb.sql b/src/tests/regress/data/sql/jsonb.sql index cd3f6c8ba9e71..59b00932db189 100644 --- a/src/tests/regress/data/sql/jsonb.sql +++ b/src/tests/regress/data/sql/jsonb.sql @@ -1047,9 +1047,9 @@ SELECT '["a","b","c",[1,2],null]'::jsonb -> -6; --@ select jsonb_strip_nulls('{"a": {"b": null, "c": null}, "d": {} }'); ---@ select jsonb_pretty('{"a": "test", "b": [1, 2, 3], "c": "test3", "d":{"dd": "test4", "dd2":{"ddd": "test5"}}}'); ---@ select jsonb_pretty('[{"f1":1,"f2":null},2,null,[[{"x":true},6,7],8],3]'); ---@ select jsonb_pretty('{"a":["b", "c"], "d": {"e":"f"}}'); +select jsonb_pretty('{"a": "test", "b": [1, 2, 3], "c": "test3", "d":{"dd": "test4", "dd2":{"ddd": "test5"}}}'); +select jsonb_pretty('[{"f1":1,"f2":null},2,null,[[{"x":true},6,7],8],3]'); +select jsonb_pretty('{"a":["b", "c"], "d": {"e":"f"}}'); --@ --@ select jsonb_concat('{"d": "test", "a": [1, 2]}', '{"g": "test2", "c": {"c1":1, "c2":2}}'); --@ From 31d6dc01b543631ed13888760d1fc8ee365d649a Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Fri, 27 Oct 2023 16:34:46 +0800 Subject: [PATCH 28/52] fix: Remove possible panic in metrics clean up. (#13004) --- src/batch/src/executor/generic_exchange.rs | 15 +- src/batch/src/executor/group_top_n.rs | 6 +- src/batch/src/executor/hash_agg.rs | 6 +- src/batch/src/executor/join/hash_join.rs | 6 +- src/batch/src/executor/row_seq_scan.rs | 18 +- src/batch/src/lib.rs | 1 + src/batch/src/monitor/stats.rs | 387 +++++------------- src/batch/src/task/context.rs | 13 +- .../src/estimate_size/collections/heap.rs | 7 +- src/common/src/memory/mem_context.rs | 66 ++- 10 files changed, 160 insertions(+), 365 deletions(-) diff --git a/src/batch/src/executor/generic_exchange.rs b/src/batch/src/executor/generic_exchange.rs index 6ac7e41b45959..3520d750bbc5e 100644 --- a/src/batch/src/executor/generic_exchange.rs +++ b/src/batch/src/executor/generic_exchange.rs @@ -201,12 +201,15 @@ impl GenericExchangeExec // create the collector let source_id = source.get_task_id(); let counter = metrics.as_ref().map(|metrics| { - metrics.create_collector_for_exchange_recv_row_number(vec![ - identity, - source_id.query_id, - source_id.stage_id.to_string(), - source_id.task_id.to_string(), - ]) + metrics + .executor_metrics() + .exchange_recv_row_number + .with_label_values(&[ + source_id.query_id.as_str(), + format!("{}", source_id.stage_id).as_str(), + format!("{}", source_id.task_id).as_str(), + identity.as_str(), + ]) }); loop { diff --git a/src/batch/src/executor/group_top_n.rs b/src/batch/src/executor/group_top_n.rs index 1b76bb6e6e997..3fb335398d89b 100644 --- a/src/batch/src/executor/group_top_n.rs +++ b/src/batch/src/executor/group_top_n.rs @@ -240,9 +240,9 @@ impl GroupTopNExecutor { #[cfg(test)] mod tests { use futures::stream::StreamExt; - use prometheus::IntGauge; use risingwave_common::array::DataChunk; use risingwave_common::catalog::{Field, Schema}; + use risingwave_common::metrics::LabelGuardedIntGauge; use risingwave_common::test_prelude::DataChunkTestExt; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; @@ -254,7 +254,7 @@ mod tests { #[tokio::test] async fn test_group_top_n_executor() { - let parent_mem = MemoryContext::root(IntGauge::new("root_memory_usage", " ").unwrap()); + let parent_mem = MemoryContext::root(LabelGuardedIntGauge::<4>::test_int_gauge()); { let schema = Schema { fields: vec![ @@ -290,7 +290,7 @@ mod tests { ]; let mem_ctx = MemoryContext::new( Some(parent_mem.clone()), - IntGauge::new("memory_usage", " ").unwrap(), + LabelGuardedIntGauge::<4>::test_int_gauge(), ); let top_n_executor = (GroupTopNExecutorBuilder { child: Box::new(mock_executor), diff --git a/src/batch/src/executor/hash_agg.rs b/src/batch/src/executor/hash_agg.rs index 03ce86d475620..7cc408485444d 100644 --- a/src/batch/src/executor/hash_agg.rs +++ b/src/batch/src/executor/hash_agg.rs @@ -308,8 +308,8 @@ mod tests { use std::sync::Arc; use futures_async_stream::for_await; - use prometheus::IntGauge; use risingwave_common::catalog::{Field, Schema}; + use risingwave_common::metrics::LabelGuardedIntGauge; use risingwave_common::test_prelude::DataChunkTestExt; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::PbDataType; @@ -323,7 +323,7 @@ mod tests { #[tokio::test] async fn execute_int32_grouped() { - let parent_mem = MemoryContext::root(IntGauge::new("root_memory_usage", " ").unwrap()); + let parent_mem = MemoryContext::root(LabelGuardedIntGauge::<4>::test_int_gauge()); { let src_exec = Box::new(MockExecutor::with_chunk( DataChunk::from_pretty( @@ -370,7 +370,7 @@ mod tests { let mem_context = MemoryContext::new( Some(parent_mem.clone()), - IntGauge::new("memory_usage", " ").unwrap(), + LabelGuardedIntGauge::<4>::test_int_gauge(), ); let actual_exec = HashAggExecutorBuilder::deserialize( &agg_prost, diff --git a/src/batch/src/executor/join/hash_join.rs b/src/batch/src/executor/join/hash_join.rs index 0f5a0788b23ec..cd9caa54e4b2a 100644 --- a/src/batch/src/executor/join/hash_join.rs +++ b/src/batch/src/executor/join/hash_join.rs @@ -1933,12 +1933,12 @@ impl HashJoinExecutor { mod tests { use futures::StreamExt; use futures_async_stream::for_await; - use prometheus::IntGauge; use risingwave_common::array::{ArrayBuilderImpl, DataChunk}; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::error::Result; use risingwave_common::hash::Key32; use risingwave_common::memory::MemoryContext; + use risingwave_common::metrics::LabelGuardedIntGauge; use risingwave_common::test_prelude::DataChunkTestExt; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqDebug; @@ -2157,7 +2157,7 @@ mod tests { }; let mem_ctx = - MemoryContext::new(parent_mem_ctx, IntGauge::new("memory_usage", " ").unwrap()); + MemoryContext::new(parent_mem_ctx, LabelGuardedIntGauge::<4>::test_int_gauge()); Box::new(HashJoinExecutor::::new( join_type, output_indices, @@ -2198,7 +2198,7 @@ mod tests { right_executor: BoxedExecutor, ) { let parent_mem_context = - MemoryContext::root(IntGauge::new("total_memory_usage", " ").unwrap()); + MemoryContext::root(LabelGuardedIntGauge::<4>::test_int_gauge()); { let join_executor = self.create_join_executor_with_chunk_size_and_executors( diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index 3c46682c139de..1a94e9ef25b4c 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -use std::ops::{Bound, RangeBounds}; +use std::ops::{Bound, Deref, RangeBounds}; use std::sync::Arc; use futures::{pin_mut, StreamExt}; @@ -310,9 +310,12 @@ impl RowSeqScanExecutor { let table = Arc::new(table); // Create collector. - let histogram = metrics - .as_ref() - .map(|metrics| metrics.create_collector_for_row_seq_scan_next_duration(vec![identity])); + let histogram = metrics.as_ref().map(|metrics| { + metrics + .executor_metrics() + .row_seq_scan_next_duration + .with_label_values(&metrics.executor_labels(&identity)) + }); if ordered { // Currently we execute range-scans concurrently so the order is not guaranteed if @@ -329,9 +332,8 @@ impl RowSeqScanExecutor { // Point Get for point_get in point_gets { let table = table.clone(); - let histogram = histogram.clone(); if let Some(row) = - Self::execute_point_get(table, point_get, epoch.clone(), histogram).await? + Self::execute_point_get(table, point_get, epoch.clone(), histogram.clone()).await? { if let Some(chunk) = data_chunk_builder.append_one_row(row) { yield chunk; @@ -365,7 +367,7 @@ impl RowSeqScanExecutor { table: Arc>, scan_range: ScanRange, epoch: BatchQueryEpoch, - histogram: Option, + histogram: Option>, ) -> Result> { let pk_prefix = scan_range.pk_prefix; assert!(pk_prefix.len() == table.pk_indices().len()); @@ -389,7 +391,7 @@ impl RowSeqScanExecutor { ordered: bool, epoch: BatchQueryEpoch, chunk_size: usize, - histogram: Option, + histogram: Option>, ) { let ScanRange { pk_prefix, diff --git a/src/batch/src/lib.rs b/src/batch/src/lib.rs index 809c096eb49df..6793b779ac754 100644 --- a/src/batch/src/lib.rs +++ b/src/batch/src/lib.rs @@ -32,6 +32,7 @@ #![feature(result_option_inspect)] #![feature(assert_matches)] #![feature(lazy_cell)] +#![feature(array_methods)] mod error; pub mod exchange_source; diff --git a/src/batch/src/monitor/stats.rs b/src/batch/src/monitor/stats.rs index f59e0217cd7a0..f286515aafbe5 100644 --- a/src/batch/src/monitor/stats.rs +++ b/src/batch/src/monitor/stats.rs @@ -12,51 +12,28 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; use std::sync::{Arc, LazyLock}; -use itertools::Itertools; -use parking_lot::Mutex; -use paste::paste; -use prometheus::core::{ - AtomicF64, AtomicU64, Collector, Desc, GenericCounter, GenericCounterVec, GenericGaugeVec, +use prometheus::{IntGauge, Registry}; +use risingwave_common::metrics::{ + LabelGuardedGaugeVec, LabelGuardedHistogramVec, LabelGuardedIntCounterVec, + LabelGuardedIntGaugeVec, TrAdderGauge, }; -use prometheus::{ - exponential_buckets, opts, proto, GaugeVec, Histogram, HistogramOpts, HistogramVec, - IntCounterVec, IntGauge, IntGaugeVec, Registry, -}; -use risingwave_common::metrics::TrAdderGauge; use risingwave_common::monitor::GLOBAL_METRICS_REGISTRY; use crate::task::TaskId; -macro_rules! for_all_task_metrics { - ($macro:ident) => { - $macro! { - { task_first_poll_delay, GenericGaugeVec }, - { task_fast_poll_duration, GenericGaugeVec }, - { task_idle_duration, GenericGaugeVec }, - { task_poll_duration, GenericGaugeVec }, - { task_scheduled_duration, GenericGaugeVec }, - { task_slow_poll_duration, GenericGaugeVec }, - { task_mem_usage, IntGaugeVec }, - } - }; -} - -macro_rules! def_task_metrics { - ($( { $metric:ident, $type:ty }, )*) => { - #[derive(Clone)] - pub struct BatchTaskMetrics { - descs: Vec, - delete_task: Arc>>, - $( pub $metric: $type, )* - } - }; +#[derive(Clone)] +pub struct BatchTaskMetrics { + pub task_first_poll_delay: LabelGuardedGaugeVec<3>, + pub task_fast_poll_duration: LabelGuardedGaugeVec<3>, + pub task_idle_duration: LabelGuardedGaugeVec<3>, + pub task_poll_duration: LabelGuardedGaugeVec<3>, + pub task_scheduled_duration: LabelGuardedGaugeVec<3>, + pub task_slow_poll_duration: LabelGuardedGaugeVec<3>, + pub task_mem_usage: LabelGuardedIntGaugeVec<3>, } -for_all_task_metrics!(def_task_metrics); - pub static GLOBAL_BATCH_TASK_METRICS: LazyLock = LazyLock::new(|| BatchTaskMetrics::new(&GLOBAL_METRICS_REGISTRY)); @@ -64,76 +41,62 @@ impl BatchTaskMetrics { /// The created [`BatchTaskMetrics`] is already registered to the `registry`. fn new(registry: &Registry) -> Self { let task_labels = ["query_id", "stage_id", "task_id"]; - let mut descs = Vec::with_capacity(8); - let task_first_poll_delay = GaugeVec::new(opts!( + let task_first_poll_delay = register_guarded_gauge_vec_with_registry!( "batch_task_first_poll_delay", "The total duration (s) elapsed between the instant tasks are instrumented, and the instant they are first polled.", - ), &task_labels[..]).unwrap(); - descs.extend(task_first_poll_delay.desc().into_iter().cloned()); - - let task_fast_poll_duration = GaugeVec::new( - opts!( - "batch_task_fast_poll_duration", - "The total duration (s) of fast polls.", - ), - &task_labels[..], + &task_labels, + registry).unwrap(); + + let task_fast_poll_duration = register_guarded_gauge_vec_with_registry!( + "batch_task_fast_poll_duration", + "The total duration (s) of fast polls.", + &task_labels, + registry ) .unwrap(); - descs.extend(task_fast_poll_duration.desc().into_iter().cloned()); - - let task_idle_duration = GaugeVec::new( - opts!( - "batch_task_idle_duration", - "The total duration (s) that tasks idled.", - ), - &task_labels[..], + + let task_idle_duration = register_guarded_gauge_vec_with_registry!( + "batch_task_idle_duration", + "The total duration (s) that tasks idled.", + &task_labels, + registry ) .unwrap(); - descs.extend(task_idle_duration.desc().into_iter().cloned()); - - let task_poll_duration = GaugeVec::new( - opts!( - "batch_task_poll_duration", - "The total duration (s) elapsed during polls.", - ), - &task_labels[..], + + let task_poll_duration = register_guarded_gauge_vec_with_registry!( + "batch_task_poll_duration", + "The total duration (s) elapsed during polls.", + &task_labels, + registry, ) .unwrap(); - descs.extend(task_poll_duration.desc().into_iter().cloned()); - - let task_scheduled_duration = GaugeVec::new( - opts!( - "batch_task_scheduled_duration", - "The total duration (s) that tasks spent waiting to be polled after awakening.", - ), - &task_labels[..], + + let task_scheduled_duration = register_guarded_gauge_vec_with_registry!( + "batch_task_scheduled_duration", + "The total duration (s) that tasks spent waiting to be polled after awakening.", + &task_labels, + registry, ) .unwrap(); - descs.extend(task_scheduled_duration.desc().into_iter().cloned()); - - let task_slow_poll_duration = GaugeVec::new( - opts!( - "batch_task_slow_poll_duration", - "The total duration (s) of slow polls.", - ), - &task_labels[..], + + let task_slow_poll_duration = register_guarded_gauge_vec_with_registry!( + "batch_task_slow_poll_duration", + "The total duration (s) of slow polls.", + &task_labels, + registry, ) .unwrap(); - descs.extend(task_slow_poll_duration.desc().into_iter().cloned()); - - let task_mem_usage = IntGaugeVec::new( - opts!( - "batch_task_mem_usage", - "Memory usage of batch tasks in bytes." - ), - &task_labels[..], + + let task_mem_usage = register_guarded_int_gauge_vec_with_registry!( + "batch_task_mem_usage", + "Memory usage of batch tasks in bytes.", + &task_labels, + registry, ) .unwrap(); - descs.extend(task_mem_usage.desc().into_iter().cloned()); - let metrics = Self { - descs, + Self { task_first_poll_delay, task_fast_poll_duration, task_idle_duration, @@ -141,197 +104,64 @@ impl BatchTaskMetrics { task_scheduled_duration, task_slow_poll_duration, task_mem_usage, - delete_task: Arc::new(Mutex::new(Vec::new())), - }; - registry.register(Box::new(metrics.clone())).unwrap(); - metrics + } } /// Create a new `BatchTaskMetrics` instance used in tests or other places. pub fn for_test() -> Self { GLOBAL_BATCH_TASK_METRICS.clone() } - - fn clean_metrics(&self) { - let delete_task: Vec = { - let mut delete_task = self.delete_task.lock(); - if delete_task.is_empty() { - return; - } - std::mem::take(delete_task.as_mut()) - }; - for id in &delete_task { - let stage_id = id.stage_id.to_string(); - let task_id = id.task_id.to_string(); - let labels = vec![id.query_id.as_str(), stage_id.as_str(), task_id.as_str()]; - - macro_rules! remove { - ($({ $metric:ident, $type:ty},)*) => { - $( - if let Err(err) = self.$metric.remove_label_values(&labels) { - warn!("Failed to remove label values: {:?}", err); - } - )* - }; - } - for_all_task_metrics!(remove); - } - } - - pub fn add_delete_task(&self, id: TaskId) { - self.delete_task.lock().push(id); - } -} - -impl Collector for BatchTaskMetrics { - fn desc(&self) -> Vec<&Desc> { - self.descs.iter().collect() - } - - fn collect(&self) -> Vec { - let mut mfs = Vec::with_capacity(8); - - macro_rules! collect { - ($({ $metric:ident, $type:ty },)*) => { - $( - mfs.extend(self.$metric.collect()); - )* - }; - } - for_all_task_metrics!(collect); - - // TODO: Every time we execute it involving get the lock, here maybe a bottleneck. - self.clean_metrics(); - - mfs - } -} - -macro_rules! for_all_executor_metrics { - ($macro:ident) => { - $macro! { - { exchange_recv_row_number, GenericCounterVec, GenericCounter}, - { row_seq_scan_next_duration, HistogramVec , Histogram}, - { mem_usage, IntGaugeVec, IntGauge }, - } - }; } -macro_rules! def_executor_metrics { - ($( { $metric:ident, $type:ty, $_t:ty }, )*) => { - #[derive(Clone)] - pub struct BatchExecutorMetrics { - descs: Vec, - delete_task: Arc>>, - register_labels: Arc>>>>, - $( pub $metric: $type, )* - } - }; +#[derive(Clone)] +pub struct BatchExecutorMetrics { + pub exchange_recv_row_number: LabelGuardedIntCounterVec<4>, + pub row_seq_scan_next_duration: LabelGuardedHistogramVec<4>, + pub mem_usage: LabelGuardedIntGaugeVec<4>, } -for_all_executor_metrics!(def_executor_metrics); - pub static GLOBAL_BATCH_EXECUTOR_METRICS: LazyLock = LazyLock::new(|| BatchExecutorMetrics::new(&GLOBAL_METRICS_REGISTRY)); impl BatchExecutorMetrics { fn new(register: &Registry) -> Self { - let executor_labels = vec!["query_id", "stage_id", "task_id", "executor_id"]; - let mut descs = Vec::with_capacity(2); - - let mut custom_labels = executor_labels.clone(); - custom_labels.extend_from_slice(&["source_query_id", "source_stage_id", "source_task_id"]); - let exchange_recv_row_number = IntCounterVec::new( - opts!( - "batch_exchange_recv_row_number", - "Total number of row that have been received from upstream source", - ), - &custom_labels, + let executor_labels = ["query_id", "stage_id", "task_id", "executor_id"]; + + let exchange_recv_row_number = register_guarded_int_counter_vec_with_registry!( + "batch_exchange_recv_row_number", + "Total number of row that have been received from upstream source", + &executor_labels, + register, ) .unwrap(); - descs.extend(exchange_recv_row_number.desc().into_iter().cloned()); - - let row_seq_scan_next_duration = HistogramVec::new( - HistogramOpts::new( - "batch_row_seq_scan_next_duration", - "Time spent deserializing into a row in cell based table.", - ) - .buckets(exponential_buckets(0.0001, 2.0, 20).unwrap()), + + let row_seq_scan_next_duration = register_guarded_histogram_vec_with_registry!( + "batch_row_seq_scan_next_duration", + "Time spent deserializing into a row in cell based table.", &executor_labels, + register, ) .unwrap(); - descs.extend(row_seq_scan_next_duration.desc().into_iter().cloned()); - let mem_usage = IntGaugeVec::new( - opts!( - "batch_executor_mem_usage", - "Batch executor memory usage in bytes." - ), + let mem_usage = register_guarded_int_gauge_vec_with_registry!( + "batch_executor_mem_usage", + "Batch executor memory usage in bytes.", &executor_labels, + register, ) .unwrap(); - descs.extend(mem_usage.desc().into_iter().cloned()); - let metrics = Self { - descs, - delete_task: Arc::new(Mutex::new(Vec::new())), + Self { exchange_recv_row_number, row_seq_scan_next_duration, mem_usage, - register_labels: Arc::new(Mutex::new(HashMap::new())), - }; - register.register(Box::new(metrics.clone())).unwrap(); - metrics + } } /// Create a new `BatchTaskMetrics` instance used in tests or other places. pub fn for_test() -> Self { GLOBAL_BATCH_EXECUTOR_METRICS.clone() } - - fn clean_metrics(&self) { - let delete_task: Vec = { - let mut delete_task = self.delete_task.lock(); - if delete_task.is_empty() { - return; - } - std::mem::take(delete_task.as_mut()) - }; - let delete_labels = { - let mut register_labels = self.register_labels.lock(); - let mut delete_labels = Vec::with_capacity(delete_task.len()); - for id in delete_task { - if let Some(callback) = register_labels.remove(&id) { - delete_labels.push(callback); - } - } - delete_labels - }; - delete_labels - .into_iter() - .for_each(|delete_labels| delete_labels.into_iter().for_each(|callback| callback())); - } - - pub fn add_delete_task(&self, task_id: TaskId) { - self.delete_task.lock().push(task_id); - } -} - -impl Collector for BatchExecutorMetrics { - fn desc(&self) -> Vec<&Desc> { - self.descs.iter().collect() - } - - fn collect(&self) -> Vec { - let mut mfs = Vec::with_capacity(2); - - mfs.extend(self.exchange_recv_row_number.collect()); - mfs.extend(self.row_seq_scan_next_duration.collect()); - - self.clean_metrics(); - - mfs - } } pub type BatchMetricsWithTaskLabels = Arc; @@ -343,46 +173,10 @@ pub struct BatchMetricsWithTaskLabelsInner { task_metrics: Arc, executor_metrics: Arc, task_id: TaskId, - task_labels: Vec, -} - -macro_rules! def_create_executor_collector { - ($( { $metric:ident, $type:ty, $collector_type:ty }, )*) => { - paste! { - $( - pub fn [](&self,executor_label: Vec) -> $collector_type { - let mut owned_task_labels = self.task_labels.clone(); - owned_task_labels.extend(executor_label); - let task_labels = owned_task_labels.iter().map(|s| s.as_str()).collect_vec(); - - let collecter = self - .executor_metrics - .$metric - .with_label_values(&task_labels); - - let metrics = self.executor_metrics.$metric.clone(); - - self.executor_metrics - .register_labels - .lock() - .entry(self.task_id.clone()) - .or_default() - .push(Box::new(move || { - metrics.remove_label_values( - &owned_task_labels.iter().map(|s| s.as_str()).collect::>(), - ).expect("Collector with same label only can be created once. It should never have case of duplicate remove"); - })); - - collecter - } - )* - } - }; + task_labels: [String; 3], } impl BatchMetricsWithTaskLabelsInner { - for_all_executor_metrics! {def_create_executor_collector} - pub fn new( task_metrics: Arc, executor_metrics: Arc, @@ -392,12 +186,12 @@ impl BatchMetricsWithTaskLabelsInner { task_metrics, executor_metrics, task_id: id.clone(), - task_labels: vec![id.query_id, id.stage_id.to_string(), id.task_id.to_string()], + task_labels: [id.query_id, id.stage_id.to_string(), id.task_id.to_string()], } } - pub fn task_labels(&self) -> Vec<&str> { - self.task_labels.iter().map(AsRef::as_ref).collect() + pub fn task_labels(&self) -> [&str; 3] { + self.task_labels.each_ref().map(String::as_str) } pub fn task_id(&self) -> TaskId { @@ -407,12 +201,21 @@ impl BatchMetricsWithTaskLabelsInner { pub fn get_task_metrics(&self) -> &Arc { &self.task_metrics } -} -impl Drop for BatchMetricsWithTaskLabelsInner { - fn drop(&mut self) { - self.task_metrics.add_delete_task(self.task_id()); - self.executor_metrics.add_delete_task(self.task_id()); + pub fn executor_metrics(&self) -> &BatchExecutorMetrics { + &self.executor_metrics + } + + pub fn executor_labels<'a>( + &'a self, + executor_id: &'a (impl AsRef + ?Sized), + ) -> [&'a str; 4] { + [ + self.task_labels[0].as_str(), + self.task_labels[1].as_str(), + self.task_labels[2].as_str(), + executor_id.as_ref(), + ] } } diff --git a/src/batch/src/task/context.rs b/src/batch/src/task/context.rs index f9648c58f7a50..70fa48b20d2e9 100644 --- a/src/batch/src/task/context.rs +++ b/src/batch/src/task/context.rs @@ -11,15 +11,14 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. - use std::sync::atomic::{AtomicUsize, Ordering}; use std::sync::Arc; -use prometheus::IntGauge; use risingwave_common::catalog::SysCatalogReaderRef; use risingwave_common::config::BatchConfig; use risingwave_common::error::Result; use risingwave_common::memory::MemoryContext; +use risingwave_common::metrics::LabelGuardedIntGauge; use risingwave_common::util::addr::{is_local_address, HostAddr}; use risingwave_connector::source::monitor::SourceMetrics; use risingwave_rpc_client::ComputeClientPoolRef; @@ -142,10 +141,10 @@ impl BatchTaskContext for ComputeNodeContext { fn create_executor_mem_context(&self, executor_id: &str) -> MemoryContext { if let Some(metrics) = &self.batch_metrics { - let mut labels = metrics.task_labels(); - labels.push(executor_id); - let executor_mem_usage = - metrics.create_collector_for_mem_usage(vec![executor_id.to_string()]); + let executor_mem_usage = metrics + .executor_metrics() + .mem_usage + .with_label_values(&metrics.executor_labels(executor_id)); MemoryContext::new(Some(self.mem_context.clone()), executor_mem_usage) } else { MemoryContext::none() @@ -195,7 +194,7 @@ impl ComputeNodeContext { cur_mem_val: Arc::new(0.into()), last_mem_val: Arc::new(0.into()), // Leave it for now, it should be None - mem_context: MemoryContext::root(IntGauge::new("test", "test").unwrap()), + mem_context: MemoryContext::root(LabelGuardedIntGauge::<4>::test_int_gauge()), } } diff --git a/src/common/src/estimate_size/collections/heap.rs b/src/common/src/estimate_size/collections/heap.rs index beb5430844c32..96bd942aa02b1 100644 --- a/src/common/src/estimate_size/collections/heap.rs +++ b/src/common/src/estimate_size/collections/heap.rs @@ -105,14 +105,13 @@ where #[cfg(test)] mod tests { - use prometheus::IntGauge; - use crate::estimate_size::collections::MemMonitoredHeap; use crate::memory::MemoryContext; + use crate::metrics::LabelGuardedIntGauge; #[test] fn test_heap() { - let gauge = IntGauge::new("test", "test").unwrap(); + let gauge = LabelGuardedIntGauge::<4>::test_int_gauge(); let mem_ctx = MemoryContext::root(gauge.clone()); let mut heap = MemMonitoredHeap::::new_with(mem_ctx); @@ -130,7 +129,7 @@ mod tests { #[test] fn test_heap_drop() { - let gauge = IntGauge::new("test", "test").unwrap(); + let gauge = LabelGuardedIntGauge::<4>::test_int_gauge(); let mem_ctx = MemoryContext::root(gauge.clone()); let vec = { diff --git a/src/common/src/memory/mem_context.rs b/src/common/src/memory/mem_context.rs index 70aa20d9e331d..c835a46732a73 100644 --- a/src/common/src/memory/mem_context.rs +++ b/src/common/src/memory/mem_context.rs @@ -12,64 +12,52 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::ops::Deref; use std::sync::Arc; -use prometheus::IntGauge; - use super::MonitoredGlobalAlloc; -use crate::metrics::TrAdderGauge; +use crate::metrics::{LabelGuardedIntGauge, TrAdderGauge}; -struct MemoryContextInner { - counter: MemCounter, - parent: Option, +pub trait MemCounter: Send + Sync + 'static { + fn add(&self, bytes: i64); + fn get_bytes_used(&self) -> i64; } -#[derive(Clone)] -pub struct MemoryContext { - /// Add None op mem context, so that we don't need to return [`Option`] in - /// `BatchTaskContext`. This helps with later `Allocator` implementation. - inner: Option>, -} - -#[derive(Debug)] -pub enum MemCounter { - /// Used when the add/sub operation don't have much conflicts. - Atomic(IntGauge), - /// Used when the add/sub operation may cause a lot of conflicts. - TrAdder(TrAdderGauge), -} +impl MemCounter for TrAdderGauge { + fn add(&self, bytes: i64) { + self.add(bytes) + } -impl From for MemCounter { - fn from(value: IntGauge) -> Self { - MemCounter::Atomic(value) + fn get_bytes_used(&self) -> i64 { + self.get() } } -impl MemCounter { +impl MemCounter for LabelGuardedIntGauge { fn add(&self, bytes: i64) { - match &self { - MemCounter::TrAdder(c) => c.add(bytes), - MemCounter::Atomic(c) => c.add(bytes), - } + self.deref().add(bytes) } fn get_bytes_used(&self) -> i64 { - match &self { - MemCounter::TrAdder(c) => c.get(), - MemCounter::Atomic(c) => c.get(), - } + self.get() } } -impl From for MemCounter { - fn from(value: TrAdderGauge) -> Self { - MemCounter::TrAdder(value) - } +struct MemoryContextInner { + counter: Box, + parent: Option, +} + +#[derive(Clone)] +pub struct MemoryContext { + /// Add None op mem context, so that we don't need to return [`Option`] in + /// `BatchTaskContext`. This helps with later `Allocator` implementation. + inner: Option>, } impl MemoryContext { - pub fn new(parent: Option, counter: impl Into) -> Self { - let c = counter.into(); + pub fn new(parent: Option, counter: impl MemCounter) -> Self { + let c = Box::new(counter); Self { inner: Some(Arc::new(MemoryContextInner { counter: c, parent })), } @@ -80,7 +68,7 @@ impl MemoryContext { Self { inner: None } } - pub fn root(counter: impl Into) -> Self { + pub fn root(counter: impl MemCounter) -> Self { Self::new(None, counter) } From f60f5b6bc64ebadd3e7b7616d2899f2ae9887c4c Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Fri, 27 Oct 2023 17:45:11 +0800 Subject: [PATCH 29/52] fix(ci): fix iceberg-cdc integration_tests (#13082) Co-authored-by: ZENOTME --- .../{docker-compose.yaml => docker-compose.yml} | 9 +++++++++ integration_tests/iceberg-cdc/python/Dockerfile | 12 ++++++++++++ .../iceberg-cdc/python/{ => script}/check.py | 0 .../iceberg-cdc/python/{ => script}/config.ini | 4 ++-- .../iceberg-cdc/python/{ => script}/init.py | 0 .../iceberg-cdc/python/{ => script}/pyproject.toml | 0 integration_tests/iceberg-cdc/run_test.sh | 14 ++++---------- integration_tests/scripts/check_data.py | 2 +- 8 files changed, 28 insertions(+), 13 deletions(-) rename integration_tests/iceberg-cdc/{docker-compose.yaml => docker-compose.yml} (93%) create mode 100644 integration_tests/iceberg-cdc/python/Dockerfile rename integration_tests/iceberg-cdc/python/{ => script}/check.py (100%) rename integration_tests/iceberg-cdc/python/{ => script}/config.ini (53%) rename integration_tests/iceberg-cdc/python/{ => script}/init.py (100%) rename integration_tests/iceberg-cdc/python/{ => script}/pyproject.toml (100%) diff --git a/integration_tests/iceberg-cdc/docker-compose.yaml b/integration_tests/iceberg-cdc/docker-compose.yml similarity index 93% rename from integration_tests/iceberg-cdc/docker-compose.yaml rename to integration_tests/iceberg-cdc/docker-compose.yml index 8e9ad1062ef38..a41441eb7f0a5 100644 --- a/integration_tests/iceberg-cdc/docker-compose.yaml +++ b/integration_tests/iceberg-cdc/docker-compose.yml @@ -79,6 +79,15 @@ services: - "./mysql_prepare.sql:/mysql_prepare.sql" container_name: prepare_mysql restart: on-failure + datagen: + build: ../datagen + depends_on: [mysql] + command: + - /bin/sh + - -c + - /datagen --mode clickstream --qps 1 mysql --user mysqluser --password mysqlpw --host mysql --port 3306 --db mydb + container_name: datagen + restart: on-failure rest: image: tabulario/iceberg-rest:0.6.0 diff --git a/integration_tests/iceberg-cdc/python/Dockerfile b/integration_tests/iceberg-cdc/python/Dockerfile new file mode 100644 index 0000000000000..b0e395ada44ed --- /dev/null +++ b/integration_tests/iceberg-cdc/python/Dockerfile @@ -0,0 +1,12 @@ +FROM python:3.8 + +RUN curl -sSL https://install.python-poetry.org | python3 - + +WORKDIR /app + +COPY script /app + +RUN $HOME/.local/bin/poetry update + +CMD $HOME/.local/bin/poetry run python init.py;sleep 40;$HOME/.local/bin/poetry run python check.py + diff --git a/integration_tests/iceberg-cdc/python/check.py b/integration_tests/iceberg-cdc/python/script/check.py similarity index 100% rename from integration_tests/iceberg-cdc/python/check.py rename to integration_tests/iceberg-cdc/python/script/check.py diff --git a/integration_tests/iceberg-cdc/python/config.ini b/integration_tests/iceberg-cdc/python/script/config.ini similarity index 53% rename from integration_tests/iceberg-cdc/python/config.ini rename to integration_tests/iceberg-cdc/python/script/config.ini index bd95eddc5b80e..d9c7331c35da8 100644 --- a/integration_tests/iceberg-cdc/python/config.ini +++ b/integration_tests/iceberg-cdc/python/script/config.ini @@ -1,8 +1,8 @@ [spark] -url=sc://localhost:15002 +url=sc://spark:15002 [risingwave] db=dev user=root -host=127.0.0.1 +host=frontend-node-0 port=4566 diff --git a/integration_tests/iceberg-cdc/python/init.py b/integration_tests/iceberg-cdc/python/script/init.py similarity index 100% rename from integration_tests/iceberg-cdc/python/init.py rename to integration_tests/iceberg-cdc/python/script/init.py diff --git a/integration_tests/iceberg-cdc/python/pyproject.toml b/integration_tests/iceberg-cdc/python/script/pyproject.toml similarity index 100% rename from integration_tests/iceberg-cdc/python/pyproject.toml rename to integration_tests/iceberg-cdc/python/script/pyproject.toml diff --git a/integration_tests/iceberg-cdc/run_test.sh b/integration_tests/iceberg-cdc/run_test.sh index 2d8b691bc7284..6e47d6ad59be4 100755 --- a/integration_tests/iceberg-cdc/run_test.sh +++ b/integration_tests/iceberg-cdc/run_test.sh @@ -6,14 +6,8 @@ docker-compose up -d --wait # To avoid exiting by unhealth, set it after start environment. set -ex -# Generate data -docker build -t iceberg-cdc-datagen ../datagen -timeout 20 docker run --network=iceberg-cdc_default iceberg-cdc-datagen /datagen --mode clickstream --qps 1 mysql --user mysqluser --password mysqlpw --host mysql --port 3306 --db mydb & +sleep 20 +docker stop datagen -cd python -poetry update --quiet -# Init source, mv, and sink. -poetry run python init.py -# Wait for sink to be finished. -sleep 40; -poetry run python check.py +docker build -t iceberg-cdc-python python +docker run --network=iceberg-cdc_default iceberg-cdc-python diff --git a/integration_tests/scripts/check_data.py b/integration_tests/scripts/check_data.py index 52cc79dc0ab0a..37679fbdacdae 100644 --- a/integration_tests/scripts/check_data.py +++ b/integration_tests/scripts/check_data.py @@ -54,7 +54,7 @@ def run_psql(sql): demo = sys.argv[1] upstream = sys.argv[2] # mysql, postgres, etc. see scripts/integration_tests.sh -if demo in ['docker', 'iceberg-sink','clickhouse-sink']: +if demo in ['docker', 'iceberg-sink','clickhouse-sink', 'iceberg-cdc']: print('Skip for running test for `%s`' % demo) sys.exit(0) From a6f1714ddab79340f8b9120d04bc510848fba26e Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 27 Oct 2023 18:03:02 +0800 Subject: [PATCH 30/52] perf(connector): do not create span for parsing each row (#13105) Signed-off-by: Bugen Zhao --- src/connector/src/parser/mod.rs | 54 ++++++++++++++++++-------------- src/connector/src/source/base.rs | 2 +- 2 files changed, 32 insertions(+), 24 deletions(-) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 7362497bb2dcb..584e0e497d1f3 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -143,30 +143,32 @@ pub struct SourceStreamChunkRowWriter<'a> { /// An optional meta data of the original message. /// /// When this is set by `with_meta`, it'll be used to fill the columns of types other than [`SourceColumnType::Normal`]. - row_meta: Option, + row_meta: Option>, } /// The meta data of the original message for a row writer. /// /// Extracted from the `SourceMessage`. -pub struct MessageMeta { - meta: SourceMeta, - offset: String, +#[derive(Clone, Copy)] +pub struct MessageMeta<'a> { + meta: &'a SourceMeta, + split_id: &'a str, + offset: &'a str, } -impl MessageMeta { +impl MessageMeta<'_> { /// Extract the value for the given column. /// /// Returns `None` if the column is not a meta column. - fn value_for_column(&self, desc: &SourceColumnDesc) -> Option { + fn value_for_column(self, desc: &SourceColumnDesc) -> Option { match desc.column_type { // Row id columns are filled with `NULL` here and will be filled with the real // row id generated by `RowIdGenExecutor` later. SourceColumnType::RowId => Datum::None.into(), // Extract the offset from the meta data. - SourceColumnType::Offset => Datum::Some(self.offset.as_str().into()).into(), + SourceColumnType::Offset => Datum::Some(self.offset.into()).into(), // Extract custom meta data per connector. - SourceColumnType::Meta if let SourceMeta::Kafka(kafka_meta) = &self.meta => { + SourceColumnType::Meta if let SourceMeta::Kafka(kafka_meta) = self.meta => { assert_eq!(desc.name.as_str(), KAFKA_TIMESTAMP_COLUMN_NAME, "unexpected meta column name"); kafka_meta.timestamp.map(|ts| { risingwave_common::cast::i64_to_timestamptz(ts) @@ -274,11 +276,11 @@ impl OpAction for OpActionUpdate { } } -impl SourceStreamChunkRowWriter<'_> { +impl<'a> SourceStreamChunkRowWriter<'a> { /// Set the meta data of the original message for this row writer. /// /// This should always be called except for tests. - fn with_meta(self, row_meta: MessageMeta) -> Self { + fn with_meta(self, row_meta: MessageMeta<'a>) -> Self { Self { row_meta: Some(row_meta), ..self @@ -308,8 +310,12 @@ impl SourceStreamChunkRowWriter<'_> { Err(error) => { // TODO: figure out a way to fill in not-null default value if user specifies one // TODO: decide whether the error should not be ignored (e.g., even not a valid Debezium message) + // TODO: not using tracing span to provide `split_id` and `offset` due to performance concern, + // see #13105 tracing::warn!( %error, + split_id = self.row_meta.as_ref().map(|m| m.split_id), + offset = self.row_meta.as_ref().map(|m| m.offset), column = desc.name, "failed to parse non-pk column, padding with `NULL`" ); @@ -527,7 +533,7 @@ async fn into_chunk_stream(mut parser: P, data_stream for (i, msg) in batch.into_iter().enumerate() { if msg.key.is_none() && msg.payload.is_none() { if parser.parser_format() == ParserFormat::Debezium { - tracing::debug!("heartbeat message {}, skip parser", msg.offset); + tracing::debug!(offset = msg.offset, "skip parsing of heartbeat message"); // empty payload means a heartbeat in cdc source // heartbeat message offset should not overwrite data messages offset split_offset_mapping @@ -537,13 +543,8 @@ async fn into_chunk_stream(mut parser: P, data_stream continue; } - let parse_span = tracing::info_span!( - "parse_one", - split_id = msg.split_id.as_ref(), - offset = msg.offset - ); - split_offset_mapping.insert(msg.split_id, msg.offset.clone()); + split_offset_mapping.insert(msg.split_id.clone(), msg.offset.clone()); let old_op_num = builder.op_num(); match parser @@ -551,11 +552,11 @@ async fn into_chunk_stream(mut parser: P, data_stream msg.key, msg.payload, builder.row_writer().with_meta(MessageMeta { - meta: msg.meta, - offset: msg.offset, + meta: &msg.meta, + split_id: &msg.split_id, + offset: &msg.offset, }), ) - .instrument(parse_span.clone()) .await { // It's possible that parsing multiple rows in a single message PARTIALLY failed. @@ -570,7 +571,14 @@ async fn into_chunk_stream(mut parser: P, data_stream } if let Err(error) = res { - tracing::error!(parent: &parse_span, %error, "failed to parse message, skipping"); + // TODO: not using tracing span to provide `split_id` and `offset` due to performance concern, + // see #13105 + tracing::error!( + %error, + split_id = &*msg.split_id, + offset = msg.offset, + "failed to parse message, skipping" + ); parser.source_ctx().report_user_source_error(error); } } @@ -579,14 +587,14 @@ async fn into_chunk_stream(mut parser: P, data_stream match txn_ctl { TransactionControl::Begin { id } => { if let Some(Transaction { id: current_id, .. }) = ¤t_transaction { - tracing::warn!(parent: &parse_span, current_id, id, "already in transaction"); + tracing::warn!(current_id, id, "already in transaction"); } current_transaction = Some(Transaction { id, len: 0 }); } TransactionControl::Commit { id } => { let current_id = current_transaction.as_ref().map(|t| &t.id); if current_id != Some(&id) { - tracing::warn!(parent: &parse_span, ?current_id, id, "transaction id mismatch"); + tracing::warn!(?current_id, id, "transaction id mismatch"); } current_transaction = None; } diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 49dc3b5d87119..e5193291a6d14 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -561,7 +561,7 @@ pub type SplitId = Arc; pub struct SourceMessage { pub key: Option>, pub payload: Option>, - pub offset: String, + pub offset: String, // TODO: use `Arc` pub split_id: SplitId, pub meta: SourceMeta, } From 72d9765bd638e62b5c5d4bdc85013ea54843deeb Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Fri, 27 Oct 2023 18:04:22 +0800 Subject: [PATCH 31/52] feat(expr): add `jsonb_object` function (#13036) Signed-off-by: Runji Wang --- proto/expr.proto | 1 + src/expr/impl/src/lib.rs | 1 + src/expr/impl/src/scalar/jsonb_object.rs | 178 +++++++++++++++++++++++ src/expr/impl/src/scalar/mod.rs | 1 + src/frontend/src/binder/expr/function.rs | 1 + src/frontend/src/expr/pure.rs | 1 + 6 files changed, 183 insertions(+) create mode 100644 src/expr/impl/src/scalar/jsonb_object.rs diff --git a/proto/expr.proto b/proto/expr.proto index 2fcc905c4b2ab..7998f2fe8128a 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -220,6 +220,7 @@ message ExprNode { JSONB_ARRAY_LENGTH = 603; IS_JSON = 604; JSONB_CAT = 605; + JSONB_OBJECT = 606; JSONB_PRETTY = 607; // Non-pure functions below (> 1000) diff --git a/src/expr/impl/src/lib.rs b/src/expr/impl/src/lib.rs index 6ea82d30ac5f1..51b9a20a75c46 100644 --- a/src/expr/impl/src/lib.rs +++ b/src/expr/impl/src/lib.rs @@ -31,6 +31,7 @@ #![feature(coroutines)] #![feature(test)] #![feature(arc_unwrap_or_clone)] +#![feature(iter_array_chunks)] mod aggregate; mod scalar; diff --git a/src/expr/impl/src/scalar/jsonb_object.rs b/src/expr/impl/src/scalar/jsonb_object.rs new file mode 100644 index 0000000000000..3eb99cbaae615 --- /dev/null +++ b/src/expr/impl/src/scalar/jsonb_object.rs @@ -0,0 +1,178 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use jsonbb::Builder; +use risingwave_common::types::{JsonbVal, ListRef}; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_expr::{function, ExprError, Result}; + +/// Builds a JSON object out of a text array. +/// +/// The array must have either exactly one dimension with an even number of members, +/// in which case they are taken as alternating key/value pairs, or two dimensions +/// such that each inner array has exactly two elements, which are taken as a key/value pair. +/// All values are converted to JSON strings. +/// +/// # Examples +/// +/// ```slt +/// query T +/// select jsonb_object('{a, 1, b, def, c, 3.5}' :: text[]); +/// ---- +/// {"a": "1", "b": "def", "c": "3.5"} +/// +/// query error array must have even number of elements +/// select jsonb_object('{a, 1, b, "def", c}' :: text[]); +/// +/// query error null value not allowed for object key +/// select jsonb_object(array[null, 'b']); +/// +/// query T +/// select jsonb_object(array['a', null]); +/// ---- +/// {"a": null} +/// ``` +#[function("jsonb_object(varchar[]) -> jsonb")] +fn jsonb_object_1d(array: ListRef<'_>) -> Result { + if array.len() % 2 == 1 { + return Err(ExprError::InvalidParam { + name: "array", + reason: "array must have even number of elements".into(), + }); + } + let mut builder = Builder::>::new(); + builder.begin_object(); + for [key, value] in array.iter().array_chunks() { + match key { + Some(s) => builder.add_string(s.into_utf8()), + None => { + return Err(ExprError::InvalidParam { + name: "array", + reason: "null value not allowed for object key".into(), + }) + } + } + match value { + Some(s) => builder.add_string(s.into_utf8()), + None => builder.add_null(), + } + } + builder.end_object(); + Ok(builder.finish().into()) +} + +/// Builds a JSON object out of a text array. +/// +/// The array must have either exactly one dimension with an even number of members, +/// in which case they are taken as alternating key/value pairs, or two dimensions +/// such that each inner array has exactly two elements, which are taken as a key/value pair. +/// All values are converted to JSON strings. +/// +/// # Examples +/// +/// ```slt +/// query T +/// select jsonb_object('{{a, 1}, {b, def}, {c, 3.5}}' :: text[][]); +/// ---- +/// {"a": "1", "b": "def", "c": "3.5"} +/// +/// # FIXME: `null` should be parsed as a null value instead of a "null" string. +/// # query error null value not allowed for object key +/// # select jsonb_object('{{a, 1}, {null, "def"}, {c, 3.5}}' :: text[][]); +/// +/// query error array must have two columns +/// select jsonb_object('{{a, 1, 2}, {b, "def"}, {c, 3.5}}' :: text[][]); +/// ``` +#[function("jsonb_object(varchar[][]) -> jsonb")] +fn jsonb_object_2d(array: ListRef<'_>) -> Result { + let mut builder = Builder::>::new(); + builder.begin_object(); + for kv in array.iter() { + let Some(kv) = kv else { + return Err(ExprError::InvalidParam { + name: "array", + reason: "Unexpected array element.".into(), + }); + }; + let kv = kv.into_list(); + if kv.len() != 2 { + return Err(ExprError::InvalidParam { + name: "array", + reason: "array must have two columns".into(), + }); + } + match kv.get(0).unwrap() { + Some(s) => builder.add_string(s.into_utf8()), + None => { + return Err(ExprError::InvalidParam { + name: "array", + reason: "null value not allowed for object key".into(), + }) + } + } + match kv.get(1).unwrap() { + Some(s) => builder.add_string(s.into_utf8()), + None => builder.add_null(), + } + } + builder.end_object(); + Ok(builder.finish().into()) +} + +/// This form of `jsonb_object` takes keys and values pairwise from separate text arrays. +/// Otherwise it is identical to the one-argument form. +/// +/// # Examples +/// +/// ```slt +/// query T +/// select jsonb_object('{a,b}', '{1,2}'); +/// ---- +/// {"a": "1", "b": "2"} +/// +/// query error mismatched array dimensions +/// select jsonb_object('{a,b}', '{1,2,3}'); +/// +/// # FIXME: `null` should be parsed as a null value instead of a "null" string. +/// # query error null value not allowed for object key +/// # select jsonb_object('{a,null}', '{1,2}'); +/// ``` +#[function("jsonb_object(varchar[], varchar[]) -> jsonb")] +fn jsonb_object_kv(keys: ListRef<'_>, values: ListRef<'_>) -> Result { + if keys.len() != values.len() { + return Err(ExprError::InvalidParam { + name: "values", + reason: "mismatched array dimensions".into(), + }); + } + let mut builder = Builder::>::new(); + builder.begin_object(); + for (key, value) in keys.iter().zip_eq_fast(values.iter()) { + match key { + Some(s) => builder.add_string(s.into_utf8()), + None => { + return Err(ExprError::InvalidParam { + name: "keys", + reason: "null value not allowed for object key".into(), + }) + } + } + match value { + Some(s) => builder.add_string(s.into_utf8()), + None => builder.add_null(), + } + } + builder.end_object(); + Ok(builder.finish().into()) +} diff --git a/src/expr/impl/src/scalar/mod.rs b/src/expr/impl/src/scalar/mod.rs index dd88a374ba966..d9d10e4548aee 100644 --- a/src/expr/impl/src/scalar/mod.rs +++ b/src/expr/impl/src/scalar/mod.rs @@ -45,6 +45,7 @@ mod int256; mod jsonb_access; mod jsonb_concat; mod jsonb_info; +mod jsonb_object; mod length; mod lower; mod md5; diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index cc2519fb4371f..50a621c0879d5 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -877,6 +877,7 @@ impl Binder { ("jsonb_array_element_text", raw_call(ExprType::JsonbAccessStr)), ("jsonb_typeof", raw_call(ExprType::JsonbTypeof)), ("jsonb_array_length", raw_call(ExprType::JsonbArrayLength)), + ("jsonb_object", raw_call(ExprType::JsonbObject)), ("jsonb_pretty", raw_call(ExprType::JsonbPretty)), // Functions that return a constant value ("pi", pi()), diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 42813ca07b354..a3c7abf1ef482 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -177,6 +177,7 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::JsonbAccessStr | expr_node::Type::JsonbTypeof | expr_node::Type::JsonbArrayLength + | expr_node::Type::JsonbObject | expr_node::Type::JsonbPretty | expr_node::Type::IsJson | expr_node::Type::Sind From c6945429a2f538262b0cc024d7780dd5e4dfaacb Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 27 Oct 2023 18:42:10 +0800 Subject: [PATCH 32/52] chore(playground): remove connector service from playground (#13114) --- src/cmd_all/src/common.rs | 6 +---- src/cmd_all/src/playground.rs | 42 ----------------------------------- 2 files changed, 1 insertion(+), 47 deletions(-) diff --git a/src/cmd_all/src/common.rs b/src/cmd_all/src/common.rs index c7df4e5821360..c0e5c10dde5a7 100644 --- a/src/cmd_all/src/common.rs +++ b/src/cmd_all/src/common.rs @@ -24,7 +24,6 @@ pub enum RisingWaveService { Frontend(Vec), #[allow(dead_code)] Compactor(Vec), - ConnectorNode(Vec), } impl RisingWaveService { @@ -34,10 +33,7 @@ impl RisingWaveService { RisingWaveService::Compute(args0) | RisingWaveService::Meta(args0) | RisingWaveService::Frontend(args0) - | RisingWaveService::Compactor(args0) - | RisingWaveService::ConnectorNode(args0) => { - args0.extend(args.iter().map(|s| s.into())) - } + | RisingWaveService::Compactor(args0) => args0.extend(args.iter().map(|s| s.into())), } } } diff --git a/src/cmd_all/src/playground.rs b/src/cmd_all/src/playground.rs index 76ca89be17c76..bb3f3d8726680 100644 --- a/src/cmd_all/src/playground.rs +++ b/src/cmd_all/src/playground.rs @@ -21,8 +21,6 @@ use std::sync::LazyLock; use anyhow::Result; use clap::Parser; use tempfile::TempPath; -use tokio::io::{AsyncBufReadExt, BufReader}; -use tokio::process::Command; use tokio::signal; use crate::common::{osstrs as common_osstrs, RisingWaveService}; @@ -62,7 +60,6 @@ fn get_services(profile: &str) -> (Vec, bool) { ])), RisingWaveService::Compute(osstrs([])), RisingWaveService::Frontend(osstrs([])), - RisingWaveService::ConnectorNode(osstrs([])), ], "playground-3cn" => vec![ RisingWaveService::Meta(osstrs([ @@ -120,7 +117,6 @@ fn get_services(profile: &str) -> (Vec, bool) { "--advertise-addr", "127.0.0.1:4566", ])), - RisingWaveService::ConnectorNode(osstrs([])), ], _ => { tracing::warn!("Unknown playground profile. All components will be started using the default command line options."); @@ -196,44 +192,6 @@ pub async fn playground(opts: PlaygroundOpts) -> Result<()> { let _compactor_handle = tokio::spawn(async move { risingwave_compactor::start(opts).await }); } - // connector node only supports in docker-playground profile - RisingWaveService::ConnectorNode(_) => { - let prefix_bin = match profile.as_str() { - "docker-playground" | "online-docker-playground" => { - "/risingwave/bin".to_string() - } - "playground" => std::env::var("PREFIX_BIN").unwrap_or_default(), - _ => "".to_string(), - }; - let cmd_path = Path::new(&prefix_bin) - .join("connector-node") - .join("start-service.sh"); - if cmd_path.exists() { - tracing::info!("start connector-node with prefix_bin {}", prefix_bin); - let mut child = Command::new(cmd_path) - .arg("-p") - .arg("50051") - .stderr(std::process::Stdio::piped()) - .spawn()?; - let stderr = child.stderr.take().unwrap(); - - let _child_handle = tokio::spawn(async move { - signal::ctrl_c().await.unwrap(); - let _ = child.start_kill(); - }); - let _stderr_handle = tokio::spawn(async move { - let mut reader = BufReader::new(stderr).lines(); - while let Ok(Some(line)) = reader.next_line().await { - tracing::info!(target: "risingwave_connector_node", "{}", line); - } - }); - } else { - tracing::warn!( - "Will not start connector node since `{}` does not exist.", - cmd_path.display() - ); - } - } } } From ae8f1b642cca618316b77a0e4b6b8915e2334162 Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 27 Oct 2023 19:13:23 +0800 Subject: [PATCH 33/52] chore(ci): remove connector start service from ci (#13112) --- ci/scripts/e2e-iceberg-cdc.sh | 29 +---------------------- ci/scripts/e2e-iceberg-sink-test.sh | 4 +--- ci/scripts/e2e-sink-test.sh | 25 +------------------- ci/scripts/e2e-source-test.sh | 36 +---------------------------- 4 files changed, 4 insertions(+), 90 deletions(-) diff --git a/ci/scripts/e2e-iceberg-cdc.sh b/ci/scripts/e2e-iceberg-cdc.sh index 081f5bbd2afcb..1cdc4abf26487 100755 --- a/ci/scripts/e2e-iceberg-cdc.sh +++ b/ci/scripts/e2e-iceberg-cdc.sh @@ -6,7 +6,6 @@ set -euo pipefail source ci/scripts/common.sh # prepare environment -export CONNECTOR_RPC_ENDPOINT="localhost:50051" export CONNECTOR_LIBS_PATH="./connector-node/libs" while getopts 'p:' opt; do @@ -34,36 +33,10 @@ tar xf ./risingwave-connector.tar.gz -C ./connector-node echo "--- e2e, ci-1cn-1fe, iceberg cdc" -node_port=50051 -node_timeout=10 - -wait_for_connector_node_start() { - start_time=$(date +%s) - while : - do - if nc -z localhost $node_port; then - echo "Port $node_port is listened! Connector Node is up!" - break - fi - - current_time=$(date +%s) - elapsed_time=$((current_time - start_time)) - if [ $elapsed_time -ge $node_timeout ]; then - echo "Timeout waiting for port $node_port to be listened!" - exit 1 - fi - sleep 0.1 - done - sleep 2 -} - -echo "--- starting risingwave cluster with connector node" +echo "--- starting risingwave cluster" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ cargo make ci-start ci-1cn-1fe-with-recovery -./connector-node/start-service.sh -p $node_port > .risingwave/log/connector-node.log 2>&1 & -echo "waiting for connector node to start" -wait_for_connector_node_start # prepare minio iceberg sink echo "--- preparing iceberg" diff --git a/ci/scripts/e2e-iceberg-sink-test.sh b/ci/scripts/e2e-iceberg-sink-test.sh index 1a12225ab5435..998ca52e51262 100755 --- a/ci/scripts/e2e-iceberg-sink-test.sh +++ b/ci/scripts/e2e-iceberg-sink-test.sh @@ -31,9 +31,8 @@ buildkite-agent artifact download risingwave-connector.tar.gz ./ mkdir ./connector-node tar xf ./risingwave-connector.tar.gz -C ./connector-node -echo "--- starting risingwave cluster with connector node" +echo "--- starting risingwave cluster" mkdir -p .risingwave/log -./connector-node/start-service.sh -p 50051 > .risingwave/log/connector-sink.log 2>&1 & cargo make ci-start ci-iceberg-test sleep 1 @@ -84,4 +83,3 @@ fi echo "--- Kill cluster" cargo make ci-kill -pkill -f connector-node diff --git a/ci/scripts/e2e-sink-test.sh b/ci/scripts/e2e-sink-test.sh index ce2cc46381eba..0dbdce47abf55 100755 --- a/ci/scripts/e2e-sink-test.sh +++ b/ci/scripts/e2e-sink-test.sh @@ -53,30 +53,8 @@ psql -h db -U postgres -d test -c "CREATE TABLE t4 (v1 int PRIMARY KEY, v2 int); psql -h db -U postgres -d test -c "create table t5 (v1 smallint primary key, v2 int, v3 bigint, v4 float4, v5 float8, v6 decimal, v7 varchar, v8 timestamp, v9 boolean);" psql -h db -U postgres -d test < ./e2e_test/sink/remote/pg_create_table.sql -node_port=50051 -node_timeout=10 - -echo "--- starting risingwave cluster with connector node" +echo "--- starting risingwave cluster" cargo make ci-start ci-1cn-1fe -./connector-node/start-service.sh -p $node_port > .risingwave/log/connector-node.log 2>&1 & - -echo "waiting for connector node to start" -start_time=$(date +%s) -while : -do - if nc -z localhost $node_port; then - echo "Port $node_port is listened! Connector Node is up!" - break - fi - - current_time=$(date +%s) - elapsed_time=$((current_time - start_time)) - if [ $elapsed_time -ge $node_timeout ]; then - echo "Timeout waiting for port $node_port to be listened!" - exit 1 - fi - sleep 0.1 -done echo "--- testing common sinks" sqllogictest -p 4566 -d dev './e2e_test/sink/append_only_sink.slt' @@ -141,7 +119,6 @@ fi echo "--- Kill cluster" cargo make ci-kill -pkill -f connector-node echo "--- e2e, ci-1cn-1fe, nexmark endless" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 2edd099e31187..12ea66f9f1ac4 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -6,7 +6,6 @@ set -euo pipefail source ci/scripts/common.sh # prepare environment -export CONNECTOR_RPC_ENDPOINT="localhost:50051" export CONNECTOR_LIBS_PATH="./connector-node/libs" while getopts 'p:' opt; do @@ -49,36 +48,9 @@ export PGHOST=db PGUSER=postgres PGPASSWORD=postgres PGDATABASE=cdc_test createdb psql < ./e2e_test/source/cdc/postgres_cdc.sql -node_port=50051 -node_timeout=10 - -wait_for_connector_node_start() { - start_time=$(date +%s) - while : - do - if nc -z localhost $node_port; then - echo "Port $node_port is listened! Connector Node is up!" - break - fi - - current_time=$(date +%s) - elapsed_time=$((current_time - start_time)) - if [ $elapsed_time -ge $node_timeout ]; then - echo "Timeout waiting for port $node_port to be listened!" - exit 1 - fi - sleep 0.1 - done - sleep 2 -} - -echo "--- starting risingwave cluster with connector node" +echo "--- starting risingwave cluster" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ cargo make ci-start ci-1cn-1fe-with-recovery -./connector-node/start-service.sh -p $node_port > .risingwave/log/connector-node.log 2>&1 & - -echo "waiting for connector node to start" -wait_for_connector_node_start echo "--- inline cdc test" export MYSQL_HOST=mysql MYSQL_TCP_PORT=3306 MYSQL_PWD=123456 @@ -96,7 +68,6 @@ sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.check.slt' # kill cluster and the connector node cargo make kill -pkill -f connector-node echo "cluster killed " # insert new rows @@ -106,10 +77,6 @@ echo "inserted new rows into mysql and postgres" # start cluster w/o clean-data RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ -touch .risingwave/log/connector-node.log -./connector-node/start-service.sh -p $node_port >> .risingwave/log/connector-node.log 2>&1 & -echo "(recovery) waiting for connector node to start" -wait_for_connector_node_start cargo make dev ci-1cn-1fe-with-recovery echo "wait for cluster recovery finish" @@ -120,7 +87,6 @@ sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.check_new_rows.slt' echo "--- Kill cluster" cargo make ci-kill -pkill -f connector-node echo "--- e2e, ci-1cn-1fe, protobuf schema registry" RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ From bc392aa71467c28ae7a8ddf512966efc508e4a2f Mon Sep 17 00:00:00 2001 From: August Date: Fri, 27 Oct 2023 19:19:19 +0800 Subject: [PATCH 34/52] feat: fix missing column description in row desc for show columns command (#13116) --- e2e_test/ddl/show.slt | 8 ++++---- e2e_test/extended_mode/basic.slt | 8 ++++---- src/frontend/src/utils/infer_stmt_row_desc.rs | 5 +++++ 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/e2e_test/ddl/show.slt b/e2e_test/ddl/show.slt index 9586731207fa7..89fd174f02b4e 100644 --- a/e2e_test/ddl/show.slt +++ b/e2e_test/ddl/show.slt @@ -33,10 +33,10 @@ table description t3 NULL volutpat vitae query TTT show columns from t3; ---- -v1 integer false -v2 integer false -v3 integer false -_row_id serial true +v1 integer false turpis vehicula +v2 integer false Lorem ipsum dolor sit amet +v3 integer false NULL +_row_id serial true consectetur adipiscing elit statement ok create index idx1 on t3 (v1,v2); diff --git a/e2e_test/extended_mode/basic.slt b/e2e_test/extended_mode/basic.slt index 054dae2f6a234..59195d093f1ce 100644 --- a/e2e_test/extended_mode/basic.slt +++ b/e2e_test/extended_mode/basic.slt @@ -53,10 +53,10 @@ table description t3 NULL NULL query TTT show columns from t3; ---- -v1 integer false -v2 integer false -v3 integer false -_row_id serial true +v1 integer false NULL +v2 integer false NULL +v3 integer false NULL +_row_id serial true NULL statement ok drop table t3; diff --git a/src/frontend/src/utils/infer_stmt_row_desc.rs b/src/frontend/src/utils/infer_stmt_row_desc.rs index fdc4b7c460fc2..dbe8968f0a293 100644 --- a/src/frontend/src/utils/infer_stmt_row_desc.rs +++ b/src/frontend/src/utils/infer_stmt_row_desc.rs @@ -35,6 +35,11 @@ pub fn infer_show_object(objects: &ShowObject) -> Vec { DataType::Varchar.to_oid(), DataType::Varchar.type_len(), ), + PgFieldDescriptor::new( + "Description".to_owned(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), ], ShowObject::Connection { .. } => vec![ PgFieldDescriptor::new( From 92470cbe7417b6ef8acfa16a1b7bb89bbea05ff9 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Fri, 27 Oct 2023 19:55:16 +0800 Subject: [PATCH 35/52] test(sink): add recovery test for sink (#12701) --- ci/scripts/deterministic-it-test.sh | 1 + src/stream/src/executor/sink.rs | 4 +- .../tests/integration_tests/sink/basic.rs | 355 ++---------------- .../tests/integration_tests/sink/mod.rs | 30 ++ .../tests/integration_tests/sink/recovery.rs | 101 +++++ .../tests/integration_tests/sink/scale.rs | 121 ++++++ .../tests/integration_tests/sink/utils.rs | 336 +++++++++++++++++ 7 files changed, 628 insertions(+), 320 deletions(-) create mode 100644 src/tests/simulation/tests/integration_tests/sink/recovery.rs create mode 100644 src/tests/simulation/tests/integration_tests/sink/scale.rs create mode 100644 src/tests/simulation/tests/integration_tests/sink/utils.rs diff --git a/ci/scripts/deterministic-it-test.sh b/ci/scripts/deterministic-it-test.sh index f281eaa467bfd..5d76a6677d580 100755 --- a/ci/scripts/deterministic-it-test.sh +++ b/ci/scripts/deterministic-it-test.sh @@ -16,6 +16,7 @@ mv target/ci-sim target/sim echo "--- Run integration tests in deterministic simulation mode" seq $TEST_NUM | parallel MADSIM_TEST_SEED={} NEXTEST_PROFILE=ci-sim \ cargo nextest run \ + --no-capture \ --no-fail-fast \ --cargo-metadata target/nextest/cargo-metadata.json \ --binaries-metadata target/nextest/binaries-metadata.json \ diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index 70e63b4b33cd0..fdc4e95ef5799 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -241,7 +241,7 @@ impl SinkExecutor { .await?; if let Some(vnode_bitmap) = barrier.as_update_vnode_bitmap(actor_context.id) { - log_writer.update_vnode_bitmap(vnode_bitmap); + log_writer.update_vnode_bitmap(vnode_bitmap).await?; } yield Message::Barrier(barrier); } @@ -276,7 +276,7 @@ impl SinkExecutor { .await?; if let Some(vnode_bitmap) = barrier.as_update_vnode_bitmap(actor_context.id) { - log_writer.update_vnode_bitmap(vnode_bitmap); + log_writer.update_vnode_bitmap(vnode_bitmap).await?; } yield Message::Barrier(barrier); } diff --git a/src/tests/simulation/tests/integration_tests/sink/basic.rs b/src/tests/simulation/tests/integration_tests/sink/basic.rs index bceb45a8a2389..24f709139f0fc 100644 --- a/src/tests/simulation/tests/integration_tests/sink/basic.rs +++ b/src/tests/simulation/tests/integration_tests/sink/basic.rs @@ -12,360 +12,79 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::io::Write; -use std::iter::once; -use std::sync::atomic::AtomicUsize; use std::sync::atomic::Ordering::Relaxed; use std::sync::Arc; use std::time::Duration; use anyhow::Result; -use async_trait::async_trait; -use futures::stream::select_all; -use futures::StreamExt; -use itertools::Itertools; -use rand::prelude::SliceRandom; -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::buffer::Bitmap; -use risingwave_common::types::{DataType, ScalarImpl}; -use risingwave_common::util::chunk_coalesce::DataChunkBuilder; -use risingwave_connector::sink::boxed::{BoxCoordinator, BoxWriter}; -use risingwave_connector::sink::test_sink::registry_build_sink; -use risingwave_connector::sink::writer::SinkWriter; -use risingwave_connector::sink::{Sink, SinkWriterParam}; -use risingwave_connector::source::test_source::{registry_test_source, BoxSource, TestSourceSplit}; -use risingwave_connector::source::StreamChunkWithState; -use risingwave_simulation::cluster::{Cluster, ConfigPath, Configuration}; -use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver}; use tokio::time::sleep; -use tokio_stream::wrappers::UnboundedReceiverStream; -struct TestWriter { - row_counter: Arc, - parallelism_counter: Arc, -} - -#[async_trait] -impl SinkWriter for TestWriter { - async fn begin_epoch(&mut self, _epoch: u64) -> risingwave_connector::sink::Result<()> { - Ok(()) - } - - async fn write_batch(&mut self, chunk: StreamChunk) -> risingwave_connector::sink::Result<()> { - let mut count = 0; - for _ in chunk.rows() { - count += 1; - } - self.row_counter.fetch_add(count, Relaxed); - Ok(()) - } - - async fn barrier( - &mut self, - _is_checkpoint: bool, - ) -> risingwave_connector::sink::Result { - sleep(Duration::from_millis(100)).await; - Ok(()) - } -} - -impl Drop for TestWriter { - fn drop(&mut self) { - self.parallelism_counter.fetch_sub(1, Relaxed); - } -} - -fn build_stream_chunk(row_iter: impl Iterator) -> StreamChunk { - let mut builder = DataChunkBuilder::new(vec![DataType::Int32, DataType::Varchar], 100000); - for (id, name) in row_iter { - assert!(builder - .append_one_row([ - Some(ScalarImpl::Int32(id)), - Some(ScalarImpl::Utf8(name.into())), - ]) - .is_none()); - } - let chunk = builder.consume_all().unwrap(); - let ops = (0..chunk.cardinality()).map(|_| Op::Insert).collect_vec(); - StreamChunk::from_parts(ops, chunk) -} - -#[tokio::test] -async fn test_sink_basic() -> Result<()> { - let config_path = { - let mut file = tempfile::NamedTempFile::new().expect("failed to create temp config file"); - file.write_all(include_bytes!("../../../../../config/ci-sim.toml")) - .expect("failed to write config file"); - file.into_temp_path() - }; - - let mut cluster = Cluster::start(Configuration { - config_path: ConfigPath::Temp(config_path.into()), - frontend_nodes: 1, - compute_nodes: 3, - meta_nodes: 1, - compactor_nodes: 1, - compute_node_cores: 2, - etcd_timeout_rate: 0.0, - etcd_data_path: None, - }) - .await?; - - let row_counter = Arc::new(AtomicUsize::new(0)); - let parallelism_counter = Arc::new(AtomicUsize::new(0)); +use crate::sink::utils::{ + start_sink_test_cluster, SimulationTestSink, SimulationTestSource, CREATE_SINK, CREATE_SOURCE, + DROP_SINK, DROP_SOURCE, +}; +use crate::{assert_eq_with_err_returned as assert_eq, assert_with_err_returned as assert}; - let _sink_guard = registry_build_sink({ - let row_counter = row_counter.clone(); - let parallelism_counter = parallelism_counter.clone(); - move |_, _| { - parallelism_counter.fetch_add(1, Relaxed); - Box::new(TestWriter { - row_counter: row_counter.clone(), - parallelism_counter: parallelism_counter.clone(), - }) - } - }); +async fn basic_test_inner(is_decouple: bool) -> Result<()> { + let mut cluster = start_sink_test_cluster().await?; - let source_parallelism = 12; - let mut txs = Vec::new(); - let mut rxs = Vec::new(); - for _ in 0..source_parallelism { - let (tx, rx): (_, UnboundedReceiver) = unbounded_channel(); - txs.push(tx); - rxs.push(Some(rx)); - } - - let _source_guard = registry_test_source(BoxSource::new( - move |_, _| { - Ok((0..source_parallelism) - .map(|i: usize| TestSourceSplit { - id: format!("{}", i).as_str().into(), - properties: Default::default(), - offset: "".to_string(), - }) - .collect_vec()) - }, - move |_, splits, _, _, _| { - select_all(splits.into_iter().map(|split| { - let id: usize = split.id.parse().unwrap(); - let rx = rxs[id].take().unwrap(); - UnboundedReceiverStream::new(rx).map(|chunk| Ok(StreamChunkWithState::from(chunk))) - })) - .boxed() - }, - )); + let test_sink = SimulationTestSink::register_new(); + let test_source = SimulationTestSource::register_new(12, 0..500000, 0.2, 50); let mut session = cluster.start_session(); session.run("set streaming_parallelism = 6").await?; - session.run("set sink_decouple = false").await?; - session - .run("create table test_table (id int primary key, name varchar) with (connector = 'test') FORMAT PLAIN ENCODE JSON") - .await?; - session - .run("create sink test_sink from test_table with (connector = 'test')") - .await?; - let mut count = 0; - let mut id_list: Vec = (0..100000).collect_vec(); - id_list.shuffle(&mut rand::thread_rng()); - let flush_freq = 50; - for id in &id_list[0..10000] { - let chunk = build_stream_chunk(once((*id as i32, format!("name-{}", id)))); - txs[id % source_parallelism].send(chunk).unwrap(); - count += 1; - if count % flush_freq == 0 { - sleep(Duration::from_millis(10)).await; - } + if is_decouple { + session.run("set sink_decouple = true").await?; + } else { + session.run("set sink_decouple = false").await?; } - sleep(Duration::from_millis(10000)).await; + session.run(CREATE_SOURCE).await?; + session.run(CREATE_SINK).await?; + assert_eq!(6, test_sink.parallelism_counter.load(Relaxed)); - assert_eq!(6, parallelism_counter.load(Relaxed)); - assert_eq!(count, row_counter.load(Relaxed)); + test_sink + .store + .wait_for_count(test_source.id_list.len()) + .await?; - session.run("drop sink test_sink").await?; + session.run(DROP_SINK).await?; + session.run(DROP_SOURCE).await?; - assert_eq!(0, parallelism_counter.load(Relaxed)); + assert_eq!(0, test_sink.parallelism_counter.load(Relaxed)); + test_sink.store.check_simple_result(&test_source.id_list)?; + assert!(test_sink.store.inner().checkpoint_count > 0); Ok(()) } #[tokio::test] -async fn test_sink_decouple_basic() -> Result<()> { - let config_path = { - let mut file = tempfile::NamedTempFile::new().expect("failed to create temp config file"); - file.write_all(include_bytes!("../../../../../config/ci-sim.toml")) - .expect("failed to write config file"); - file.into_temp_path() - }; - - let mut cluster = Cluster::start(Configuration { - config_path: ConfigPath::Temp(config_path.into()), - frontend_nodes: 1, - compute_nodes: 3, - meta_nodes: 1, - compactor_nodes: 1, - compute_node_cores: 2, - etcd_timeout_rate: 0.0, - etcd_data_path: None, - }) - .await?; - - let row_counter = Arc::new(AtomicUsize::new(0)); - let parallelism_counter = Arc::new(AtomicUsize::new(0)); - - let _sink_guard = registry_build_sink({ - let row_counter = row_counter.clone(); - let parallelism_counter = parallelism_counter.clone(); - move |_, _| { - parallelism_counter.fetch_add(1, Relaxed); - Box::new(TestWriter { - row_counter: row_counter.clone(), - parallelism_counter: parallelism_counter.clone(), - }) - } - }); - - let source_parallelism = 12; - let mut txs = Vec::new(); - let mut rxs = Vec::new(); - for _ in 0..source_parallelism { - let (tx, rx): (_, UnboundedReceiver) = unbounded_channel(); - txs.push(tx); - rxs.push(Some(rx)); - } - - let _source_guard = registry_test_source(BoxSource::new( - move |_, _| { - Ok((0..source_parallelism) - .map(|i: usize| TestSourceSplit { - id: format!("{}", i).as_str().into(), - properties: Default::default(), - offset: "".to_string(), - }) - .collect_vec()) - }, - move |_, splits, _, _, _| { - select_all(splits.into_iter().map(|split| { - let id: usize = split.id.parse().unwrap(); - let rx = rxs[id].take().unwrap(); - UnboundedReceiverStream::new(rx).map(|chunk| Ok(StreamChunkWithState::from(chunk))) - })) - .boxed() - }, - )); - - let mut session = cluster.start_session(); - - session.run("set streaming_parallelism = 6").await?; - session.run("set sink_decouple = true").await?; - session - .run("create table test_table (id int primary key, name varchar) with (connector = 'test') FORMAT PLAIN ENCODE JSON") - .await?; - session - .run("create sink test_sink from test_table with (connector = 'test')") - .await?; - assert_eq!(6, parallelism_counter.load(Relaxed)); - - let mut count = 0; - let mut id_list = (0..100000).collect_vec(); - id_list.shuffle(&mut rand::thread_rng()); - let flush_freq = 50; - for id in &id_list[0..10000] { - let chunk = build_stream_chunk(once((*id as i32, format!("name-{}", id)))); - txs[id % source_parallelism].send(chunk).unwrap(); - count += 1; - if count % flush_freq == 0 { - sleep(Duration::from_millis(10)).await; - } - } - - while row_counter.load(Relaxed) < count { - sleep(Duration::from_millis(1000)).await - } - - assert_eq!(count, row_counter.load(Relaxed)); - - session.run("drop sink test_sink").await?; - - assert_eq!(0, parallelism_counter.load(Relaxed)); +async fn test_sink_basic() -> Result<()> { + basic_test_inner(false).await +} - Ok(()) +#[tokio::test] +async fn test_sink_decouple_basic() -> Result<()> { + basic_test_inner(true).await } #[tokio::test] async fn test_sink_decouple_blackhole() -> Result<()> { - let config_path = { - let mut file = tempfile::NamedTempFile::new().expect("failed to create temp config file"); - file.write_all(include_bytes!("../../../../../config/ci-sim.toml")) - .expect("failed to write config file"); - file.into_temp_path() - }; - - let mut cluster = Cluster::start(Configuration { - config_path: ConfigPath::Temp(config_path.into()), - frontend_nodes: 1, - compute_nodes: 3, - meta_nodes: 1, - compactor_nodes: 1, - compute_node_cores: 2, - etcd_timeout_rate: 0.0, - etcd_data_path: None, - }) - .await?; - - let source_parallelism = 12; - let mut txs = Vec::new(); - let mut rxs = Vec::new(); - for _ in 0..source_parallelism { - let (tx, rx): (_, UnboundedReceiver) = unbounded_channel(); - txs.push(tx); - rxs.push(Some(rx)); - } + let mut cluster = start_sink_test_cluster().await?; - let _source_guard = registry_test_source(BoxSource::new( - move |_, _| { - Ok((0..source_parallelism) - .map(|i: usize| TestSourceSplit { - id: format!("{}", i).as_str().into(), - properties: Default::default(), - offset: "".to_string(), - }) - .collect_vec()) - }, - move |_, splits, _, _, _| { - select_all(splits.into_iter().map(|split| { - let id: usize = split.id.parse().unwrap(); - let rx = rxs[id].take().unwrap(); - UnboundedReceiverStream::new(rx).map(|chunk| Ok(StreamChunkWithState::from(chunk))) - })) - .boxed() - }, - )); + let test_source = SimulationTestSource::register_new(12, 0..500000, 0.2, 50); let mut session = cluster.start_session(); session.run("set streaming_parallelism = 6").await?; session.run("set sink_decouple = true").await?; + session.run(CREATE_SOURCE).await?; session - .run("create table test_table (id int primary key, name varchar) with (connector = 'test') FORMAT PLAIN ENCODE JSON") - .await?; - session - .run("create sink test_sink from test_table with (connector = 'blackhole')") + .run("create sink test_sink from test_source with (connector = 'blackhole')") .await?; - let mut count = 0; - let mut id_list = (0..100000).collect_vec(); - id_list.shuffle(&mut rand::thread_rng()); - let flush_freq = 50; - for id in &id_list[0..10000] { - let chunk = build_stream_chunk(once((*id as i32, format!("name-{}", id)))); - txs[id % source_parallelism].send(chunk).unwrap(); - count += 1; - if count % flush_freq == 0 { - sleep(Duration::from_millis(10)).await; - } - } + session.run(DROP_SINK).await?; + session.run(DROP_SOURCE).await?; - session.run("drop sink test_sink").await?; Ok(()) } diff --git a/src/tests/simulation/tests/integration_tests/sink/mod.rs b/src/tests/simulation/tests/integration_tests/sink/mod.rs index 71a65bf062d9c..0d0de28b84dc8 100644 --- a/src/tests/simulation/tests/integration_tests/sink/mod.rs +++ b/src/tests/simulation/tests/integration_tests/sink/mod.rs @@ -14,3 +14,33 @@ #[cfg(madsim)] mod basic; +#[cfg(madsim)] +mod recovery; +#[cfg(madsim)] +mod scale; +#[cfg(madsim)] +mod utils; + +#[macro_export] +macro_rules! assert_with_err_returned { + ($condition:expr, $($rest:tt)*) => {{ + if !$condition { + return Err(anyhow::anyhow!($($rest)*).into()); + } + }}; + ($condition:expr) => {{ + if !$condition { + return Err(anyhow::anyhow!("fail assertion {}", stringify! {$condition}).into()); + } + }}; +} + +#[macro_export] +macro_rules! assert_eq_with_err_returned { + ($first:expr, $second:expr $(,$($rest:tt)*)?) => {{ + $crate::assert_with_err_returned ! { + {$first == $second} + $(, $($rest:tt)*)? + } + }}; +} diff --git a/src/tests/simulation/tests/integration_tests/sink/recovery.rs b/src/tests/simulation/tests/integration_tests/sink/recovery.rs new file mode 100644 index 0000000000000..c23ea7fe5fa78 --- /dev/null +++ b/src/tests/simulation/tests/integration_tests/sink/recovery.rs @@ -0,0 +1,101 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::atomic::Ordering::Relaxed; +use std::sync::Arc; +use std::time::Duration; + +use anyhow::Result; +use risingwave_simulation::cluster::{Cluster, KillOpts}; +use tokio::time::sleep; + +use crate::sink::utils::{ + start_sink_test_cluster, SimulationTestSink, SimulationTestSource, CREATE_SINK, CREATE_SOURCE, + DROP_SINK, DROP_SOURCE, +}; +use crate::{assert_eq_with_err_returned as assert_eq, assert_with_err_returned as assert}; + +async fn kill_and_check( + cluster: &mut Cluster, + test_sink: &SimulationTestSink, + target: usize, +) -> anyhow::Result<()> { + let mut prev_count = 0; + sleep(Duration::from_secs(2)).await; + for i in 0..5 { + let curr_count = test_sink.store.id_count(); + if curr_count == target { + assert!(i > 0, "test finish without kill"); + break; + } + assert!( + curr_count > prev_count, + "not make progress between kill. Prev count {}, curr count {}, i {}", + prev_count, + curr_count, + i + ); + prev_count = curr_count; + cluster.kill_node(&KillOpts::ALL).await; + sleep(Duration::from_secs(10)).await; + } + Ok(()) +} + +async fn recovery_test_inner(is_decouple: bool) -> Result<()> { + let mut cluster = start_sink_test_cluster().await?; + + let test_sink = SimulationTestSink::register_new(); + let test_source = SimulationTestSource::register_new(12, 0..500000, 0.2, 50); + + let mut session = cluster.start_session(); + + session.run("set streaming_parallelism = 6").await?; + if is_decouple { + session.run("set sink_decouple = true").await?; + } else { + session.run("set sink_decouple = false").await?; + } + session.run(CREATE_SOURCE).await?; + session.run(CREATE_SINK).await?; + assert_eq!(6, test_sink.parallelism_counter.load(Relaxed)); + + let count = test_source.id_list.len(); + + kill_and_check(&mut cluster, &test_sink, count).await?; + + test_sink.store.wait_for_count(count).await?; + + let mut session = cluster.start_session(); + session.run(DROP_SINK).await?; + session.run(DROP_SOURCE).await?; + + assert_eq!(0, test_sink.parallelism_counter.load(Relaxed)); + assert!(test_sink.store.inner().checkpoint_count > 0); + + test_sink.store.check_simple_result(&test_source.id_list)?; + assert!(test_sink.store.inner().checkpoint_count > 0); + + Ok(()) +} + +#[tokio::test] +async fn test_sink_recovery() -> Result<()> { + recovery_test_inner(false).await +} + +#[tokio::test] +async fn test_sink_decouple_recovery() -> Result<()> { + recovery_test_inner(true).await +} diff --git a/src/tests/simulation/tests/integration_tests/sink/scale.rs b/src/tests/simulation/tests/integration_tests/sink/scale.rs new file mode 100644 index 0000000000000..259678636d535 --- /dev/null +++ b/src/tests/simulation/tests/integration_tests/sink/scale.rs @@ -0,0 +1,121 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::atomic::Ordering::Relaxed; +use std::sync::Arc; +use std::time::Duration; + +use anyhow::Result; +use itertools::Itertools; +use rand::{thread_rng, Rng}; +use risingwave_simulation::cluster::{Cluster, KillOpts}; +use risingwave_simulation::ctl_ext::predicate::identity_contains; +use tokio::time::sleep; + +use crate::sink::utils::{ + start_sink_test_cluster, SimulationTestSink, SimulationTestSource, CREATE_SINK, CREATE_SOURCE, + DROP_SINK, DROP_SOURCE, +}; +use crate::{assert_eq_with_err_returned as assert_eq, assert_with_err_returned as assert}; + +async fn scale_and_check( + cluster: &mut Cluster, + test_sink: &SimulationTestSink, + target_count: usize, + schedule_plan: impl Iterator, +) -> Result<()> { + for (plan, expected_parallelism) in schedule_plan { + let prev_count = test_sink.store.id_count(); + assert!(prev_count <= target_count); + if prev_count == target_count { + return Ok(()); + } + cluster.reschedule(plan).await?; + let after_count = test_sink.store.id_count(); + sleep(Duration::from_secs(10)).await; + if thread_rng().gen_bool(0.5) { + sleep(Duration::from_secs(10)).await; + let before_kill_count = test_sink.store.id_count(); + cluster.kill_node(&KillOpts::ALL).await; + sleep(Duration::from_secs(10)).await; + } + } + Ok(()) +} + +async fn scale_test_inner(is_decouple: bool) -> Result<()> { + let mut cluster = start_sink_test_cluster().await?; + + let test_sink = SimulationTestSink::register_new(); + let test_source = SimulationTestSource::register_new(12, 0..500000, 0.2, 20); + + let mut session = cluster.start_session(); + + session.run("set streaming_parallelism = 6").await?; + if is_decouple { + session.run("set sink_decouple = true").await?; + } else { + session.run("set sink_decouple = false").await?; + } + session.run(CREATE_SOURCE).await?; + session.run(CREATE_SINK).await?; + assert_eq!(6, test_sink.parallelism_counter.load(Relaxed)); + + let mut sink_fragments = cluster + .locate_fragments([identity_contains("Sink")]) + .await?; + + assert_eq!(sink_fragments.len(), 1); + let framgment = sink_fragments.pop().unwrap(); + let id = framgment.id(); + + let count = test_source.id_list.len(); + + scale_and_check( + &mut cluster, + &test_sink, + count, + vec![ + (format!("{id}-[1,2,3]"), 3), + (format!("{id}-[4,5]+[1,2]"), 3), + (format!("{id}+[3,4,5]"), 6), + ] + .into_iter(), + ) + .await?; + + test_sink.store.wait_for_count(count).await?; + + let mut session = cluster.start_session(); + session.run(DROP_SINK).await?; + session.run(DROP_SOURCE).await?; + + assert_eq!(0, test_sink.parallelism_counter.load(Relaxed)); + assert!(test_sink.store.inner().checkpoint_count > 0); + + test_sink.store.check_simple_result(&test_source.id_list)?; + assert!(test_sink.store.inner().checkpoint_count > 0); + + Ok(()) +} + +#[tokio::test] +async fn test_sink_scale() -> Result<()> { + scale_test_inner(false).await +} + +#[tokio::test] +async fn test_sink_decouple_scale() -> Result<()> { + scale_test_inner(true).await +} diff --git a/src/tests/simulation/tests/integration_tests/sink/utils.rs b/src/tests/simulation/tests/integration_tests/sink/utils.rs new file mode 100644 index 0000000000000..965c74b1e972a --- /dev/null +++ b/src/tests/simulation/tests/integration_tests/sink/utils.rs @@ -0,0 +1,336 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::io::Write; +use std::iter::once; +use std::sync::atomic::Ordering::Relaxed; +use std::sync::atomic::{AtomicI64, AtomicUsize}; +use std::sync::{Arc, LazyLock, Mutex, MutexGuard}; +use std::time::{Duration, Instant}; + +use anyhow::Result; +use async_trait::async_trait; +use futures::future::pending; +use futures::stream::{empty, select_all, BoxStream}; +use futures::{stream, FutureExt, StreamExt}; +use itertools::Itertools; +use rand::prelude::SliceRandom; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::row::Row; +use risingwave_common::types::{DataType, ScalarImpl, Serial}; +use risingwave_common::util::chunk_coalesce::DataChunkBuilder; +use risingwave_connector::sink::test_sink::{registry_build_sink, TestSinkRegistryGuard}; +use risingwave_connector::sink::writer::SinkWriter; +use risingwave_connector::source::test_source::{ + registry_test_source, BoxSource, TestSourceRegistryGuard, TestSourceSplit, +}; +use risingwave_connector::source::StreamChunkWithState; +use risingwave_simulation::cluster::{Cluster, ConfigPath, Configuration}; +use tokio::time::sleep; + +use crate::{assert_eq_with_err_returned as assert_eq, assert_with_err_returned as assert}; + +pub const CREATE_SOURCE: &str = "create source test_source (id int, name varchar) with (connector = 'test') FORMAT PLAIN ENCODE JSON"; +pub const CREATE_SINK: &str = "create sink test_sink from test_source with (connector = 'test')"; +pub const DROP_SINK: &str = "drop sink test_sink"; +pub const DROP_SOURCE: &str = "drop source test_source"; + +pub struct TestSinkStoreInner { + pub id_name: HashMap>, + pub epochs: Vec, + pub checkpoint_count: usize, +} + +#[derive(Clone)] +pub struct TestSinkStore { + inner: Arc>, +} + +impl TestSinkStore { + pub fn new() -> Self { + Self { + inner: Arc::new(Mutex::new(TestSinkStoreInner { + id_name: HashMap::new(), + epochs: Vec::new(), + checkpoint_count: 0, + })), + } + } + + pub fn insert(&self, id: i32, name: String) { + self.inner().id_name.entry(id).or_default().push(name); + } + + pub fn begin_epoch(&self, epoch: u64) { + self.inner().epochs.push(epoch) + } + + pub fn inner(&self) -> MutexGuard<'_, TestSinkStoreInner> { + self.inner.lock().unwrap() + } + + pub fn check_simple_result(&self, id_list: &[i32]) -> anyhow::Result<()> { + let inner = self.inner(); + assert_eq!(inner.id_name.len(), id_list.len()); + for id in id_list { + let names = inner.id_name.get(id).unwrap(); + assert!(!names.is_empty()); + for name in names { + assert_eq!(name, &simple_name_of_id(*id)); + } + } + Ok(()) + } + + pub fn id_count(&self) -> usize { + self.inner().id_name.len() + } + + pub async fn wait_for_count(&self, count: usize) -> anyhow::Result<()> { + let mut prev_count = 0; + loop { + sleep(Duration::from_secs(1)).await; + let curr_count = self.id_count(); + if curr_count >= count { + assert_eq!(count, curr_count); + break; + } + assert!( + curr_count > prev_count, + "not making progress: curr {}, prev {}", + curr_count, + prev_count + ); + prev_count = curr_count; + } + Ok(()) + } +} + +pub struct TestWriter { + store: TestSinkStore, + parallelism_counter: Arc, +} + +impl TestWriter { + pub fn new(store: TestSinkStore, parallelism_counter: Arc) -> Self { + Self { + store, + parallelism_counter, + } + } +} + +#[async_trait] +impl SinkWriter for TestWriter { + async fn begin_epoch(&mut self, epoch: u64) -> risingwave_connector::sink::Result<()> { + self.store.begin_epoch(epoch); + Ok(()) + } + + async fn write_batch(&mut self, chunk: StreamChunk) -> risingwave_connector::sink::Result<()> { + for (op, row) in chunk.rows() { + assert_eq!(op, Op::Insert); + assert_eq!(row.len(), 2); + let id = row.datum_at(0).unwrap().into_int32(); + let name = row.datum_at(1).unwrap().into_utf8().to_string(); + self.store.insert(id, name); + } + Ok(()) + } + + async fn barrier( + &mut self, + is_checkpoint: bool, + ) -> risingwave_connector::sink::Result { + if is_checkpoint { + self.store.inner().checkpoint_count += 1; + sleep(Duration::from_millis(100)).await; + } + Ok(()) + } +} + +impl Drop for TestWriter { + fn drop(&mut self) { + self.parallelism_counter.fetch_sub(1, Relaxed); + } +} + +pub fn simple_name_of_id(id: i32) -> String { + format!("name-{}", id) +} + +pub struct SimulationTestSink { + _sink_guard: TestSinkRegistryGuard, + pub store: TestSinkStore, + pub parallelism_counter: Arc, +} + +impl SimulationTestSink { + pub fn register_new() -> Self { + let parallelism_counter = Arc::new(AtomicUsize::new(0)); + let store = TestSinkStore::new(); + + let _sink_guard = registry_build_sink({ + let parallelism_counter = parallelism_counter.clone(); + let store = store.clone(); + move |_, _| { + parallelism_counter.fetch_add(1, Relaxed); + Box::new(TestWriter::new(store.clone(), parallelism_counter.clone())) + } + }); + + Self { + _sink_guard, + parallelism_counter, + store, + } + } +} + +pub fn build_stream_chunk(row_iter: impl Iterator) -> StreamChunk { + static ROW_ID_GEN: LazyLock> = LazyLock::new(|| Arc::new(AtomicI64::new(0))); + + let mut builder = DataChunkBuilder::new( + vec![DataType::Int32, DataType::Varchar, DataType::Serial], + 100000, + ); + for (id, name) in row_iter { + let row_id = ROW_ID_GEN.fetch_add(1, Relaxed); + std::assert!(builder + .append_one_row([ + Some(ScalarImpl::Int32(id)), + Some(ScalarImpl::Utf8(name.into())), + Some(ScalarImpl::Serial(Serial::from(row_id))), + ]) + .is_none()); + } + let chunk = builder.consume_all().unwrap(); + let ops = (0..chunk.cardinality()).map(|_| Op::Insert).collect_vec(); + StreamChunk::from_parts(ops, chunk) +} + +pub struct SimulationTestSource { + _source_guard: TestSourceRegistryGuard, + pub id_list: Vec, +} + +impl SimulationTestSource { + pub fn register_new( + source_parallelism: usize, + id_list: impl Iterator, + sample_rate: f32, + pause_interval: usize, + ) -> Self { + let mut id_list: Vec = id_list.collect_vec(); + let count = (id_list.len() as f32 * sample_rate) as usize; + id_list.shuffle(&mut rand::thread_rng()); + let id_list = id_list[0..count].iter().cloned().collect_vec(); + let mut id_lists = vec![vec![]; source_parallelism]; + for id in &id_list { + id_lists[*id as usize % source_parallelism].push(*id); + } + let id_lists_clone = id_lists.iter().map(|l| Arc::new(l.clone())).collect_vec(); + let _source_guard = registry_test_source(BoxSource::new( + move |_, _| { + Ok((0..source_parallelism) + .map(|i: usize| TestSourceSplit { + id: format!("{}", i).as_str().into(), + properties: Default::default(), + offset: "".to_string(), + }) + .collect_vec()) + }, + move |_, splits, _, _, _| { + select_all(splits.into_iter().map(|split| { + let split_id: usize = split.id.parse().unwrap(); + let id_list = id_lists_clone[split_id].clone(); + let mut offset = if split.offset == "" { + 0 + } else { + split.offset.parse::().unwrap() + 1 + }; + + let mut stream: BoxStream<'static, StreamChunkWithState> = empty().boxed(); + + while offset < id_list.len() { + let mut chunks = Vec::new(); + while offset < id_list.len() && chunks.len() < pause_interval { + let id = id_list[offset]; + let chunk = build_stream_chunk(once((id, simple_name_of_id(id)))); + let mut split_offset = HashMap::new(); + split_offset.insert(split.id.clone(), offset.to_string()); + let chunk_with_state = StreamChunkWithState { + chunk, + split_offset_mapping: Some(split_offset), + }; + chunks.push(chunk_with_state); + + offset += 1; + } + + stream = stream + .chain( + async move { stream::iter(chunks) } + .into_stream() + .chain( + async move { + sleep(Duration::from_millis(100)).await; + stream::iter(Vec::new()) + } + .into_stream(), + ) + .flatten(), + ) + .boxed(); + } + + stream + .chain(async { pending::().await }.into_stream()) + .map(|chunk| Ok(chunk)) + .boxed() + })) + .boxed() + }, + )); + + Self { + _source_guard, + id_list, + } + } +} + +pub async fn start_sink_test_cluster() -> Result { + let config_path = { + let mut file = tempfile::NamedTempFile::new().expect("failed to create temp config file"); + file.write_all(include_bytes!("../../../../../config/ci-sim.toml")) + .expect("failed to write config file"); + file.into_temp_path() + }; + + Cluster::start(Configuration { + config_path: ConfigPath::Temp(config_path.into()), + frontend_nodes: 1, + compute_nodes: 3, + meta_nodes: 1, + compactor_nodes: 1, + compute_node_cores: 2, + etcd_timeout_rate: 0.0, + etcd_data_path: None, + }) + .await +} From 8cbe568167504f827cf5d926d98198f681bf0f60 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Sun, 29 Oct 2023 23:51:32 +0800 Subject: [PATCH 36/52] fix(expr): include data type in parse error message (#13121) --- src/expr/impl/src/scalar/cast.rs | 14 +++++++++----- .../tests/testdata/output/range_scan.yaml | 6 +++--- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/src/expr/impl/src/scalar/cast.rs b/src/expr/impl/src/scalar/cast.rs index f22b643bb9a09..ae2ed50472b86 100644 --- a/src/expr/impl/src/scalar/cast.rs +++ b/src/expr/impl/src/scalar/cast.rs @@ -37,14 +37,14 @@ use risingwave_pb::expr::expr_node::PbType; #[function("cast(varchar) -> timestamp")] #[function("cast(varchar) -> interval")] #[function("cast(varchar) -> jsonb")] -pub fn str_parse(elem: &str) -> Result +pub fn str_parse(elem: &str, ctx: &Context) -> Result where T: FromStr, ::Err: std::fmt::Display, { - elem.trim() - .parse() - .map_err(|err: ::Err| ExprError::Parse(err.to_string().into())) + elem.trim().parse().map_err(|err: ::Err| { + ExprError::Parse(format!("{} {}", ctx.return_type, err).into()) + }) } // TODO: introduce `FromBinary` and support all types @@ -521,7 +521,11 @@ mod tests { async fn test_unary() { test_unary_bool::(|x| !x, PbType::Not).await; test_unary_date::(|x| try_cast(x).unwrap(), PbType::Cast).await; - test_str_to_int16::(|x| str_parse(x).unwrap()).await; + let ctx_str_to_int16 = Context { + arg_types: vec![DataType::Varchar], + return_type: DataType::Int16, + }; + test_str_to_int16::(|x| str_parse(x, &ctx_str_to_int16).unwrap()).await; } #[tokio::test] diff --git a/src/frontend/planner_test/tests/testdata/output/range_scan.yaml b/src/frontend/planner_test/tests/testdata/output/range_scan.yaml index d16b98f101d3b..f95075c8823a8 100644 --- a/src/frontend/planner_test/tests/testdata/output/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/output/range_scan.yaml @@ -42,12 +42,12 @@ - create_table_and_mv sql: | SELECT * FROM orders_count_by_user WHERE user_id = 'a' - batch_error: 'Expr error: Parse error: invalid digit found in string' + batch_error: 'Expr error: Parse error: bigint invalid digit found in string' - before: - create_table_and_mv sql: | SELECT * FROM orders_count_by_user WHERE user_id > 'a' - batch_error: 'Expr error: Parse error: invalid digit found in string' + batch_error: 'Expr error: Parse error: bigint invalid digit found in string' - before: - create_table_and_mv sql: | @@ -129,7 +129,7 @@ - create_table_and_mv sql: | SELECT * FROM orders_count_by_user WHERE user_id in ('42', '43.0') - batch_error: 'Expr error: Parse error: invalid digit found in string' + batch_error: 'Expr error: Parse error: bigint invalid digit found in string' - before: - create_table_and_mv sql: | From b0166944d335f16d7ffd5294fea706a3db6ce2b0 Mon Sep 17 00:00:00 2001 From: Yingjun Wu Date: Sun, 29 Oct 2023 18:27:25 -0700 Subject: [PATCH 37/52] chore: Update README.md (#13120) Co-authored-by: hengm3467 <100685635+hengm3467@users.noreply.github.com> --- README.md | 60 ++++++++++++++++++++++++++++++------------------------- 1 file changed, 33 insertions(+), 27 deletions(-) diff --git a/README.md b/README.md index 29a7d7e51888a..091bccc0de9b4 100644 --- a/README.md +++ b/README.md @@ -54,52 +54,62 @@ -RisingWave is a distributed SQL streaming database that enables simple, efficient, and reliable processing of streaming data. +RisingWave is a distributed SQL streaming database that enables cost-efficient and reliable processing of streaming data. ![RisingWave](https://github.com/risingwavelabs/risingwave-docs/blob/0f7e1302b22493ba3c1c48e78810750ce9a5ff42/docs/images/archi_simple.png) -## How to install -**Ubuntu** -``` -wget https://github.com/risingwavelabs/risingwave/releases/download/v1.3.0/risingwave-v1.3.0-x86_64-unknown-linux.tar.gz -tar xvf risingwave-v1.3.0-x86_64-unknown-linux.tar.gz -./risingwave playground -``` +## Try it out in 5 minutes **Mac** ``` brew tap risingwavelabs/risingwave brew install risingwave risingwave playground ``` +**Ubuntu** +``` +wget https://github.com/risingwavelabs/risingwave/releases/download/v1.3.0/risingwave-v1.3.0-x86_64-unknown-linux.tar.gz +tar xvf risingwave-v1.3.0-x86_64-unknown-linux.tar.gz +./risingwave playground +``` Now connect to RisingWave using `psql`: ``` psql -h localhost -p 4566 -d dev -U root ``` - Learn more at [Quick Start](https://docs.risingwave.com/docs/current/get-started/). +## Production deployments +For **single-node Docker deployments**, please refer to [Docker Compose](https://docs.risingwave.com/docs/current/risingwave-trial/?method=docker-compose). + +For **Kubernetes deployments**, please refer to [Kubernetes with Helm](https://docs.risingwave.com/docs/current/risingwave-k8s-helm/) or [Kubernetes with Operator](https://docs.risingwave.com/docs/current/risingwave-kubernetes/). + +**RisingWave Cloud** the easiest way to run a fully-fledged RisingWave cluster. Try it out for free at: [cloud.risingwave.com](https://cloud.risingwave.com). + + ## Why RisingWave for stream processing? -RisingWave adaptly tackles some of the most challenging problems in stream processing. Compared to existing stream processing systems, RisingWave shines through with the following key features: -* **Easy to learn** +RisingWave adaptly addresses some of the most challenging problems in stream processing. Compared to existing stream processing systems like [Apache Flink](https://flink.apache.org/), [Apache Spark Streaming](https://spark.apache.org/docs/latest/streaming-programming-guide.html), and [KsqlDB](https://ksqldb.io/), RisingWave stands out in two primary dimensions: **Ease-of-use** and **efficiency**, thanks to its **[PostgreSQL](https://www.postgresql.org/)-style interaction experience** and **[Snowflake](https://snowflake.com/)-like architectural design** (i.e., compute-storage decoupling). +### Ease-of-use +* **Simple to learn** * RisingWave speaks PostgreSQL-style SQL, enabling users to dive into stream processing in much the same way as operating a PostgreSQL database. -* **Highly efficient in multi-stream joins** - * RisingWave has made significant optimizations for multiple stream join scenarios. Users can easily join 10-20 streams (or more) efficiently in a production environment. +* **Simple to verify correctness** + * RisingWave persists results in materialized views and allow users to break down complex stream computation programs into stacked materialized views, simplifying program development and result verification. +* **Simple to maintain and operate** + * RisingWave abstracts away unnecessary low-level details, allowing users to concentrate solely on SQL code-level issues. +* **Simple to integrate** + * With integrations to a diverse range of cloud systems and the PostgreSQL ecosystem, RisingWave boasts a rich and expansive ecosystem, making it straightforward to incorporate into existing infrastructures. + +### Efficiency * **High resource utilization** * Queries in RisingWave leverage shared computational resources, eliminating the need for users to manually allocate resources for each query. * **No compromise on large state management** * The decoupled compute-storage architecture of RisingWave ensures remote persistence of internal states, and users never need to worry about the size of internal states when handling complex queries. +* **Highly efficient in multi-stream joins** + * RisingWave has made significant optimizations for multiple stream join scenarios. Users can easily join 10-20 streams (or more) efficiently in a production environment. * **Transparent dynamic scaling** * RisingWave supports near-instantaneous dynamic scaling without any service interruptions. * **Instant failure recovery** * RisingWave's state management mechanism allows it to recover from failure in seconds, not minutes or hours. -* **Easy to verify correctness** - * RisingWave persists results in materialized views and allow users to break down complex stream computation programs into stacked materialized views, simplifying program development and result verification. * **Simplified data stack** * RisingWave's ability to store data and serve queries eliminates the need for separate maintenance of stream processors and databases. Users can effortlessly connect RisingWave to their preferred BI tools or through client libraries. -* **Simple to maintain and operate** - * RisingWave abstracts away unnecessary low-level details, allowing users to concentrate solely on SQL code-level issues. -* **Rich ecosystem** - * With integrations to a diverse range of cloud systems and the PostgreSQL ecosystem, RisingWave boasts a rich and expansive ecosystem. ## RisingWave's limitations RisingWave isn’t a panacea for all data engineering hurdles. It has its own set of limitations: @@ -111,14 +121,6 @@ RisingWave isn’t a panacea for all data engineering hurdles. It has its own se * RisingWave's row store design is tailored for optimal stream processing performance rather than interactive analytical workloads. Hence, it's not a suitable replacement for OLAP databases. Yet, a reliable integration with many OLAP databases exists, and a collaborative use of RisingWave and OLAP databases is a common practice among many users. -## RisingWave Cloud - -RisingWave Cloud is a fully-managed and scalable stream processing platform powered by the open-source RisingWave project. Try it out for free at: [cloud.risingwave.com](https://cloud.risingwave.com). - -## Notes on telemetry - -RisingWave collects anonymous usage statistics to better understand how the community is using RisingWave. The sole intention of this exercise is to help improve the product. Users may opt out easily at any time. Please refer to the [user documentation](https://docs.risingwave.com/docs/current/telemetry/) for more details. - ## In-production use cases Like other stream processing systems, the primary use cases of RisingWave include monitoring, alerting, real-time dashboard reporting, streaming ETL (Extract, Transform, Load), machine learning feature engineering, and more. It has already been adopted in fields such as financial trading, manufacturing, new media, logistics, gaming, and more. Check out [customer stories](https://www.risingwave.com/use-cases/). @@ -126,6 +128,10 @@ Like other stream processing systems, the primary use cases of RisingWave includ Looking for help, discussions, collaboration opportunities, or a casual afternoon chat with our fellow engineers and community members? Join our [Slack workspace](https://risingwave.com/slack)! +## Notes on telemetry + +RisingWave collects anonymous usage statistics to better understand how the community is using RisingWave. The sole intention of this exercise is to help improve the product. Users may opt out easily at any time. Please refer to the [user documentation](https://docs.risingwave.com/docs/current/telemetry/) for more details. + ## License RisingWave is distributed under the Apache License (Version 2.0). Please refer to [LICENSE](LICENSE) for more information. From 4b7d19453152de7d474c5b5cbc336f5a282f0034 Mon Sep 17 00:00:00 2001 From: Renjie Liu Date: Mon, 30 Oct 2023 10:53:51 +0800 Subject: [PATCH 38/52] fix: iceberg.enabled in pulsar connector should be optional (#13124) --- src/connector/src/source/pulsar/mod.rs | 4 ++-- src/connector/src/source/pulsar/source/reader.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/connector/src/source/pulsar/mod.rs b/src/connector/src/source/pulsar/mod.rs index c9db87fd3bb94..5a1d73963bd23 100644 --- a/src/connector/src/source/pulsar/mod.rs +++ b/src/connector/src/source/pulsar/mod.rs @@ -49,8 +49,8 @@ pub struct PulsarProperties { pub common: PulsarCommon, #[serde(rename = "iceberg.enabled")] - #[serde_as(as = "Option")] - pub iceberg_loader_enabled: bool, + #[serde_as(as = "DisplayFromStr")] + pub iceberg_loader_enabled: Option, #[serde(rename = "iceberg.bucket", default)] pub iceberg_bucket: Option, diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 0568a8935932e..04fee26f42aa7 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -65,7 +65,7 @@ impl SplitReader for PulsarSplitReader { tracing::debug!("creating consumer for pulsar split topic {}", topic,); - if props.iceberg_loader_enabled + if props.iceberg_loader_enabled.unwrap_or(false) && matches!(split.start_offset, PulsarEnumeratorOffset::Earliest) && !topic.starts_with("non-persistent://") { From 8b47f9f2b59c00c8e9bf305ee5a60042236528ba Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Mon, 30 Oct 2023 11:10:33 +0800 Subject: [PATCH 39/52] refactor(object store): bump OpenDAL to v0.41 (#13101) --- Cargo.lock | 24 ++- src/object_store/Cargo.toml | 2 +- src/object_store/src/object/mod.rs | 7 +- .../src/object/opendal_engine/gcs.rs | 8 - .../opendal_engine/opendal_object_store.rs | 150 ++++++------------ 5 files changed, 64 insertions(+), 127 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0b7d0573bf22a..498002620379d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5269,9 +5269,9 @@ checksum = "0ab1bc2a289d34bd04a330323ac98a1b4bc82c9d9fcb1e66b63caa84da26b575" [[package]] name = "opendal" -version = "0.39.0" +version = "0.40.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1ad95e460e5976ab1b74f398ab856c59f8417b3dd32202329e3491dcbe3a6b84" +checksum = "ddba7299bab261d3ae2f37617fb7f45b19ed872752bb4e22cf93a69d979366c5" dependencies = [ "anyhow", "async-compat", @@ -5290,6 +5290,7 @@ dependencies = [ "parking_lot 0.12.1", "percent-encoding", "pin-project", + "prometheus", "quick-xml 0.29.0", "reqsign", "reqwest", @@ -5302,9 +5303,9 @@ dependencies = [ [[package]] name = "opendal" -version = "0.40.0" +version = "0.41.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ddba7299bab261d3ae2f37617fb7f45b19ed872752bb4e22cf93a69d979366c5" +checksum = "e31b48f0af6de5b3b344c1acc1e06c4581dca3e13cd5ba05269927fc2abf953a" dependencies = [ "anyhow", "async-compat", @@ -5323,8 +5324,7 @@ dependencies = [ "parking_lot 0.12.1", "percent-encoding", "pin-project", - "prometheus", - "quick-xml 0.29.0", + "quick-xml 0.30.0", "reqsign", "reqwest", "serde", @@ -6564,6 +6564,16 @@ dependencies = [ "serde", ] +[[package]] +name = "quick-xml" +version = "0.30.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "eff6510e86862b57b210fd8cbe8ed3f0d7d600b9c2863cd4549a2e033c66e956" +dependencies = [ + "memchr", + "serde", +] + [[package]] name = "quote" version = "1.0.33" @@ -7841,7 +7851,7 @@ dependencies = [ "itertools 0.11.0", "madsim-aws-sdk-s3", "madsim-tokio", - "opendal 0.39.0", + "opendal 0.41.0", "prometheus", "risingwave_common", "spin 0.9.8", diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index f117c272a9afc..2459bf83b5af6 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -24,7 +24,7 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] } hyper = "0.14" hyper-tls = "0.5.0" itertools = "0.11" -opendal = "0.39" +opendal = "0.41" prometheus = { version = "0.13", features = ["process"] } risingwave_common = { workspace = true } spin = "0.9" diff --git a/src/object_store/src/object/mod.rs b/src/object_store/src/object/mod.rs index 96e58397dfa82..fe80794756246 100644 --- a/src/object_store/src/object/mod.rs +++ b/src/object_store/src/object/mod.rs @@ -242,12 +242,7 @@ impl ObjectStoreImpl { match self { ObjectStoreImpl::InMem(_) => true, ObjectStoreImpl::Opendal(store) => { - store - .inner - .op - .info() - .capability() - .write_without_content_length + store.inner.op.info().native_capability().write_can_multi } ObjectStoreImpl::S3(_) => true, } diff --git a/src/object_store/src/object/opendal_engine/gcs.rs b/src/object_store/src/object/opendal_engine/gcs.rs index bb6ef8eee0446..b497f85c263a5 100644 --- a/src/object_store/src/object/opendal_engine/gcs.rs +++ b/src/object_store/src/object/opendal_engine/gcs.rs @@ -19,12 +19,6 @@ use opendal::Operator; use super::{EngineType, OpendalObjectStore}; use crate::object::ObjectResult; -/// The fixed number of bytes that is buffered before they are uploaded as a part, will be used in -/// streaing upload. -/// -/// Reference: -const GCS_PART_SIZE: usize = 16 * 1024 * 1024; - impl OpendalObjectStore { /// create opendal gcs engine. pub fn new_gcs_engine(bucket: String, root: String) -> ObjectResult { @@ -35,8 +29,6 @@ impl OpendalObjectStore { builder.root(&root); - builder.write_fixed_size(GCS_PART_SIZE); - // if credential env is set, use it. Otherwise, ADC will be used. let cred = std::env::var("GOOGLE_APPLICATION_CREDENTIALS"); if let Ok(cred) = cred { diff --git a/src/object_store/src/object/opendal_engine/opendal_object_store.rs b/src/object_store/src/object/opendal_engine/opendal_object_store.rs index b829dbd544abf..ff682946b0651 100644 --- a/src/object_store/src/object/opendal_engine/opendal_object_store.rs +++ b/src/object_store/src/object/opendal_engine/opendal_object_store.rs @@ -12,15 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::pin::Pin; -use std::task::{ready, Context, Poll}; - use bytes::Bytes; use fail::fail_point; -use futures::future::BoxFuture; -use futures::{FutureExt, Stream, StreamExt}; +use futures::{stream, StreamExt}; use opendal::services::Memory; -use opendal::{Entry, Error, Lister, Metakey, Operator, Writer}; +use opendal::{Metakey, Operator, Writer}; use risingwave_common::range::RangeBoundsExt; use tokio::io::AsyncRead; @@ -84,7 +80,10 @@ impl ObjectStore for OpendalObjectStore { let data = if range.is_full() { self.op.read(path).await? } else { - self.op.range_read(path, range.map(|v| *v as u64)).await? + self.op + .read_with(path) + .range(range.map(|v| *v as u64)) + .await? }; if let Some(len) = range.len() && len != data.len() { @@ -112,7 +111,12 @@ impl ObjectStore for OpendalObjectStore { ObjectError::internal("opendal streaming read error") )); let reader = match start_pos { - Some(start_position) => self.op.range_reader(path, start_position as u64..).await?, + Some(start_position) => { + self.op + .reader_with(path) + .range(start_position as u64..) + .await? + } None => self.op.reader(path).await?, }; @@ -149,8 +153,36 @@ impl ObjectStore for OpendalObjectStore { } async fn list(&self, prefix: &str) -> ObjectResult { - let lister = self.op.scan(prefix).await?; - Ok(Box::pin(OpenDalObjectIter::new(lister, self.op.clone()))) + let object_lister = self + .op + .lister_with(prefix) + .delimiter("") + .metakey(Metakey::ContentLength | Metakey::ContentType) + .await?; + + let stream = stream::unfold(object_lister, |mut object_lister| async move { + match object_lister.next().await { + Some(Ok(object)) => { + let key = object.path().to_string(); + let om = object.metadata(); + let last_modified = match om.last_modified() { + Some(t) => t.timestamp() as f64, + None => 0_f64, + }; + let total_size = om.content_length() as usize; + let metadata = ObjectMetadata { + key, + last_modified, + total_size, + }; + Some((Ok(metadata), object_lister)) + } + Some(Err(err)) => Some((Err(err.into()), object_lister)), + None => None, + } + }); + + Ok(stream.boxed()) } fn store_media_type(&self) -> &'static str { @@ -172,12 +204,12 @@ pub struct OpenDalStreamingUploader { } impl OpenDalStreamingUploader { pub async fn new(op: Operator, path: String) -> ObjectResult { - let writer = op.writer(&path).await?; + let writer = op.writer_with(&path).buffer(OPENDAL_BUFFER_SIZE).await?; Ok(Self { writer }) } } -const OPENDAL_BUFFER_SIZE: u64 = 8 * 1024 * 1024; +const OPENDAL_BUFFER_SIZE: usize = 16 * 1024 * 1024; #[async_trait::async_trait] impl StreamingUploader for OpenDalStreamingUploader { @@ -199,99 +231,7 @@ impl StreamingUploader for OpenDalStreamingUploader { } fn get_memory_usage(&self) -> u64 { - OPENDAL_BUFFER_SIZE - } -} - -struct OpenDalObjectIter { - lister: Option, - op: Option, - #[allow(clippy::type_complexity)] - next_future: Option>, Lister)>>, - #[allow(clippy::type_complexity)] - metadata_future: Option, Operator)>>, -} - -impl OpenDalObjectIter { - fn new(lister: Lister, op: Operator) -> Self { - Self { - lister: Some(lister), - op: Some(op), - next_future: None, - metadata_future: None, - } - } -} - -impl Stream for OpenDalObjectIter { - type Item = ObjectResult; - - fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll> { - if let Some(metadata_future) = self.metadata_future.as_mut() { - let (result, op) = ready!(metadata_future.poll_unpin(cx)); - self.op = Some(op); - return match result { - Ok(m) => { - self.metadata_future = None; - Poll::Ready(Some(Ok(m))) - } - Err(e) => { - self.metadata_future = None; - Poll::Ready(Some(Err(e.into()))) - } - }; - } - if let Some(next_future) = self.next_future.as_mut() { - let (option, lister) = ready!(next_future.poll_unpin(cx)); - self.lister = Some(lister); - return match option { - None => { - self.next_future = None; - Poll::Ready(None) - } - Some(result) => { - self.next_future = None; - match result { - Ok(object) => { - let op = self.op.take().expect("op should not be None"); - let f = async move { - let key = object.path().to_string(); - // FIXME: How does opendal metadata cache work? - // Will below line result in one IO per object? - let om = match op - .metadata( - &object, - Metakey::LastModified | Metakey::ContentLength, - ) - .await - { - Ok(om) => om, - Err(e) => return (Err(e), op), - }; - let last_modified = match om.last_modified() { - Some(t) => t.timestamp() as f64, - None => 0_f64, - }; - let total_size = om.content_length() as usize; - let metadata = ObjectMetadata { - key, - last_modified, - total_size, - }; - (Ok(metadata), op) - }; - self.metadata_future = Some(Box::pin(f)); - self.poll_next(cx) - } - Err(e) => Poll::Ready(Some(Err(e.into()))), - } - } - }; - } - let mut lister = self.lister.take().expect("list should not be None"); - let f = async move { (lister.next().await, lister) }; - self.next_future = Some(Box::pin(f)); - self.poll_next(cx) + OPENDAL_BUFFER_SIZE as u64 } } From af6b4c6a4f7a6354a607088c2bcdf5d51473b18a Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Mon, 30 Oct 2023 11:17:04 +0800 Subject: [PATCH 40/52] chore(stream): change `FlowControlExecutor` to print WARN logs in `madsim` (#13125) --- src/stream/src/executor/flow_control.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/stream/src/executor/flow_control.rs b/src/stream/src/executor/flow_control.rs index 230359109fca0..320d93b9810f1 100644 --- a/src/stream/src/executor/flow_control.rs +++ b/src/stream/src/executor/flow_control.rs @@ -39,7 +39,7 @@ impl FlowControlExecutor { #[allow(clippy::too_many_arguments)] pub fn new(input: Box, rate_limit: Option) -> Self { #[cfg(madsim)] - println!("FlowControlExecutor rate limiter is disabled in madsim as it will spawn system threads"); + tracing::warn!("FlowControlExecutor rate limiter is disabled in madsim as it will spawn system threads"); Self { input, rate_limit } } From 9a5e2b886e823e038edd6f42a632112236d5b8d1 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Mon, 30 Oct 2023 11:28:50 +0800 Subject: [PATCH 41/52] feat(parser): support "jsonb" as typed string (#13087) Signed-off-by: Runji Wang --- e2e_test/batch/types/jsonb.slt.part | 6 ++++++ src/frontend/src/binder/expr/mod.rs | 2 +- src/frontend/src/binder/select.rs | 1 + src/sqlparser/src/ast/data_type.rs | 3 +++ src/sqlparser/src/parser.rs | 7 ++++++- 5 files changed, 17 insertions(+), 2 deletions(-) diff --git a/e2e_test/batch/types/jsonb.slt.part b/e2e_test/batch/types/jsonb.slt.part index fc2f60d48db35..9812f254ff0cc 100644 --- a/e2e_test/batch/types/jsonb.slt.part +++ b/e2e_test/batch/types/jsonb.slt.part @@ -33,6 +33,12 @@ null true NULL +# typed string +query TT +select jsonb 'true', JsonB '{}'; +---- +true {} + query T select 'true'::jsonb::bool; ---- diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 221056f3a4822..4433605b03a54 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -642,7 +642,6 @@ pub fn bind_data_type(data_type: &AstDataType) -> Result { "float4" => DataType::Float32, "float8" => DataType::Float64, "timestamptz" => DataType::Timestamptz, - "jsonb" => DataType::Jsonb, "serial" => { return Err(ErrorCode::NotSupported( "Column type SERIAL is not supported".into(), @@ -654,6 +653,7 @@ pub fn bind_data_type(data_type: &AstDataType) -> Result { } } AstDataType::Bytea => DataType::Bytea, + AstDataType::Jsonb => DataType::Jsonb, AstDataType::Regclass | AstDataType::Regproc | AstDataType::Uuid diff --git a/src/frontend/src/binder/select.rs b/src/frontend/src/binder/select.rs index 48c4290ee7e05..ac1a53e75f63f 100644 --- a/src/frontend/src/binder/select.rs +++ b/src/frontend/src/binder/select.rs @@ -905,6 +905,7 @@ fn data_type_to_alias(data_type: &AstDataType) -> Option { AstDataType::Regproc => "regproc".to_string(), AstDataType::Text => "text".to_string(), AstDataType::Bytea => "bytea".to_string(), + AstDataType::Jsonb => "jsonb".to_string(), AstDataType::Array(ty) => return data_type_to_alias(ty), AstDataType::Custom(ty) => format!("{}", ty), AstDataType::Struct(_) => { diff --git a/src/sqlparser/src/ast/data_type.rs b/src/sqlparser/src/ast/data_type.rs index e8ad404d4d7d6..1e588955f093b 100644 --- a/src/sqlparser/src/ast/data_type.rs +++ b/src/sqlparser/src/ast/data_type.rs @@ -62,6 +62,8 @@ pub enum DataType { Text, /// Bytea Bytea, + /// JSONB + Jsonb, /// Custom type such as enums Custom(ObjectName), /// Arrays @@ -102,6 +104,7 @@ impl fmt::Display for DataType { DataType::Regproc => write!(f, "REGPROC"), DataType::Text => write!(f, "TEXT"), DataType::Bytea => write!(f, "BYTEA"), + DataType::Jsonb => write!(f, "JSONB"), DataType::Array(ty) => write!(f, "{}[]", ty), DataType::Custom(ty) => write!(f, "{}", ty), DataType::Struct(defs) => { diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 5cc094a204268..d87488ac30648 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -3254,7 +3254,12 @@ impl Parser { _ => { self.prev_token(); let type_name = self.parse_object_name()?; - Ok(DataType::Custom(type_name)) + // JSONB is not a keyword + if type_name.to_string().eq_ignore_ascii_case("jsonb") { + Ok(DataType::Jsonb) + } else { + Ok(DataType::Custom(type_name)) + } } }, unexpected => { From 7db34d516356d11ed2562e798cbb28f66108f715 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Mon, 30 Oct 2023 12:32:01 +0800 Subject: [PATCH 42/52] feat(expr): support jsonb `@>`, `<@`, `?`, `?|` and `?&` operator (#13056) Signed-off-by: Runji Wang --- proto/expr.proto | 10 + src/expr/impl/src/scalar/jsonb_contains.rs | 273 +++++++++++++++++++++ src/expr/impl/src/scalar/mod.rs | 1 + src/frontend/src/binder/expr/binary_op.rs | 5 + src/frontend/src/binder/expr/function.rs | 5 + src/frontend/src/expr/pure.rs | 5 + src/sqlparser/src/ast/operator.rs | 10 + src/sqlparser/src/parser.rs | 12 +- src/sqlparser/src/tokenizer.rs | 34 ++- src/tests/regress/data/sql/jsonb.sql | 206 ++++++++-------- 10 files changed, 456 insertions(+), 105 deletions(-) create mode 100644 src/expr/impl/src/scalar/jsonb_contains.rs diff --git a/proto/expr.proto b/proto/expr.proto index 7998f2fe8128a..cab83e0ea45ce 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -222,6 +222,16 @@ message ExprNode { JSONB_CAT = 605; JSONB_OBJECT = 606; JSONB_PRETTY = 607; + // jsonb @> jsonb + JSONB_CONTAINS = 608; + // jsonb <@ jsonb + JSONB_CONTAINED = 609; + // jsonb ? text + JSONB_EXISTS = 610; + // jsonb ?| text[] + JSONB_EXISTS_ANY = 611; + // jsonb ?& text[] + JSONB_EXISTS_ALL = 612; // Non-pure functions below (> 1000) // ------------------------ diff --git a/src/expr/impl/src/scalar/jsonb_contains.rs b/src/expr/impl/src/scalar/jsonb_contains.rs new file mode 100644 index 0000000000000..02dfffe9d2d18 --- /dev/null +++ b/src/expr/impl/src/scalar/jsonb_contains.rs @@ -0,0 +1,273 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use jsonbb::ValueRef; +use risingwave_common::types::{JsonbRef, ListRef}; +use risingwave_expr::function; + +/// Does the first JSON value contain the second? +/// +/// Examples: +/// +/// ```slt +/// # Simple scalar/primitive values contain only the identical value: +/// query B +/// SELECT '"foo"'::jsonb @> '"foo"'::jsonb; +/// ---- +/// t +/// +/// # The array on the right side is contained within the one on the left: +/// query B +/// SELECT '[1, 2, 3]'::jsonb @> '[1, 3]'::jsonb; +/// ---- +/// t +/// +/// # Order of array elements is not significant, so this is also true: +/// query B +/// SELECT '[1, 2, 3]'::jsonb @> '[3, 1]'::jsonb; +/// ---- +/// t +/// +/// # Duplicate array elements don't matter either: +/// query B +/// SELECT '[1, 2, 3]'::jsonb @> '[1, 2, 2]'::jsonb; +/// ---- +/// t +/// +/// # The object with a single pair on the right side is contained +/// # within the object on the left side: +/// query B +/// SELECT '{"product": "PostgreSQL", "version": 9.4, "jsonb": true}'::jsonb @> '{"version": 9.4}'::jsonb; +/// ---- +/// t +/// +/// # The array on the right side is not considered contained within the +/// # array on the left, even though a similar array is nested within it: +/// query B +/// SELECT '[1, 2, [1, 3]]'::jsonb @> '[1, 3]'::jsonb; +/// ---- +/// f +/// +/// # But with a layer of nesting, it is contained: +/// query B +/// SELECT '[1, 2, [1, 3]]'::jsonb @> '[[1, 3]]'::jsonb; +/// ---- +/// t +/// +/// # Similarly, containment is not reported here: +/// query B +/// SELECT '{"foo": {"bar": "baz"}}'::jsonb @> '{"bar": "baz"}'::jsonb; +/// ---- +/// f +/// +/// # A top-level key and an empty object is contained: +/// query B +/// SELECT '{"foo": {"bar": "baz"}}'::jsonb @> '{"foo": {}}'::jsonb; +/// ---- +/// t +/// +/// # This array contains the primitive string value: +/// query B +/// SELECT '["foo", "bar"]'::jsonb @> '"bar"'::jsonb; +/// ---- +/// t +/// +/// # This exception is not reciprocal -- non-containment is reported here: +/// query B +/// SELECT '"bar"'::jsonb @> '["bar"]'::jsonb; +/// ---- +/// f +/// +/// # Object is not primitive: +/// query B +/// SELECT '[1, {"a":2}]'::jsonb @> '{"a":2}'; +/// ---- +/// f +/// +/// # Array can be nested: +/// query B +/// SELECT '[1, [3, 4]]'::jsonb @> '[[3]]'; +/// ---- +/// t +/// +/// # Recursion shall not include the special rule of array containing primitive: +/// query B +/// SELECT '{"a": [3, 4]}'::jsonb @> '{"a": 3}'; +/// ---- +/// f +/// ``` +#[function("jsonb_contains(jsonb, jsonb) -> boolean")] +fn jsonb_contains(left: JsonbRef<'_>, right: JsonbRef<'_>) -> bool { + jsonbb_contains(left.into(), right.into(), true) +} + +/// Performs `jsonb_contains` on `jsonbb::ValueRef`. +/// `root` indicates whether the current recursion is at the root level. +fn jsonbb_contains(left: ValueRef<'_>, right: ValueRef<'_>, root: bool) -> bool { + match (left, right) { + // Both left and right are objects. + (ValueRef::Object(left_obj), ValueRef::Object(right_obj)) => { + // Every key-value pair in right should be present in left. + right_obj.iter().all(|(key, value)| { + left_obj + .get(key) + .map_or(false, |left_val| jsonbb_contains(left_val, value, false)) + }) + } + + // Both left and right are arrays. + (ValueRef::Array(left_arr), ValueRef::Array(right_arr)) => { + // For every value in right, there should be an equivalent in left. + right_arr.iter().all(|right_val| { + left_arr + .iter() + .any(|left_val| jsonbb_contains(left_val, right_val, false)) + }) + } + + // Left is an array and right is an object. + (ValueRef::Array(_), ValueRef::Object(_)) => false, + + // Left is an array and right is a primitive value. only at root level. + (ValueRef::Array(left_arr), right_val) if root => { + // The right should be present in left. + left_arr.iter().any(|left_val| left_val == right_val) + } + + // Both left and right are primitive values. + (left_val, right_val) => left_val == right_val, + } +} + +/// Is the first JSON value contained in the second? +/// +/// Examples: +/// +/// ```slt +/// query B +/// select '{"b":2}'::jsonb <@ '{"a":1, "b":2}'::jsonb; +/// ---- +/// t +/// ``` +#[function("jsonb_contained(jsonb, jsonb) -> boolean")] +fn jsonb_contained(left: JsonbRef<'_>, right: JsonbRef<'_>) -> bool { + jsonb_contains(right, left) +} + +/// Does the text string exist as a top-level key or array element within the JSON value? +/// +/// Examples: +/// +/// ```slt +/// # String exists as array element: +/// query B +/// SELECT '["foo", "bar", "baz"]'::jsonb ? 'bar'; +/// ---- +/// t +/// +/// # String exists as object key: +/// query B +/// SELECT '{"foo": "bar"}'::jsonb ? 'foo'; +/// ---- +/// t +/// +/// # Object values are not considered: +/// query B +/// SELECT '{"foo": "bar"}'::jsonb ? 'bar'; +/// ---- +/// f +/// +/// # As with containment, existence must match at the top level: +/// query B +/// SELECT '{"foo": {"bar": "baz"}}'::jsonb ? 'bar'; +/// ---- +/// f +/// +/// # A string is considered to exist if it matches a primitive JSON string: +/// query B +/// SELECT '"foo"'::jsonb ? 'foo'; +/// ---- +/// t +/// ``` +#[function("jsonb_exists(jsonb, varchar) -> boolean")] +fn jsonb_exists(left: JsonbRef<'_>, key: &str) -> bool { + match left.into() { + ValueRef::Object(object) => object.get(key).is_some(), + ValueRef::Array(array) => array.iter().any(|val| val.as_str() == Some(key)), + ValueRef::String(str) => str == key, + _ => false, + } +} + +/// Do any of the strings in the text array exist as top-level keys or array elements? +/// +/// Examples: +/// +/// ```slt +/// query B +/// select '{"a":1, "b":2, "c":3}'::jsonb ?| array['b', 'd']; +/// ---- +/// t +/// +/// query B +/// select '["a", "b", "c"]'::jsonb ?| array['b', 'd']; +/// ---- +/// t +/// +/// query B +/// select '"b"'::jsonb ?| array['b', 'd']; +/// ---- +/// t +/// ``` +#[function("jsonb_exists_any(jsonb, varchar[]) -> boolean")] +fn jsonb_exists_any(left: JsonbRef<'_>, keys: ListRef<'_>) -> bool { + let mut keys = keys.iter().flatten().map(|val| val.into_utf8()); + match left.into() { + ValueRef::Object(object) => keys.any(|key| object.get(key).is_some()), + ValueRef::Array(array) => keys.any(|key| array.iter().any(|val| val.as_str() == Some(key))), + ValueRef::String(str) => keys.any(|key| str == key), + _ => false, + } +} + +/// Do all of the strings in the text array exist as top-level keys or array elements? +/// +/// Examples: +/// +/// ```slt +/// query B +/// select '{"a":1, "b":2, "c":3}'::jsonb ?& array['a', 'b']; +/// ---- +/// t +/// +/// query B +/// select '["a", "b", "c"]'::jsonb ?& array['a', 'b']; +/// ---- +/// t +/// +/// query B +/// select '"b"'::jsonb ?& array['b']; +/// ---- +/// t +/// ``` +#[function("jsonb_exists_all(jsonb, varchar[]) -> boolean")] +fn jsonb_exists_all(left: JsonbRef<'_>, keys: ListRef<'_>) -> bool { + let mut keys = keys.iter().flatten().map(|val| val.into_utf8()); + match left.into() { + ValueRef::Object(object) => keys.all(|key| object.get(key).is_some()), + ValueRef::Array(array) => keys.all(|key| array.iter().any(|val| val.as_str() == Some(key))), + ValueRef::String(str) => keys.all(|key| str == key), + _ => false, + } +} diff --git a/src/expr/impl/src/scalar/mod.rs b/src/expr/impl/src/scalar/mod.rs index d9d10e4548aee..d1b89d3c75d6c 100644 --- a/src/expr/impl/src/scalar/mod.rs +++ b/src/expr/impl/src/scalar/mod.rs @@ -44,6 +44,7 @@ mod format_type; mod int256; mod jsonb_access; mod jsonb_concat; +mod jsonb_contains; mod jsonb_info; mod jsonb_object; mod length; diff --git a/src/frontend/src/binder/expr/binary_op.rs b/src/frontend/src/binder/expr/binary_op.rs index f7c8a86144fc9..bd85089b18cce 100644 --- a/src/frontend/src/binder/expr/binary_op.rs +++ b/src/frontend/src/binder/expr/binary_op.rs @@ -92,6 +92,11 @@ impl Binder { BinaryOperator::Arrow => ExprType::JsonbAccessInner, BinaryOperator::LongArrow => ExprType::JsonbAccessStr, BinaryOperator::Prefix => ExprType::StartsWith, + BinaryOperator::Contains => ExprType::JsonbContains, + BinaryOperator::Contained => ExprType::JsonbContained, + BinaryOperator::Exists => ExprType::JsonbExists, + BinaryOperator::ExistsAny => ExprType::JsonbExistsAny, + BinaryOperator::ExistsAll => ExprType::JsonbExistsAll, BinaryOperator::Concat => { let left_type = (!bound_left.is_untyped()).then(|| bound_left.return_type()); let right_type = (!bound_right.is_untyped()).then(|| bound_right.return_type()); diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 50a621c0879d5..7ca509e6e3af9 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -879,6 +879,11 @@ impl Binder { ("jsonb_array_length", raw_call(ExprType::JsonbArrayLength)), ("jsonb_object", raw_call(ExprType::JsonbObject)), ("jsonb_pretty", raw_call(ExprType::JsonbPretty)), + ("jsonb_contains", raw_call(ExprType::JsonbContains)), + ("jsonb_contained", raw_call(ExprType::JsonbContained)), + ("jsonb_exists", raw_call(ExprType::JsonbExists)), + ("jsonb_exists_any", raw_call(ExprType::JsonbExistsAny)), + ("jsonb_exists_all", raw_call(ExprType::JsonbExistsAll)), // Functions that return a constant value ("pi", pi()), // greatest and least diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index a3c7abf1ef482..038bd94ffcf07 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -179,6 +179,11 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::JsonbArrayLength | expr_node::Type::JsonbObject | expr_node::Type::JsonbPretty + | expr_node::Type::JsonbContains + | expr_node::Type::JsonbContained + | expr_node::Type::JsonbExists + | expr_node::Type::JsonbExistsAny + | expr_node::Type::JsonbExistsAll | expr_node::Type::IsJson | expr_node::Type::Sind | expr_node::Type::Cosd diff --git a/src/sqlparser/src/ast/operator.rs b/src/sqlparser/src/ast/operator.rs index ad084a59425b0..147a78d0b7174 100644 --- a/src/sqlparser/src/ast/operator.rs +++ b/src/sqlparser/src/ast/operator.rs @@ -99,6 +99,11 @@ pub enum BinaryOperator { LongArrow, HashArrow, HashLongArrow, + Contains, + Contained, + Exists, + ExistsAny, + ExistsAll, PGQualified(Box), } @@ -143,6 +148,11 @@ impl fmt::Display for BinaryOperator { BinaryOperator::LongArrow => "->>", BinaryOperator::HashArrow => "#>", BinaryOperator::HashLongArrow => "#>>", + BinaryOperator::Contains => "@>", + BinaryOperator::Contained => "<@", + BinaryOperator::Exists => "?", + BinaryOperator::ExistsAny => "?|", + BinaryOperator::ExistsAll => "?&", BinaryOperator::PGQualified(_) => unreachable!(), }) } diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index d87488ac30648..d73ec295b0f90 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -1397,6 +1397,11 @@ impl Parser { Token::LongArrow => Some(BinaryOperator::LongArrow), Token::HashArrow => Some(BinaryOperator::HashArrow), Token::HashLongArrow => Some(BinaryOperator::HashLongArrow), + Token::AtArrow => Some(BinaryOperator::Contains), + Token::ArrowAt => Some(BinaryOperator::Contained), + Token::QuestionMark => Some(BinaryOperator::Exists), + Token::QuestionMarkPipe => Some(BinaryOperator::ExistsAny), + Token::QuestionMarkAmpersand => Some(BinaryOperator::ExistsAll), Token::Word(w) => match w.keyword { Keyword::AND => Some(BinaryOperator::And), Keyword::OR => Some(BinaryOperator::Or), @@ -1735,7 +1740,12 @@ impl Parser { | Token::Arrow | Token::LongArrow | Token::HashArrow - | Token::HashLongArrow => Ok(P::Other), + | Token::HashLongArrow + | Token::AtArrow + | Token::ArrowAt + | Token::QuestionMark + | Token::QuestionMarkPipe + | Token::QuestionMarkAmpersand => Ok(P::Other), Token::Word(w) if w.keyword == Keyword::OPERATOR && self.peek_nth_token(1) == Token::LParen => { diff --git a/src/sqlparser/src/tokenizer.rs b/src/sqlparser/src/tokenizer.rs index d0d1e096f8f73..914f0e5f0f8d4 100644 --- a/src/sqlparser/src/tokenizer.rs +++ b/src/sqlparser/src/tokenizer.rs @@ -164,6 +164,16 @@ pub enum Token { HashArrow, /// `#>>`, extract JSON sub-object at the specified path as text in PostgreSQL HashLongArrow, + /// `@>`, does the left JSON value contain the right JSON path/value entries at the top level + AtArrow, + /// `<@`, does the right JSON value contain the left JSON path/value entries at the top level + ArrowAt, + /// `?`, does the string exist as a top-level key within the JSON value + QuestionMark, + /// `?|`, do any of the strings exist as top-level keys or array elements? + QuestionMarkPipe, + /// `?&`, do all of the strings exist as top-level keys or array elements? + QuestionMarkAmpersand, } impl fmt::Display for Token { @@ -231,6 +241,11 @@ impl fmt::Display for Token { Token::LongArrow => f.write_str("->>"), Token::HashArrow => f.write_str("#>"), Token::HashLongArrow => f.write_str("#>>"), + Token::AtArrow => f.write_str("@>"), + Token::ArrowAt => f.write_str("<@"), + Token::QuestionMark => f.write_str("?"), + Token::QuestionMarkPipe => f.write_str("?|"), + Token::QuestionMarkAmpersand => f.write_str("?&"), } } } @@ -693,6 +708,7 @@ impl<'a> Tokenizer<'a> { } Some('>') => self.consume_and_return(chars, Token::Neq), Some('<') => self.consume_and_return(chars, Token::ShiftLeft), + Some('@') => self.consume_and_return(chars, Token::ArrowAt), _ => Ok(Some(Token::Lt)), } } @@ -759,7 +775,23 @@ impl<'a> Tokenizer<'a> { _ => Ok(Some(Token::Sharp)), } } - '@' => self.consume_and_return(chars, Token::AtSign), + '@' => { + chars.next(); // consume the '@' + match chars.peek() { + Some('>') => self.consume_and_return(chars, Token::AtArrow), + // a regular '@' operator + _ => Ok(Some(Token::AtSign)), + } + } + '?' => { + chars.next(); // consume the '?' + match chars.peek() { + Some('|') => self.consume_and_return(chars, Token::QuestionMarkPipe), + Some('&') => self.consume_and_return(chars, Token::QuestionMarkAmpersand), + // a regular '?' operator + _ => Ok(Some(Token::QuestionMark)), + } + } other => self.consume_and_return(chars, Token::Char(other)), }, None => Ok(None), diff --git a/src/tests/regress/data/sql/jsonb.sql b/src/tests/regress/data/sql/jsonb.sql index 59b00932db189..26c25af897bf4 100644 --- a/src/tests/regress/data/sql/jsonb.sql +++ b/src/tests/regress/data/sql/jsonb.sql @@ -211,53 +211,53 @@ select '"foo"'::jsonb ->> 'z'; --@ SELECT '{"x":"y"}'::jsonb <> '{"x":"z"}'::jsonb; -- containment ---@ SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"b"}'); ---@ SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"b", "c":null}'); ---@ SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"b", "g":null}'); ---@ SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"g":null}'); ---@ SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"c"}'); ---@ SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"b"}'); ---@ SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"b", "c":"q"}'); ---@ SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"b"}'; ---@ SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"b", "c":null}'; ---@ SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"b", "g":null}'; ---@ SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"g":null}'; ---@ SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"c"}'; ---@ SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"b"}'; ---@ SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"b", "c":"q"}'; ---@ ---@ SELECT '[1,2]'::jsonb @> '[1,2,2]'::jsonb; ---@ SELECT '[1,1,2]'::jsonb @> '[1,2,2]'::jsonb; ---@ SELECT '[[1,2]]'::jsonb @> '[[1,2,2]]'::jsonb; ---@ SELECT '[1,2,2]'::jsonb <@ '[1,2]'::jsonb; ---@ SELECT '[1,2,2]'::jsonb <@ '[1,1,2]'::jsonb; ---@ SELECT '[[1,2,2]]'::jsonb <@ '[[1,2]]'::jsonb; ---@ ---@ SELECT jsonb_contained('{"a":"b"}', '{"a":"b", "b":1, "c":null}'); ---@ SELECT jsonb_contained('{"a":"b", "c":null}', '{"a":"b", "b":1, "c":null}'); ---@ SELECT jsonb_contained('{"a":"b", "g":null}', '{"a":"b", "b":1, "c":null}'); ---@ SELECT jsonb_contained('{"g":null}', '{"a":"b", "b":1, "c":null}'); ---@ SELECT jsonb_contained('{"a":"c"}', '{"a":"b", "b":1, "c":null}'); ---@ SELECT jsonb_contained('{"a":"b"}', '{"a":"b", "b":1, "c":null}'); ---@ SELECT jsonb_contained('{"a":"b", "c":"q"}', '{"a":"b", "b":1, "c":null}'); ---@ SELECT '{"a":"b"}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; ---@ SELECT '{"a":"b", "c":null}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; ---@ SELECT '{"a":"b", "g":null}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; ---@ SELECT '{"g":null}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; ---@ SELECT '{"a":"c"}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; ---@ SELECT '{"a":"b"}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; ---@ SELECT '{"a":"b", "c":"q"}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; ---@ -- Raw scalar may contain another raw scalar, array may contain a raw scalar ---@ SELECT '[5]'::jsonb @> '[5]'; ---@ SELECT '5'::jsonb @> '5'; ---@ SELECT '[5]'::jsonb @> '5'; ---@ -- But a raw scalar cannot contain an array ---@ SELECT '5'::jsonb @> '[5]'; ---@ -- In general, one thing should always contain itself. Test array containment: ---@ SELECT '["9", ["7", "3"], 1]'::jsonb @> '["9", ["7", "3"], 1]'::jsonb; ---@ SELECT '["9", ["7", "3"], ["1"]]'::jsonb @> '["9", ["7", "3"], ["1"]]'::jsonb; ---@ -- array containment string matching confusion bug ---@ SELECT '{ "name": "Bob", "tags": [ "enim", "qui"]}'::jsonb @> '{"tags":["qu"]}'; +SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"b"}'); +SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"b", "c":null}'); +SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"b", "g":null}'); +SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"g":null}'); +SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"c"}'); +SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"b"}'); +SELECT jsonb_contains('{"a":"b", "b":1, "c":null}', '{"a":"b", "c":"q"}'); +SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"b"}'; +SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"b", "c":null}'; +SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"b", "g":null}'; +SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"g":null}'; +SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"c"}'; +SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"b"}'; +SELECT '{"a":"b", "b":1, "c":null}'::jsonb @> '{"a":"b", "c":"q"}'; + +SELECT '[1,2]'::jsonb @> '[1,2,2]'::jsonb; +SELECT '[1,1,2]'::jsonb @> '[1,2,2]'::jsonb; +SELECT '[[1,2]]'::jsonb @> '[[1,2,2]]'::jsonb; +SELECT '[1,2,2]'::jsonb <@ '[1,2]'::jsonb; +SELECT '[1,2,2]'::jsonb <@ '[1,1,2]'::jsonb; +SELECT '[[1,2,2]]'::jsonb <@ '[[1,2]]'::jsonb; + +SELECT jsonb_contained('{"a":"b"}', '{"a":"b", "b":1, "c":null}'); +SELECT jsonb_contained('{"a":"b", "c":null}', '{"a":"b", "b":1, "c":null}'); +SELECT jsonb_contained('{"a":"b", "g":null}', '{"a":"b", "b":1, "c":null}'); +SELECT jsonb_contained('{"g":null}', '{"a":"b", "b":1, "c":null}'); +SELECT jsonb_contained('{"a":"c"}', '{"a":"b", "b":1, "c":null}'); +SELECT jsonb_contained('{"a":"b"}', '{"a":"b", "b":1, "c":null}'); +SELECT jsonb_contained('{"a":"b", "c":"q"}', '{"a":"b", "b":1, "c":null}'); +SELECT '{"a":"b"}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; +SELECT '{"a":"b", "c":null}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; +SELECT '{"a":"b", "g":null}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; +SELECT '{"g":null}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; +SELECT '{"a":"c"}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; +SELECT '{"a":"b"}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; +SELECT '{"a":"b", "c":"q"}'::jsonb <@ '{"a":"b", "b":1, "c":null}'; +-- Raw scalar may contain another raw scalar, array may contain a raw scalar +SELECT '[5]'::jsonb @> '[5]'; +SELECT '5'::jsonb @> '5'; +SELECT '[5]'::jsonb @> '5'; +-- But a raw scalar cannot contain an array +SELECT '5'::jsonb @> '[5]'; +-- In general, one thing should always contain itself. Test array containment: +SELECT '["9", ["7", "3"], 1]'::jsonb @> '["9", ["7", "3"], 1]'::jsonb; +SELECT '["9", ["7", "3"], ["1"]]'::jsonb @> '["9", ["7", "3"], ["1"]]'::jsonb; +-- array containment string matching confusion bug +SELECT '{ "name": "Bob", "tags": [ "enim", "qui"]}'::jsonb @> '{"tags":["qu"]}'; -- array length SELECT jsonb_array_length('[1,2,3,{"f1":1,"f2":[5,6]},4]'); @@ -277,14 +277,14 @@ SELECT jsonb_array_length('4'); --@ SELECT * FROM jsonb_each_text('{"a":{"b":"c","c":"b","1":"first"},"b":[1,2],"c":"cc","1":"first","n":null}'::jsonb) AS q; -- exists ---@ SELECT jsonb_exists('{"a":null, "b":"qq"}', 'a'); ---@ SELECT jsonb_exists('{"a":null, "b":"qq"}', 'b'); ---@ SELECT jsonb_exists('{"a":null, "b":"qq"}', 'c'); ---@ SELECT jsonb_exists('{"a":"null", "b":"qq"}', 'a'); ---@ SELECT jsonb '{"a":null, "b":"qq"}' ? 'a'; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ? 'b'; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ? 'c'; ---@ SELECT jsonb '{"a":"null", "b":"qq"}' ? 'a'; +SELECT jsonb_exists('{"a":null, "b":"qq"}', 'a'); +SELECT jsonb_exists('{"a":null, "b":"qq"}', 'b'); +SELECT jsonb_exists('{"a":null, "b":"qq"}', 'c'); +SELECT jsonb_exists('{"a":"null", "b":"qq"}', 'a'); +SELECT jsonb '{"a":null, "b":"qq"}' ? 'a'; +SELECT jsonb '{"a":null, "b":"qq"}' ? 'b'; +SELECT jsonb '{"a":null, "b":"qq"}' ? 'c'; +SELECT jsonb '{"a":"null", "b":"qq"}' ? 'a'; --@ -- array exists - array elements should behave as keys --@ SELECT count(*) from testjsonb WHERE j->'array' ? 'bar'; --@ -- type sensitive array exists - should return no rows (since "exists" only @@ -292,29 +292,29 @@ SELECT jsonb_array_length('4'); --@ SELECT count(*) from testjsonb WHERE j->'array' ? '5'::text; --@ -- However, a raw scalar is *contained* within the array --@ SELECT count(*) from testjsonb WHERE j->'array' @> '5'::jsonb; ---@ ---@ SELECT jsonb_exists_any('{"a":null, "b":"qq"}', ARRAY['a','b']); ---@ SELECT jsonb_exists_any('{"a":null, "b":"qq"}', ARRAY['b','a']); ---@ SELECT jsonb_exists_any('{"a":null, "b":"qq"}', ARRAY['c','a']); ---@ SELECT jsonb_exists_any('{"a":null, "b":"qq"}', ARRAY['c','d']); ---@ SELECT jsonb_exists_any('{"a":null, "b":"qq"}', '{}'::text[]); ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?| ARRAY['a','b']; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?| ARRAY['b','a']; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?| ARRAY['c','a']; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?| ARRAY['c','d']; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?| '{}'::text[]; ---@ ---@ SELECT jsonb_exists_all('{"a":null, "b":"qq"}', ARRAY['a','b']); ---@ SELECT jsonb_exists_all('{"a":null, "b":"qq"}', ARRAY['b','a']); ---@ SELECT jsonb_exists_all('{"a":null, "b":"qq"}', ARRAY['c','a']); ---@ SELECT jsonb_exists_all('{"a":null, "b":"qq"}', ARRAY['c','d']); ---@ SELECT jsonb_exists_all('{"a":null, "b":"qq"}', '{}'::text[]); ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?& ARRAY['a','b']; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?& ARRAY['b','a']; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?& ARRAY['c','a']; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?& ARRAY['c','d']; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?& ARRAY['a','a', 'b', 'b', 'b']; ---@ SELECT jsonb '{"a":null, "b":"qq"}' ?& '{}'::text[]; + +SELECT jsonb_exists_any('{"a":null, "b":"qq"}', ARRAY['a','b']); +SELECT jsonb_exists_any('{"a":null, "b":"qq"}', ARRAY['b','a']); +SELECT jsonb_exists_any('{"a":null, "b":"qq"}', ARRAY['c','a']); +SELECT jsonb_exists_any('{"a":null, "b":"qq"}', ARRAY['c','d']); +SELECT jsonb_exists_any('{"a":null, "b":"qq"}', '{}'::text[]); +SELECT jsonb '{"a":null, "b":"qq"}' ?| ARRAY['a','b']; +SELECT jsonb '{"a":null, "b":"qq"}' ?| ARRAY['b','a']; +SELECT jsonb '{"a":null, "b":"qq"}' ?| ARRAY['c','a']; +SELECT jsonb '{"a":null, "b":"qq"}' ?| ARRAY['c','d']; +SELECT jsonb '{"a":null, "b":"qq"}' ?| '{}'::text[]; + +SELECT jsonb_exists_all('{"a":null, "b":"qq"}', ARRAY['a','b']); +SELECT jsonb_exists_all('{"a":null, "b":"qq"}', ARRAY['b','a']); +SELECT jsonb_exists_all('{"a":null, "b":"qq"}', ARRAY['c','a']); +SELECT jsonb_exists_all('{"a":null, "b":"qq"}', ARRAY['c','d']); +SELECT jsonb_exists_all('{"a":null, "b":"qq"}', '{}'::text[]); +SELECT jsonb '{"a":null, "b":"qq"}' ?& ARRAY['a','b']; +SELECT jsonb '{"a":null, "b":"qq"}' ?& ARRAY['b','a']; +SELECT jsonb '{"a":null, "b":"qq"}' ?& ARRAY['c','a']; +SELECT jsonb '{"a":null, "b":"qq"}' ?& ARRAY['c','d']; +SELECT jsonb '{"a":null, "b":"qq"}' ?& ARRAY['a','a', 'b', 'b', 'b']; +SELECT jsonb '{"a":null, "b":"qq"}' ?& '{}'::text[]; -- typeof SELECT jsonb_typeof('{}') AS object; @@ -941,25 +941,25 @@ SELECT '{"ff":{"a":12,"b":16},"qq":123,"x":[1,2],"Y":null}'::jsonb -> 'x'; -- nested containment ---@ SELECT '{"a":[1,2],"c":"b"}'::jsonb @> '{"a":[1,2]}'; ---@ SELECT '{"a":[2,1],"c":"b"}'::jsonb @> '{"a":[1,2]}'; ---@ SELECT '{"a":{"1":2},"c":"b"}'::jsonb @> '{"a":[1,2]}'; ---@ SELECT '{"a":{"2":1},"c":"b"}'::jsonb @> '{"a":[1,2]}'; ---@ SELECT '{"a":{"1":2},"c":"b"}'::jsonb @> '{"a":{"1":2}}'; ---@ SELECT '{"a":{"2":1},"c":"b"}'::jsonb @> '{"a":{"1":2}}'; ---@ SELECT '["a","b"]'::jsonb @> '["a","b","c","b"]'; ---@ SELECT '["a","b","c","b"]'::jsonb @> '["a","b"]'; ---@ SELECT '["a","b","c",[1,2]]'::jsonb @> '["a",[1,2]]'; ---@ SELECT '["a","b","c",[1,2]]'::jsonb @> '["b",[1,2]]'; ---@ ---@ SELECT '{"a":[1,2],"c":"b"}'::jsonb @> '{"a":[1]}'; ---@ SELECT '{"a":[1,2],"c":"b"}'::jsonb @> '{"a":[2]}'; ---@ SELECT '{"a":[1,2],"c":"b"}'::jsonb @> '{"a":[3]}'; ---@ ---@ SELECT '{"a":[1,2,{"c":3,"x":4}],"c":"b"}'::jsonb @> '{"a":[{"c":3}]}'; ---@ SELECT '{"a":[1,2,{"c":3,"x":4}],"c":"b"}'::jsonb @> '{"a":[{"x":4}]}'; ---@ SELECT '{"a":[1,2,{"c":3,"x":4}],"c":"b"}'::jsonb @> '{"a":[{"x":4},3]}'; ---@ SELECT '{"a":[1,2,{"c":3,"x":4}],"c":"b"}'::jsonb @> '{"a":[{"x":4},1]}'; +SELECT '{"a":[1,2],"c":"b"}'::jsonb @> '{"a":[1,2]}'; +SELECT '{"a":[2,1],"c":"b"}'::jsonb @> '{"a":[1,2]}'; +SELECT '{"a":{"1":2},"c":"b"}'::jsonb @> '{"a":[1,2]}'; +SELECT '{"a":{"2":1},"c":"b"}'::jsonb @> '{"a":[1,2]}'; +SELECT '{"a":{"1":2},"c":"b"}'::jsonb @> '{"a":{"1":2}}'; +SELECT '{"a":{"2":1},"c":"b"}'::jsonb @> '{"a":{"1":2}}'; +SELECT '["a","b"]'::jsonb @> '["a","b","c","b"]'; +SELECT '["a","b","c","b"]'::jsonb @> '["a","b"]'; +SELECT '["a","b","c",[1,2]]'::jsonb @> '["a",[1,2]]'; +SELECT '["a","b","c",[1,2]]'::jsonb @> '["b",[1,2]]'; + +SELECT '{"a":[1,2],"c":"b"}'::jsonb @> '{"a":[1]}'; +SELECT '{"a":[1,2],"c":"b"}'::jsonb @> '{"a":[2]}'; +SELECT '{"a":[1,2],"c":"b"}'::jsonb @> '{"a":[3]}'; + +SELECT '{"a":[1,2,{"c":3,"x":4}],"c":"b"}'::jsonb @> '{"a":[{"c":3}]}'; +SELECT '{"a":[1,2,{"c":3,"x":4}],"c":"b"}'::jsonb @> '{"a":[{"x":4}]}'; +SELECT '{"a":[1,2,{"c":3,"x":4}],"c":"b"}'::jsonb @> '{"a":[{"x":4},3]}'; +SELECT '{"a":[1,2,{"c":3,"x":4}],"c":"b"}'::jsonb @> '{"a":[{"x":4},1]}'; -- check some corner cases for indexed nested containment (bug #13756) --@ create temp table nestjsonb (j jsonb); @@ -1020,12 +1020,12 @@ SELECT '["a","b","c",[1,2],null]'::jsonb -> -6; --@ SELECT '[0,1,2,[3,4],{"5":"five"}]'::jsonb #> '{4,5}'; --nested exists ---@ SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'n'; ---@ SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'a'; ---@ SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'b'; ---@ SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'c'; ---@ SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'd'; ---@ SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'e'; +SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'n'; +SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'a'; +SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'b'; +SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'c'; +SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'd'; +SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'e'; -- jsonb_strip_nulls From 98edea69b37499c697935c3e9aab3327458b9b63 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Mon, 30 Oct 2023 13:02:35 +0800 Subject: [PATCH 43/52] fix(psql_conn): Add row desc with 'SHOW PARAMETERS' (#13099) Co-authored-by: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> --- src/frontend/src/handler/variable.rs | 81 ++++--------------- src/frontend/src/session.rs | 28 +------ src/frontend/src/utils/infer_stmt_row_desc.rs | 46 +++++++++++ 3 files changed, 63 insertions(+), 92 deletions(-) diff --git a/src/frontend/src/handler/variable.rs b/src/frontend/src/handler/variable.rs index 9060ff2209413..7d108ae35128e 100644 --- a/src/frontend/src/handler/variable.rs +++ b/src/frontend/src/handler/variable.rs @@ -13,7 +13,6 @@ // limitations under the License. use itertools::Itertools; -use pgwire::pg_field_descriptor::PgFieldDescriptor; use pgwire::pg_protocol::ParameterStatus; use pgwire::pg_response::{PgResponse, StatementType}; use pgwire::types::Row; @@ -25,6 +24,7 @@ use risingwave_sqlparser::ast::{Ident, SetTimeZoneValue, SetVariableValue, Value use super::RwPgResponse; use crate::handler::HandlerArgs; +use crate::utils::infer_stmt_row_desc::infer_show_variable; pub fn handle_set( handler_args: HandlerArgs, @@ -96,29 +96,22 @@ pub(super) async fn handle_show( ) -> Result { // TODO: Verify that the name used in `show` command is indeed always case-insensitive. let name = variable.iter().map(|e| e.real_value()).join(" "); - if name.eq_ignore_ascii_case("PARAMETERS") { - return handle_show_system_params(handler_args).await; - } - // Show session config. - let config_reader = handler_args.session.config(); - if name.eq_ignore_ascii_case("ALL") { - return handle_show_all(handler_args.clone()); - } - let row = Row::new(vec![Some(config_reader.get(&name)?.into())]); + let row_desc = infer_show_variable(&name); + let rows = if name.eq_ignore_ascii_case("PARAMETERS") { + handle_show_system_params(handler_args).await? + } else if name.eq_ignore_ascii_case("ALL") { + handle_show_all(handler_args.clone())? + } else { + let config_reader = handler_args.session.config(); + vec![Row::new(vec![Some(config_reader.get(&name)?.into())])] + }; Ok(PgResponse::builder(StatementType::SHOW_VARIABLE) - .values( - vec![row].into(), - vec![PgFieldDescriptor::new( - name.to_ascii_lowercase(), - DataType::Varchar.to_oid(), - DataType::Varchar.type_len(), - )], - ) + .values(rows.into(), row_desc) .into()) } -fn handle_show_all(handler_args: HandlerArgs) -> Result { +fn handle_show_all(handler_args: HandlerArgs) -> Result> { let config_reader = handler_args.session.config(); let all_variables = config_reader.get_all(); @@ -133,32 +126,10 @@ fn handle_show_all(handler_args: HandlerArgs) -> Result { ]) }) .collect_vec(); - - Ok(RwPgResponse::builder(StatementType::SHOW_VARIABLE) - .values( - rows.into(), - vec![ - PgFieldDescriptor::new( - "Name".to_string(), - DataType::Varchar.to_oid(), - DataType::Varchar.type_len(), - ), - PgFieldDescriptor::new( - "Setting".to_string(), - DataType::Varchar.to_oid(), - DataType::Varchar.type_len(), - ), - PgFieldDescriptor::new( - "Description".to_string(), - DataType::Varchar.to_oid(), - DataType::Varchar.type_len(), - ), - ], - ) - .into()) + Ok(rows) } -async fn handle_show_system_params(handler_args: HandlerArgs) -> Result { +async fn handle_show_system_params(handler_args: HandlerArgs) -> Result> { let params = handler_args .session .env() @@ -175,27 +146,5 @@ async fn handle_show_system_params(handler_args: HandlerArgs) -> Result, stmt: Statement) -> Result { let name = &variable[0].real_value().to_lowercase(); - if name.eq_ignore_ascii_case("ALL") { - Ok(vec![ - PgFieldDescriptor::new( - "Name".to_string(), - DataType::Varchar.to_oid(), - DataType::Varchar.type_len(), - ), - PgFieldDescriptor::new( - "Setting".to_string(), - DataType::Varchar.to_oid(), - DataType::Varchar.type_len(), - ), - PgFieldDescriptor::new( - "Description".to_string(), - DataType::Varchar.to_oid(), - DataType::Varchar.type_len(), - ), - ]) - } else { - Ok(vec![PgFieldDescriptor::new( - name.to_ascii_lowercase(), - DataType::Varchar.to_oid(), - DataType::Varchar.type_len(), - )]) - } + Ok(infer_show_variable(name)) } Statement::Describe { name: _ } => Ok(vec![ PgFieldDescriptor::new( diff --git a/src/frontend/src/utils/infer_stmt_row_desc.rs b/src/frontend/src/utils/infer_stmt_row_desc.rs index dbe8968f0a293..8ebb7ac5c7d7a 100644 --- a/src/frontend/src/utils/infer_stmt_row_desc.rs +++ b/src/frontend/src/utils/infer_stmt_row_desc.rs @@ -168,3 +168,49 @@ pub fn infer_show_object(objects: &ShowObject) -> Vec { )], } } + +pub fn infer_show_variable(name: &str) -> Vec { + if name.eq_ignore_ascii_case("ALL") { + vec![ + PgFieldDescriptor::new( + "Name".to_string(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + PgFieldDescriptor::new( + "Setting".to_string(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + PgFieldDescriptor::new( + "Description".to_string(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + ] + } else if name.eq_ignore_ascii_case("PARAMETERS") { + vec![ + PgFieldDescriptor::new( + "Name".to_string(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + PgFieldDescriptor::new( + "Value".to_string(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + ), + PgFieldDescriptor::new( + "Mutable".to_string(), + DataType::Boolean.to_oid(), + DataType::Boolean.type_len(), + ), + ] + } else { + vec![PgFieldDescriptor::new( + name.to_ascii_lowercase(), + DataType::Varchar.to_oid(), + DataType::Varchar.type_len(), + )] + } +} From e392db0c6f468caf89597294a94d4e4134c61eaf Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Mon, 30 Oct 2023 13:31:35 +0800 Subject: [PATCH 44/52] feat(expr): support `#>` and `#>>` operator for extracting jsonb at a path (#13110) Signed-off-by: Runji Wang --- proto/expr.proto | 10 +- src/common/src/types/jsonb.rs | 10 ++ src/expr/impl/src/scalar/jsonb_access.rs | 142 +++++++++++++++++- .../tests/testdata/output/cse_expr.yaml | 12 +- src/frontend/src/binder/expr/binary_op.rs | 4 +- src/frontend/src/binder/expr/function.rs | 4 +- src/frontend/src/expr/pure.rs | 4 +- src/tests/regress/data/sql/jsonb.sql | 132 ++++++++-------- 8 files changed, 236 insertions(+), 82 deletions(-) diff --git a/proto/expr.proto b/proto/expr.proto index cab83e0ea45ce..fecefc12b4ee7 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -212,10 +212,14 @@ message ExprNode { // Jsonb functions - // jsonb -> int, jsonb -> text, jsonb #> text[] that returns jsonb - JSONB_ACCESS_INNER = 600; - // jsonb ->> int, jsonb ->> text, jsonb #>> text[] that returns text + // jsonb -> int, jsonb -> text that returns jsonb + JSONB_ACCESS = 600; + // jsonb ->> int, jsonb ->> text that returns text JSONB_ACCESS_STR = 601; + // jsonb #> text[] -> jsonb + JSONB_ACCESS_MULTI = 613; + // jsonb #>> text[] -> text + JSONB_ACCESS_MULTI_STR = 614; JSONB_TYPEOF = 602; JSONB_ARRAY_LENGTH = 603; IS_JSON = 604; diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index be708ac9013a8..664af6c0b1921 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -293,6 +293,16 @@ impl<'a> JsonbRef<'a> { self.0.as_null().is_some() } + /// Returns true if this is a jsonb array. + pub fn is_array(&self) -> bool { + matches!(self.0, ValueRef::Array(_)) + } + + /// Returns true if this is a jsonb object. + pub fn is_object(&self) -> bool { + matches!(self.0, ValueRef::Object(_)) + } + /// Returns the type name of this jsonb. /// /// Possible values are: `null`, `boolean`, `number`, `string`, `array`, `object`. diff --git a/src/expr/impl/src/scalar/jsonb_access.rs b/src/expr/impl/src/scalar/jsonb_access.rs index 8115c1d7214ab..08e36bedf83cc 100644 --- a/src/expr/impl/src/scalar/jsonb_access.rs +++ b/src/expr/impl/src/scalar/jsonb_access.rs @@ -14,15 +14,45 @@ use std::fmt::Write; -use risingwave_common::types::JsonbRef; +use risingwave_common::types::{JsonbRef, ListRef}; use risingwave_expr::function; -#[function("jsonb_access_inner(jsonb, varchar) -> jsonb")] +/// Extracts JSON object field with the given key. +/// +/// `jsonb -> text → jsonb` +/// +/// # Examples +/// +/// ```slt +/// query T +/// select '{"a": {"b":"foo"}}'::jsonb -> 'a'; +/// ---- +/// {"b": "foo"} +/// ``` +#[function("jsonb_access(jsonb, varchar) -> jsonb")] pub fn jsonb_object_field<'a>(v: JsonbRef<'a>, p: &str) -> Option> { v.access_object_field(p) } -#[function("jsonb_access_inner(jsonb, int4) -> jsonb")] +/// Extracts n'th element of JSON array (array elements are indexed from zero, +/// but negative integers count from the end). +/// +/// `jsonb -> integer → jsonb` +/// +/// # Examples +/// +/// ```slt +/// query T +/// select '[{"a":"foo"},{"b":"bar"},{"c":"baz"}]'::jsonb -> 2; +/// ---- +/// {"c": "baz"} +/// +/// query T +/// select '[{"a":"foo"},{"b":"bar"},{"c":"baz"}]'::jsonb -> -3; +/// ---- +/// {"a": "foo"} +/// ``` +#[function("jsonb_access(jsonb, int4) -> jsonb")] pub fn jsonb_array_element(v: JsonbRef<'_>, p: i32) -> Option> { let idx = if p < 0 { let Ok(len) = v.array_len() else { @@ -39,6 +69,59 @@ pub fn jsonb_array_element(v: JsonbRef<'_>, p: i32) -> Option> { v.access_array_element(idx) } +/// Extracts JSON sub-object at the specified path, where path elements can be either field keys or array indexes. +/// +/// `jsonb #> text[] → jsonb` +/// +/// # Examples +/// +/// ```slt +/// query T +/// select '{"a": {"b": ["foo","bar"]}}'::jsonb #> '{a,b,1}'::text[]; +/// ---- +/// "bar" +/// +/// query T +/// select '{"a": {"b": ["foo","bar"]}}'::jsonb #> '{a,b,null}'::text[]; +/// ---- +/// NULL +/// ``` +#[function("jsonb_access_multi(jsonb, varchar[]) -> jsonb")] +pub fn jsonb_access_multi<'a>(v: JsonbRef<'a>, path: ListRef<'_>) -> Option> { + let mut jsonb = v; + for key in path.iter() { + // return null if any element is null + let key = key?.into_utf8(); + if jsonb.is_array() { + // return null if the key is not an integer + let idx = key.parse().ok()?; + jsonb = jsonb_array_element(jsonb, idx)?; + } else if jsonb.is_object() { + jsonb = jsonb_object_field(jsonb, key)?; + } else { + return None; + } + } + Some(jsonb) +} + +/// Extracts JSON object field with the given key, as text. +/// +/// `jsonb ->> text → text` +/// +/// # Examples +/// +/// ```slt +/// query T +/// select '{"a":1,"b":2}'::jsonb ->> 'b'; +/// ---- +/// 2 +/// +/// query T +/// select '{"a":1,"b":null}'::jsonb ->> 'b'; +/// ---- +/// NULL +/// ``` #[function("jsonb_access_str(jsonb, varchar) -> varchar")] pub fn jsonb_object_field_str(v: JsonbRef<'_>, p: &str, writer: &mut impl Write) -> Option<()> { let jsonb = jsonb_object_field(v, p)?; @@ -49,6 +132,23 @@ pub fn jsonb_object_field_str(v: JsonbRef<'_>, p: &str, writer: &mut impl Write) Some(()) } +/// Extracts n'th element of JSON array, as text. +/// +/// `jsonb ->> integer → text` +/// +/// # Examples +/// +/// ```slt +/// query T +/// select '[1,2,3]'::jsonb ->> 2; +/// ---- +/// 3 +/// +/// query T +/// select '[1,2,null]'::jsonb ->> 2; +/// ---- +/// NULL +/// ``` #[function("jsonb_access_str(jsonb, int4) -> varchar")] pub fn jsonb_array_element_str(v: JsonbRef<'_>, p: i32, writer: &mut impl Write) -> Option<()> { let jsonb = jsonb_array_element(v, p)?; @@ -58,3 +158,39 @@ pub fn jsonb_array_element_str(v: JsonbRef<'_>, p: i32, writer: &mut impl Write) jsonb.force_str(writer).unwrap(); Some(()) } + +/// Extracts JSON sub-object at the specified path as text. +/// +/// `jsonb #>> text[] → text` +/// +/// # Examples +/// +/// ```slt +/// query T +/// select '{"a": {"b": ["foo","bar"]}}'::jsonb #>> '{a,b,1}'::text[]; +/// ---- +/// bar +/// +/// query T +/// select '{"a": {"b": ["foo",null]}}'::jsonb #>> '{a,b,1}'::text[]; +/// ---- +/// NULL +/// +/// query T +/// select '{"a": {"b": ["foo","bar"]}}'::jsonb #>> '{a,b,null}'::text[]; +/// ---- +/// NULL +/// ``` +#[function("jsonb_access_multi_str(jsonb, varchar[]) -> varchar")] +pub fn jsonb_access_multi_str( + v: JsonbRef<'_>, + path: ListRef<'_>, + writer: &mut impl Write, +) -> Option<()> { + let jsonb = jsonb_access_multi(v, path)?; + if jsonb.is_jsonb_null() { + return None; + } + jsonb.force_str(writer).unwrap(); + Some(()) +} diff --git a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml index f54f6a837343f..09e0e7872e7c7 100644 --- a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml @@ -5,13 +5,13 @@ select v1->'a'->'c' x, v1->'a'->'b' y from t; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [JsonbAccessInner($expr1, 'c':Varchar) as $expr2, JsonbAccessInner($expr1, 'b':Varchar) as $expr3] } - └─BatchProject { exprs: [t.v1, JsonbAccessInner(t.v1, 'a':Varchar) as $expr1] } + └─BatchProject { exprs: [JsonbAccess($expr1, 'c':Varchar) as $expr2, JsonbAccess($expr1, 'b':Varchar) as $expr3] } + └─BatchProject { exprs: [t.v1, JsonbAccess(t.v1, 'a':Varchar) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [t._row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [JsonbAccessInner($expr1, 'c':Varchar) as $expr2, JsonbAccessInner($expr1, 'b':Varchar) as $expr3, t._row_id] } - └─StreamProject { exprs: [t.v1, JsonbAccessInner(t.v1, 'a':Varchar) as $expr1, t._row_id] } + └─StreamProject { exprs: [JsonbAccess($expr1, 'c':Varchar) as $expr2, JsonbAccess($expr1, 'b':Varchar) as $expr3, t._row_id] } + └─StreamProject { exprs: [t.v1, JsonbAccess(t.v1, 'a':Varchar) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: Common sub expression extract2 sql: | @@ -20,12 +20,12 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [$expr1, $expr1] } - └─BatchProject { exprs: [t.v1, JsonbAccessInner(JsonbAccessInner(t.v1, 'a':Varchar), 'c':Varchar) as $expr1] } + └─BatchProject { exprs: [t.v1, JsonbAccess(JsonbAccess(t.v1, 'a':Varchar), 'c':Varchar) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [t._row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr1, $expr1, t._row_id] } - └─StreamProject { exprs: [t.v1, JsonbAccessInner(JsonbAccessInner(t.v1, 'a':Varchar), 'c':Varchar) as $expr1, t._row_id] } + └─StreamProject { exprs: [t.v1, JsonbAccess(JsonbAccess(t.v1, 'a':Varchar), 'c':Varchar) as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: Common sub expression shouldn't extract impure function sql: | diff --git a/src/frontend/src/binder/expr/binary_op.rs b/src/frontend/src/binder/expr/binary_op.rs index bd85089b18cce..352d2bfbfd246 100644 --- a/src/frontend/src/binder/expr/binary_op.rs +++ b/src/frontend/src/binder/expr/binary_op.rs @@ -89,8 +89,10 @@ impl Binder { BinaryOperator::PGBitwiseXor => ExprType::BitwiseXor, BinaryOperator::PGBitwiseShiftLeft => ExprType::BitwiseShiftLeft, BinaryOperator::PGBitwiseShiftRight => ExprType::BitwiseShiftRight, - BinaryOperator::Arrow => ExprType::JsonbAccessInner, + BinaryOperator::Arrow => ExprType::JsonbAccess, BinaryOperator::LongArrow => ExprType::JsonbAccessStr, + BinaryOperator::HashArrow => ExprType::JsonbAccessMulti, + BinaryOperator::HashLongArrow => ExprType::JsonbAccessMultiStr, BinaryOperator::Prefix => ExprType::StartsWith, BinaryOperator::Contains => ExprType::JsonbContains, BinaryOperator::Contained => ExprType::JsonbContained, diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 7ca509e6e3af9..6dfab2c3bc283 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -871,8 +871,8 @@ impl Binder { // int256 ("hex_to_int256", raw_call(ExprType::HexToInt256)), // jsonb - ("jsonb_object_field", raw_call(ExprType::JsonbAccessInner)), - ("jsonb_array_element", raw_call(ExprType::JsonbAccessInner)), + ("jsonb_object_field", raw_call(ExprType::JsonbAccess)), + ("jsonb_array_element", raw_call(ExprType::JsonbAccess)), ("jsonb_object_field_text", raw_call(ExprType::JsonbAccessStr)), ("jsonb_array_element_text", raw_call(ExprType::JsonbAccessStr)), ("jsonb_typeof", raw_call(ExprType::JsonbTypeof)), diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 038bd94ffcf07..7a63c7f95ae99 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -173,8 +173,10 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::ArrayPosition | expr_node::Type::HexToInt256 | expr_node::Type::JsonbCat - | expr_node::Type::JsonbAccessInner + | expr_node::Type::JsonbAccess | expr_node::Type::JsonbAccessStr + | expr_node::Type::JsonbAccessMulti + | expr_node::Type::JsonbAccessMultiStr | expr_node::Type::JsonbTypeof | expr_node::Type::JsonbArrayLength | expr_node::Type::JsonbObject diff --git a/src/tests/regress/data/sql/jsonb.sql b/src/tests/regress/data/sql/jsonb.sql index 26c25af897bf4..69bd9a928a0f5 100644 --- a/src/tests/regress/data/sql/jsonb.sql +++ b/src/tests/regress/data/sql/jsonb.sql @@ -465,58 +465,58 @@ SELECT jsonb_typeof('"1.0"') AS string; --@ SELECT jsonb_extract_path_text('{"f2":{"f3":1},"f4":[0,1,2,null]}','f4','3') IS NULL AS expect_true; -- extract_path operators ---@ SELECT '{"f2":{"f3":1},"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>array['f4','f6']; ---@ SELECT '{"f2":{"f3":1},"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>array['f2']; ---@ SELECT '{"f2":["f3",1],"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>array['f2','0']; ---@ SELECT '{"f2":["f3",1],"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>array['f2','1']; ---@ ---@ SELECT '{"f2":{"f3":1},"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>>array['f4','f6']; ---@ SELECT '{"f2":{"f3":1},"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>>array['f2']; ---@ SELECT '{"f2":["f3",1],"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>>array['f2','0']; ---@ SELECT '{"f2":["f3",1],"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>>array['f2','1']; ---@ +SELECT '{"f2":{"f3":1},"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>array['f4','f6']; +SELECT '{"f2":{"f3":1},"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>array['f2']; +SELECT '{"f2":["f3",1],"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>array['f2','0']; +SELECT '{"f2":["f3",1],"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>array['f2','1']; + +SELECT '{"f2":{"f3":1},"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>>array['f4','f6']; +SELECT '{"f2":{"f3":1},"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>>array['f2']; +SELECT '{"f2":["f3",1],"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>>array['f2','0']; +SELECT '{"f2":["f3",1],"f4":{"f5":99,"f6":"stringy"}}'::jsonb#>>array['f2','1']; + --@ -- corner cases for same ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #> '{}'; ---@ select '[1,2,3]'::jsonb #> '{}'; ---@ select '"foo"'::jsonb #> '{}'; ---@ select '42'::jsonb #> '{}'; ---@ select 'null'::jsonb #> '{}'; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a']; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a', null]; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a', '']; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a','b']; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a','b','c']; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a','b','c','d']; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a','z','c']; ---@ select '{"a": [{"b": "c"}, {"b": "cc"}]}'::jsonb #> array['a','1','b']; ---@ select '{"a": [{"b": "c"}, {"b": "cc"}]}'::jsonb #> array['a','z','b']; ---@ select '[{"b": "c"}, {"b": "cc"}]'::jsonb #> array['1','b']; ---@ select '[{"b": "c"}, {"b": "cc"}]'::jsonb #> array['z','b']; ---@ select '[{"b": "c"}, {"b": null}]'::jsonb #> array['1','b']; ---@ select '"foo"'::jsonb #> array['z']; ---@ select '42'::jsonb #> array['f2']; ---@ select '42'::jsonb #> array['0']; ---@ ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> '{}'; ---@ select '[1,2,3]'::jsonb #>> '{}'; ---@ select '"foo"'::jsonb #>> '{}'; ---@ select '42'::jsonb #>> '{}'; ---@ select 'null'::jsonb #>> '{}'; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a']; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a', null]; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a', '']; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a','b']; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a','b','c']; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a','b','c','d']; ---@ select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a','z','c']; ---@ select '{"a": [{"b": "c"}, {"b": "cc"}]}'::jsonb #>> array['a','1','b']; ---@ select '{"a": [{"b": "c"}, {"b": "cc"}]}'::jsonb #>> array['a','z','b']; ---@ select '[{"b": "c"}, {"b": "cc"}]'::jsonb #>> array['1','b']; ---@ select '[{"b": "c"}, {"b": "cc"}]'::jsonb #>> array['z','b']; ---@ select '[{"b": "c"}, {"b": null}]'::jsonb #>> array['1','b']; ---@ select '"foo"'::jsonb #>> array['z']; ---@ select '42'::jsonb #>> array['f2']; ---@ select '42'::jsonb #>> array['0']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #> '{}'; +select '[1,2,3]'::jsonb #> '{}'; +select '"foo"'::jsonb #> '{}'; +select '42'::jsonb #> '{}'; +select 'null'::jsonb #> '{}'; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a', null]; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a', '']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a','b']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a','b','c']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a','b','c','d']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #> array['a','z','c']; +select '{"a": [{"b": "c"}, {"b": "cc"}]}'::jsonb #> array['a','1','b']; +select '{"a": [{"b": "c"}, {"b": "cc"}]}'::jsonb #> array['a','z','b']; +select '[{"b": "c"}, {"b": "cc"}]'::jsonb #> array['1','b']; +select '[{"b": "c"}, {"b": "cc"}]'::jsonb #> array['z','b']; +select '[{"b": "c"}, {"b": null}]'::jsonb #> array['1','b']; +select '"foo"'::jsonb #> array['z']; +select '42'::jsonb #> array['f2']; +select '42'::jsonb #> array['0']; + +select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> '{}'; +select '[1,2,3]'::jsonb #>> '{}'; +select '"foo"'::jsonb #>> '{}'; +select '42'::jsonb #>> '{}'; +select 'null'::jsonb #>> '{}'; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a', null]; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a', '']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a','b']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a','b','c']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a','b','c','d']; +select '{"a": {"b":{"c": "foo"}}}'::jsonb #>> array['a','z','c']; +select '{"a": [{"b": "c"}, {"b": "cc"}]}'::jsonb #>> array['a','1','b']; +select '{"a": [{"b": "c"}, {"b": "cc"}]}'::jsonb #>> array['a','z','b']; +select '[{"b": "c"}, {"b": "cc"}]'::jsonb #>> array['1','b']; +select '[{"b": "c"}, {"b": "cc"}]'::jsonb #>> array['z','b']; +select '[{"b": "c"}, {"b": null}]'::jsonb #>> array['1','b']; +select '"foo"'::jsonb #>> array['z']; +select '42'::jsonb #>> array['f2']; +select '42'::jsonb #>> array['0']; -- array_elements --@ SELECT jsonb_array_elements('[1,true,[1,[2,3]],null,{"f1":1,"f2":[7,8,9]},false]'); @@ -1003,21 +1003,21 @@ SELECT '["a","b","c",[1,2],null]'::jsonb -> -5; SELECT '["a","b","c",[1,2],null]'::jsonb -> -6; --nested path extraction ---@ SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{0}'; ---@ SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{a}'; ---@ SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c}'; ---@ SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,0}'; ---@ SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,1}'; ---@ SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,2}'; ---@ SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,3}'; ---@ SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,-1}'; ---@ SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,-3}'; ---@ SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,-4}'; ---@ ---@ SELECT '[0,1,2,[3,4],{"5":"five"}]'::jsonb #> '{0}'; ---@ SELECT '[0,1,2,[3,4],{"5":"five"}]'::jsonb #> '{3}'; ---@ SELECT '[0,1,2,[3,4],{"5":"five"}]'::jsonb #> '{4}'; ---@ SELECT '[0,1,2,[3,4],{"5":"five"}]'::jsonb #> '{4,5}'; +SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{0}'; +SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{a}'; +SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c}'; +SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,0}'; +SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,1}'; +SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,2}'; +SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,3}'; +SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,-1}'; +SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,-3}'; +SELECT '{"a":"b","c":[1,2,3]}'::jsonb #> '{c,-4}'; + +SELECT '[0,1,2,[3,4],{"5":"five"}]'::jsonb #> '{0}'; +SELECT '[0,1,2,[3,4],{"5":"five"}]'::jsonb #> '{3}'; +SELECT '[0,1,2,[3,4],{"5":"five"}]'::jsonb #> '{4}'; +SELECT '[0,1,2,[3,4],{"5":"five"}]'::jsonb #> '{4,5}'; --nested exists SELECT '{"n":null,"a":1,"b":[1,2],"c":{"1":2},"d":{"1":[2,3]}}'::jsonb ? 'n'; From 67056a542e0f85aa318e9a34e091183540c8fe4e Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Mon, 30 Oct 2023 16:10:06 +0800 Subject: [PATCH 45/52] refactor(optimizer): remove some ColIndexMapping::without_tar_size usage (#13134) Co-authored-by: Dylan Chen --- src/common/src/util/column_index_mapping.rs | 32 +++++++------------ .../src/handler/alter_table_column.rs | 2 +- src/frontend/src/optimizer/mod.rs | 2 +- .../optimizer/plan_node/eq_join_predicate.rs | 4 +-- .../src/optimizer/plan_node/generic/agg.rs | 4 +-- .../src/optimizer/plan_node/generic/expand.rs | 2 +- .../src/optimizer/plan_node/generic/join.rs | 2 +- .../optimizer/plan_node/generic/project.rs | 4 +-- .../plan_node/generic/project_set.rs | 4 +-- .../src/optimizer/plan_node/logical_agg.rs | 2 +- .../src/optimizer/plan_node/logical_expand.rs | 5 +-- .../src/optimizer/plan_node/logical_join.rs | 3 +- .../optimizer/plan_node/logical_multi_join.rs | 4 +-- .../src/optimizer/plan_node/logical_now.rs | 5 +-- .../optimizer/plan_node/logical_project.rs | 2 +- .../plan_node/logical_project_set.rs | 2 +- .../src/optimizer/plan_node/logical_scan.rs | 6 ++-- .../src/optimizer/plan_node/logical_source.rs | 2 +- .../rule/apply_join_transpose_rule.rs | 27 ++++++++++------ .../optimizer/rule/apply_offset_rewriter.rs | 9 ++++-- .../rule/push_calculation_of_join_rule.rs | 4 +-- .../optimizer/rule/translate_apply_rule.rs | 3 +- 22 files changed, 63 insertions(+), 67 deletions(-) diff --git a/src/common/src/util/column_index_mapping.rs b/src/common/src/util/column_index_mapping.rs index 49030a06c5001..2c12dc47efb11 100644 --- a/src/common/src/util/column_index_mapping.rs +++ b/src/common/src/util/column_index_mapping.rs @@ -32,19 +32,9 @@ pub struct ColIndexMapping { } impl ColIndexMapping { - /// Create a partial mapping which maps the subscripts range `(0..map.len())` to the - /// corresponding element. **This method is not recommended**, please use `with_target_size` instead, see for more information** - pub fn without_target_size(map: Vec>) -> Self { - let target_size = match map.iter().filter_map(|x| *x).max_by_key(|x| *x) { - Some(target_max) => target_max + 1, - None => 0, - }; - Self { target_size, map } - } - /// Create a partial mapping which maps from the subscripts range `(0..map.len())` to /// `(0..target_size)`. Each subscript is mapped to the corresponding element. - pub fn with_target_size(map: Vec>, target_size: usize) -> Self { + pub fn new(map: Vec>, target_size: usize) -> Self { if let Some(target_max) = map.iter().filter_map(|x| *x).max_by_key(|x| *x) { assert!(target_max < target_size) }; @@ -69,7 +59,7 @@ impl ColIndexMapping { pub fn identity(size: usize) -> Self { let map = (0..size).map(Some).collect(); - Self::without_target_size(map) + Self::new(map, size) } pub fn is_identity(&self) -> bool { @@ -90,12 +80,12 @@ impl ColIndexMapping { let map = (0..source_size) .map(|i| if i < target_size { Some(i) } else { None }) .collect(); - Self::with_target_size(map, target_size) + Self::new(map, target_size) } pub fn empty(source_size: usize, target_size: usize) -> Self { let map = vec![None; source_size]; - Self::with_target_size(map, target_size) + Self::new(map, target_size) } /// Create a partial mapping which maps range `(0..source_num)` to range @@ -134,7 +124,7 @@ impl ColIndexMapping { }) .collect_vec(); let target_size = usize::try_from(source_num as isize + offset).unwrap(); - Self::with_target_size(map, target_size) + Self::new(map, target_size) } /// Maps the smallest index to 0, the next smallest to 1, and so on. @@ -159,7 +149,7 @@ impl ColIndexMapping { for (tar, &src) in cols.iter().enumerate() { map[src] = Some(tar); } - Self::without_target_size(map) + Self::new(map, cols.len()) } // TODO(yuchao): isn't this the same as `with_remaining_columns`? @@ -170,7 +160,7 @@ impl ColIndexMapping { map[src] = Some(tar); } } - Self::without_target_size(map) + Self::new(map, cols.len()) } /// Remove the given columns, and maps the remaining columns to a consecutive range starting @@ -205,7 +195,7 @@ impl ColIndexMapping { for target in &mut map { *target = target.and_then(|index| following.try_map(index)); } - Self::with_target_size(map, following.target_size()) + Self::new(map, following.target_size()) } pub fn clone_with_offset(&self, offset: usize) -> Self { @@ -213,7 +203,7 @@ impl ColIndexMapping { for target in &mut map { *target = target.and_then(|index| index.checked_add(offset)); } - Self::with_target_size(map, self.target_size() + offset) + Self::new(map, self.target_size() + offset) } /// Union two mapping, the result mapping `target_size` and source size will be the max size @@ -236,7 +226,7 @@ impl ColIndexMapping { assert_eq!(map[src], None); map[src] = Some(dst); } - Self::with_target_size(map, target_size) + Self::new(map, target_size) } /// Inverse the mapping. If a target corresponds to more than one source, return `None`. @@ -249,7 +239,7 @@ impl ColIndexMapping { } map[dst] = Some(src); } - Some(Self::with_target_size(map, self.source_size())) + Some(Self::new(map, self.source_size())) } /// return iter of (src, dst) order by src diff --git a/src/frontend/src/handler/alter_table_column.rs b/src/frontend/src/handler/alter_table_column.rs index 18313e0458a04..1b3babc41ceaf 100644 --- a/src/frontend/src/handler/alter_table_column.rs +++ b/src/frontend/src/handler/alter_table_column.rs @@ -240,7 +240,7 @@ pub async fn handle_alter_table_column( }; // Calculate the mapping from the original columns to the new columns. - let col_index_mapping = ColIndexMapping::with_target_size( + let col_index_mapping = ColIndexMapping::new( original_catalog .columns() .iter() diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 4004748a2f4f9..20da006dcc992 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -378,7 +378,7 @@ impl PlanRoot { } let plan = LogicalProject::with_out_col_idx(plan, output_indices.into_iter()); - let out_col_change = ColIndexMapping::with_target_size(map, target_size); + let out_col_change = ColIndexMapping::new(map, target_size); (plan.into(), out_col_change) } }; diff --git a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs index 0811bc0531a25..2c714bea0f7de 100644 --- a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs +++ b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs @@ -238,7 +238,7 @@ impl EqJoinPredicate { for (left, right, _) in self.eq_keys() { map[right.index - left_cols_num] = Some(left.index); } - ColIndexMapping::with_target_size(map, left_cols_num) + ColIndexMapping::new(map, left_cols_num) } /// return the eq columns index mapping from left inputs to right inputs @@ -251,7 +251,7 @@ impl EqJoinPredicate { for (left, right, _) in self.eq_keys() { map[left.index] = Some(right.index - left_cols_num); } - ColIndexMapping::with_target_size(map, right_cols_num) + ColIndexMapping::new(map, right_cols_num) } /// Reorder the `eq_keys` according to the `reorder_idx`. diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index c7a0ea4f39389..18ac5525182f3 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -74,7 +74,7 @@ impl Agg { for (i, key) in self.group_key.indices().enumerate() { map[i] = Some(key); } - ColIndexMapping::with_target_size(map, self.input.schema().len()) + ColIndexMapping::new(map, self.input.schema().len()) } /// get the Mapping of columnIndex from input column index to out column index @@ -83,7 +83,7 @@ impl Agg { for (i, key) in self.group_key.indices().enumerate() { map[key] = Some(i); } - ColIndexMapping::with_target_size(map, self.output_len()) + ColIndexMapping::new(map, self.output_len()) } fn two_phase_agg_forced(&self) -> bool { diff --git a/src/frontend/src/optimizer/plan_node/generic/expand.rs b/src/frontend/src/optimizer/plan_node/generic/expand.rs index d346677ac2d62..426589d3983e6 100644 --- a/src/frontend/src/optimizer/plan_node/generic/expand.rs +++ b/src/frontend/src/optimizer/plan_node/generic/expand.rs @@ -134,7 +134,7 @@ impl Expand { let map = (0..input_len) .map(|source| Some(source + input_len)) .collect_vec(); - ColIndexMapping::with_target_size(map, self.output_len()) + ColIndexMapping::new(map, self.output_len()) } pub fn o2i_col_mapping(&self) -> ColIndexMapping { diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index 2536cee984558..ee8c43b388684 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -431,7 +431,7 @@ impl Join { pub fn o2i_col_mapping(&self) -> ColIndexMapping { // If output_indices = [0, 0, 1], we should use it as `o2i_col_mapping` directly. // If we use `self.i2o_col_mapping().inverse()`, we will lose the first 0. - ColIndexMapping::with_target_size( + ColIndexMapping::new( self.output_indices.iter().map(|x| Some(*x)).collect(), self.internal_column_num(), ) diff --git a/src/frontend/src/optimizer/plan_node/generic/project.rs b/src/frontend/src/optimizer/plan_node/generic/project.rs index d8b6988af4391..e58ac40e918a8 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project.rs @@ -210,7 +210,7 @@ impl Project { map[i] = Some(input.index()) } } - ColIndexMapping::with_target_size(map, input_len) + ColIndexMapping::new(map, input_len) } /// get the Mapping of columnIndex from input column index to output column index,if a input @@ -224,7 +224,7 @@ impl Project { map[input.index()] = Some(i) } } - ColIndexMapping::with_target_size(map, exprs.len()) + ColIndexMapping::new(map, exprs.len()) } pub fn is_all_inputref(&self) -> bool { diff --git a/src/frontend/src/optimizer/plan_node/generic/project_set.rs b/src/frontend/src/optimizer/plan_node/generic/project_set.rs index fef26d1b32993..195b420802076 100644 --- a/src/frontend/src/optimizer/plan_node/generic/project_set.rs +++ b/src/frontend/src/optimizer/plan_node/generic/project_set.rs @@ -122,7 +122,7 @@ impl ProjectSet { map[1 + i] = Some(input.index()) } } - ColIndexMapping::with_target_size(map, input_len) + ColIndexMapping::new(map, input_len) } /// Gets the Mapping of columnIndex from input column index to output column index,if a input @@ -135,7 +135,7 @@ impl ProjectSet { map[input.index()] = Some(1 + i) } } - ColIndexMapping::with_target_size(map, 1 + self.select_list.len()) + ColIndexMapping::new(map, 1 + self.select_list.len()) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 0aed32abec40e..b0e04f0598778 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -1214,7 +1214,7 @@ impl ToStream for LogicalAgg { let (input, input_col_change) = self.input().logical_rewrite_for_stream(ctx)?; let (agg, out_col_change) = self.rewrite_with_input(input, input_col_change); let (map, _) = out_col_change.into_parts(); - let out_col_change = ColIndexMapping::with_target_size(map, agg.schema().len()); + let out_col_change = ColIndexMapping::new(map, agg.schema().len()); Ok((agg.into(), out_col_change)) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_expand.rs b/src/frontend/src/optimizer/plan_node/logical_expand.rs index 054621ef967bf..77388ee33cf4a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_expand.rs +++ b/src/frontend/src/optimizer/plan_node/logical_expand.rs @@ -121,10 +121,7 @@ impl PlanTreeNodeUnary for LogicalExpand { let expand = Self::new(input, column_subsets); let output_col_num = expand.schema().len(); - ( - expand, - ColIndexMapping::with_target_size(mapping, output_col_num), - ) + (expand, ColIndexMapping::new(mapping, output_col_num)) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index a928481230d3c..c594ededa40cf 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -518,8 +518,7 @@ impl PlanTreeNodeBinary for LogicalJoin { *i += left.schema().len(); } map.append(&mut right_map); - let mut mapping = - ColIndexMapping::with_target_size(map, left.schema().len() + right.schema().len()); + let mut mapping = ColIndexMapping::new(map, left.schema().len() + right.schema().len()); let new_output_indices = self .output_indices() diff --git a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs index 819f84e963cfd..0bbe59dd70754 100644 --- a/src/frontend/src/optimizer/plan_node/logical_multi_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_multi_join.rs @@ -87,7 +87,7 @@ impl LogicalMultiJoinBuilder { /// add a predicate above the plan, so they will be rewritten from the `output_indices` to the /// input indices pub fn add_predicate_above(&mut self, exprs: impl Iterator) { - let mut mapping = ColIndexMapping::with_target_size( + let mut mapping = ColIndexMapping::new( self.output_indices.iter().map(|i| Some(*i)).collect(), self.tot_input_col_num, ); @@ -240,7 +240,7 @@ impl LogicalMultiJoin { i2o_maps .into_iter() - .map(|map| ColIndexMapping::with_target_size(map, tot_col_num)) + .map(|map| ColIndexMapping::new(map, tot_col_num)) .collect_vec() }; diff --git a/src/frontend/src/optimizer/plan_node/logical_now.rs b/src/frontend/src/optimizer/plan_node/logical_now.rs index c13a0c93a6e70..9e2d586c0fd2a 100644 --- a/src/frontend/src/optimizer/plan_node/logical_now.rs +++ b/src/frontend/src/optimizer/plan_node/logical_now.rs @@ -81,10 +81,7 @@ impl ToStream for LogicalNow { &self, _ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping)> { - Ok(( - self.clone().into(), - ColIndexMapping::with_target_size(vec![Some(0)], 1), - )) + Ok((self.clone().into(), ColIndexMapping::new(vec![Some(0)], 1))) } /// `to_stream` is equivalent to `to_stream_with_dist_required(RequiredDist::Any)` diff --git a/src/frontend/src/optimizer/plan_node/logical_project.rs b/src/frontend/src/optimizer/plan_node/logical_project.rs index 59a5509ebcd70..aee136e57ca9c 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project.rs @@ -284,7 +284,7 @@ impl ToStream for LogicalProject { // But the target size of `out_col_change` should be the same as the length of the new // schema. let (map, _) = out_col_change.into_parts(); - let out_col_change = ColIndexMapping::with_target_size(map, proj.base.schema().len()); + let out_col_change = ColIndexMapping::new(map, proj.base.schema().len()); Ok((proj.into(), out_col_change)) } } diff --git a/src/frontend/src/optimizer/plan_node/logical_project_set.rs b/src/frontend/src/optimizer/plan_node/logical_project_set.rs index fc894713c2f43..727c641d52974 100644 --- a/src/frontend/src/optimizer/plan_node/logical_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/logical_project_set.rs @@ -385,7 +385,7 @@ impl ToStream for LogicalProjectSet { // But the target size of `out_col_change` should be the same as the length of the new // schema. let (map, _) = out_col_change.into_parts(); - let out_col_change = ColIndexMapping::with_target_size(map, project_set.schema().len()); + let out_col_change = ColIndexMapping::new(map, project_set.schema().len()); Ok((project_set.into(), out_col_change)) } diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 200302ae70f97..c9f5494b86be6 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -233,7 +233,7 @@ impl LogicalScan { } let mut inverse_mapping = { - let mapping = ColIndexMapping::with_target_size( + let mapping = ColIndexMapping::new( self.required_col_idx().iter().map(|i| Some(*i)).collect(), self.table_desc().columns.len(), ); @@ -242,7 +242,7 @@ impl LogicalScan { for (src, dst) in mapping.mapping_pairs() { inverse_map[dst] = Some(src); } - ColIndexMapping::with_target_size(inverse_map, mapping.source_size()) + ColIndexMapping::new(inverse_map, mapping.source_size()) }; predicate = predicate.rewrite_expr(&mut inverse_mapping); @@ -412,7 +412,7 @@ impl PredicatePushdown for LogicalScan { .conjunctions .extract_if(|expr| expr.count_nows() > 0 || HasCorrelated {}.visit_expr(expr)) .collect(); - let predicate = predicate.rewrite_expr(&mut ColIndexMapping::with_target_size( + let predicate = predicate.rewrite_expr(&mut ColIndexMapping::new( self.output_col_idx().iter().map(|i| Some(*i)).collect(), self.table_desc().columns.len(), )); diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 01166e74f1359..cac051957b0a5 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -127,7 +127,7 @@ impl LogicalSource { mapping[idx] = None; } } - ColIndexMapping::with_target_size(mapping, columns.len()) + ColIndexMapping::new(mapping, columns.len()) }; let mut rewriter = IndexRewriter::new(col_mapping); diff --git a/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs index 7ac121692c81d..98363f2aba213 100644 --- a/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_join_transpose_rule.rs @@ -356,7 +356,7 @@ impl ApplyJoinTransposeRule { correlated_indices, false, ); - let output_indices: Vec<_> = { + let (output_indices, target_size) = { let (apply_left_len, join_right_len) = match apply_join_type { JoinType::LeftSemi | JoinType::LeftAnti => (apply_left_len, 0), JoinType::RightSemi | JoinType::RightAnti => (0, join.right().schema().len()), @@ -368,14 +368,19 @@ impl ApplyJoinTransposeRule { join_left_len + apply_left_len..join_left_len + apply_left_len + join_right_len, ); - match join.join_type() { + let output_indices: Vec<_> = match join.join_type() { JoinType::LeftSemi | JoinType::LeftAnti => left_iter.collect(), JoinType::RightSemi | JoinType::RightAnti => right_iter.collect(), _ => left_iter.chain(right_iter).collect(), - } + }; + + let target_size = join_left_len + apply_left_len + join_right_len; + (output_indices, target_size) }; - let mut output_indices_mapping = - ColIndexMapping::without_target_size(output_indices.iter().map(|x| Some(*x)).collect()); + let mut output_indices_mapping = ColIndexMapping::new( + output_indices.iter().map(|x| Some(*x)).collect(), + target_size, + ); let new_join = LogicalJoin::new( join.left(), new_join_right, @@ -518,7 +523,7 @@ impl ApplyJoinTransposeRule { false, ); - let output_indices: Vec<_> = { + let (output_indices, target_size) = { let (apply_left_len, join_right_len) = match apply_join_type { JoinType::LeftSemi | JoinType::LeftAnti => (apply_left_len, 0), JoinType::RightSemi | JoinType::RightAnti => (0, join.right().schema().len()), @@ -529,11 +534,14 @@ impl ApplyJoinTransposeRule { let right_iter = join_left_len + apply_left_len * 2 ..join_left_len + apply_left_len * 2 + join_right_len; - match join.join_type() { + let output_indices: Vec<_> = match join.join_type() { JoinType::LeftSemi | JoinType::LeftAnti => left_iter.collect(), JoinType::RightSemi | JoinType::RightAnti => right_iter.collect(), _ => left_iter.chain(right_iter).collect(), - } + }; + + let target_size = join_left_len + apply_left_len * 2 + join_right_len; + (output_indices, target_size) }; let new_join = LogicalJoin::new( new_join_left, @@ -548,8 +556,9 @@ impl ApplyJoinTransposeRule { new_join.into() } JoinType::Inner | JoinType::LeftOuter | JoinType::RightOuter | JoinType::FullOuter => { - let mut output_indices_mapping = ColIndexMapping::without_target_size( + let mut output_indices_mapping = ColIndexMapping::new( output_indices.iter().map(|x| Some(*x)).collect(), + target_size, ); // Leave other condition for predicate push down to deal with LogicalFilter::create( diff --git a/src/frontend/src/optimizer/rule/apply_offset_rewriter.rs b/src/frontend/src/optimizer/rule/apply_offset_rewriter.rs index 4a18f1ba37e44..559ace9d5dbe0 100644 --- a/src/frontend/src/optimizer/rule/apply_offset_rewriter.rs +++ b/src/frontend/src/optimizer/rule/apply_offset_rewriter.rs @@ -74,13 +74,18 @@ pub struct ApplyCorrelatedIndicesConverter {} impl ApplyCorrelatedIndicesConverter { pub fn convert_to_index_mapping(correlated_indices: &[usize]) -> ColIndexMapping { // Inverse anyway. - let col_mapping = ColIndexMapping::without_target_size( + let target_size = match correlated_indices.iter().max_by_key(|&&x| x) { + Some(target_max) => target_max + 1, + None => 0, + }; + let col_mapping = ColIndexMapping::new( correlated_indices.iter().copied().map(Some).collect_vec(), + target_size, ); let mut map = vec![None; col_mapping.target_size()]; for (src, dst) in col_mapping.mapping_pairs() { map[dst] = Some(src); } - ColIndexMapping::with_target_size(map, col_mapping.source_size()) + ColIndexMapping::new(map, col_mapping.source_size()) } } diff --git a/src/frontend/src/optimizer/rule/push_calculation_of_join_rule.rs b/src/frontend/src/optimizer/rule/push_calculation_of_join_rule.rs index ce5004b069a31..a4cdb78990a77 100644 --- a/src/frontend/src/optimizer/rule/push_calculation_of_join_rule.rs +++ b/src/frontend/src/optimizer/rule/push_calculation_of_join_rule.rs @@ -63,7 +63,7 @@ impl Rule for PushCalculationOfJoinRule { ) .map(Some) .collect_vec(); - ColIndexMapping::with_target_size(map, new_internal_col_num) + ColIndexMapping::new(map, new_internal_col_num) }; let (mut exprs, new_output_indices) = Self::remap_exprs_and_output_indices(exprs, output_indices, &mut col_index_mapping); @@ -82,7 +82,7 @@ impl Rule for PushCalculationOfJoinRule { .map(|i| i + left_col_num + left_exprs_non_input_ref.len()) .map(Some) .collect_vec(); - ColIndexMapping::with_target_size(map, new_internal_col_num) + ColIndexMapping::new(map, new_internal_col_num) }; // replace chosen function calls. for (((index_of_func_call, ty), left_expr), right_expr) in indices_and_ty_of_func_calls diff --git a/src/frontend/src/optimizer/rule/translate_apply_rule.rs b/src/frontend/src/optimizer/rule/translate_apply_rule.rs index aee6a2a06e1a9..ec6f58a2422d9 100644 --- a/src/frontend/src/optimizer/rule/translate_apply_rule.rs +++ b/src/frontend/src/optimizer/rule/translate_apply_rule.rs @@ -59,8 +59,7 @@ impl Rule for TranslateApplyRule { let apply_left_len = left.schema().len(); let correlated_indices = apply.correlated_indices(); - let mut index_mapping = - ColIndexMapping::with_target_size(vec![None; apply_left_len], apply_left_len); + let mut index_mapping = ColIndexMapping::new(vec![None; apply_left_len], apply_left_len); let mut data_types = HashMap::new(); let mut index = 0; From 85274f1dc323d1980c18d869f27a17a4d49bb116 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Mon, 30 Oct 2023 16:14:49 +0800 Subject: [PATCH 46/52] refactor(source): remove `upsert_avro_primary_key ` and separate extracting columns and bind pk (#13023) --- e2e_test/source/basic/ddl.slt | 2 +- e2e_test/source/basic/nosim_kafka.slt | 78 +-- .../basic/old_row_format_syntax/ddl.slt | 2 +- .../old_row_format_syntax/nosim_kafka.slt | 81 +-- proto/catalog.proto | 3 +- src/connector/src/parser/avro/parser.rs | 10 +- src/connector/src/parser/mod.rs | 2 - src/connector/src/parser/upsert_parser.rs | 24 +- .../tests/testdata/output/create_source.yaml | 2 +- src/frontend/src/handler/create_sink.rs | 2 +- src/frontend/src/handler/create_source.rs | 578 +++++++++--------- src/frontend/src/handler/create_table.rs | 32 +- src/sqlparser/src/ast/ddl.rs | 6 + 13 files changed, 391 insertions(+), 431 deletions(-) diff --git a/e2e_test/source/basic/ddl.slt b/e2e_test/source/basic/ddl.slt index c1941d4697ffa..c6c1c0590d558 100644 --- a/e2e_test/source/basic/ddl.slt +++ b/e2e_test/source/basic/ddl.slt @@ -28,7 +28,7 @@ create source invalid_startup_timestamp ( properties.bootstrap.server = 'message_queue:29092' ) FORMAT PLAIN ENCODE JSON; -statement error db error: ERROR: QueryError: Invalid input syntax: schema definition is required for ENCODE JSON +statement error db error: ERROR: QueryError: Protocol error: Schema definition is required, either from SQL or schema registry. create source invalid_schema_definition with ( connector = 'kafka', diff --git a/e2e_test/source/basic/nosim_kafka.slt b/e2e_test/source/basic/nosim_kafka.slt index 945f60e732fb8..beafb02cb96ba 100644 --- a/e2e_test/source/basic/nosim_kafka.slt +++ b/e2e_test/source/basic/nosim_kafka.slt @@ -30,26 +30,17 @@ WITH ( FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); -statement ok -CREATE TABLE upsert_avro_json ( - PRIMARY KEY("ID") -) -WITH ( -connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', -topic = 'upsert_avro_json') -FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); +# TODO: Uncomment this when we add test data kafka key with format `"ID":id` +# statement ok +# CREATE TABLE upsert_avro_json ( +# PRIMARY KEY("ID") +# ) +# WITH ( +# connector = 'kafka', +# properties.bootstrap.server = 'message_queue:29092', +# topic = 'upsert_avro_json') +# FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); -# Just ignore the kafka key, it works -statement ok -CREATE TABLE upsert_avro_json2 ( - PRIMARY KEY("ID") -) -WITH ( - connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', - topic = 'upsert_avro_json') -FORMAT UPSERT ENCODE AVRO (schema.registry = 'http://message_queue:8081'); statement ok CREATE TABLE debezium_non_compact (PRIMARY KEY(order_id)) with ( @@ -89,7 +80,7 @@ statement ok flush; # Wait enough time to ensure SourceExecutor consumes all Kafka data. -sleep 5s +sleep 8s query II SELECT @@ -104,33 +95,18 @@ delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -query II -SELECT - * -FROM - upsert_avro_json -ORDER BY - "ID"; ----- -update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z -delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z - - -query II -SELECT - * -FROM - upsert_avro_json2 -ORDER BY - "ID"; ----- -update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z -delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z - +# query II +# SELECT +# * +# FROM +# upsert_avro_json +# ORDER BY +# "ID"; +# ---- +# update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z +# delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z +# delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z +# delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z query II SELECT @@ -172,12 +148,8 @@ select * from kafka_json_schema_upsert order by id statement ok DROP TABLE upsert_avro_json_default_key; -statement ok -DROP TABLE upsert_avro_json; - - -statement ok -DROP TABLE upsert_avro_json2; +# statement ok +# DROP TABLE upsert_avro_json; statement ok DROP TABLE upsert_student_avro_json; diff --git a/e2e_test/source/basic/old_row_format_syntax/ddl.slt b/e2e_test/source/basic/old_row_format_syntax/ddl.slt index d0a8cd9ba08ea..6d1290463beeb 100644 --- a/e2e_test/source/basic/old_row_format_syntax/ddl.slt +++ b/e2e_test/source/basic/old_row_format_syntax/ddl.slt @@ -28,7 +28,7 @@ create source invalid_startup_timestamp ( properties.bootstrap.server = 'message_queue:29092' ) ROW FORMAT JSON; -statement error db error: ERROR: QueryError: Invalid input syntax: schema definition is required for ENCODE JSON +statement error db error: ERROR: QueryError: Protocol error: Schema definition is required, either from SQL or schema registry. create source invalid_schema_definition with ( connector = 'kafka', diff --git a/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt b/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt index 582aff7d958fb..37e2ef2266ff0 100644 --- a/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt +++ b/e2e_test/source/basic/old_row_format_syntax/nosim_kafka.slt @@ -33,28 +33,17 @@ ROW FORMAT UPSERT_AVRO row schema location confluent schema registry 'http://message_queue:8081' -statement ok -CREATE TABLE upsert_avro_json ( - PRIMARY KEY("ID") -) -WITH ( -connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', -topic = 'upsert_avro_json') -ROW FORMAT UPSERT_AVRO -row schema location confluent schema registry 'http://message_queue:8081' - -# Just ignore the kafka key, it works -statement ok -CREATE TABLE upsert_avro_json2 ( - PRIMARY KEY("ID") -) -WITH ( - connector = 'kafka', - properties.bootstrap.server = 'message_queue:29092', - topic = 'upsert_avro_json') -ROW FORMAT UPSERT_AVRO -row schema location confluent schema registry 'http://message_queue:8081' +# TODO: Uncomment this when we add test data kafka key with format `"ID":id` +# statement ok +# CREATE TABLE upsert_avro_json ( +# PRIMARY KEY("ID") +# ) +# WITH ( +# connector = 'kafka', +# properties.bootstrap.server = 'message_queue:29092', +# topic = 'upsert_avro_json') +# ROW FORMAT UPSERT_AVRO +# row schema location confluent schema registry 'http://message_queue:8081' statement ok CREATE TABLE debezium_non_compact (PRIMARY KEY(order_id)) with ( @@ -65,7 +54,6 @@ CREATE TABLE debezium_non_compact (PRIMARY KEY(order_id)) with ( ) ROW FORMAT DEBEZIUM_AVRO ROW SCHEMA LOCATION CONFLUENT SCHEMA REGISTRY 'http://message_queue:8081'; - statement ok CREATE TABLE debezium_compact (PRIMARY KEY(order_id)) with ( connector = 'kafka', @@ -78,7 +66,7 @@ statement ok flush; # Wait enough time to ensure SourceExecutor consumes all Kafka data. -sleep 5s +sleep 10s query II SELECT @@ -93,32 +81,18 @@ delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -query II -SELECT - * -FROM - upsert_avro_json -ORDER BY - "ID"; ----- -update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z -delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z - - -query II -SELECT - * -FROM - upsert_avro_json2 -ORDER BY - "ID"; ----- -update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z -delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z -delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z +# query II +# SELECT +# * +# FROM +# upsert_avro_json +# ORDER BY +# "ID"; +# ---- +# update id1 -1 6768 6970 value9 7172 info9 2021-05-18T07:59:58.714Z +# delete id2 2 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z +# delete id3 3 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z +# delete id5 5 7778 7980 value10 8182 info10 2021-05-19T15:22:45.539Z query II @@ -150,12 +124,9 @@ select count(*) from debezium_compact; statement ok DROP TABLE upsert_avro_json_default_key; -statement ok -DROP TABLE upsert_avro_json; - +# statement ok +# DROP TABLE upsert_avro_json; -statement ok -DROP TABLE upsert_avro_json2; statement ok DROP TABLE upsert_student_avro_json; diff --git a/proto/catalog.proto b/proto/catalog.proto index c966b7bbe5eb0..4f421305c4eea 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -55,7 +55,8 @@ message StreamSourceInfo { string proto_message_name = 4; int32 csv_delimiter = 5; bool csv_has_header = 6; - string upsert_avro_primary_key = 7; + reserved 7; + reserved "upsert_avro_primary_key"; // deprecated plan_common.FormatType format = 8; plan_common.EncodeType row_encode = 9; SchemaRegistryNameStrategy name_strategy = 10; diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index e02bcca7fe2ae..61c55decfc040 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -108,7 +108,6 @@ pub struct AvroParserConfig { pub schema: Arc, pub key_schema: Option>, pub schema_resolver: Option>, - pub upsert_primary_key_column_name: Option, } impl AvroParserConfig { @@ -120,12 +119,7 @@ impl AvroParserConfig { if avro_config.use_schema_registry { let client = Client::new(url, &avro_config.client_config)?; let resolver = ConfluentSchemaResolver::new(client); - let upsert_primary_key_column_name = - if enable_upsert && !avro_config.upsert_primary_key.is_empty() { - Some(avro_config.upsert_primary_key.clone()) - } else { - None - }; + let subject_key = if enable_upsert { Some(get_subject_by_strategy( &avro_config.name_strategy, @@ -157,7 +151,6 @@ impl AvroParserConfig { None }, schema_resolver: Some(Arc::new(resolver)), - upsert_primary_key_column_name, }) } else { if enable_upsert { @@ -184,7 +177,6 @@ impl AvroParserConfig { schema: Arc::new(schema), key_schema: None, schema_resolver: None, - upsert_primary_key_column_name: None, }) } } diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 584e0e497d1f3..c7b8bf702e1cc 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -794,7 +794,6 @@ impl SpecificParserConfig { pub struct AvroProperties { pub use_schema_registry: bool, pub row_schema_location: String, - pub upsert_primary_key: String, pub client_config: SchemaRegistryAuth, pub aws_auth_props: Option, pub topic: String, @@ -895,7 +894,6 @@ impl SpecificParserConfig { .unwrap(), use_schema_registry: info.use_schema_registry, row_schema_location: info.row_schema_location.clone(), - upsert_primary_key: info.upsert_avro_primary_key.clone(), ..Default::default() }; if format == SourceFormat::Upsert { diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index 214775851103b..f9ce0caa7e254 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -34,7 +34,6 @@ pub struct UpsertParser { payload_builder: AccessBuilderImpl, pub(crate) rw_columns: Vec, source_ctx: SourceContextRef, - avro_primary_key_column_name: Option, } async fn build_accessor_builder( @@ -68,23 +67,18 @@ impl UpsertParser { rw_columns: Vec, source_ctx: SourceContextRef, ) -> Result { - let mut avro_primary_key_column_name = None; - let key_builder: AccessBuilderImpl; // check whether columns has `DEFAULT_KEY_COLUMN_NAME`, if so, the key accessor should be // bytes - if check_rw_default_key(&rw_columns) { - key_builder = AccessBuilderImpl::Bytes(BytesAccessBuilder::new( - EncodingProperties::Bytes(BytesProperties { + let key_builder = if check_rw_default_key(&rw_columns) { + AccessBuilderImpl::Bytes(BytesAccessBuilder::new(EncodingProperties::Bytes( + BytesProperties { column_name: Some(DEFAULT_KEY_COLUMN_NAME.into()), - }), - )?); + }, + ))?) } else { - if let EncodingProperties::Avro(config) = &props.encoding_config { - avro_primary_key_column_name = Some(config.upsert_primary_key.clone()) - } let (key_config, key_type) = extract_key_config!(props); - key_builder = build_accessor_builder(key_config, key_type).await?; - } + build_accessor_builder(key_config, key_type).await? + }; let payload_builder = build_accessor_builder(props.encoding_config, EncodingType::Value).await?; Ok(Self { @@ -92,7 +86,6 @@ impl UpsertParser { payload_builder, rw_columns, source_ctx, - avro_primary_key_column_name, }) } @@ -113,9 +106,6 @@ impl UpsertParser { row_op = row_op.with_value(self.payload_builder.generate_accessor(data).await?); change_event_op = ChangeEventOperation::Upsert; } - if let Some(primary_key_name) = &self.avro_primary_key_column_name { - row_op = row_op.with_key_as_column_name(primary_key_name); - } apply_row_operation_on_stream_chunk_writer_with_op(row_op, &mut writer, change_event_op) .map_err(Into::into) diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index d1d6c314595b8..50178ccb1c6e5 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -10,7 +10,7 @@ - id: create_source_without_schema_in_json sql: | create source s with(connector='kafka') FORMAT PLAIN ENCODE JSON; - planner_error: 'Invalid input syntax: schema definition is required for ENCODE JSON' + planner_error: 'Protocol error: Schema definition is required, either from SQL or schema registry.' - id: csv_delimiter_tab sql: | explain create table s0 (v1 int, v2 varchar) with ( diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 85d867d538e7f..e4081fbee4fcf 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -226,7 +226,7 @@ pub async fn handle_create_sink( } /// Transforms the (format, encode, options) from sqlparser AST into an internal struct `SinkFormatDesc`. -/// This is an analogy to (part of) [`crate::handler::create_source::try_bind_columns_from_source`] +/// This is an analogy to (part of) [`crate::handler::create_source::bind_columns_from_source`] /// which transforms sqlparser AST `SourceSchemaV2` into `StreamSourceInfo`. fn bind_sink_format_desc(value: ConnectorSchema) -> Result { use risingwave_connector::sink::catalog::{SinkEncode, SinkFormat}; diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 0ce3e32ed584e..8bca367351641 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -45,8 +45,8 @@ use risingwave_pb::catalog::{ }; use risingwave_pb::plan_common::{EncodeType, FormatType}; use risingwave_sqlparser::ast::{ - self, get_delimiter, AstString, AvroSchema, ColumnDef, ColumnOption, ConnectorSchema, - CreateSourceStatement, DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceWatermark, + get_delimiter, AstString, AvroSchema, ColumnDef, ConnectorSchema, CreateSourceStatement, + DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceWatermark, }; use super::RwPgResponse; @@ -54,7 +54,7 @@ use crate::binder::Binder; use crate::catalog::ColumnId; use crate::expr::Expr; use crate::handler::create_table::{ - bind_pk_names, bind_pk_on_relation, bind_sql_column_constraints, bind_sql_columns, + bind_pk_on_relation, bind_sql_column_constraints, bind_sql_columns, bind_sql_pk_names, ensure_table_constraints_supported, ColumnIdGenerator, }; use crate::handler::util::{get_connector, is_kafka_connector}; @@ -110,45 +110,35 @@ async fn extract_avro_table_schema( .collect_vec()) } -/// Map an Avro schema to a relational schema. And extract primary key columns. -async fn extract_upsert_avro_table_schema( +/// Extract Avro primary key columns. +async fn extract_upsert_avro_table_pk_columns( info: &StreamSourceInfo, with_properties: &HashMap, -) -> Result<(Vec, Vec)> { +) -> Result>> { let parser_config = SpecificParserConfig::new(info, with_properties)?; let conf = AvroParserConfig::new(parser_config.encoding_config).await?; let vec_column_desc = conf.map_to_columns()?; - let mut vec_column_catalog = vec_column_desc - .clone() - .into_iter() - .map(|col| ColumnCatalog { - column_desc: col.into(), - is_hidden: false, - }) - .collect_vec(); - // For upsert avro, if we can't extract pk from schema, use message key as primary key - let pks = if let Ok(pk_desc) = conf.extract_pks() { - pk_desc - .into_iter() - .map(|desc| { - vec_column_desc - .iter() - .find(|x| x.name == desc.name) - .ok_or_else(|| { - RwError::from(ErrorCode::InternalError(format!( - "Can not found primary key column {} in value schema", - desc.name - ))) - }) - }) - .map_ok(|desc| desc.name.clone()) - .collect::>>()? - } else { - add_upsert_default_key_column(&mut vec_column_catalog); - vec![DEFAULT_KEY_COLUMN_NAME.into()] - }; - Ok((vec_column_catalog, pks)) + conf.extract_pks() + .ok() + .map(|pk_desc| { + pk_desc + .into_iter() + .map(|desc| { + vec_column_desc + .iter() + .find(|x| x.name == desc.name) + .ok_or_else(|| { + RwError::from(ErrorCode::InternalError(format!( + "Can not found primary key column {} in value schema", + desc.name + ))) + }) + }) + .map_ok(|desc| desc.name.clone()) + .collect::>>() + }) + .transpose() } async fn extract_debezium_avro_table_pk_columns( @@ -208,11 +198,7 @@ async fn extract_protobuf_table_schema( fn non_generated_sql_columns(columns: &[ColumnDef]) -> Vec { columns .iter() - .filter(|c| { - c.options - .iter() - .all(|option| !matches!(option.option, ColumnOption::GeneratedColumns(_))) - }) + .filter(|c| !c.is_generated()) .cloned() .collect() } @@ -275,19 +261,15 @@ fn get_name_strategy_or_default(name_strategy: Option) -> Result for more information. -/// return `(columns, pk_names, source info)` -pub(crate) async fn try_bind_columns_from_source( +/// return `(columns, source info)` +pub(crate) async fn bind_columns_from_source( source_schema: &ConnectorSchema, - sql_defined_pk_names: Vec, - sql_defined_columns: &[ColumnDef], with_properties: &HashMap, -) -> Result<(Option>, Vec, StreamSourceInfo)> { +) -> Result<(Option>, StreamSourceInfo)> { const MESSAGE_NAME_KEY: &str = "message"; const KEY_MESSAGE_NAME_KEY: &str = "key.message"; const NAME_STRATEGY_KEY: &str = "schema.registry.name.strategy"; - let sql_defined_pk = !sql_defined_pk_names.is_empty(); - let sql_defined_schema = !sql_defined_columns.is_empty(); let is_kafka: bool = is_kafka_connector(with_properties); let mut options = WithOptions::try_from(source_schema.row_options())?.into_inner(); @@ -314,7 +296,6 @@ pub(crate) async fn try_bind_columns_from_source( let res = match (&source_schema.format, &source_schema.row_encode) { (Format::Native, Encode::Native) => ( None, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Native as i32, row_encode: EncodeType::Native as i32, @@ -322,10 +303,6 @@ pub(crate) async fn try_bind_columns_from_source( }, ), (Format::Plain, Encode::Protobuf) => { - if sql_defined_schema { - return Err(RwError::from(ProtocolError( - "User-defined schema is not allowed with FORMAT PLAIN ENCODE PROTOBUF. Please refer to https://www.risingwave.dev/docs/current/sql-create-source/#protobuf for more information.".to_string()))); - }; let (row_schema_location, use_schema_registry) = get_schema_location(&mut options)?; let protobuf_schema = ProtobufSchema { message_name: consume_string_from_options(&mut options, MESSAGE_NAME_KEY)?, @@ -340,7 +317,6 @@ pub(crate) async fn try_bind_columns_from_source( extract_protobuf_table_schema(&protobuf_schema, with_properties.clone()) .await?, ), - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Plain as i32, row_encode: EncodeType::Protobuf as i32, @@ -357,19 +333,8 @@ pub(crate) async fn try_bind_columns_from_source( } (Format::Plain, Encode::Json) => { let schema_config = get_json_schema_location(&mut options)?; - if schema_config.is_some() && sql_defined_schema { - return Err(RwError::from(ProtocolError( - "User-defined schema is not allowed with schema registry.".to_string(), - ))); - } - if schema_config.is_none() && sql_defined_columns.is_empty() { - return Err(RwError::from(InvalidInputSyntax( - "schema definition is required for ENCODE JSON".to_owned(), - ))); - } ( extract_json_table_schema(&schema_config, with_properties).await?, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Plain as i32, row_encode: EncodeType::Json as i32, @@ -384,10 +349,7 @@ pub(crate) async fn try_bind_columns_from_source( row_schema_location, use_schema_registry, }; - if sql_defined_schema { - return Err(RwError::from(ProtocolError( - "User-defined schema is not allowed with FORMAT PLAIN ENCODE AVRO. Please refer to https://www.risingwave.dev/docs/current/sql-create-source/#avro for more information.".to_string()))); - } + let key_message_name = get_key_message_name(&mut options); let message_name = try_consume_string_from_options(&mut options, MESSAGE_NAME_KEY); let name_strategy = @@ -405,7 +367,6 @@ pub(crate) async fn try_bind_columns_from_source( }; ( Some(extract_avro_table_schema(&stream_source_info, with_properties).await?), - sql_defined_pk_names, stream_source_info, ) } @@ -425,7 +386,6 @@ pub(crate) async fn try_bind_columns_from_source( } ( None, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Plain as i32, row_encode: EncodeType::Csv as i32, @@ -435,48 +395,20 @@ pub(crate) async fn try_bind_columns_from_source( }, ) } - (Format::Plain, Encode::Bytes) => { - if !sql_defined_schema || sql_defined_columns.len() != 1 { - return Err(RwError::from(ProtocolError( - "BYTES format only accepts one column".to_string(), - ))); - } - - match sql_defined_columns[0].data_type { - Some(ast::DataType::Bytea) => {} - _ => { - return Err(RwError::from(ProtocolError( - "BYTES format only accepts BYTEA type".to_string(), - ))) - } - } - - ( - None, - sql_defined_pk_names, - StreamSourceInfo { - format: FormatType::Plain as i32, - row_encode: EncodeType::Bytes as i32, - ..Default::default() - }, - ) - } + (Format::Plain, Encode::Bytes) => ( + None, + StreamSourceInfo { + format: FormatType::Plain as i32, + row_encode: EncodeType::Bytes as i32, + ..Default::default() + }, + ), (Format::Upsert, Encode::Json) => { let schema_config = get_json_schema_location(&mut options)?; let columns = extract_json_table_schema(&schema_config, with_properties).await?; - let (columns, pk_names) = if !sql_defined_pk { - let mut columns = match columns { - None => bind_sql_columns(sql_defined_columns)?, - Some(columns) => columns, - }; - add_upsert_default_key_column(&mut columns); - (Some(columns), vec![DEFAULT_KEY_COLUMN_NAME.into()]) - } else { - (columns, sql_defined_pk_names) - }; + ( columns, - pk_names, StreamSourceInfo { format: FormatType::Upsert as i32, row_encode: EncodeType::Json as i32, @@ -491,10 +423,6 @@ pub(crate) async fn try_bind_columns_from_source( row_schema_location, use_schema_registry, }; - if sql_defined_schema { - return Err(RwError::from(ProtocolError( - "User-defined schema is not allowed with row format upsert avro. Please refer to https://www.risingwave.dev/docs/current/sql-create-source/#avro for more information.".to_string()))); - } let name_strategy = get_sr_name_strategy_check(&mut options, avro_schema.use_schema_registry)? @@ -502,57 +430,25 @@ pub(crate) async fn try_bind_columns_from_source( let key_message_name = get_key_message_name(&mut options); let message_name = try_consume_string_from_options(&mut options, MESSAGE_NAME_KEY); - if sql_defined_pk { - if sql_defined_pk_names.len() != 1 { - return Err(RwError::from(ProtocolError( - "upsert avro supports only one primary key column.".to_string(), - ))); - } - let upsert_avro_primary_key = sql_defined_pk_names[0].clone(); - - let stream_source_info = StreamSourceInfo { - key_message_name, - format: FormatType::Upsert as i32, - row_encode: EncodeType::Avro as i32, - row_schema_location: avro_schema.row_schema_location.0.clone(), - use_schema_registry: avro_schema.use_schema_registry, - proto_message_name: message_name.unwrap_or(AstString("".into())).0, - upsert_avro_primary_key, - name_strategy, - ..Default::default() - }; - let columns = - extract_avro_table_schema(&stream_source_info, with_properties).await?; + let stream_source_info = StreamSourceInfo { + key_message_name, + format: FormatType::Upsert as i32, + row_encode: EncodeType::Avro as i32, + row_schema_location: avro_schema.row_schema_location.0.clone(), + use_schema_registry: avro_schema.use_schema_registry, + proto_message_name: message_name.unwrap_or(AstString("".into())).0, + name_strategy, + ..Default::default() + }; + let columns = extract_avro_table_schema(&stream_source_info, with_properties).await?; - (Some(columns), sql_defined_pk_names, stream_source_info) - } else { - let stream_source_info = StreamSourceInfo { - format: FormatType::Upsert as i32, - row_encode: EncodeType::Avro as i32, - row_schema_location: avro_schema.row_schema_location.0.clone(), - use_schema_registry: avro_schema.use_schema_registry, - proto_message_name: message_name.unwrap_or(AstString("".into())).0, - name_strategy, - key_message_name, - ..Default::default() - }; - let (columns, pk_from_avro) = - extract_upsert_avro_table_schema(&stream_source_info, with_properties).await?; - (Some(columns), pk_from_avro, stream_source_info) - } + (Some(columns), stream_source_info) } (Format::Debezium, Encode::Json) => { - if !sql_defined_pk { - return Err(RwError::from(ProtocolError( - "Primary key must be specified when creating source with format debezium." - .to_string(), - ))); - } let schema_config = get_json_schema_location(&mut options)?; ( extract_json_table_schema(&schema_config, with_properties).await?, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Debezium as i32, row_encode: EncodeType::Json as i32, @@ -571,11 +467,6 @@ pub(crate) async fn try_bind_columns_from_source( let avro_schema = DebeziumAvroSchema { row_schema_location, }; - if sql_defined_schema { - return Err(RwError::from(ProtocolError( - "User-defined schema is not allowed with row format debezium avro.".to_string(), - ))); - } // no need to check whether works schema registry because debezium avro always work with // schema registry @@ -597,121 +488,22 @@ pub(crate) async fn try_bind_columns_from_source( let full_columns = extract_debezium_avro_table_schema(&stream_source_info, with_properties).await?; - let pk_names = if sql_defined_pk { - sql_defined_pk_names - } else { - let pk_names = - extract_debezium_avro_table_pk_columns(&stream_source_info, with_properties) - .await?; - // extract pk(s) from schema registry - for pk_name in &pk_names { - full_columns - .iter() - .find(|c: &&ColumnCatalog| c.name().eq(pk_name)) - .ok_or_else(|| { - RwError::from(ProtocolError(format!( - "avro's key column {} not exists in avro's row schema", - pk_name - ))) - })?; - } - pk_names - }; - (Some(full_columns), pk_names, stream_source_info) - } - (Format::DebeziumMongo, Encode::Json) => { - let mut columns = vec![ - ColumnCatalog { - column_desc: ColumnDesc { - data_type: DataType::Varchar, - column_id: 0.into(), - name: "_id".to_string(), - field_descs: vec![], - type_name: "".to_string(), - generated_or_default_column: None, - description: None, - }, - is_hidden: false, - }, - ColumnCatalog { - column_desc: ColumnDesc { - data_type: DataType::Jsonb, - column_id: 0.into(), - name: "payload".to_string(), - field_descs: vec![], - type_name: "".to_string(), - generated_or_default_column: None, - description: None, - }, - is_hidden: false, - }, - ]; - if sql_defined_schema { - let non_generated_sql_defined_columns = - non_generated_sql_columns(sql_defined_columns); - if non_generated_sql_defined_columns.len() != 2 - && non_generated_sql_defined_columns[0].name.real_value() != columns[0].name() - && non_generated_sql_defined_columns[1].name.real_value() != columns[1].name() - { - return Err(RwError::from(ProtocolError( - "the not generated columns of the source with row format DebeziumMongoJson - must be (_id [Jsonb | Varchar | Int32 | Int64], payload jsonb)." - .to_string(), - ))); - } - if let Some(key_data_type) = &non_generated_sql_defined_columns[0].data_type { - let key_data_type = bind_data_type(key_data_type)?; - match key_data_type { - DataType::Jsonb | DataType::Varchar | DataType::Int32 | DataType::Int64 => { - columns[0].column_desc.data_type = key_data_type; - } - _ => { - return Err(RwError::from(ProtocolError( - "the `_id` column of the source with row format DebeziumMongoJson - must be [Jsonb | Varchar | Int32 | Int64]" - .to_string(), - ))); - } - } - } - if let Some(value_data_type) = &non_generated_sql_defined_columns[1].data_type { - if !matches!(bind_data_type(value_data_type)?, DataType::Jsonb) { - return Err(RwError::from(ProtocolError( - "the `payload` column of the source with row format DebeziumMongoJson - must be Jsonb datatype" - .to_string(), - ))); - } - } - } - let pk_names = if sql_defined_pk { - sql_defined_pk_names - } else { - vec!["_id".to_string()] - }; - ( - Some(columns), - pk_names, - StreamSourceInfo { - format: FormatType::DebeziumMongo as i32, - row_encode: EncodeType::Json as i32, - ..Default::default() - }, - ) + (Some(full_columns), stream_source_info) } + (Format::DebeziumMongo, Encode::Json) => ( + None, + StreamSourceInfo { + format: FormatType::DebeziumMongo as i32, + row_encode: EncodeType::Json as i32, + ..Default::default() + }, + ), (Format::Maxwell, Encode::Json) => { - if !sql_defined_pk { - return Err(RwError::from(ProtocolError( - "Primary key must be specified when creating source with FORMAT MAXWELL ENCODE JSON." - .to_string(), - ))); - } let schema_config = get_json_schema_location(&mut options)?; ( extract_json_table_schema(&schema_config, with_properties).await?, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Maxwell as i32, row_encode: EncodeType::Json as i32, @@ -722,16 +514,9 @@ pub(crate) async fn try_bind_columns_from_source( } (Format::Canal, Encode::Json) => { - if !sql_defined_pk { - return Err(RwError::from(ProtocolError( - "Primary key must be specified when creating source with row format cannal_json." - .to_string(), - ))); - } let schema_config = get_json_schema_location(&mut options)?; ( extract_json_table_schema(&schema_config, with_properties).await?, - sql_defined_pk_names, StreamSourceInfo { format: FormatType::Canal as i32, row_encode: EncodeType::Json as i32, @@ -762,6 +547,225 @@ pub(crate) async fn try_bind_columns_from_source( Ok(res) } +/// Bind columns from both source and sql defined. +pub(crate) fn bind_all_columns( + source_schema: &ConnectorSchema, + cols_from_source: Option>, + cols_from_sql: Vec, + col_defs_from_sql: &[ColumnDef], +) -> Result> { + if let Some(cols_from_source) = cols_from_source { + if cols_from_sql.is_empty() { + Ok(cols_from_source) + } else { + // TODO(yuhao): https://github.com/risingwavelabs/risingwave/issues/12209 + Err(RwError::from(ProtocolError( + format!("User-defined schema from SQL is not allowed with FORMAT {} ENCODE {}. \ + Please refer to https://www.risingwave.dev/docs/current/sql-create-source/ for more information.", source_schema.format, source_schema.row_encode)))) + } + } else { + // FIXME(yuhao): cols_from_sql should be None is no `()` is given. + if cols_from_sql.is_empty() { + return Err(RwError::from(ProtocolError( + "Schema definition is required, either from SQL or schema registry.".to_string(), + ))); + } + match (&source_schema.format, &source_schema.row_encode) { + (Format::DebeziumMongo, Encode::Json) => { + let mut columns = vec![ + ColumnCatalog { + column_desc: ColumnDesc { + data_type: DataType::Varchar, + column_id: 0.into(), + name: "_id".to_string(), + field_descs: vec![], + type_name: "".to_string(), + generated_or_default_column: None, + description: None, + }, + is_hidden: false, + }, + ColumnCatalog { + column_desc: ColumnDesc { + data_type: DataType::Jsonb, + column_id: 0.into(), + name: "payload".to_string(), + field_descs: vec![], + type_name: "".to_string(), + generated_or_default_column: None, + description: None, + }, + is_hidden: false, + }, + ]; + let non_generated_sql_defined_columns = + non_generated_sql_columns(col_defs_from_sql); + if non_generated_sql_defined_columns.len() != 2 + || non_generated_sql_defined_columns[0].name.real_value() != columns[0].name() + || non_generated_sql_defined_columns[1].name.real_value() != columns[1].name() + { + return Err(RwError::from(ProtocolError( + "the not generated columns of the source with row format DebeziumMongoJson + must be (_id [Jsonb | Varchar | Int32 | Int64], payload jsonb)." + .to_string(), + ))); + } + // ok to unwrap since it was checked at `bind_sql_columns` + let key_data_type = bind_data_type( + non_generated_sql_defined_columns[0] + .data_type + .as_ref() + .unwrap(), + )?; + match key_data_type { + DataType::Jsonb | DataType::Varchar | DataType::Int32 | DataType::Int64 => { + columns[0].column_desc.data_type = key_data_type.clone(); + } + _ => { + return Err(RwError::from(ProtocolError( + "the `_id` column of the source with row format DebeziumMongoJson + must be [Jsonb | Varchar | Int32 | Int64]" + .to_string(), + ))); + } + } + + // ok to unwrap since it was checked at `bind_sql_columns` + let value_data_type = bind_data_type( + non_generated_sql_defined_columns[1] + .data_type + .as_ref() + .unwrap(), + )?; + if !matches!(value_data_type, DataType::Jsonb) { + return Err(RwError::from(ProtocolError( + "the `payload` column of the source with row format DebeziumMongoJson + must be Jsonb datatype" + .to_string(), + ))); + } + Ok(columns) + } + (Format::Plain, Encode::Bytes) => { + if cols_from_sql.len() != 1 || cols_from_sql[0].data_type() != &DataType::Bytea { + return Err(RwError::from(ProtocolError( + "ENCODE BYTES only accepts one BYTEA type column".to_string(), + ))); + } + Ok(cols_from_sql) + } + (_, _) => Ok(cols_from_sql), + } + } +} + +/// Bind column from source. Add key column to table columns if necessary. +/// Return (columns, pks) +pub(crate) async fn bind_source_pk( + source_schema: &ConnectorSchema, + source_info: &StreamSourceInfo, + columns: &mut Vec, + sql_defined_pk_names: Vec, + with_properties: &HashMap, +) -> Result> { + let sql_defined_pk = !sql_defined_pk_names.is_empty(); + + let res = match (&source_schema.format, &source_schema.row_encode) { + (Format::Native, Encode::Native) | (Format::Plain, _) => sql_defined_pk_names, + (Format::Upsert, Encode::Json) => { + if sql_defined_pk { + sql_defined_pk_names + } else { + add_upsert_default_key_column(columns); + vec![DEFAULT_KEY_COLUMN_NAME.into()] + } + } + (Format::Upsert, Encode::Avro) => { + if sql_defined_pk { + if sql_defined_pk_names.len() != 1 { + return Err(RwError::from(ProtocolError( + "upsert avro supports only one primary key column.".to_string(), + ))); + } + sql_defined_pk_names + } else if let Some(extracted_pk_names) = + extract_upsert_avro_table_pk_columns(source_info, with_properties).await? + { + extracted_pk_names + } else { + // For upsert avro, if we can't extract pk from schema, use message key as primary key + add_upsert_default_key_column(columns); + vec![DEFAULT_KEY_COLUMN_NAME.into()] + } + } + + (Format::Debezium, Encode::Json) => { + if !sql_defined_pk { + return Err(RwError::from(ProtocolError( + "Primary key must be specified when creating source with FORMAT DEBEZIUM." + .to_string(), + ))); + } + sql_defined_pk_names + } + (Format::Debezium, Encode::Avro) => { + if sql_defined_pk { + sql_defined_pk_names + } else { + let pk_names = + extract_debezium_avro_table_pk_columns(source_info, with_properties).await?; + // extract pk(s) from schema registry + for pk_name in &pk_names { + columns + .iter() + .find(|c: &&ColumnCatalog| c.name().eq(pk_name)) + .ok_or_else(|| { + RwError::from(ProtocolError(format!( + "avro's key column {} not exists in avro's row schema", + pk_name + ))) + })?; + } + pk_names + } + } + (Format::DebeziumMongo, Encode::Json) => { + if sql_defined_pk { + sql_defined_pk_names + } else { + vec!["_id".to_string()] + } + } + + (Format::Maxwell, Encode::Json) => { + if !sql_defined_pk { + return Err(RwError::from(ProtocolError( + "Primary key must be specified when creating source with FORMAT MAXWELL ENCODE JSON." + .to_string(), + ))); + } + sql_defined_pk_names + } + + (Format::Canal, Encode::Json) => { + if !sql_defined_pk { + return Err(RwError::from(ProtocolError( + "Primary key must be specified when creating source with FORMAT CANAL ENCODE JSON." + .to_string(), + ))); + } + sql_defined_pk_names + } + (format, encoding) => { + return Err(RwError::from(ProtocolError(format!( + "Unknown combination {:?} {:?}", + format, encoding + )))); + } + }; + Ok(res) +} + // Add a hidden column `_rw_kafka_timestamp` to each message from Kafka source. fn check_and_add_timestamp_column( with_properties: &HashMap, @@ -1070,14 +1074,26 @@ pub async fn handle_create_source( validate_compatibility(&source_schema, &mut with_properties)?; ensure_table_constraints_supported(&stmt.constraints)?; - let pk_names = bind_pk_names(&stmt.columns, &stmt.constraints)?; + let sql_pk_names = bind_sql_pk_names(&stmt.columns, &stmt.constraints)?; - let (columns_from_resolve_source, pk_names, source_info) = - try_bind_columns_from_source(&source_schema, pk_names, &stmt.columns, &with_properties) - .await?; + let (columns_from_resolve_source, source_info) = + bind_columns_from_source(&source_schema, &with_properties).await?; let columns_from_sql = bind_sql_columns(&stmt.columns)?; - let mut columns = columns_from_resolve_source.unwrap_or(columns_from_sql); + let mut columns = bind_all_columns( + &source_schema, + columns_from_resolve_source, + columns_from_sql, + &stmt.columns, + )?; + let pk_names = bind_source_pk( + &source_schema, + &source_info, + &mut columns, + sql_pk_names, + &with_properties, + ) + .await?; check_and_add_timestamp_column(&with_properties, &mut columns); diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index e412658cb712f..efa4278bcbc99 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -43,8 +43,8 @@ use crate::catalog::table_catalog::TableVersion; use crate::catalog::{check_valid_column_name, CatalogError, ColumnId}; use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; use crate::handler::create_source::{ - bind_source_watermark, check_source_schema, try_bind_columns_from_source, - validate_compatibility, UPSTREAM_SOURCE_KEY, + bind_all_columns, bind_columns_from_source, bind_source_pk, bind_source_watermark, + check_source_schema, validate_compatibility, UPSTREAM_SOURCE_KEY, }; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::LogicalSource; @@ -346,7 +346,7 @@ pub fn ensure_table_constraints_supported(table_constraints: &[TableConstraint]) Ok(()) } -pub fn bind_pk_names( +pub fn bind_sql_pk_names( columns_defs: &[ColumnDef], table_constraints: &[TableConstraint], ) -> Result> { @@ -457,13 +457,27 @@ pub(crate) async fn gen_create_table_plan_with_source( validate_compatibility(&source_schema, &mut properties)?; ensure_table_constraints_supported(&constraints)?; - let pk_names = bind_pk_names(&column_defs, &constraints)?; - let (columns_from_resolve_source, pk_names, mut source_info) = - try_bind_columns_from_source(&source_schema, pk_names, &column_defs, &properties).await?; + let sql_pk_names = bind_sql_pk_names(&column_defs, &constraints)?; + + let (columns_from_resolve_source, mut source_info) = + bind_columns_from_source(&source_schema, &properties).await?; let columns_from_sql = bind_sql_columns(&column_defs)?; - let mut columns = columns_from_resolve_source.unwrap_or(columns_from_sql); + let mut columns = bind_all_columns( + &source_schema, + columns_from_resolve_source, + columns_from_sql, + &column_defs, + )?; + let pk_names = bind_source_pk( + &source_schema, + &source_info, + &mut columns, + sql_pk_names, + &properties, + ) + .await?; for c in &mut columns { c.column_desc.column_id = col_id_gen.generate(c.name()) @@ -605,7 +619,7 @@ pub(crate) fn gen_create_table_plan_without_bind( version: Option, ) -> Result<(PlanRef, Option, PbTable)> { ensure_table_constraints_supported(&constraints)?; - let pk_names = bind_pk_names(&column_defs, &constraints)?; + let pk_names = bind_sql_pk_names(&column_defs, &constraints)?; let (mut columns, pk_column_ids, row_id_index) = bind_pk_on_relation(columns, pk_names)?; let watermark_descs = bind_source_watermark( @@ -976,7 +990,7 @@ mod tests { c.column_desc.column_id = col_id_gen.generate(c.name()) } ensure_table_constraints_supported(&constraints)?; - let pk_names = bind_pk_names(&column_defs, &constraints)?; + let pk_names = bind_sql_pk_names(&column_defs, &constraints)?; let (_, pk_column_ids, _) = bind_pk_on_relation(columns, pk_names)?; Ok(pk_column_ids) })(); diff --git a/src/sqlparser/src/ast/ddl.rs b/src/sqlparser/src/ast/ddl.rs index 8cdc4ac2aa865..ae497d777c4d0 100644 --- a/src/sqlparser/src/ast/ddl.rs +++ b/src/sqlparser/src/ast/ddl.rs @@ -363,6 +363,12 @@ impl ColumnDef { options, } } + + pub fn is_generated(&self) -> bool { + self.options + .iter() + .any(|option| matches!(option.option, ColumnOption::GeneratedColumns(_))) + } } impl fmt::Display for ColumnDef { From a9d9bc980248d0bc4a81d5ab29d4d029494d6eef Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Mon, 30 Oct 2023 16:58:18 +0800 Subject: [PATCH 47/52] fix(expr): `null` input of `~` / `!~` (#13145) --- e2e_test/batch/basic/pg_regrex_match_op.slt.part | 10 ++++++++++ proto/expr.proto | 1 + src/expr/impl/src/scalar/regexp.rs | 9 +++++++++ .../planner_test/tests/testdata/output/subquery.yaml | 4 ++-- src/frontend/src/binder/expr/binary_op.rs | 9 +++------ src/frontend/src/expr/pure.rs | 1 + 6 files changed, 26 insertions(+), 8 deletions(-) diff --git a/e2e_test/batch/basic/pg_regrex_match_op.slt.part b/e2e_test/batch/basic/pg_regrex_match_op.slt.part index 0988f078d3401..cd12a5296b868 100644 --- a/e2e_test/batch/basic/pg_regrex_match_op.slt.part +++ b/e2e_test/batch/basic/pg_regrex_match_op.slt.part @@ -7,3 +7,13 @@ query T select 'foobarbequebazilbarfbonk' !~ '(b[^b]+)(b[^b]+)'; ---- f + +query T +select 'foobarbequebazilbarfbonk' ~ null; +---- +NULL + +query T +select null !~ '(b[^b]+)(b[^b]+)'; +---- +NULL diff --git a/proto/expr.proto b/proto/expr.proto index fecefc12b4ee7..4437a548beec3 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -107,6 +107,7 @@ message ExprNode { REGEXP_REPLACE = 280; REGEXP_COUNT = 281; REGEXP_SPLIT_TO_ARRAY = 282; + REGEXP_EQ = 283; POW = 233; EXP = 234; CHR = 235; diff --git a/src/expr/impl/src/scalar/regexp.rs b/src/expr/impl/src/scalar/regexp.rs index d919334fe2c98..64234a8384dfe 100644 --- a/src/expr/impl/src/scalar/regexp.rs +++ b/src/expr/impl/src/scalar/regexp.rs @@ -134,6 +134,15 @@ impl FromStr for RegexpOptions { } } +#[function( + // source ~ pattern + "regexp_eq(varchar, varchar) -> boolean", + prebuild = "RegexpContext::from_pattern($1)?" +)] +fn regexp_eq(text: &str, regex: &RegexpContext) -> bool { + regex.regex.is_match(text).unwrap() +} + #[function( // regexp_match(source, pattern) "regexp_match(varchar, varchar) -> varchar[]", diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 69b6e21592c0a..309892e692b02 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -237,7 +237,7 @@ logical_plan: |- LogicalProject { exprs: [rw_schemas.name, rw_tables.name, Case(($expr1 = 'r':Varchar), 'table':Varchar, ($expr1 = 'v':Varchar), 'view':Varchar, ($expr1 = 'm':Varchar), 'materialized view':Varchar, ($expr1 = 'i':Varchar), 'index':Varchar, ($expr1 = 'S':Varchar), 'sequence':Varchar, ($expr1 = 's':Varchar), 'special':Varchar, ($expr1 = 't':Varchar), 'TOAST table':Varchar, ($expr1 = 'f':Varchar), 'foreign table':Varchar, ($expr1 = 'p':Varchar), 'partitioned table':Varchar, ($expr1 = 'I':Varchar), 'partitioned index':Varchar) as $expr3, rw_users.name] } └─LogicalApply { type: LeftOuter, on: true, correlated_id: 1, max_one_row: true } - ├─LogicalFilter { predicate: In($expr1, 'r':Varchar, 'p':Varchar, 'v':Varchar, 'm':Varchar, 'S':Varchar, 'f':Varchar, '':Varchar) AND (rw_schemas.name <> 'pg_catalog':Varchar) AND IsNull(RegexpMatch(rw_schemas.name, '^pg_toast':Varchar)) AND (rw_schemas.name <> 'information_schema':Varchar) } + ├─LogicalFilter { predicate: In($expr1, 'r':Varchar, 'p':Varchar, 'v':Varchar, 'm':Varchar, 'S':Varchar, 'f':Varchar, '':Varchar) AND (rw_schemas.name <> 'pg_catalog':Varchar) AND Not(RegexpEq(rw_schemas.name, '^pg_toast':Varchar)) AND (rw_schemas.name <> 'information_schema':Varchar) } │ └─LogicalJoin { type: LeftOuter, on: (rw_schemas.id = rw_tables.schema_id), output: all } │ ├─LogicalShare { id: 16 } │ │ └─LogicalProject { exprs: [rw_tables.id, rw_tables.name, rw_tables.schema_id, rw_tables.owner, Case(('table':Varchar = 'table':Varchar), 'r':Varchar, ('table':Varchar = 'system table':Varchar), 'r':Varchar, ('table':Varchar = 'index':Varchar), 'i':Varchar, ('table':Varchar = 'view':Varchar), 'v':Varchar, ('table':Varchar = 'materialized view':Varchar), 'm':Varchar) as $expr1, 0:Int32, 0:Int32, Array as $expr2] } @@ -302,7 +302,7 @@ │ │ └─BatchFilter { predicate: true } │ │ └─BatchScan { table: rw_views, columns: [rw_views.name, rw_views.schema_id, rw_views.owner], distribution: Single } │ └─BatchExchange { order: [], dist: HashShard(rw_schemas.id) } - │ └─BatchFilter { predicate: (rw_schemas.name <> 'pg_catalog':Varchar) AND IsNull(RegexpMatch(rw_schemas.name, '^pg_toast':Varchar)) AND (rw_schemas.name <> 'information_schema':Varchar) } + │ └─BatchFilter { predicate: (rw_schemas.name <> 'pg_catalog':Varchar) AND Not(RegexpEq(rw_schemas.name, '^pg_toast':Varchar)) AND (rw_schemas.name <> 'information_schema':Varchar) } │ └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } └─BatchExchange { order: [], dist: HashShard(rw_users.id) } └─BatchProject { exprs: [rw_users.name, rw_users.id] } diff --git a/src/frontend/src/binder/expr/binary_op.rs b/src/frontend/src/binder/expr/binary_op.rs index 352d2bfbfd246..33f6f366e6496 100644 --- a/src/frontend/src/binder/expr/binary_op.rs +++ b/src/frontend/src/binder/expr/binary_op.rs @@ -142,13 +142,10 @@ impl Binder { } } } - BinaryOperator::PGRegexMatch => { - func_types.push(ExprType::IsNotNull); - ExprType::RegexpMatch - } + BinaryOperator::PGRegexMatch => ExprType::RegexpEq, BinaryOperator::PGRegexNotMatch => { - func_types.push(ExprType::IsNull); - ExprType::RegexpMatch + func_types.push(ExprType::Not); + ExprType::RegexpEq } _ => { return Err( diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 7a63c7f95ae99..cebd59322c2b9 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -108,6 +108,7 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::RegexpReplace | expr_node::Type::RegexpCount | expr_node::Type::RegexpSplitToArray + | expr_node::Type::RegexpEq | expr_node::Type::Pow | expr_node::Type::Exp | expr_node::Type::Ln From c0ab3ca01c05a95c4d1b07be22684f6b1fbffea3 Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Mon, 30 Oct 2023 17:46:14 +0800 Subject: [PATCH 48/52] feat(expr): support jsonb `-` and `#-` operator (#13118) Signed-off-by: Runji Wang --- Cargo.lock | 4 +- proto/expr.proto | 7 + src/common/src/types/jsonb.rs | 8 + src/expr/impl/src/scalar/jsonb_delete.rs | 369 ++++++++++++++++++++++ src/expr/impl/src/scalar/mod.rs | 1 + src/frontend/src/binder/expr/binary_op.rs | 1 + src/frontend/src/binder/expr/function.rs | 2 + src/frontend/src/expr/pure.rs | 1 + src/sqlparser/src/ast/operator.rs | 2 + src/sqlparser/src/parser.rs | 2 + src/sqlparser/src/tokenizer.rs | 4 + src/tests/regress/data/sql/jsonb.sql | 91 +++--- 12 files changed, 444 insertions(+), 48 deletions(-) create mode 100644 src/expr/impl/src/scalar/jsonb_delete.rs diff --git a/Cargo.lock b/Cargo.lock index 498002620379d..92bbc847031cc 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4137,9 +4137,9 @@ dependencies = [ [[package]] name = "jsonbb" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "44376417b2ff0cd879b5c84976fa9e0855c316321b4e0502e33e52963bf84f74" +checksum = "efd95430271266a57cbb8fd31115559c853fcaa5f367d32c4720034f7bd37b7f" dependencies = [ "bytes", "serde", diff --git a/proto/expr.proto b/proto/expr.proto index 4437a548beec3..0d29cabd494fb 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -237,6 +237,13 @@ message ExprNode { JSONB_EXISTS_ANY = 611; // jsonb ?& text[] JSONB_EXISTS_ALL = 612; + // see SUBTRACT for: + // jsonb - text -> jsonb + // jsonb - text[] -> jsonb + // jsonb - integer -> jsonb + // + // jsonb #- text[] -> jsonb + JSONB_DELETE_PATH = 615; // Non-pure functions below (> 1000) // ------------------------ diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 664af6c0b1921..15345ac7cfe20 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -293,6 +293,14 @@ impl<'a> JsonbRef<'a> { self.0.as_null().is_some() } + /// Returns true if this is a jsonb null, boolean, number or string. + pub fn is_scalar(&self) -> bool { + matches!( + self.0, + ValueRef::Null | ValueRef::Bool(_) | ValueRef::Number(_) | ValueRef::String(_) + ) + } + /// Returns true if this is a jsonb array. pub fn is_array(&self) -> bool { matches!(self.0, ValueRef::Array(_)) diff --git a/src/expr/impl/src/scalar/jsonb_delete.rs b/src/expr/impl/src/scalar/jsonb_delete.rs new file mode 100644 index 0000000000000..dff60f34f722b --- /dev/null +++ b/src/expr/impl/src/scalar/jsonb_delete.rs @@ -0,0 +1,369 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashSet; + +use jsonbb::{Value, ValueRef}; +use risingwave_common::types::{JsonbRef, JsonbVal, ListRef}; +use risingwave_expr::{function, ExprError, Result}; + +/// Removes a key (and its value) from a JSON object, or matching string value(s) from a JSON array. +/// +/// Examples: +/// +/// ```slt +/// # remove key from object +/// query T +/// SELECT '{"a": "b", "c": "d"}'::jsonb - 'a'; +/// ---- +/// {"c": "d"} +/// +/// # remove matching value from array +/// query T +/// SELECT '["a", "b", "c", "b"]'::jsonb - 'b'; +/// ---- +/// ["a", "c"] +/// +/// query error cannot delete from scalar +/// SELECT '1'::jsonb - 'b'; +/// ``` +#[function("subtract(jsonb, varchar) -> jsonb")] +fn jsonb_remove(v: JsonbRef<'_>, key: &str) -> Result { + match v.into() { + ValueRef::Object(obj) => Ok(JsonbVal::from(Value::object( + obj.iter().filter(|(k, _)| *k != key), + ))), + ValueRef::Array(arr) => Ok(JsonbVal::from(Value::array( + arr.iter().filter(|value| value.as_str() != Some(key)), + ))), + _ => Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot delete from scalar".into(), + }), + } +} + +/// Deletes all matching keys or array elements from the left operand. +/// +/// Examples: +/// +/// ```slt +/// query T +/// SELECT '{"a": "b", "c": "d"}'::jsonb - '{a,c}'::text[]; +/// ---- +/// {} +/// +/// query error cannot delete from scalar +/// SELECT '1'::jsonb - '{a,c}'::text[]; +/// ``` +#[function("subtract(jsonb, varchar[]) -> jsonb")] +fn jsonb_remove_keys(v: JsonbRef<'_>, keys: ListRef<'_>) -> Result { + let keys_set: HashSet<&str> = keys.iter().flatten().map(|s| s.into_utf8()).collect(); + + match v.into() { + ValueRef::Object(obj) => Ok(JsonbVal::from(Value::object( + obj.iter().filter(|(k, _)| !keys_set.contains(*k)), + ))), + ValueRef::Array(arr) => { + Ok(JsonbVal::from(Value::array(arr.iter().filter( + |value| match value.as_str() { + Some(s) => !keys_set.contains(s), + None => true, + }, + )))) + } + _ => Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot delete from scalar".into(), + }), + } +} + +/// Deletes the array element with the specified index (negative integers count from the end). +/// Throws an error if JSON value is not an array. +/// +/// Examples: +/// +/// ```slt +/// query T +/// SELECT '["a", "b"]'::jsonb - 1; +/// ---- +/// ["a"] +/// +/// query T +/// SELECT '["a", "b"]'::jsonb - -1; +/// ---- +/// ["a"] +/// +/// query T +/// SELECT '["a", "b"]'::jsonb - 2; +/// ---- +/// ["a", "b"] +/// +/// query T +/// SELECT '["a", "b"]'::jsonb - -3; +/// ---- +/// ["a", "b"] +/// +/// query error cannot delete from scalar +/// SELECT '1'::jsonb - 1; +/// +/// query error cannot delete from object using integer index +/// SELECT '{"a": 1}'::jsonb - 1; +/// ``` +#[function("subtract(jsonb, int4) -> jsonb")] +fn jsonb_remove_index(v: JsonbRef<'_>, index: i32) -> Result { + let array = match v.into() { + ValueRef::Array(array) => array, + ValueRef::Object(_) => { + return Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot delete from object using integer index".into(), + }) + } + _ => { + return Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot delete from scalar".into(), + }) + } + }; + let Some(idx) = normalize_array_index(array.len(), index) else { + // out of bounds index returns original value + return Ok(JsonbVal::from(v)); + }; + Ok(JsonbVal::from(Value::array( + array + .iter() + .enumerate() + .filter(|&(i, _)| i != idx) + .map(|(_, v)| v), + ))) +} + +/// Deletes the field or array element at the specified path, where path elements can be +/// either field keys or array indexes. +/// +/// Examples: +/// +/// ```slt +/// # Basic test case +/// query T +/// SELECT '["a", {"b":1}]'::jsonb #- '{1,b}'; +/// ---- +/// ["a", {}] +/// +/// # Invalid path +/// query error path element at position 1 is null +/// SELECT '["a", {"b":1}]'::jsonb #- array[null]; +/// +/// # Removing non-existent key from an object +/// query T +/// SELECT '{"a": 1, "b": 2}'::jsonb #- '{c}'; +/// ---- +/// {"a": 1, "b": 2} +/// +/// # Removing an existing key from an object +/// query T +/// SELECT '{"a": 1, "b": 2}'::jsonb #- '{a}'; +/// ---- +/// {"b": 2} +/// +/// # Removing an item from an array by positive index +/// query T +/// SELECT '["a", "b", "c"]'::jsonb #- '{1}'; +/// ---- +/// ["a", "c"] +/// +/// # Removing an item from an array by negative index +/// query T +/// SELECT '["a", "b", "c"]'::jsonb #- '{-1}'; +/// ---- +/// ["a", "b"] +/// +/// # Removing a non-existent index from an array +/// query T +/// SELECT '["a", "b", "c"]'::jsonb #- '{3}'; +/// ---- +/// ["a", "b", "c"] +/// +/// # Path element is not an integer for array +/// query error path element at position 1 is not an integer: "a" +/// SELECT '["a", "b", "c"]'::jsonb #- '{a}'; +/// +/// # Path to deeply nested value +/// query T +/// SELECT '{"a": {"b": {"c": [1, 2, 3]}}}'::jsonb #- '{a,b,c,1}'; +/// ---- +/// {"a": {"b": {"c": [1, 3]}}} +/// +/// # Path terminates early (before reaching the final depth of the JSON) +/// query T +/// SELECT '{"a": {"b": {"c": [1, 2, 3]}}}'::jsonb #- '{a}'; +/// ---- +/// {} +/// +/// # Removing non-existent path in nested structure +/// query T +/// SELECT '{"a": {"b": {"c": [1, 2, 3]}}}'::jsonb #- '{a,x}'; +/// ---- +/// {"a": {"b": {"c": [1, 2, 3]}}} +/// +/// # Path is longer than the depth of the JSON structure +/// query T +/// SELECT '{"a": 1}'::jsonb #- '{a,b}'; +/// ---- +/// {"a": 1} +/// +/// # Edge case: Removing root +/// query T +/// SELECT '{"a": 1}'::jsonb #- '{}'; +/// ---- +/// {"a": 1} +/// +/// # Edge case: Empty array +/// query T +/// SELECT '[]'::jsonb #- '{a}'; +/// ---- +/// [] +/// +/// # Edge case: Empty object +/// query T +/// SELECT '{}'::jsonb #- '{null}'; +/// ---- +/// {} +/// +/// query error cannot delete path in scalar +/// SELECT '1'::jsonb #- '{}'; +/// ``` +#[function("jsonb_delete_path(jsonb, varchar[]) -> jsonb")] +fn jsonb_delete_path(v: JsonbRef<'_>, path: ListRef<'_>) -> Result { + if v.is_scalar() { + return Err(ExprError::InvalidParam { + name: "jsonb", + reason: "cannot delete path in scalar".into(), + }); + } + if path.is_empty() { + return Ok(JsonbVal::from(v)); + } + let jsonb: ValueRef<'_> = v.into(); + let mut builder = jsonbb::Builder::>::with_capacity(jsonb.capacity()); + jsonbb_remove_path(jsonb, path, 0, &mut builder)?; + Ok(JsonbVal::from(builder.finish())) +} + +// Recursively remove `path[i..]` from `jsonb` and write the result to `builder`. +// Panics if `i` is out of bounds. +fn jsonbb_remove_path( + jsonb: ValueRef<'_>, + path: ListRef<'_>, + i: usize, + builder: &mut jsonbb::Builder, +) -> Result<()> { + match jsonb { + ValueRef::Object(obj) => { + if obj.is_empty() { + builder.add_value(jsonb); + return Ok(()); + } + let key = path + .get(i) + .unwrap() + .ok_or_else(|| ExprError::InvalidParam { + name: "path", + reason: format!("path element at position {} is null", i + 1).into(), + })? + .into_utf8(); + if !obj.contains_key(key) { + builder.add_value(jsonb); + return Ok(()); + } + builder.begin_object(); + for (k, v) in obj.iter() { + if k != key { + builder.add_string(k); + builder.add_value(v); + continue; + } + if i != path.len() - 1 { + builder.add_string(k); + // recursively remove path[i+1..] from v + jsonbb_remove_path(v, path, i + 1, builder)?; + } + } + builder.end_object(); + Ok(()) + } + ValueRef::Array(array) => { + if array.is_empty() { + builder.add_value(jsonb); + return Ok(()); + } + let key = path + .get(i) + .unwrap() + .ok_or_else(|| ExprError::InvalidParam { + name: "path", + reason: format!("path element at position {} is null", i + 1).into(), + })? + .into_utf8(); + let idx = key.parse::().map_err(|_| ExprError::InvalidParam { + name: "path", + reason: format!( + "path element at position {} is not an integer: \"{}\"", + i + 1, + key + ) + .into(), + })?; + let Some(idx) = normalize_array_index(array.len(), idx) else { + // out of bounds index returns original value + builder.add_value(jsonb); + return Ok(()); + }; + builder.begin_array(); + for (j, v) in array.iter().enumerate() { + if j != idx { + builder.add_value(v); + continue; + } + if i != path.len() - 1 { + // recursively remove path[i+1..] from v + jsonbb_remove_path(v, path, i + 1, builder)?; + } + } + builder.end_array(); + Ok(()) + } + _ => { + builder.add_value(jsonb); + Ok(()) + } + } +} + +/// Normalizes an array index to `0..len`. +/// Negative indices count from the end. i.e. `-len..0 => 0..len`. +/// Returns `None` if index is out of bounds. +fn normalize_array_index(len: usize, index: i32) -> Option { + if index < -(len as i32) || index >= (len as i32) { + return None; + } + if index >= 0 { + Some(index as usize) + } else { + Some((len as i32 + index) as usize) + } +} diff --git a/src/expr/impl/src/scalar/mod.rs b/src/expr/impl/src/scalar/mod.rs index d1b89d3c75d6c..564091897ae1a 100644 --- a/src/expr/impl/src/scalar/mod.rs +++ b/src/expr/impl/src/scalar/mod.rs @@ -45,6 +45,7 @@ mod int256; mod jsonb_access; mod jsonb_concat; mod jsonb_contains; +mod jsonb_delete; mod jsonb_info; mod jsonb_object; mod length; diff --git a/src/frontend/src/binder/expr/binary_op.rs b/src/frontend/src/binder/expr/binary_op.rs index 33f6f366e6496..8718ea74e5b5d 100644 --- a/src/frontend/src/binder/expr/binary_op.rs +++ b/src/frontend/src/binder/expr/binary_op.rs @@ -91,6 +91,7 @@ impl Binder { BinaryOperator::PGBitwiseShiftRight => ExprType::BitwiseShiftRight, BinaryOperator::Arrow => ExprType::JsonbAccess, BinaryOperator::LongArrow => ExprType::JsonbAccessStr, + BinaryOperator::HashMinus => ExprType::JsonbDeletePath, BinaryOperator::HashArrow => ExprType::JsonbAccessMulti, BinaryOperator::HashLongArrow => ExprType::JsonbAccessMultiStr, BinaryOperator::Prefix => ExprType::StartsWith, diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 6dfab2c3bc283..a32e07f72ce51 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -884,6 +884,8 @@ impl Binder { ("jsonb_exists", raw_call(ExprType::JsonbExists)), ("jsonb_exists_any", raw_call(ExprType::JsonbExistsAny)), ("jsonb_exists_all", raw_call(ExprType::JsonbExistsAll)), + ("jsonb_delete", raw_call(ExprType::Subtract)), + ("jsonb_delete_path", raw_call(ExprType::JsonbDeletePath)), // Functions that return a constant value ("pi", pi()), // greatest and least diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index cebd59322c2b9..94d38dcff56f1 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -182,6 +182,7 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::JsonbArrayLength | expr_node::Type::JsonbObject | expr_node::Type::JsonbPretty + | expr_node::Type::JsonbDeletePath | expr_node::Type::JsonbContains | expr_node::Type::JsonbContained | expr_node::Type::JsonbExists diff --git a/src/sqlparser/src/ast/operator.rs b/src/sqlparser/src/ast/operator.rs index 147a78d0b7174..9929d56b28679 100644 --- a/src/sqlparser/src/ast/operator.rs +++ b/src/sqlparser/src/ast/operator.rs @@ -99,6 +99,7 @@ pub enum BinaryOperator { LongArrow, HashArrow, HashLongArrow, + HashMinus, Contains, Contained, Exists, @@ -148,6 +149,7 @@ impl fmt::Display for BinaryOperator { BinaryOperator::LongArrow => "->>", BinaryOperator::HashArrow => "#>", BinaryOperator::HashLongArrow => "#>>", + BinaryOperator::HashMinus => "#-", BinaryOperator::Contains => "@>", BinaryOperator::Contained => "<@", BinaryOperator::Exists => "?", diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index d73ec295b0f90..87af26bfb1750 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -1397,6 +1397,7 @@ impl Parser { Token::LongArrow => Some(BinaryOperator::LongArrow), Token::HashArrow => Some(BinaryOperator::HashArrow), Token::HashLongArrow => Some(BinaryOperator::HashLongArrow), + Token::HashMinus => Some(BinaryOperator::HashMinus), Token::AtArrow => Some(BinaryOperator::Contains), Token::ArrowAt => Some(BinaryOperator::Contained), Token::QuestionMark => Some(BinaryOperator::Exists), @@ -1741,6 +1742,7 @@ impl Parser { | Token::LongArrow | Token::HashArrow | Token::HashLongArrow + | Token::HashMinus | Token::AtArrow | Token::ArrowAt | Token::QuestionMark diff --git a/src/sqlparser/src/tokenizer.rs b/src/sqlparser/src/tokenizer.rs index 914f0e5f0f8d4..4fafde820f414 100644 --- a/src/sqlparser/src/tokenizer.rs +++ b/src/sqlparser/src/tokenizer.rs @@ -164,6 +164,8 @@ pub enum Token { HashArrow, /// `#>>`, extract JSON sub-object at the specified path as text in PostgreSQL HashLongArrow, + /// `#-`, delete a key from a JSON object in PostgreSQL + HashMinus, /// `@>`, does the left JSON value contain the right JSON path/value entries at the top level AtArrow, /// `<@`, does the right JSON value contain the left JSON path/value entries at the top level @@ -241,6 +243,7 @@ impl fmt::Display for Token { Token::LongArrow => f.write_str("->>"), Token::HashArrow => f.write_str("#>"), Token::HashLongArrow => f.write_str("#>>"), + Token::HashMinus => f.write_str("#-"), Token::AtArrow => f.write_str("@>"), Token::ArrowAt => f.write_str("<@"), Token::QuestionMark => f.write_str("?"), @@ -761,6 +764,7 @@ impl<'a> Tokenizer<'a> { '#' => { chars.next(); // consume the '#' match chars.peek() { + Some('-') => self.consume_and_return(chars, Token::HashMinus), Some('>') => { chars.next(); // consume first '>' match chars.peek() { diff --git a/src/tests/regress/data/sql/jsonb.sql b/src/tests/regress/data/sql/jsonb.sql index 69bd9a928a0f5..8ae2b473b0dd4 100644 --- a/src/tests/regress/data/sql/jsonb.sql +++ b/src/tests/regress/data/sql/jsonb.sql @@ -1090,31 +1090,30 @@ select jsonb_pretty('{"a":["b", "c"], "d": {"e":"f"}}'); --@ select pg_column_size('{"aa":1, "b":2}'::jsonb || '{}'::jsonb) = pg_column_size('{"aa":1, "b":2}'::jsonb); --@ select pg_column_size('{}'::jsonb || '{"aa":1, "b":2}'::jsonb) = pg_column_size('{"aa":1, "b":2}'::jsonb); ---@ select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'a'); ---@ select jsonb_delete('{"a":null , "b":2, "c":3}'::jsonb, 'a'); ---@ select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'b'); ---@ select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'c'); ---@ select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'd'); ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - 'a'; ---@ select '{"a":null , "b":2, "c":3}'::jsonb - 'a'; ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - 'b'; ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - 'c'; ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - 'd'; +select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'a'); +select jsonb_delete('{"a":null , "b":2, "c":3}'::jsonb, 'a'); +select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'b'); +select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'c'); +select jsonb_delete('{"a":1 , "b":2, "c":3}'::jsonb, 'd'); +select '{"a":1 , "b":2, "c":3}'::jsonb - 'a'; +select '{"a":null , "b":2, "c":3}'::jsonb - 'a'; +select '{"a":1 , "b":2, "c":3}'::jsonb - 'b'; +select '{"a":1 , "b":2, "c":3}'::jsonb - 'c'; +select '{"a":1 , "b":2, "c":3}'::jsonb - 'd'; --@ select pg_column_size('{"a":1 , "b":2, "c":3}'::jsonb - 'b') = pg_column_size('{"a":1, "b":2}'::jsonb); ---@ ---@ select '["a","b","c"]'::jsonb - 3; ---@ select '["a","b","c"]'::jsonb - 2; ---@ select '["a","b","c"]'::jsonb - 1; ---@ select '["a","b","c"]'::jsonb - 0; ---@ select '["a","b","c"]'::jsonb - -1; ---@ select '["a","b","c"]'::jsonb - -2; ---@ select '["a","b","c"]'::jsonb - -3; ---@ select '["a","b","c"]'::jsonb - -4; ---@ ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - '{b}'::text[]; ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - '{c,b}'::text[]; ---@ select '{"a":1 , "b":2, "c":3}'::jsonb - '{}'::text[]; ---@ + +select '["a","b","c"]'::jsonb - 3; +select '["a","b","c"]'::jsonb - 2; +select '["a","b","c"]'::jsonb - 1; +select '["a","b","c"]'::jsonb - 0; +select '["a","b","c"]'::jsonb - -1; +select '["a","b","c"]'::jsonb - -2; +select '["a","b","c"]'::jsonb - -3; +select '["a","b","c"]'::jsonb - -4; +select '{"a":1 , "b":2, "c":3}'::jsonb - '{b}'::text[]; +select '{"a":1 , "b":2, "c":3}'::jsonb - '{c,b}'::text[]; +select '{"a":1 , "b":2, "c":3}'::jsonb - '{}'::text[]; + --@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{n}', '[1,2,3]'); --@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '[1,2,3]'); --@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{d,1,0}', '[1,2,3]'); @@ -1127,28 +1126,28 @@ select jsonb_pretty('{"a":["b", "c"], "d": {"e":"f"}}'); --@ --@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '"test"'); --@ select jsonb_set('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb, '{b,-1}', '{"f": "test"}'); ---@ ---@ select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{n}'); ---@ select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{b,-1}'); ---@ select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{d,1,0}'); ---@ ---@ select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{n}'; ---@ select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{b,-1}'; ---@ select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{b,-1e}'; -- invalid array subscript ---@ select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{d,1,0}'; ---@ ---@ ---@ -- empty structure and error conditions for delete and replace ---@ ---@ select '"a"'::jsonb - 'a'; -- error ---@ select '{}'::jsonb - 'a'; ---@ select '[]'::jsonb - 'a'; ---@ select '"a"'::jsonb - 1; -- error ---@ select '{}'::jsonb - 1; -- error ---@ select '[]'::jsonb - 1; ---@ select '"a"'::jsonb #- '{a}'; -- error ---@ select '{}'::jsonb #- '{a}'; ---@ select '[]'::jsonb #- '{a}'; + +select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{n}'); +select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{b,-1}'); +select jsonb_delete_path('{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}', '{d,1,0}'); + +select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{n}'; +select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{b,-1}'; +select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{b,-1e}'; -- invalid array subscript +select '{"n":null, "a":1, "b":[1,2], "c":{"1":2}, "d":{"1":[2,3]}}'::jsonb #- '{d,1,0}'; + + +-- empty structure and error conditions for delete and replace + +select '"a"'::jsonb - 'a'; -- error +select '{}'::jsonb - 'a'; +select '[]'::jsonb - 'a'; +select '"a"'::jsonb - 1; -- error +select '{}'::jsonb - 1; -- error +select '[]'::jsonb - 1; +select '"a"'::jsonb #- '{a}'; -- error +select '{}'::jsonb #- '{a}'; +select '[]'::jsonb #- '{a}'; --@ select jsonb_set('"a"','{a}','"b"'); --error --@ select jsonb_set('{}','{a}','"b"', false); --@ select jsonb_set('[]','{1}','"b"', false); From 4fde65f08098e7c6d7272131ea56259d636a6b38 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Oct 2023 10:16:35 +0000 Subject: [PATCH 49/52] chore(deps): Bump simd-json from 0.12.0 to 0.13.3 (#13119) Signed-off-by: dependabot[bot] Signed-off-by: Runji Wang Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Runji Wang --- Cargo.lock | 29 ++++++++++++++++--- src/connector/Cargo.toml | 2 +- .../src/parser/canal/simd_json_parser.rs | 3 +- src/connector/src/parser/common.rs | 3 +- .../src/parser/debezium/mongo_json_parser.rs | 3 +- .../src/parser/debezium/simd_json_parser.rs | 3 +- src/connector/src/parser/unified/json.rs | 5 +++- 7 files changed, 38 insertions(+), 10 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 92bbc847031cc..73e8eab121d60 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -6721,6 +6721,26 @@ dependencies = [ "bitflags 1.3.2", ] +[[package]] +name = "ref-cast" +version = "1.0.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "acde58d073e9c79da00f2b5b84eed919c8326832648a5b109b3fce1bb1175280" +dependencies = [ + "ref-cast-impl", +] + +[[package]] +name = "ref-cast-impl" +version = "1.0.20" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "7f7473c2cfcf90008193dd0e3e16599455cb601a9fce322b5bb55de799664925" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.37", +] + [[package]] name = "regex" version = "1.10.0" @@ -9130,13 +9150,14 @@ dependencies = [ [[package]] name = "simd-json" -version = "0.12.0" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f0f07a84c7456b901b8dd2c1d44caca8b0fd2c2616206ee5acc9d9da61e8d9ec" +checksum = "d123f285a3635e423ec2ef5b67e0168dcf86c0d62fffbcea88fcd1c926e47413" dependencies = [ "getrandom", "halfbrown", "lexical-core", + "ref-cast", "serde", "serde_json", "simdutf8", @@ -10596,9 +10617,9 @@ checksum = "d92ccd67fb88503048c01b59152a04effd0782d035a83a6d256ce6085f08f4a3" [[package]] name = "value-trait" -version = "0.6.1" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "09a5b6c8ceb01263b969cac48d4a6705134d490ded13d889e52c0cfc80c6945e" +checksum = "ea87257cfcbedcb9444eda79c59fdfea71217e6305afee8ee33f500375c2ac97" dependencies = [ "float-cmp", "halfbrown", diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 8331adb8dc20a..fe05cfaf523bd 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -108,7 +108,7 @@ serde = { version = "1", features = ["derive", "rc"] } serde_derive = "1" serde_json = "1" serde_with = { version = "3", features = ["json"] } -simd-json = "0.12.0" +simd-json = "0.13.3" strum = "0.25" strum_macros = "0.25" tempfile = "3" diff --git a/src/connector/src/parser/canal/simd_json_parser.rs b/src/connector/src/parser/canal/simd_json_parser.rs index f238af2053ceb..bea0561330774 100644 --- a/src/connector/src/parser/canal/simd_json_parser.rs +++ b/src/connector/src/parser/canal/simd_json_parser.rs @@ -15,7 +15,8 @@ use itertools::Itertools; use risingwave_common::error::ErrorCode::{self, ProtocolError}; use risingwave_common::error::{Result, RwError}; -use simd_json::{BorrowedValue, Mutable, ValueAccess}; +use simd_json::prelude::{MutableObject, ValueAsScalar, ValueObjectAccess}; +use simd_json::BorrowedValue; use crate::only_parse_payload; use crate::parser::canal::operators::*; diff --git a/src/connector/src/parser/common.rs b/src/connector/src/parser/common.rs index 5a288dfd80b8d..cc30fabe0f475 100644 --- a/src/connector/src/parser/common.rs +++ b/src/connector/src/parser/common.rs @@ -12,7 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use simd_json::{BorrowedValue, ValueAccess}; +use simd_json::prelude::ValueAsContainer; +use simd_json::BorrowedValue; /// Get a value from a json object by key, case insensitive. /// diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 1217a49c42c9c..e4baacb895b62 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -17,7 +17,8 @@ use std::fmt::Debug; use risingwave_common::error::ErrorCode::{self, ProtocolError}; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::DataType; -use simd_json::{BorrowedValue, Mutable}; +use simd_json::prelude::MutableObject; +use simd_json::BorrowedValue; use crate::only_parse_payload; use crate::parser::unified::debezium::{DebeziumChangeEvent, MongoProjection}; diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index ca4313c43edcb..9ee966456799b 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -15,7 +15,8 @@ use std::fmt::Debug; use risingwave_common::error::{ErrorCode, Result, RwError}; -use simd_json::{BorrowedValue, Mutable}; +use simd_json::prelude::MutableObject; +use simd_json::BorrowedValue; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; use crate::parser::unified::AccessImpl; diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index 79590749351d2..b770649b052c9 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -23,7 +23,10 @@ use risingwave_common::types::{ DataType, Date, Decimal, Int256, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, }; use risingwave_common::util::iter_util::ZipEqFast; -use simd_json::{BorrowedValue, ValueAccess, ValueType}; +use simd_json::prelude::{ + TypedValue, ValueAsContainer, ValueAsScalar, ValueObjectAccess, ValueTryAsScalar, +}; +use simd_json::{BorrowedValue, ValueType}; use super::{Access, AccessError, AccessResult}; use crate::parser::common::json_object_get_case_insensitive; From 9e06e8828fdee621423a44dabfffa4e2453b4e09 Mon Sep 17 00:00:00 2001 From: emile-00 <106993396+emile-00@users.noreply.github.com> Date: Mon, 30 Oct 2023 11:15:29 -0700 Subject: [PATCH 50/52] chore: Update README architecture image (#13158) --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 091bccc0de9b4..c6d4a8b14c8ba 100644 --- a/README.md +++ b/README.md @@ -56,7 +56,7 @@ RisingWave is a distributed SQL streaming database that enables cost-efficient and reliable processing of streaming data. -![RisingWave](https://github.com/risingwavelabs/risingwave-docs/blob/0f7e1302b22493ba3c1c48e78810750ce9a5ff42/docs/images/archi_simple.png) +![RisingWave](https://github.com/risingwavelabs/risingwave-docs/blob/main/docs/images/new_archi_grey.png) ## Try it out in 5 minutes **Mac** From 16aabb212bd467c2d4bc14c9eb5208ee89a238d3 Mon Sep 17 00:00:00 2001 From: Yingjun Wu Date: Mon, 30 Oct 2023 11:48:09 -0700 Subject: [PATCH 51/52] Add docker info in README.md (#13160) --- README.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/README.md b/README.md index c6d4a8b14c8ba..51ee4b19a5cc0 100644 --- a/README.md +++ b/README.md @@ -40,6 +40,12 @@ > Slack + + Docker + Date: Tue, 31 Oct 2023 09:04:57 +0800 Subject: [PATCH 52/52] chore(deps): bump reqsign to 0.14.3 and remove patches (#13157) Signed-off-by: TennyZhuang --- Cargo.lock | 32 ++++++++++++++++++++++---------- Cargo.toml | 5 ----- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 73e8eab121d60..477cbf02bc3c0 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2551,8 +2551,9 @@ checksum = "86e3bdc80eee6e16b2b6b0f87fbc98c04bee3455e35174c0de1a125d0688c632" [[package]] name = "dlv-list" -version = "0.5.1" -source = "git+https://github.com/sgodwincs/dlv-list-rs.git?rev=5bbc5d0#5bbc5d0cc84f257e173d851f8dc1674fb6e46f95" +version = "0.5.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "442039f5147480ba31067cb00ada1adae6892028e40e45fc5de7b7df6dcc1b5f" dependencies = [ "const-random", ] @@ -5524,11 +5525,12 @@ dependencies = [ [[package]] name = "ordered-multimap" -version = "0.6.0" -source = "git+https://github.com/risingwavelabs/ordered-multimap-rs.git?rev=19c743f#19c743f3e3d106c99ba37628f06a2ca6faa2284f" +version = "0.7.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a4d6a8c22fc714f0c2373e6091bf6f5e9b37b1bc0b1184874b7e0a4e303d318f" dependencies = [ "dlv-list", - "hashbrown 0.13.2", + "hashbrown 0.14.0", ] [[package]] @@ -6574,6 +6576,16 @@ dependencies = [ "serde", ] +[[package]] +name = "quick-xml" +version = "0.31.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1004a344b30a54e2ee58d66a71b32d2db2feb0a31f9a2d302bf0536f15de2a33" +dependencies = [ + "memchr", + "serde", +] + [[package]] name = "quote" version = "1.0.33" @@ -6802,9 +6814,9 @@ dependencies = [ [[package]] name = "reqsign" -version = "0.14.1" +version = "0.14.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3228e570df74d69d3d3236a71371f1edd748a3e4eb728ea1f29d403bc10fc727" +checksum = "1ad14258ddd8ef6e564d57a94613e138cc9c21ef8a1fec547206d853213c7959" dependencies = [ "anyhow", "async-trait", @@ -6819,7 +6831,7 @@ dependencies = [ "log", "once_cell", "percent-encoding", - "quick-xml 0.29.0", + "quick-xml 0.31.0", "rand", "reqwest", "rsa", @@ -8353,9 +8365,9 @@ dependencies = [ [[package]] name = "rust-ini" -version = "0.19.0" +version = "0.20.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7e2a3bcec1f113553ef1c88aae6c020a369d03d55b58de9869a0908930385091" +checksum = "3e0698206bcb8882bf2a9ecb4c1e7785db57ff052297085a6efd4fe42302068a" dependencies = [ "cfg-if", "ordered-multimap", diff --git a/Cargo.toml b/Cargo.toml index f0fbb8485d318..a312a5fe045af 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -243,8 +243,3 @@ tokio-retry = { git = "https://github.com/madsim-rs/rust-tokio-retry.git", rev = tokio-postgres = { git = "https://github.com/madsim-rs/rust-postgres.git", rev = "ac00d88" } # patch: unlimit 4MB message size for grpc client etcd-client = { git = "https://github.com/risingwavelabs/etcd-client.git", rev = "4e84d40" } - -# Patch for coverage_attribute. -# https://github.com/sgodwincs/dlv-list-rs/pull/19#issuecomment-1774786289 -dlv-list = { git = "https://github.com/sgodwincs/dlv-list-rs.git", rev = "5bbc5d0" } -ordered-multimap = { git = "https://github.com/risingwavelabs/ordered-multimap-rs.git", rev = "19c743f" }