From 2513cbab0fff771eb61bf130be5ef1a0f0bcf694 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 5 Jan 2024 12:08:12 +0800 Subject: [PATCH 01/12] test: fix flaky sink_into_table test (#14372) --- e2e_test/sink/sink_into_table/rename.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e_test/sink/sink_into_table/rename.slt b/e2e_test/sink/sink_into_table/rename.slt index 2d6f35ca31dd7..19b24c9c20b4a 100644 --- a/e2e_test/sink/sink_into_table/rename.slt +++ b/e2e_test/sink/sink_into_table/rename.slt @@ -27,11 +27,11 @@ ALTER TABLE source RENAME TO src; statement ok ALTER TABLE target RENAME TO tar; -query TT +query TT rowsort SELECT name, definition from rw_tables; ---- -tar CREATE TABLE tar (v INT) src CREATE TABLE src (v INT) APPEND ONLY +tar CREATE TABLE tar (v INT) query TT SELECT name, definition from rw_sinks; From cb825f0f3f1611b8ef82daf032bc0f184fc9ac88 Mon Sep 17 00:00:00 2001 From: xxchan Date: Fri, 5 Jan 2024 12:08:43 +0800 Subject: [PATCH 02/12] test: bump sqllogictest & fix backwards-compat-test/e2e-tests (#14354) --- Makefile.toml | 2 +- .../{delete.slt => delete.slt.part} | 0 .../validate_restart.slt | 8 ++ .../{delete.slt => delete.slt.part} | 0 .../{insert.slt => insert.slt.part} | 0 .../validate_restart.slt | 8 ++ ci/Dockerfile | 4 +- ci/build-ci-image.sh | 2 +- ci/docker-compose.yml | 10 +-- e2e_test/batch/duckdb/all.slt.part | 9 +- e2e_test/batch/local_mode.slt | 1 - e2e_test/extended_mode/type.slt | 2 +- .../batch/rank_func/row_number_old.slt.part | 84 +++++++++++++++++++ .../rank_func/row_number_old.slt.part | 84 +++++++++++++++++++ ...umber_old.part => row_number_old.slt.part} | 0 15 files changed, 197 insertions(+), 17 deletions(-) rename backwards-compat-tests/slt/nexmark-backwards-compat/{delete.slt => delete.slt.part} (100%) rename backwards-compat-tests/slt/tpch-backwards-compat/{delete.slt => delete.slt.part} (100%) rename backwards-compat-tests/slt/tpch-backwards-compat/{insert.slt => insert.slt.part} (100%) create mode 100644 e2e_test/over_window/generated/batch/rank_func/row_number_old.slt.part create mode 100644 e2e_test/over_window/generated/streaming/rank_func/row_number_old.slt.part rename e2e_test/over_window/templates/rank_func/{row_number_old.part => row_number_old.slt.part} (100%) diff --git a/Makefile.toml b/Makefile.toml index 9aec344d74d1f..347f2234e5fda 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -1332,7 +1332,7 @@ echo "All processes has exited." [tasks.slt] category = "RiseDev - SQLLogicTest" -install_crate = { version = "0.18.0", crate_name = "sqllogictest-bin", binary = "sqllogictest", test_arg = [ +install_crate = { version = "0.19.1", crate_name = "sqllogictest-bin", binary = "sqllogictest", test_arg = [ "--help", ], install_command = "binstall" } dependencies = ["check-risedev-env-file"] diff --git a/backwards-compat-tests/slt/nexmark-backwards-compat/delete.slt b/backwards-compat-tests/slt/nexmark-backwards-compat/delete.slt.part similarity index 100% rename from backwards-compat-tests/slt/nexmark-backwards-compat/delete.slt rename to backwards-compat-tests/slt/nexmark-backwards-compat/delete.slt.part diff --git a/backwards-compat-tests/slt/nexmark-backwards-compat/validate_restart.slt b/backwards-compat-tests/slt/nexmark-backwards-compat/validate_restart.slt index 093f88f4fdbbe..7a0aede6e2236 100644 --- a/backwards-compat-tests/slt/nexmark-backwards-compat/validate_restart.slt +++ b/backwards-compat-tests/slt/nexmark-backwards-compat/validate_restart.slt @@ -1,9 +1,17 @@ include ../nexmark/test_mv_result.slt.part include ./delete.slt.part + +statement ok +flush; + include ../nexmark/insert_person.slt.part include ../nexmark/insert_auction.slt.part include ../nexmark/insert_bid.slt.part + +statement ok +flush; + include ../nexmark/test_mv_result.slt.part include ../nexmark/drop_views.slt.part diff --git a/backwards-compat-tests/slt/tpch-backwards-compat/delete.slt b/backwards-compat-tests/slt/tpch-backwards-compat/delete.slt.part similarity index 100% rename from backwards-compat-tests/slt/tpch-backwards-compat/delete.slt rename to backwards-compat-tests/slt/tpch-backwards-compat/delete.slt.part diff --git a/backwards-compat-tests/slt/tpch-backwards-compat/insert.slt b/backwards-compat-tests/slt/tpch-backwards-compat/insert.slt.part similarity index 100% rename from backwards-compat-tests/slt/tpch-backwards-compat/insert.slt rename to backwards-compat-tests/slt/tpch-backwards-compat/insert.slt.part diff --git a/backwards-compat-tests/slt/tpch-backwards-compat/validate_restart.slt b/backwards-compat-tests/slt/tpch-backwards-compat/validate_restart.slt index 7c7334cc222d3..81e00f92f0ca1 100644 --- a/backwards-compat-tests/slt/tpch-backwards-compat/validate_restart.slt +++ b/backwards-compat-tests/slt/tpch-backwards-compat/validate_restart.slt @@ -23,7 +23,15 @@ include ../tpch/q22.slt.part # Test deletes and updates should work as per normal. include ./delete.slt.part + +statement ok +flush; + include ./insert.slt.part + +statement ok +flush; + include ../tpch/q1.slt.part include ../tpch/q2.slt.part include ../tpch/q3.slt.part diff --git a/ci/Dockerfile b/ci/Dockerfile index 427e6d68dc116..810a3289b66a8 100644 --- a/ci/Dockerfile +++ b/ci/Dockerfile @@ -48,8 +48,8 @@ ENV CARGO_REGISTRIES_CRATES_IO_PROTOCOL=sparse RUN curl -L --proto '=https' --tlsv1.2 -sSf https://raw.githubusercontent.com/cargo-bins/cargo-binstall/main/install-from-binstall-release.sh | bash RUN cargo binstall -y --no-symlinks cargo-llvm-cov cargo-nextest cargo-hakari cargo-sort cargo-cache cargo-audit \ cargo-make@0.36.10 \ - sqllogictest-bin@0.18.0 \ - && cargo install sccache \ + sqllogictest-bin@0.19.1 \ + sccache@0.7.4 \ && cargo cache -a \ && rm -rf "/root/.cargo/registry/index" \ && rm -rf "/root/.cargo/registry/cache" \ diff --git a/ci/build-ci-image.sh b/ci/build-ci-image.sh index 93718b1e181be..bbdc5d7bfa9ed 100755 --- a/ci/build-ci-image.sh +++ b/ci/build-ci-image.sh @@ -10,7 +10,7 @@ cat ../rust-toolchain # shellcheck disable=SC2155 # REMEMBER TO ALSO UPDATE ci/docker-compose.yml -export BUILD_ENV_VERSION=v20231226 +export BUILD_ENV_VERSION=v20240104_1 export BUILD_TAG="public.ecr.aws/x5u3w5h6/rw-build-env:${BUILD_ENV_VERSION}" diff --git a/ci/docker-compose.yml b/ci/docker-compose.yml index e4ba36d5b79a2..0ebb9e77eeb20 100644 --- a/ci/docker-compose.yml +++ b/ci/docker-compose.yml @@ -71,7 +71,7 @@ services: retries: 5 source-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231226 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20240104_1 depends_on: - mysql - db @@ -81,7 +81,7 @@ services: - ..:/risingwave sink-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231226 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20240104_1 depends_on: - mysql - db @@ -93,12 +93,12 @@ services: - ..:/risingwave rw-build-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231226 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20240104_1 volumes: - ..:/risingwave ci-flamegraph-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231226 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20240104_1 # NOTE(kwannoel): This is used in order to permit # syscalls for `nperf` (perf_event_open), # so it can do CPU profiling. @@ -109,7 +109,7 @@ services: - ..:/risingwave regress-test-env: - image: public.ecr.aws/x5u3w5h6/rw-build-env:v20231226 + image: public.ecr.aws/x5u3w5h6/rw-build-env:v20240104_1 depends_on: db: condition: service_healthy diff --git a/e2e_test/batch/duckdb/all.slt.part b/e2e_test/batch/duckdb/all.slt.part index 68adce73dea24..ca5d1b5201c45 100644 --- a/e2e_test/batch/duckdb/all.slt.part +++ b/e2e_test/batch/duckdb/all.slt.part @@ -1,9 +1,6 @@ -include ./aggregate/*.slt.part -include ./aggregate/*/*.slt.part -include ./join/*.slt.part -include ./join/*/*.slt.part -include ./conjunction/*.slt.part -include ./conjunction/*/*.slt.part +include ./aggregate/**/*.slt.part +include ./join/**/*.slt.part +include ./conjunction/**/*.slt.part include ./limit/*.slt.part include ./select/*.slt.part include ./cte/*.slt.part diff --git a/e2e_test/batch/local_mode.slt b/e2e_test/batch/local_mode.slt index 5de337518d433..c3818989443a7 100644 --- a/e2e_test/batch/local_mode.slt +++ b/e2e_test/batch/local_mode.slt @@ -5,7 +5,6 @@ statement ok SET QUERY_MODE TO local; include ./basic/*.slt.part -include ./basic/local/*.slt.part include ./duckdb/all.slt.part include ./order/*.slt.part include ./join/*.slt.part diff --git a/e2e_test/extended_mode/type.slt b/e2e_test/extended_mode/type.slt index 2271ecb51c5c9..b172fcf389abc 100644 --- a/e2e_test/extended_mode/type.slt +++ b/e2e_test/extended_mode/type.slt @@ -21,7 +21,7 @@ SET RW_IMPLICIT_FLUSH TO true; include ../batch/types/boolean.slt.part include ../batch/types/cast.slt.part include ../batch/types/date.slt -include ../batch/types/intercal.slt.part +include ../batch/types/interval.slt.part include ../batch/types/number_arithmetic.slt.part include ../batch/types/temporal_arithmetic.slt.part include ../batch/types/time.slt.part diff --git a/e2e_test/over_window/generated/batch/rank_func/row_number_old.slt.part b/e2e_test/over_window/generated/batch/rank_func/row_number_old.slt.part new file mode 100644 index 0000000000000..24f47a8116451 --- /dev/null +++ b/e2e_test/over_window/generated/batch/rank_func/row_number_old.slt.part @@ -0,0 +1,84 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create view v as +select + * + , row_number() over (partition by p1 order by time, id) as out1 + , row_number() over (partition by p1 order by p2 desc, id) as out2 +from t; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 +100005 100 200 3 717 810 4 4 +100006 105 204 5 703 828 1 1 + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100002 100 200 2 799 806 3 2 +100003 100 200 2 723 807 4 3 +100004 103 200 2 702 808 1 1 +100005 100 200 1 717 810 2 4 +100006 105 204 5 703 828 1 1 + +statement ok +delete from t where time = 2; + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100005 100 200 1 717 810 2 2 +100006 105 204 5 703 828 1 1 + +statement ok +drop view v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/generated/streaming/rank_func/row_number_old.slt.part b/e2e_test/over_window/generated/streaming/rank_func/row_number_old.slt.part new file mode 100644 index 0000000000000..39dec472b5b12 --- /dev/null +++ b/e2e_test/over_window/generated/streaming/rank_func/row_number_old.slt.part @@ -0,0 +1,84 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +statement ok +create table t ( + id int + , p1 int + , p2 int + , time int + , v1 int + , v2 int +); + +statement ok +create materialized view v as +select + * + , row_number() over (partition by p1 order by time, id) as out1 + , row_number() over (partition by p1 order by p2 desc, id) as out2 +from t; + +statement ok +insert into t values + (100001, 100, 200, 1, 701, 805) +, (100002, 100, 200, 2, 700, 806) +, (100003, 100, 208, 2, 723, 807) +, (100004, 103, 200, 2, 702, 808); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 + +statement ok +insert into t values + (100005, 100, 200, 3, 717, 810) +, (100006, 105, 204, 5, 703, 828); + +query II +select * from v order by id; +---- +100001 100 200 1 701 805 1 2 +100002 100 200 2 700 806 2 3 +100003 100 208 2 723 807 3 1 +100004 103 200 2 702 808 1 1 +100005 100 200 3 717 810 4 4 +100006 105 204 5 703 828 1 1 + +statement ok +update t set v1 = 799 where id = 100002; -- value change + +statement ok +update t set p2 = 200 where id = 100003; -- partition change + +statement ok +update t set "time" = 1 where id = 100005; -- order change + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100002 100 200 2 799 806 3 2 +100003 100 200 2 723 807 4 3 +100004 103 200 2 702 808 1 1 +100005 100 200 1 717 810 2 4 +100006 105 204 5 703 828 1 1 + +statement ok +delete from t where time = 2; + +query iiiiiii +select * from v order by id; +---- +100001 100 200 1 701 805 1 1 +100005 100 200 1 717 810 2 2 +100006 105 204 5 703 828 1 1 + +statement ok +drop materialized view v; + +statement ok +drop table t; diff --git a/e2e_test/over_window/templates/rank_func/row_number_old.part b/e2e_test/over_window/templates/rank_func/row_number_old.slt.part similarity index 100% rename from e2e_test/over_window/templates/rank_func/row_number_old.part rename to e2e_test/over_window/templates/rank_func/row_number_old.slt.part From 7a00e58eb76954ece665bacce2cca2e2330dacc8 Mon Sep 17 00:00:00 2001 From: August Date: Fri, 5 Jan 2024 12:16:42 +0800 Subject: [PATCH 03/12] fix: Do not recover background streaming jobs whose table fragments have been marked as created (#14367) Co-authored-by: zwang28 <84491488@qq.com> --- ci/scripts/cron-e2e-test.sh | 2 +- src/meta/src/barrier/recovery.rs | 66 +++++++++++------------ src/meta/src/manager/catalog/fragment.rs | 69 ------------------------ 3 files changed, 31 insertions(+), 106 deletions(-) diff --git a/ci/scripts/cron-e2e-test.sh b/ci/scripts/cron-e2e-test.sh index 2e8c56c3c1d5f..52c7a6faa366c 100755 --- a/ci/scripts/cron-e2e-test.sh +++ b/ci/scripts/cron-e2e-test.sh @@ -4,7 +4,7 @@ set -euo pipefail source ci/scripts/common.sh -export RUN_COMPACTION=1; +export RUN_COMPACTION=0; export RUN_META_BACKUP=1; export RUN_DELETE_RANGE=1; source ci/scripts/run-e2e-test.sh diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index a122699bf56c3..4a5b13c03caee 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -44,7 +44,7 @@ use crate::barrier::progress::CreateMviewProgressTracker; use crate::barrier::{CheckpointControl, Command, GlobalBarrierManager}; use crate::controller::catalog::ReleaseContext; use crate::manager::{MetadataManager, WorkerId}; -use crate::model::{ActorId, BarrierManagerState, MetadataModel, MigrationPlan, TableFragments}; +use crate::model::{BarrierManagerState, MetadataModel, MigrationPlan, TableFragments}; use crate::stream::{build_actor_connector_splits, RescheduleOptions, TableResizePolicy}; use crate::MetaResult; @@ -149,48 +149,44 @@ impl GlobalBarrierManager { unreachable!() }; let mviews = mgr.catalog_manager.list_creating_background_mvs().await; - let creating_mview_ids = mviews.iter().map(|m| TableId::new(m.id)).collect_vec(); - let mview_definitions = mviews - .into_iter() - .map(|m| (TableId::new(m.id), m.definition)) - .collect::>(); + let mut mview_definitions = HashMap::new(); + let mut table_map = HashMap::new(); + let mut table_fragment_map = HashMap::new(); + let mut upstream_mv_counts = HashMap::new(); let mut senders = HashMap::new(); let mut receivers = Vec::new(); - for table_id in creating_mview_ids.iter().copied() { - let (finished_tx, finished_rx) = oneshot::channel(); - senders.insert( - table_id, - Notifier { - finished: Some(finished_tx), - ..Default::default() - }, - ); - + for mview in mviews { + let table_id = TableId::new(mview.id); let fragments = mgr .fragment_manager .select_table_fragments_by_table_id(&table_id) .await?; let internal_table_ids = fragments.internal_table_ids(); let internal_tables = mgr.catalog_manager.get_tables(&internal_table_ids).await; - let table = mgr.catalog_manager.get_tables(&[table_id.table_id]).await; - assert_eq!(table.len(), 1, "should only have 1 materialized table"); - let table = table.into_iter().next().unwrap(); - receivers.push((table, internal_tables, finished_rx)); + if fragments.is_created() { + // If the mview is already created, we don't need to recover it. + mgr.catalog_manager + .finish_create_table_procedure(internal_tables, mview) + .await?; + tracing::debug!("notified frontend for stream job {}", table_id.table_id); + } else { + table_map.insert(table_id, fragments.backfill_actor_ids()); + mview_definitions.insert(table_id, mview.definition.clone()); + upstream_mv_counts.insert(table_id, fragments.dependent_table_ids()); + table_fragment_map.insert(table_id, fragments); + let (finished_tx, finished_rx) = oneshot::channel(); + senders.insert( + table_id, + Notifier { + finished: Some(finished_tx), + ..Default::default() + }, + ); + receivers.push((mview, internal_tables, finished_rx)); + } } - let table_map = mgr - .fragment_manager - .get_table_id_stream_scan_actor_mapping(&creating_mview_ids) - .await; - let table_fragment_map = mgr - .fragment_manager - .get_table_id_table_fragment_map(&creating_mview_ids) - .await?; - let upstream_mv_counts = mgr - .fragment_manager - .get_upstream_relation_counts(&creating_mview_ids) - .await; let version_stats = self.hummock_manager.get_version_stats().await; // If failed, enter recovery mode. { @@ -252,6 +248,7 @@ impl GlobalBarrierManager { let mut receivers = Vec::new(); let mut table_fragment_map = HashMap::new(); let mut mview_definitions = HashMap::new(); + let mut table_map = HashMap::new(); let mut upstream_mv_counts = HashMap::new(); for mview in &mviews { let (finished_tx, finished_rx) = oneshot::channel(); @@ -270,15 +267,12 @@ impl GlobalBarrierManager { .await?; let table_fragments = TableFragments::from_protobuf(table_fragments); upstream_mv_counts.insert(table_id, table_fragments.dependent_table_ids()); + table_map.insert(table_id, table_fragments.backfill_actor_ids()); table_fragment_map.insert(table_id, table_fragments); mview_definitions.insert(table_id, mview.definition.clone()); receivers.push((mview.table_id, finished_rx)); } - let table_map: HashMap> = table_fragment_map - .iter() - .map(|(id, tf)| (*id, tf.actor_ids().into_iter().collect())) - .collect(); let version_stats = self.hummock_manager.get_version_stats().await; // If failed, enter recovery mode. { diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index a4128aaeb8f95..e5ba6503d79de 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -33,7 +33,6 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::update_mutation::MergeUpdate; use risingwave_pb::stream_plan::{ DispatchStrategy, Dispatcher, DispatcherType, FragmentTypeFlag, StreamActor, StreamNode, - StreamScanType, }; use tokio::sync::{RwLock, RwLockReadGuard}; @@ -170,74 +169,6 @@ impl FragmentManager { map.values().cloned().collect() } - /// The `table_ids` here should correspond to stream jobs. - /// We get their corresponding table fragment, and from there, - /// we get the actors that are in the table fragment. - pub async fn get_table_id_stream_scan_actor_mapping( - &self, - table_ids: &[TableId], - ) -> HashMap> { - let map = &self.core.read().await.table_fragments; - let mut table_map = HashMap::new(); - // TODO(kwannoel): Can this be unified with `PlanVisitor`? - fn has_backfill(stream_node: &StreamNode) -> bool { - let is_backfill = if let Some(node) = &stream_node.node_body - && let Some(node) = node.as_stream_scan() - { - node.stream_scan_type == StreamScanType::Backfill as i32 - || node.stream_scan_type == StreamScanType::ArrangementBackfill as i32 - } else { - false - }; - is_backfill || stream_node.get_input().iter().any(has_backfill) - } - for table_id in table_ids { - if let Some(table_fragment) = map.get(table_id) { - let mut actors = HashSet::new(); - for fragment in table_fragment.fragments.values() { - for actor in &fragment.actors { - if let Some(node) = &actor.nodes - && has_backfill(node) - { - actors.insert(actor.actor_id); - } else { - tracing::trace!("ignoring actor: {:?}", actor); - } - } - } - table_map.insert(*table_id, actors); - } - } - table_map - } - - /// Gets the counts for each upstream relation that each stream job - /// indicated by `table_ids` depends on. - /// For example in the following query: - /// ```sql - /// CREATE MATERIALIZED VIEW m1 AS - /// SELECT * FROM t1 JOIN t2 ON t1.a = t2.a JOIN t3 ON t2.b = t3.b - /// ``` - /// - /// We have t1 occurring once, and t2 occurring once. - pub async fn get_upstream_relation_counts( - &self, - table_ids: &[TableId], - ) -> HashMap> { - let map = &self.core.read().await.table_fragments; - let mut upstream_relation_counts = HashMap::new(); - for table_id in table_ids { - if let Some(table_fragments) = map.get(table_id) { - let dependent_ids = table_fragments.dependent_table_ids(); - let r = upstream_relation_counts.insert(*table_id, dependent_ids); - assert!(r.is_none(), "Each table_id should be unique!") - } else { - upstream_relation_counts.insert(*table_id, HashMap::new()); - } - } - upstream_relation_counts - } - pub fn get_mv_id_to_internal_table_ids_mapping(&self) -> Option)>> { match self.core.try_read() { Ok(core) => Some( From 19b996b11282aa8bd929b708c47f78954ddf17d3 Mon Sep 17 00:00:00 2001 From: Li0k Date: Fri, 5 Jan 2024 12:36:08 +0800 Subject: [PATCH 04/12] chore(storage): ctl support cancel specific task (#14325) --- proto/hummock.proto | 10 ++++++++++ src/ctl/src/cmd_impl/hummock/compaction_group.rs | 11 +++++++++++ src/ctl/src/lib.rs | 8 ++++++++ src/meta/service/src/hummock_service.rs | 14 ++++++++++++++ src/rpc_client/src/meta_client.rs | 11 +++++++++++ 5 files changed, 54 insertions(+) diff --git a/proto/hummock.proto b/proto/hummock.proto index ccbfef42278f1..58007117811d5 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -750,6 +750,15 @@ message ListCompactTaskProgressResponse { repeated CompactTaskProgress task_progress = 1; } +message CancelCompactTaskRequest { + uint64 task_id = 1; + CompactTask.TaskStatus task_status = 2; +} + +message CancelCompactTaskResponse { + bool ret = 1; +} + service HummockManagerService { rpc UnpinVersionBefore(UnpinVersionBeforeRequest) returns (UnpinVersionBeforeResponse); rpc GetCurrentVersion(GetCurrentVersionRequest) returns (GetCurrentVersionResponse); @@ -788,6 +797,7 @@ service HummockManagerService { rpc GetCompactionScore(GetCompactionScoreRequest) returns (GetCompactionScoreResponse); rpc ListCompactTaskAssignment(ListCompactTaskAssignmentRequest) returns (ListCompactTaskAssignmentResponse); rpc ListCompactTaskProgress(ListCompactTaskProgressRequest) returns (ListCompactTaskProgressResponse); + rpc CancelCompactTask(CancelCompactTaskRequest) returns (CancelCompactTaskResponse); } message CompactionConfig { diff --git a/src/ctl/src/cmd_impl/hummock/compaction_group.rs b/src/ctl/src/cmd_impl/hummock/compaction_group.rs index 60ec391e9b7c5..068472dd1ce4a 100644 --- a/src/ctl/src/cmd_impl/hummock/compaction_group.rs +++ b/src/ctl/src/cmd_impl/hummock/compaction_group.rs @@ -18,6 +18,7 @@ use comfy_table::{Row, Table}; use itertools::Itertools; use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::{CompactionGroupId, HummockContextId}; +use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_pb::hummock::rise_ctl_update_compaction_config_request::mutable_config::MutableConfig; use crate::CtlContext; @@ -260,3 +261,13 @@ pub async fn get_compaction_score( println!("{table}"); Ok(()) } + +pub async fn cancel_compact_task(context: &CtlContext, task_id: u64) -> anyhow::Result<()> { + let meta_client = context.meta_client().await?; + let ret = meta_client + .cancel_compact_task(task_id, TaskStatus::ManualCanceled) + .await?; + println!("cancel_compact_task {} ret {:?}", task_id, ret); + + Ok(()) +} diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index 004b36302cf3b..95c9aa25a6b7a 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -266,6 +266,11 @@ enum HummockCommands { ValidateVersion, /// Rebuild table stats RebuildTableStats, + + CancelCompactTask { + #[clap(short, long)] + task_id: u64, + }, } #[derive(Subcommand)] @@ -655,6 +660,9 @@ pub async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { Commands::Hummock(HummockCommands::RebuildTableStats) => { cmd_impl::hummock::rebuild_table_stats(context).await?; } + Commands::Hummock(HummockCommands::CancelCompactTask { task_id }) => { + cmd_impl::hummock::cancel_compact_task(context, task_id).await?; + } Commands::Table(TableCommands::Scan { mv_name, data_dir }) => { cmd_impl::table::scan(context, mv_name, data_dir).await? } diff --git a/src/meta/service/src/hummock_service.rs b/src/meta/service/src/hummock_service.rs index 599b7a7cdb79d..a082b723dd124 100644 --- a/src/meta/service/src/hummock_service.rs +++ b/src/meta/service/src/hummock_service.rs @@ -651,6 +651,20 @@ impl HummockManagerService for HummockServiceImpl { task_progress, })) } + + async fn cancel_compact_task( + &self, + request: Request, + ) -> Result, Status> { + let request = request.into_inner(); + let ret = self + .hummock_manager + .cancel_compact_task(request.task_id, request.task_status()) + .await?; + + let response = Response::new(CancelCompactTaskResponse { ret }); + return Ok(response); + } } #[cfg(test)] diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index d8318d79b63b1..2dc51961c901e 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -55,6 +55,7 @@ use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::ddl_service::ddl_service_client::DdlServiceClient; use risingwave_pb::ddl_service::drop_table_request::SourceId; use risingwave_pb::ddl_service::*; +use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_pb::hummock::get_compaction_score_response::PickerInfo; use risingwave_pb::hummock::hummock_manager_service_client::HummockManagerServiceClient; use risingwave_pb::hummock::rise_ctl_update_compaction_config_request::mutable_config::MutableConfig; @@ -1250,6 +1251,15 @@ impl MetaClient { }) .join(); } + + pub async fn cancel_compact_task(&self, task_id: u64, task_status: TaskStatus) -> Result { + let req = CancelCompactTaskRequest { + task_id, + task_status: task_status as _, + }; + let resp = self.inner.cancel_compact_task(req).await?; + Ok(resp.ret) + } } #[async_trait] @@ -1891,6 +1901,7 @@ macro_rules! for_all_meta_rpc { ,{ hummock_client, list_hummock_meta_config, ListHummockMetaConfigRequest, ListHummockMetaConfigResponse } ,{ hummock_client, list_compact_task_assignment, ListCompactTaskAssignmentRequest, ListCompactTaskAssignmentResponse } ,{ hummock_client, list_compact_task_progress, ListCompactTaskProgressRequest, ListCompactTaskProgressResponse } + ,{ hummock_client, cancel_compact_task, CancelCompactTaskRequest, CancelCompactTaskResponse} ,{ user_client, create_user, CreateUserRequest, CreateUserResponse } ,{ user_client, update_user, UpdateUserRequest, UpdateUserResponse } ,{ user_client, drop_user, DropUserRequest, DropUserResponse } From 5a3d2652a63c9d75e596b2a21e7b732d279fcfce Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Fri, 5 Jan 2024 00:19:07 -0500 Subject: [PATCH 05/12] feat(connector): support local fs source (#14312) Co-authored-by: KeXiangWang --- ci/scripts/e2e-source-test.sh | 3 + ci/scripts/notify.py | 3 +- ci/workflows/main-cron.yml | 16 +++ e2e_test/s3/posix_fs_source.py | 136 ++++++++++++++++++ e2e_test/s3/run_csv.py | 1 - e2e_test/source/opendal/data/data1.csv | 6 + e2e_test/source/opendal/data/data2.csv | 6 + e2e_test/source/opendal/posix_fs.slt | 33 +++++ src/connector/src/macros.rs | 1 + src/connector/src/source/base.rs | 12 +- .../source/filesystem/opendal_source/mod.rs | 41 ++++++ .../opendal_source/posix_fs_source.rs | 58 ++++++++ src/connector/src/source/mod.rs | 4 +- src/connector/with_options_source.yaml | 9 ++ src/frontend/src/handler/create_source.rs | 5 +- src/source/src/connector_source.rs | 7 +- .../src/executor/source/fetch_executor.rs | 7 +- src/stream/src/from_proto/source/fs_fetch.rs | 15 +- 18 files changed, 353 insertions(+), 10 deletions(-) create mode 100644 e2e_test/s3/posix_fs_source.py create mode 100644 e2e_test/source/opendal/data/data1.csv create mode 100644 e2e_test/source/opendal/data/data2.csv create mode 100644 e2e_test/source/opendal/posix_fs.slt create mode 100644 src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index c1624230abc7e..64144d051ad58 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -56,6 +56,9 @@ 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 "--- opendal source test" +sqllogictest -p 4566 -d dev './e2e_test/source/opendal/**/*.slt' + echo "--- mysql & postgres cdc validate test" sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.validate.mysql.slt' sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.validate.postgres.slt' diff --git a/ci/scripts/notify.py b/ci/scripts/notify.py index 818dfce72143a..5266998b0045f 100755 --- a/ci/scripts/notify.py +++ b/ci/scripts/notify.py @@ -19,7 +19,8 @@ "e2e-java-binding-tests": ["yiming"], "e2e-clickhouse-sink-tests": ["bohan"], "e2e-pulsar-sink-tests": ["renjie"], - "s3-source-test-for-opendal-fs-engine": ["congyi"], + "s3-source-test-for-opendal-fs-engine": ["congyi", "kexiang"], + "s3-source-tests": ["congyi", "kexiang"], "pulsar-source-tests": ["renjie"], "connector-node-integration-test": ["siyuan"], } diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 75f58eadf2492..653578e4688e2 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -489,6 +489,22 @@ steps: timeout_in_minutes: 25 retry: *auto-retry + - label: "PosixFs source on OpenDAL fs engine (csv parser)" + command: "ci/scripts/s3-source-test.sh -p ci-release -s 'posix_fs_source.py csv_without_header'" + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-s3-source-tests" + || build.env("CI_STEPS") =~ /(^|,)s3-source-tests?(,|$$)/ + depends_on: build + plugins: + - docker-compose#v4.9.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 25 + retry: *auto-retry + - label: "S3 source on OpenDAL fs engine" key: "s3-source-test-for-opendal-fs-engine" command: "ci/scripts/s3-source-test-for-opendal-fs-engine.sh -p ci-release -s run" diff --git a/e2e_test/s3/posix_fs_source.py b/e2e_test/s3/posix_fs_source.py new file mode 100644 index 0000000000000..a7cea46fa496a --- /dev/null +++ b/e2e_test/s3/posix_fs_source.py @@ -0,0 +1,136 @@ +import os +import sys +import csv +import random +import psycopg2 +import opendal + +from time import sleep +from io import StringIO +from functools import partial + +def gen_data(file_num, item_num_per_file): + assert item_num_per_file % 2 == 0, \ + f'item_num_per_file should be even to ensure sum(mark) == 0: {item_num_per_file}' + return [ + [{ + 'id': file_id * item_num_per_file + item_id, + 'name': f'{file_id}_{item_id}', + 'sex': item_id % 2, + 'mark': (-1) ** (item_id % 2), + } for item_id in range(item_num_per_file)] + for file_id in range(file_num) + ] + +def format_csv(data, with_header): + csv_files = [] + + for file_data in data: + ostream = StringIO() + writer = csv.DictWriter(ostream, fieldnames=file_data[0].keys()) + if with_header: + writer.writeheader() + for item_data in file_data: + writer.writerow(item_data) + csv_files.append(ostream.getvalue()) + return csv_files + + +def do_test(file_num, item_num_per_file, prefix, fmt): + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + # Open a cursor to execute SQL statements + cur = conn.cursor() + + def _table(): + return f'posix_fs_test_{fmt}' + + def _encode(): + return f"CSV (delimiter = ',', without_header = {str('without' in fmt).lower()})" + + # Execute a SELECT statement + cur.execute(f'''CREATE TABLE {_table()}( + id int, + name TEXT, + sex int, + mark int, + ) WITH ( + connector = 'posix_fs', + match_pattern = '{prefix}*.{fmt}', + posix_fs.root = '/tmp', + ) FORMAT PLAIN ENCODE {_encode()};''') + + total_rows = file_num * item_num_per_file + MAX_RETRIES = 40 + for retry_no in range(MAX_RETRIES): + cur.execute(f'select count(*) from {_table()}') + result = cur.fetchone() + if result[0] == total_rows: + break + print(f"[retry {retry_no}] Now got {result[0]} rows in table, {total_rows} expected, wait 30s") + sleep(30) + + stmt = f'select count(*), sum(id), sum(sex), sum(mark) from {_table()}' + print(f'Execute {stmt}') + cur.execute(stmt) + result = cur.fetchone() + + print('Got:', result) + + def _assert_eq(field, got, expect): + assert got == expect, f'{field} assertion failed: got {got}, expect {expect}.' + + _assert_eq('count(*)', result[0], total_rows) + _assert_eq('sum(id)', result[1], (total_rows - 1) * total_rows / 2) + _assert_eq('sum(sex)', result[2], total_rows / 2) + _assert_eq('sum(mark)', result[3], 0) + + print('Test pass') + + cur.execute(f'drop table {_table()}') + cur.close() + conn.close() + + +if __name__ == "__main__": + FILE_NUM = 4001 + ITEM_NUM_PER_FILE = 2 + data = gen_data(FILE_NUM, ITEM_NUM_PER_FILE) + + fmt = sys.argv[1] + FORMATTER = { + 'csv_with_header': partial(format_csv, with_header=True), + 'csv_without_header': partial(format_csv, with_header=False), + } + assert fmt in FORMATTER, f"Unsupported format: {fmt}" + formatted_files = FORMATTER[fmt](data) + + run_id = str(random.randint(1000, 9999)) + _local = lambda idx: f'data_{idx}.{fmt}' + _posix = lambda idx: f"{run_id}_data_{idx}.{fmt}" + # put local files + op = opendal.Operator("fs", root="/tmp") + + print("write file to /tmp") + for idx, file_str in enumerate(formatted_files): + with open(_local(idx), "w") as f: + f.write(file_str) + os.fsync(f.fileno()) + file_name = _posix(idx) + file_bytes = file_str.encode('utf-8') + op.write(file_name, file_bytes) + + # do test + print("do test") + do_test(FILE_NUM, ITEM_NUM_PER_FILE, run_id, fmt) + + # clean up local files + print("clean up local files in /tmp") + for idx, _ in enumerate(formatted_files): + file_name = _posix(idx) + op.delete(file_name) diff --git a/e2e_test/s3/run_csv.py b/e2e_test/s3/run_csv.py index b721e3c796066..a6c0dc37bc4ca 100644 --- a/e2e_test/s3/run_csv.py +++ b/e2e_test/s3/run_csv.py @@ -1,5 +1,4 @@ import os -import string import json import string from time import sleep diff --git a/e2e_test/source/opendal/data/data1.csv b/e2e_test/source/opendal/data/data1.csv new file mode 100644 index 0000000000000..9279ffa24aa25 --- /dev/null +++ b/e2e_test/source/opendal/data/data1.csv @@ -0,0 +1,6 @@ +carat,cut,color,depth +0.25,Ideal,E,61.4 +0.22,Premium,I,62.0 +0.28,Good,J,63.1 +0.23,Very Good,H,57.5 +0.30,Fair,E,64.7 diff --git a/e2e_test/source/opendal/data/data2.csv b/e2e_test/source/opendal/data/data2.csv new file mode 100644 index 0000000000000..b5e39f73f2d51 --- /dev/null +++ b/e2e_test/source/opendal/data/data2.csv @@ -0,0 +1,6 @@ +carat,cut,color,depth +1.25,Ideal,E,61.4 +1.22,Premium,I,62.0 +1.28,Good,J,63.1 +1.23,Very Good,H,57.5 +1.30,Fair,E,64.7 diff --git a/e2e_test/source/opendal/posix_fs.slt b/e2e_test/source/opendal/posix_fs.slt new file mode 100644 index 0000000000000..3fc572a1a1cc8 --- /dev/null +++ b/e2e_test/source/opendal/posix_fs.slt @@ -0,0 +1,33 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +CREATE TABLE diamonds ( + carat FLOAT, + cut TEXT, + color TEXT, + depth FLOAT, +) WITH ( + connector = 'posix_fs', + match_pattern = 'data*.csv', + posix_fs.root = 'e2e_test/source/opendal/data', +) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ','); + +sleep 10s + +query TTTT rowsort +select * from diamonds; +---- +0.22 Premium I 62 +0.23 Very Good H 57.5 +0.25 Ideal E 61.4 +0.28 Good J 63.1 +0.3 Fair E 64.7 +1.22 Premium I 62 +1.23 Very Good H 57.5 +1.25 Ideal E 61.4 +1.28 Good J 63.1 +1.3 Fair E 64.7 + +statement ok +DROP TABLE diamonds; diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 47839f6dc5d5a..9a2383dbb4a96 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -35,6 +35,7 @@ macro_rules! for_all_classified_sources { { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, { Gcs, $crate::source::filesystem::opendal_source::GcsProperties , $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalGcs> }, { OpendalS3, $crate::source::filesystem::opendal_source::OpendalS3Properties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalS3> }, + { PosixFs, $crate::source::filesystem::opendal_source::PosixFsProperties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalPosixFs> }, { Test, $crate::source::test_source::TestSourceProperties, $crate::source::test_source::TestSourceSplit} } $( diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 1a4a594d44285..b6093a351783b 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -42,7 +42,7 @@ use super::google_pubsub::GooglePubsubMeta; use super::kafka::KafkaMeta; use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; -use super::OPENDAL_S3_CONNECTOR; +use super::{OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR}; use crate::parser::ParserConfig; pub(crate) use crate::source::common::CommonSplitReader; use crate::source::filesystem::FsPageItem; @@ -386,14 +386,20 @@ impl ConnectorProperties { pub fn is_new_fs_connector_b_tree_map(with_properties: &BTreeMap) -> bool { with_properties .get(UPSTREAM_SOURCE_KEY) - .map(|s| s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR)) + .map(|s| { + s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR) + || s.eq_ignore_ascii_case(POSIX_FS_CONNECTOR) + }) .unwrap_or(false) } pub fn is_new_fs_connector_hash_map(with_properties: &HashMap) -> bool { with_properties .get(UPSTREAM_SOURCE_KEY) - .map(|s| s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR)) + .map(|s| { + s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR) + || s.eq_ignore_ascii_case(POSIX_FS_CONNECTOR) + }) .unwrap_or(false) } } diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index 93e09be83ebf1..d6223c467d08b 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -15,6 +15,7 @@ use std::collections::HashMap; pub mod gcs_source; +pub mod posix_fs_source; pub mod s3_source; use serde::Deserialize; @@ -31,6 +32,7 @@ use crate::source::{SourceProperties, UnknownFields}; pub const GCS_CONNECTOR: &str = "gcs"; // The new s3_v2 will use opendal. pub const OPENDAL_S3_CONNECTOR: &str = "s3_v2"; +pub const POSIX_FS_CONNECTOR: &str = "posix_fs"; #[derive(Clone, Debug, Deserialize, PartialEq, WithOptions)] pub struct GcsProperties { @@ -89,6 +91,17 @@ impl OpendalSource for OpendalGcs { } } +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub struct OpendalPosixFs; + +impl OpendalSource for OpendalPosixFs { + type Properties = PosixFsProperties; + + fn new_enumerator(properties: Self::Properties) -> anyhow::Result> { + OpendalEnumerator::new_posix_fs_source(properties) + } +} + #[derive(Clone, Debug, Deserialize, PartialEq, with_options::WithOptions)] pub struct OpendalS3Properties { #[serde(flatten)] @@ -115,3 +128,31 @@ impl SourceProperties for OpendalS3Properties { const SOURCE_NAME: &'static str = OPENDAL_S3_CONNECTOR; } + +#[derive(Clone, Debug, Deserialize, PartialEq, WithOptions)] +pub struct PosixFsProperties { + // The root directly of the files to search. The files will be searched recursively. + #[serde(rename = "posix_fs.root")] + pub root: String, + + // The regex pattern to match files under root directory. + #[serde(rename = "match_pattern", default)] + pub match_pattern: Option, + + #[serde(flatten)] + pub unknown_fields: HashMap, +} + +impl UnknownFields for PosixFsProperties { + fn unknown_fields(&self) -> HashMap { + self.unknown_fields.clone() + } +} + +impl SourceProperties for PosixFsProperties { + type Split = OpendalFsSplit; + type SplitEnumerator = OpendalEnumerator; + type SplitReader = OpendalReader; + + const SOURCE_NAME: &'static str = POSIX_FS_CONNECTOR; +} diff --git a/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs b/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs new file mode 100644 index 0000000000000..748230ba5a16e --- /dev/null +++ b/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs @@ -0,0 +1,58 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::marker::PhantomData; + +use anyhow::Context; +use opendal::layers::{LoggingLayer, RetryLayer}; +use opendal::services::Fs; +use opendal::Operator; + +use super::opendal_enumerator::OpendalEnumerator; +use super::{OpendalSource, PosixFsProperties}; + +// Posix fs source should only be used for testing. +// For a single-CN cluster, the behavior is well-defined. It will read from the local file system. +// For a multi-CN cluster, each CN will read from its own local file system under the given directory. + +impl OpendalEnumerator { + /// create opendal posix fs source. + pub fn new_posix_fs_source(posix_fs_properties: PosixFsProperties) -> anyhow::Result { + // Create Fs builder. + let mut builder = Fs::default(); + + builder.root(&posix_fs_properties.root); + + let op: Operator = Operator::new(builder)? + .layer(LoggingLayer::default()) + .layer(RetryLayer::default()) + .finish(); + + let (prefix, matcher) = if let Some(pattern) = posix_fs_properties.match_pattern.as_ref() { + // TODO(Kexiang): Currently, FsListnenr in opendal does not support a prefix. (Seems a bug in opendal) + // So we assign prefix to empty string. + let matcher = glob::Pattern::new(pattern) + .with_context(|| format!("Invalid match_pattern: {}", pattern))?; + (Some(String::new()), Some(matcher)) + } else { + (None, None) + }; + Ok(Self { + op, + prefix, + matcher, + marker: PhantomData, + }) + } +} diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 9dd1ef5b76cde..6cdc7d30e277a 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -39,7 +39,9 @@ pub use manager::{SourceColumnDesc, SourceColumnType}; pub use crate::parser::additional_columns::{ get_connector_compatible_additional_columns, CompatibleAdditionalColumnsFn, }; -pub use crate::source::filesystem::opendal_source::{GCS_CONNECTOR, OPENDAL_S3_CONNECTOR}; +pub use crate::source::filesystem::opendal_source::{ + GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, +}; pub use crate::source::filesystem::S3_CONNECTOR; pub use crate::source::nexmark::NEXMARK_CONNECTOR; pub use crate::source::pulsar::PULSAR_CONNECTOR; diff --git a/src/connector/with_options_source.yaml b/src/connector/with_options_source.yaml index 6ee69ccd2ab18..98a45599b56f2 100644 --- a/src/connector/with_options_source.yaml +++ b/src/connector/with_options_source.yaml @@ -455,6 +455,15 @@ OpendalS3Properties: field_type: String required: false default: Default::default +PosixFsProperties: + fields: + - name: posix_fs.root + field_type: String + required: true + - name: match_pattern + field_type: String + required: false + default: Default::default PubsubProperties: fields: - name: pubsub.split_count diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 80e526b01c15f..c0d3248b5af8a 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -45,7 +45,7 @@ use risingwave_connector::source::test_source::TEST_CONNECTOR; use risingwave_connector::source::{ get_connector_compatible_additional_columns, GCS_CONNECTOR, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, OPENDAL_S3_CONNECTOR, - PULSAR_CONNECTOR, S3_CONNECTOR, + POSIX_FS_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, }; use risingwave_pb::catalog::{ PbSchemaRegistryNameStrategy, PbSource, StreamSourceInfo, WatermarkDesc, @@ -917,6 +917,9 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock hashmap!( Format::Plain => vec![Encode::Csv, Encode::Json], ), + POSIX_FS_CONNECTOR => hashmap!( + Format::Plain => vec![Encode::Csv], + ), MYSQL_CDC_CONNECTOR => hashmap!( Format::Debezium => vec![Encode::Json], // support source stream job diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index ecb1846c01249..441a91836bb0a 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -30,7 +30,7 @@ use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use risingwave_connector::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; use risingwave_connector::source::filesystem::opendal_source::{ - OpendalGcs, OpendalS3, OpendalSource, + OpendalGcs, OpendalPosixFs, OpendalS3, OpendalSource, }; use risingwave_connector::source::filesystem::FsPageItem; use risingwave_connector::source::{ @@ -103,6 +103,11 @@ impl ConnectorSource { OpendalEnumerator::new_s3_source(prop.s3_properties, prop.assume_role)?; Ok(build_opendal_fs_list_stream(lister)) } + ConnectorProperties::PosixFs(prop) => { + let lister: OpendalEnumerator = + OpendalEnumerator::new_posix_fs_source(*prop)?; + Ok(build_opendal_fs_list_stream(lister)) + } other => bail!("Unsupported source: {:?}", other), } } diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 32cd08e9c87fe..633dce3b0b9a8 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -26,7 +26,7 @@ use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{ScalarRef, ScalarRefImpl}; use risingwave_connector::source::filesystem::opendal_source::{ - OpendalGcs, OpendalS3, OpendalSource, + OpendalGcs, OpendalPosixFs, OpendalS3, OpendalSource, }; use risingwave_connector::source::filesystem::OpendalFsSplit; use risingwave_connector::source::{ @@ -125,6 +125,11 @@ impl FsFetchExecutor { OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?; SplitImpl::from(split) } + risingwave_connector::source::ConnectorProperties::PosixFs(_) => { + let split: OpendalFsSplit = + OpendalFsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?; + SplitImpl::from(split) + } _ => unreachable!(), }, _ => unreachable!(), diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index dc6b718f566ea..8d8cb78a80b19 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -15,7 +15,9 @@ use std::sync::Arc; use risingwave_common::catalog::{ColumnId, TableId}; -use risingwave_connector::source::filesystem::opendal_source::{OpendalGcs, OpendalS3}; +use risingwave_connector::source::filesystem::opendal_source::{ + OpendalGcs, OpendalPosixFs, OpendalS3, +}; use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; use risingwave_pb::stream_plan::StreamFsFetchNode; use risingwave_source::source_desc::SourceDescBuilder; @@ -110,6 +112,17 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { ) .boxed() } + risingwave_connector::source::ConnectorProperties::PosixFs(_) => { + FsFetchExecutor::<_, OpendalPosixFs>::new( + params.actor_context.clone(), + params.info, + stream_source_core, + upstream, + source_ctrl_opts, + params.env.connector_params(), + ) + .boxed() + } _ => unreachable!(), }; let rate_limit = source.rate_limit.map(|x| x as _); From 17daad73d032609addd5d048c430671539e15662 Mon Sep 17 00:00:00 2001 From: Dylan Date: Fri, 5 Jan 2024 14:22:10 +0800 Subject: [PATCH 06/12] feat(stream): recognize no op update for table (#14358) --- src/stream/src/executor/mview/materialize.rs | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index 9b8707a5ea52a..0916a9edd46ed 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -277,10 +277,13 @@ fn generate_output( new_rows.push(old_value); } KeyOp::Update((old_value, new_value)) => { - new_ops.push(Op::UpdateDelete); - new_ops.push(Op::UpdateInsert); - new_rows.push(old_value); - new_rows.push(new_value); + // if old_value == new_value, we don't need to emit updates to downstream. + if old_value != new_value { + new_ops.push(Op::UpdateDelete); + new_ops.push(Op::UpdateInsert); + new_rows.push(old_value); + new_rows.push(new_value); + } } } } From c160a9ca26563c7d8bc0dab234c6ceb8aed47056 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Fri, 5 Jan 2024 15:33:22 +0800 Subject: [PATCH 07/12] fix(docker): s3 docker compose use correct env file (#14379) --- docker/docker-compose-with-s3.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/docker-compose-with-s3.yml b/docker/docker-compose-with-s3.yml index ce10a981a8b33..1d4c9600bf1b1 100644 --- a/docker/docker-compose-with-s3.yml +++ b/docker/docker-compose-with-s3.yml @@ -59,7 +59,7 @@ services: - "2222:2222" depends_on: - etcd-0 - env_file: multiple_object_storage.env + env_file: aws.env volumes: - "./risingwave.toml:/risingwave.toml" environment: From c749f7b9557adf37af87c043e15617561d756e48 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 5 Jan 2024 15:34:36 +0800 Subject: [PATCH 08/12] feat(dashboard): allow specifying API endpoint for development (#14361) Signed-off-by: Bugen Zhao --- dashboard/mock-server.js | 3 +++ dashboard/next.config.js | 10 ------- dashboard/package-lock.json | 43 ++++++++++++++++++++++++++++++ dashboard/package.json | 2 ++ dashboard/pages/api/api.ts | 28 ++++++++++++++++++- dashboard/pages/api/cluster.ts | 6 ++--- dashboard/pages/api/metric.ts | 2 +- dashboard/pages/api/streaming.ts | 14 +++++----- dashboard/pages/await_tree.tsx | 2 +- dashboard/pages/heap_profiling.tsx | 6 ++--- dashboard/pages/settings.tsx | 36 ++++++++++++++++++------- 11 files changed, 116 insertions(+), 36 deletions(-) diff --git a/dashboard/mock-server.js b/dashboard/mock-server.js index ba10e464e54aa..31d67bd87b772 100644 --- a/dashboard/mock-server.js +++ b/dashboard/mock-server.js @@ -16,8 +16,11 @@ */ const express = require("express") +const cors = require("cors") const app = express() +app.use(cors()) + app.listen(32333, () => { console.log("Server running on port 32333") }) diff --git a/dashboard/next.config.js b/dashboard/next.config.js index 17622bb794440..ca73de39b634e 100644 --- a/dashboard/next.config.js +++ b/dashboard/next.config.js @@ -20,16 +20,6 @@ */ const nextConfig = { trailingSlash: true, - - rewrites: () => { - return [ - { - source: "/api/:path*", - // To test with a RisingWave Meta node, use "http://127.0.0.1:5691/api/:path*" - destination: "http://localhost:32333/:path*", - }, - ] - }, } module.exports = nextConfig diff --git a/dashboard/package-lock.json b/dashboard/package-lock.json index 7ef31ee76bd76..bcb4687ce202c 100644 --- a/dashboard/package-lock.json +++ b/dashboard/package-lock.json @@ -13,6 +13,7 @@ "@monaco-editor/react": "^4.4.6", "@types/d3": "^7.4.0", "@types/lodash": "^4.14.184", + "@uidotdev/usehooks": "^2.4.1", "base64url": "^3.0.1", "bootstrap-icons": "^1.9.1", "d3": "^7.6.1", @@ -42,6 +43,7 @@ "@types/node": "^18.7.14", "@types/styled-components": "^5.1.26", "@typescript-eslint/eslint-plugin": "^5.52.0", + "cors": "^2.8.5", "eslint": "^8.45.0", "eslint-config-next": "13.4.12", "eslint-config-prettier": "^8.8.0", @@ -3224,6 +3226,18 @@ "url": "https://opencollective.com/typescript-eslint" } }, + "node_modules/@uidotdev/usehooks": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/@uidotdev/usehooks/-/usehooks-2.4.1.tgz", + "integrity": "sha512-1I+RwWyS+kdv3Mv0Vmc+p0dPYH0DTRAo04HLyXReYBL9AeseDWUJyi4THuksBJcu9F0Pih69Ak150VDnqbVnXg==", + "engines": { + "node": ">=16" + }, + "peerDependencies": { + "react": ">=18.0.0", + "react-dom": ">=18.0.0" + } + }, "node_modules/@zag-js/element-size": { "version": "0.1.0", "resolved": "https://registry.npmjs.org/@zag-js/element-size/-/element-size-0.1.0.tgz", @@ -4118,6 +4132,19 @@ "url": "https://opencollective.com/core-js" } }, + "node_modules/cors": { + "version": "2.8.5", + "resolved": "https://registry.npmjs.org/cors/-/cors-2.8.5.tgz", + "integrity": "sha512-KIHbLJqu73RGr/hnbrO9uBeixNGuvSQjul/jdFvS/KFSIH1hWVd1ng7zOHx+YrEfInLG7q4n6GHQ9cDtxv/P6g==", + "dev": true, + "dependencies": { + "object-assign": "^4", + "vary": "^1" + }, + "engines": { + "node": ">= 0.10" + } + }, "node_modules/cosmiconfig": { "version": "7.0.1", "resolved": "https://registry.npmjs.org/cosmiconfig/-/cosmiconfig-7.0.1.tgz", @@ -13342,6 +13369,12 @@ "eslint-visitor-keys": "^3.3.0" } }, + "@uidotdev/usehooks": { + "version": "2.4.1", + "resolved": "https://registry.npmjs.org/@uidotdev/usehooks/-/usehooks-2.4.1.tgz", + "integrity": "sha512-1I+RwWyS+kdv3Mv0Vmc+p0dPYH0DTRAo04HLyXReYBL9AeseDWUJyi4THuksBJcu9F0Pih69Ak150VDnqbVnXg==", + "requires": {} + }, "@zag-js/element-size": { "version": "0.1.0", "resolved": "https://registry.npmjs.org/@zag-js/element-size/-/element-size-0.1.0.tgz", @@ -14004,6 +14037,16 @@ "integrity": "sha512-IeHpLwk3uoci37yoI2Laty59+YqH9x5uR65/yiA0ARAJrTrN4YU0rmauLWfvqOuk77SlNJXj2rM6oT/dBD87+A==", "dev": true }, + "cors": { + "version": "2.8.5", + "resolved": "https://registry.npmjs.org/cors/-/cors-2.8.5.tgz", + "integrity": "sha512-KIHbLJqu73RGr/hnbrO9uBeixNGuvSQjul/jdFvS/KFSIH1hWVd1ng7zOHx+YrEfInLG7q4n6GHQ9cDtxv/P6g==", + "dev": true, + "requires": { + "object-assign": "^4", + "vary": "^1" + } + }, "cosmiconfig": { "version": "7.0.1", "resolved": "https://registry.npmjs.org/cosmiconfig/-/cosmiconfig-7.0.1.tgz", diff --git a/dashboard/package.json b/dashboard/package.json index 67d2ff0ef1715..18377b4ef25e5 100644 --- a/dashboard/package.json +++ b/dashboard/package.json @@ -19,6 +19,7 @@ "@monaco-editor/react": "^4.4.6", "@types/d3": "^7.4.0", "@types/lodash": "^4.14.184", + "@uidotdev/usehooks": "^2.4.1", "base64url": "^3.0.1", "bootstrap-icons": "^1.9.1", "d3": "^7.6.1", @@ -48,6 +49,7 @@ "@types/node": "^18.7.14", "@types/styled-components": "^5.1.26", "@typescript-eslint/eslint-plugin": "^5.52.0", + "cors": "^2.8.5", "eslint": "^8.45.0", "eslint-config-next": "13.4.12", "eslint-config-prettier": "^8.8.0", diff --git a/dashboard/pages/api/api.ts b/dashboard/pages/api/api.ts index 4164172b89e76..aa63e28eb67f1 100644 --- a/dashboard/pages/api/api.ts +++ b/dashboard/pages/api/api.ts @@ -15,8 +15,34 @@ * */ +const PROD_API_ENDPOINT = "/api" +const MOCK_API_ENDPOINT = "http://localhost:32333" +const EXTERNAL_META_NODE_API_ENDPOINT = "http://localhost:5691/api" + +export const PREDEFINED_API_ENDPOINTS = [ + PROD_API_ENDPOINT, + MOCK_API_ENDPOINT, + EXTERNAL_META_NODE_API_ENDPOINT, +] + +export const DEFAULT_API_ENDPOINT: string = + process.env.NODE_ENV === "production" ? PROD_API_ENDPOINT : MOCK_API_ENDPOINT + +export const API_ENDPOINT_KEY = "risingwave.dashboard.api.endpoint" + class Api { - async get(url: string) { + urlFor(path: string) { + let apiEndpoint: string = ( + JSON.parse(localStorage.getItem(API_ENDPOINT_KEY) || "null") || + DEFAULT_API_ENDPOINT + ).replace(/\/+$/, "") // remove trailing slashes + + return `${apiEndpoint}${path}` + } + + async get(path: string) { + const url = this.urlFor(path) + try { const res = await fetch(url) const data = await res.json() diff --git a/dashboard/pages/api/cluster.ts b/dashboard/pages/api/cluster.ts index 6d8d4850529ed..0efc083309389 100644 --- a/dashboard/pages/api/cluster.ts +++ b/dashboard/pages/api/cluster.ts @@ -18,19 +18,19 @@ import { WorkerNode } from "../../proto/gen/common" import api from "./api" export async function getClusterMetrics() { - const res = await api.get("/api/metrics/cluster") + const res = await api.get("/metrics/cluster") return res } export async function getClusterInfoFrontend() { - const res: WorkerNode[] = (await api.get("/api/clusters/1")).map( + const res: WorkerNode[] = (await api.get("/clusters/1")).map( WorkerNode.fromJSON ) return res } export async function getClusterInfoComputeNode() { - const res: WorkerNode[] = (await api.get("/api/clusters/2")).map( + const res: WorkerNode[] = (await api.get("/clusters/2")).map( WorkerNode.fromJSON ) return res diff --git a/dashboard/pages/api/metric.ts b/dashboard/pages/api/metric.ts index 55103881277c2..c799e52e71a70 100644 --- a/dashboard/pages/api/metric.ts +++ b/dashboard/pages/api/metric.ts @@ -18,7 +18,7 @@ import { MetricsSample } from "../../components/metrics" import api from "./api" export async function getActorBackPressures() { - const res = await api.get("/api/metrics/actor/back_pressures") + const res = await api.get("/metrics/actor/back_pressures") return res } diff --git a/dashboard/pages/api/streaming.ts b/dashboard/pages/api/streaming.ts index 97d8a45582da5..a77a165357b9f 100644 --- a/dashboard/pages/api/streaming.ts +++ b/dashboard/pages/api/streaming.ts @@ -23,11 +23,11 @@ import { ColumnCatalog, Field } from "../../proto/gen/plan_common" import api from "./api" export async function getActors(): Promise { - return (await api.get("/api/actors")).map(ActorLocation.fromJSON) + return (await api.get("/actors")).map(ActorLocation.fromJSON) } export async function getFragments(): Promise { - let fragmentList: TableFragments[] = (await api.get("/api/fragments2")).map( + let fragmentList: TableFragments[] = (await api.get("/fragments2")).map( TableFragments.fromJSON ) fragmentList = sortBy(fragmentList, (x) => x.tableId) @@ -75,7 +75,7 @@ export async function getRelations() { async function getTableCatalogsInner( path: "tables" | "materialized_views" | "indexes" | "internal_tables" ) { - let list: Table[] = (await api.get(`/api/${path}`)).map(Table.fromJSON) + let list: Table[] = (await api.get(`/${path}`)).map(Table.fromJSON) list = sortBy(list, (x) => x.id) return list } @@ -97,21 +97,19 @@ export async function getInternalTables() { } export async function getSinks() { - let sinkList: Sink[] = (await api.get("/api/sinks")).map(Sink.fromJSON) + let sinkList: Sink[] = (await api.get("/sinks")).map(Sink.fromJSON) sinkList = sortBy(sinkList, (x) => x.id) return sinkList } export async function getSources() { - let sourceList: Source[] = (await api.get("/api/sources")).map( - Source.fromJSON - ) + let sourceList: Source[] = (await api.get("/sources")).map(Source.fromJSON) sourceList = sortBy(sourceList, (x) => x.id) return sourceList } export async function getViews() { - let views: View[] = (await api.get("/api/views")).map(View.fromJSON) + let views: View[] = (await api.get("/views")).map(View.fromJSON) views = sortBy(views, (x) => x.id) return views } diff --git a/dashboard/pages/await_tree.tsx b/dashboard/pages/await_tree.tsx index 8a3e00d2f89f8..6fc899f4b7935 100644 --- a/dashboard/pages/await_tree.tsx +++ b/dashboard/pages/await_tree.tsx @@ -67,7 +67,7 @@ export default function AwaitTreeDump() { try { const response: StackTraceResponse = StackTraceResponse.fromJSON( - await api.get(`/api/monitor/await_tree/${computeNodeId}`) + await api.get(`/monitor/await_tree/${computeNodeId}`) ) const actorTraces = _(response.actorTraces) diff --git a/dashboard/pages/heap_profiling.tsx b/dashboard/pages/heap_profiling.tsx index 88b737dc4f927..a457d84824593 100644 --- a/dashboard/pages/heap_profiling.tsx +++ b/dashboard/pages/heap_profiling.tsx @@ -75,7 +75,7 @@ export default function HeapProfiling() { try { let list: ListHeapProfilingResponse = ListHeapProfilingResponse.fromJSON( - await api.get(`/api/monitor/list_heap_profile/${computeNodeId}`) + await api.get(`/monitor/list_heap_profile/${computeNodeId}`) ) setProfileList(list) } catch (e: any) { @@ -119,7 +119,7 @@ export default function HeapProfiling() { }, [selectedProfileList]) async function dumpProfile() { - api.get(`/api/monitor/dump_heap_profile/${computeNodeId}`) + api.get(`/monitor/dump_heap_profile/${computeNodeId}`) getProfileList(computeNodes, computeNodeId) } @@ -149,7 +149,7 @@ export default function HeapProfiling() { try { let analyzeFilePathBase64 = base64url(analyzeFilePath) let resObj = await fetch( - `/api/monitor/analyze/${computeNodeId}/${analyzeFilePathBase64}` + `/monitor/analyze/${computeNodeId}/${analyzeFilePathBase64}` ).then(async (res) => ({ filename: res.headers.get("content-disposition"), blob: await res.blob(), diff --git a/dashboard/pages/settings.tsx b/dashboard/pages/settings.tsx index 8301aa227fe14..3214bbdf5c746 100644 --- a/dashboard/pages/settings.tsx +++ b/dashboard/pages/settings.tsx @@ -16,11 +16,28 @@ */ import { Box, FormControl, FormLabel, Input, VStack } from "@chakra-ui/react" +import { useIsClient, useLocalStorage } from "@uidotdev/usehooks" import Head from "next/head" import { Fragment } from "react" import Title from "../components/Title" +import { + API_ENDPOINT_KEY, + DEFAULT_API_ENDPOINT, + PREDEFINED_API_ENDPOINTS, +} from "./api/api" export default function Settings() { + const isClient = useIsClient() + return isClient && +} + +// Local storage is only available on the client side. +function ClientSettings() { + const [apiEndpoint, saveApiEndpoint] = useLocalStorage( + API_ENDPOINT_KEY, + DEFAULT_API_ENDPOINT + ) + return ( @@ -31,15 +48,16 @@ export default function Settings() { RisingWave Meta Node HTTP API - - - - Grafana HTTP API - - - - Prometheus HTTP API - + saveApiEndpoint(event.target.value)} + list="predefined" + /> + + {PREDEFINED_API_ENDPOINTS.map((endpoint) => ( + From 75735e2b15c82429d69529683d56136fc80de1bd Mon Sep 17 00:00:00 2001 From: Zihao Xu Date: Fri, 5 Jan 2024 03:08:25 -0500 Subject: [PATCH 09/12] feat(sql-udf): support basic anonymous sql udf (#14139) Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> --- e2e_test/udf/sql_udf.slt | 192 ++++++++++++++++++ proto/catalog.proto | 1 + src/frontend/src/binder/expr/column.rs | 12 ++ src/frontend/src/binder/expr/function.rs | 168 ++++++++++++++- src/frontend/src/binder/expr/mod.rs | 8 + src/frontend/src/binder/mod.rs | 7 +- src/frontend/src/catalog/function_catalog.rs | 2 + src/frontend/src/catalog/root_catalog.rs | 2 +- .../src/expr/user_defined_function.rs | 2 + src/frontend/src/handler/create_function.rs | 2 + .../src/handler/create_sql_function.rs | 181 +++++++++++++++++ src/frontend/src/handler/mod.rs | 44 +++- .../migration/src/m20230908_072257_init.rs | 2 + src/meta/model_v2/src/function.rs | 2 + src/meta/src/controller/mod.rs | 1 + src/sqlparser/README.md | 4 +- src/sqlparser/tests/sqlparser_postgres.rs | 47 +++++ 17 files changed, 653 insertions(+), 24 deletions(-) create mode 100644 e2e_test/udf/sql_udf.slt create mode 100644 src/frontend/src/handler/create_sql_function.rs diff --git a/e2e_test/udf/sql_udf.slt b/e2e_test/udf/sql_udf.slt new file mode 100644 index 0000000000000..8b89010f70a93 --- /dev/null +++ b/e2e_test/udf/sql_udf.slt @@ -0,0 +1,192 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +# Create an anonymous function with double dollar as clause +statement ok +create function add(INT, INT) returns int language sql as $$select $1 + $2$$; + +# Create an anonymous function with single quote as clause +statement ok +create function sub(INT, INT) returns int language sql as 'select $1 - $2'; + +# Create an anonymous function that calls other pre-defined sql udfs +statement ok +create function add_sub_binding() returns int language sql as 'select add(1, 1) + sub(2, 2)'; + +# Create an anonymous function that calls built-in functions +# Note that double dollar signs should be used otherwise the parsing will fail, as illutrates below +statement ok +create function call_regexp_replace() returns varchar language sql as $$select regexp_replace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥', 'baz(...)', '这是🥵', 'ic')$$; + +statement error Expected end of statement, found: 💩 +create function call_regexp_replace() returns varchar language sql as 'select regexp_replace('💩💩💩💩💩foo🤔️bar亲爱的😭baz这不是爱情❤️‍🔥', 'baz(...)', '这是🥵', 'ic')'; + +# Create an anonymous function with return expression +statement ok +create function add_return(INT, INT) returns int language sql return $1 + $2; + +statement ok +create function add_return_binding() returns int language sql return add_return(1, 1) + add_return(1, 1); + +# Recursive definition is forbidden +statement error recursive definition is forbidden, please recheck your function syntax +create function recursive(INT, INT) returns int language sql as 'select recursive($1, $2) + recursive($1, $2)'; + +# Create a wrapper function for `add` & `sub` +statement ok +create function add_sub_wrapper(INT, INT) returns int language sql as 'select add($1, $2) + sub($1, $2) + 114512'; + +# Call the defined sql udf +query I +select add(1, -1); +---- +0 + +query I +select sub(1, 1); +---- +0 + +query I +select add_sub_binding(); +---- +2 + +query III +select add(1, -1), sub(1, 1), add_sub_binding(); +---- +0 0 2 + +query I +select add_return(1, 1); +---- +2 + +query I +select add_return_binding(); +---- +4 + +query T +select call_regexp_replace(); +---- +💩💩💩💩💩foo🤔️bar亲爱的😭这是🥵爱情❤️‍🔥 + +query I +select add_sub_wrapper(1, 1); +---- +114514 + +# Create a mock table +statement ok +create table t1 (c1 INT, c2 INT); + +# Insert some data into the mock table +statement ok +insert into t1 values (1, 1), (2, 2), (3, 3), (4, 4), (5, 5); + +query III +select sub(c1, c2), c1, c2, add(c1, c2) from t1 order by c1 asc; +---- +0 1 1 2 +0 2 2 4 +0 3 3 6 +0 4 4 8 +0 5 5 10 + +query I +select c1, c2, add_return(c1, c2) from t1 order by c1 asc; +---- +1 1 2 +2 2 4 +3 3 6 +4 4 8 +5 5 10 + +# Invalid function body syntax +statement error Expected an expression:, found: EOF at the end +create function add_error(INT, INT) returns int language sql as $$select $1 + $2 +$$; + +# Multiple type interleaving sql udf +statement ok +create function add_sub(INT, FLOAT, INT) returns float language sql as $$select -$1 + $2 - $3$$; + +statement ok +create function add_sub_return(INT, FLOAT, INT) returns float language sql return -$1 + $2 - $3; + +# Note: need EXPLICIT type cast in order to call the multiple types interleaving sql udf +query I +select add_sub(1::INT, 5.1415926::FLOAT, 1::INT); +---- +3.1415926 + +# Without EXPLICIT type cast +statement error unsupported function: "add_sub" +select add_sub(1, 3.14, 2); + +# Same as above, need EXPLICIT type cast to make the binding works +query I +select add_sub_return(1::INT, 5.1415926::FLOAT, 1::INT); +---- +3.1415926 + +query III +select add(1, -1), sub(1, 1), add_sub(1::INT, 5.1415926::FLOAT, 1::INT); +---- +0 0 3.1415926 + +# Create another mock table +statement ok +create table t2 (c1 INT, c2 FLOAT, c3 INT); + +statement ok +insert into t2 values (1, 3.14, 2), (2, 4.44, 5), (20, 10.30, 02); + +query IIIIII +select c1, c2, c3, add(c1, c3), sub(c1, c3), add_sub(c1::INT, c2::FLOAT, c3::INT) from t2 order by c1 asc; +---- +1 3.14 2 3 -1 0.14000000000000012 +2 4.44 5 7 -3 -2.5599999999999996 +20 10.3 2 22 18 -11.7 + +query IIIIII +select c1, c2, c3, add(c1, c3), sub(c1, c3), add_sub_return(c1::INT, c2::FLOAT, c3::INT) from t2 order by c1 asc; +---- +1 3.14 2 3 -1 0.14000000000000012 +2 4.44 5 7 -3 -2.5599999999999996 +20 10.3 2 22 18 -11.7 + +# Drop the functions +statement ok +drop function add; + +statement ok +drop function sub; + +statement ok +drop function add_sub_binding; + +statement ok +drop function add_sub; + +statement ok +drop function add_sub_return; + +statement ok +drop function add_return; + +statement ok +drop function add_return_binding; + +statement ok +drop function call_regexp_replace; + +statement ok +drop function add_sub_wrapper; + +# Drop the mock table +statement ok +drop table t1; + +statement ok +drop table t2; diff --git a/proto/catalog.proto b/proto/catalog.proto index 01a7893383232..ec7c68a3802ba 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -218,6 +218,7 @@ message Function { string language = 7; string link = 8; string identifier = 10; + optional string body = 14; oneof kind { ScalarFunction scalar = 11; diff --git a/src/frontend/src/binder/expr/column.rs b/src/frontend/src/binder/expr/column.rs index 16053208ec8d3..2f2a8d9335256 100644 --- a/src/frontend/src/binder/expr/column.rs +++ b/src/frontend/src/binder/expr/column.rs @@ -37,6 +37,18 @@ impl Binder { } }; + // Special check for sql udf + // Note: The check in `bind_column` is to inline the identifiers, + // which, in the context of sql udf, will NOT be perceived as normal + // columns, but the actual named input parameters. + // Thus, we need to figure out if the current "column name" corresponds + // to the name of the defined sql udf parameters stored in `udf_context`. + // If so, we will treat this bind as an special bind, the actual expression + // stored in `udf_context` will then be bound instead of binding the non-existing column. + if let Some(expr) = self.udf_context.get(&column_name) { + return self.bind_expr(expr.clone()); + } + match self .context .get_column_binding_indices(&table_name, &column_name) diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index c8558b0756b5d..7244a8527f857 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::iter::once; use std::str::FromStr; -use std::sync::LazyLock; +use std::sync::{Arc, LazyLock}; use bk_tree::{metrics, BKTree}; use itertools::Itertools; @@ -30,13 +30,15 @@ use risingwave_expr::window_function::{ Frame, FrameBound, FrameBounds, FrameExclusion, WindowFuncKind, }; use risingwave_sqlparser::ast::{ - self, Function, FunctionArg, FunctionArgExpr, Ident, WindowFrameBound, WindowFrameExclusion, - WindowFrameUnits, WindowSpec, + self, Expr as AstExpr, Function, FunctionArg, FunctionArgExpr, Ident, SelectItem, SetExpr, + Statement, WindowFrameBound, WindowFrameExclusion, WindowFrameUnits, WindowSpec, }; +use risingwave_sqlparser::parser::ParserError; use thiserror_ext::AsReport; use crate::binder::bind_context::Clause; use crate::binder::{Binder, BoundQuery, BoundSetExpr}; +use crate::catalog::function_catalog::FunctionCatalog; use crate::expr::{ AggCall, Expr, ExprImpl, ExprType, FunctionCall, FunctionCallWithLambda, Literal, Now, OrderBy, Subquery, SubqueryKind, TableFunction, TableFunctionType, UserDefinedFunction, WindowFunction, @@ -117,6 +119,9 @@ impl Binder { return self.bind_array_transform(f); } + // Used later in sql udf expression evaluation + let args = f.args.clone(); + let inputs = f .args .into_iter() @@ -149,6 +154,73 @@ impl Binder { return Ok(TableFunction::new(function_type, inputs)?.into()); } + /// TODO: add name related logic + /// NOTE: need to think of a way to prevent naming conflict + /// e.g., when existing column names conflict with parameter names in sql udf + fn create_udf_context( + args: &[FunctionArg], + _catalog: &Arc, + ) -> Result> { + let mut ret: HashMap = HashMap::new(); + for (i, current_arg) in args.iter().enumerate() { + if let FunctionArg::Unnamed(arg) = current_arg { + let FunctionArgExpr::Expr(e) = arg else { + return Err( + ErrorCode::InvalidInputSyntax("invalid syntax".to_string()).into() + ); + }; + // if catalog.arg_names.is_some() { + // todo!() + // } + ret.insert(format!("${}", i + 1), e.clone()); + continue; + } + return Err(ErrorCode::InvalidInputSyntax("invalid syntax".to_string()).into()); + } + Ok(ret) + } + + fn extract_udf_expression(ast: Vec) -> Result { + if ast.len() != 1 { + return Err(ErrorCode::InvalidInputSyntax( + "the query for sql udf should contain only one statement".to_string(), + ) + .into()); + } + + // Extract the expression out + let Statement::Query(query) = ast[0].clone() else { + return Err(ErrorCode::InvalidInputSyntax( + "invalid function definition, please recheck the syntax".to_string(), + ) + .into()); + }; + + let SetExpr::Select(select) = query.body else { + return Err(ErrorCode::InvalidInputSyntax( + "missing `select` body for sql udf expression, please recheck the syntax" + .to_string(), + ) + .into()); + }; + + if select.projection.len() != 1 { + return Err(ErrorCode::InvalidInputSyntax( + "`projection` should contain only one `SelectItem`".to_string(), + ) + .into()); + } + + let SelectItem::UnnamedExpr(expr) = select.projection[0].clone() else { + return Err(ErrorCode::InvalidInputSyntax( + "expect `UnnamedExpr` for `projection`".to_string(), + ) + .into()); + }; + + Ok(expr) + } + // user defined function // TODO: resolve schema name https://github.com/risingwavelabs/risingwave/issues/12422 if let Ok(schema) = self.first_valid_schema() @@ -158,13 +230,89 @@ impl Binder { ) { use crate::catalog::function_catalog::FunctionKind::*; - match &func.kind { - Scalar { .. } => return Ok(UserDefinedFunction::new(func.clone(), inputs).into()), - Table { .. } => { - self.ensure_table_function_allowed()?; - return Ok(TableFunction::new_user_defined(func.clone(), inputs).into()); + if func.language == "sql" { + if func.body.is_none() { + return Err(ErrorCode::InvalidInputSyntax( + "`body` must exist for sql udf".to_string(), + ) + .into()); + } + // This represents the current user defined function is `language sql` + let parse_result = risingwave_sqlparser::parser::Parser::parse_sql( + func.body.as_ref().unwrap().as_str(), + ); + if let Err(ParserError::ParserError(err)) | Err(ParserError::TokenizerError(err)) = + parse_result + { + // Here we just return the original parse error message + return Err(ErrorCode::InvalidInputSyntax(err).into()); + } + debug_assert!(parse_result.is_ok()); + + // We can safely unwrap here + let ast = parse_result.unwrap(); + + let mut clean_flag = true; + + // We need to check if the `udf_context` is empty first, consider the following example: + // - create function add(INT, INT) returns int language sql as 'select $1 + $2'; + // - create function add_wrapper(INT, INT) returns int language sql as 'select add($1, $2)'; + // - select add_wrapper(1, 1); + // When binding `add($1, $2)` in `add_wrapper`, the input args are [$1, $2] instead of + // the original [1, 1], thus we need to check `udf_context` to see if the input + // args already exist in the context. If so, we do NOT need to create the context again. + // Otherwise the current `udf_context` will be corrupted. + if self.udf_context.is_empty() { + // The actual inline logic for sql udf + if let Ok(context) = create_udf_context(&args, &Arc::clone(func)) { + self.udf_context = context; + } else { + return Err(ErrorCode::InvalidInputSyntax( + "failed to create the `udf_context`, please recheck your function definition and syntax".to_string() + ) + .into()); + } + } else { + // If the `udf_context` is not empty, this means the current binding + // function is not the root binding sql udf, thus we should NOT + // clean the context after binding. + clean_flag = false; + } + + if let Ok(expr) = extract_udf_expression(ast) { + let bind_result = self.bind_expr(expr); + // Clean the `udf_context` after inlining, + // which makes sure the subsequent binding will not be affected + if clean_flag { + self.udf_context.clear(); + } + return bind_result; + } else { + return Err(ErrorCode::InvalidInputSyntax( + "failed to parse the input query and extract the udf expression, + please recheck the syntax" + .to_string(), + ) + .into()); + } + } else { + // Note that `language` may be empty for external udf + if !func.language.is_empty() { + debug_assert!( + func.language == "python" || func.language == "java", + "only `python` and `java` are currently supported for general udf" + ); + } + match &func.kind { + Scalar { .. } => { + return Ok(UserDefinedFunction::new(func.clone(), inputs).into()) + } + Table { .. } => { + self.ensure_table_function_allowed()?; + return Ok(TableFunction::new_user_defined(func.clone(), inputs).into()); + } + Aggregate => todo!("support UDAF"), } - Aggregate => todo!("support UDAF"), } } @@ -1213,7 +1361,7 @@ impl Binder { static FUNCTIONS_BKTREE: LazyLock> = LazyLock::new(|| { let mut tree = BKTree::new(metrics::Levenshtein); - // TODO: Also hint other functinos, e,g, Agg or UDF. + // TODO: Also hint other functinos, e.g., Agg or UDF. for k in HANDLES.keys() { tree.add(*k); } diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 179469e545c2b..cacd2d80dcfe4 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -378,6 +378,14 @@ impl Binder { } fn bind_parameter(&mut self, index: u64) -> Result { + // Special check for sql udf + // Note: This is specific to anonymous sql udf, since the + // parameters will be parsed and treated as `Parameter`. + // For detailed explanation, consider checking `bind_column`. + if let Some(expr) = self.udf_context.get(&format!("${index}")) { + return self.bind_expr(expr.clone()); + } + Ok(Parameter::new(index, self.param_types.clone()).into()) } diff --git a/src/frontend/src/binder/mod.rs b/src/frontend/src/binder/mod.rs index be0f441c0743a..6ba891aa6b513 100644 --- a/src/frontend/src/binder/mod.rs +++ b/src/frontend/src/binder/mod.rs @@ -21,7 +21,7 @@ use risingwave_common::error::Result; use risingwave_common::session_config::{ConfigMap, SearchPath}; use risingwave_common::types::DataType; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_sqlparser::ast::Statement; +use risingwave_sqlparser::ast::{Expr as AstExpr, Statement}; mod bind_context; mod bind_param; @@ -115,6 +115,10 @@ pub struct Binder { included_relations: HashSet, param_types: ParameterTypes, + + /// The mapping from sql udf parameters to ast expressions + /// Note: The expressions are constructed during runtime, correspond to the actual users' input + udf_context: HashMap, } /// `ParameterTypes` is used to record the types of the parameters during binding. It works @@ -216,6 +220,7 @@ impl Binder { shared_views: HashMap::new(), included_relations: HashSet::new(), param_types: ParameterTypes::new(param_types), + udf_context: HashMap::new(), } } diff --git a/src/frontend/src/catalog/function_catalog.rs b/src/frontend/src/catalog/function_catalog.rs index 7197821b33ce6..d0f037bcb47b5 100644 --- a/src/frontend/src/catalog/function_catalog.rs +++ b/src/frontend/src/catalog/function_catalog.rs @@ -30,6 +30,7 @@ pub struct FunctionCatalog { pub return_type: DataType, pub language: String, pub identifier: String, + pub body: Option, pub link: String, } @@ -63,6 +64,7 @@ impl From<&PbFunction> for FunctionCatalog { return_type: prost.return_type.as_ref().expect("no return type").into(), language: prost.language.clone(), identifier: prost.identifier.clone(), + body: prost.body.clone(), link: prost.link.clone(), } } diff --git a/src/frontend/src/catalog/root_catalog.rs b/src/frontend/src/catalog/root_catalog.rs index bd411577476ba..9d2045f5dc61f 100644 --- a/src/frontend/src/catalog/root_catalog.rs +++ b/src/frontend/src/catalog/root_catalog.rs @@ -88,7 +88,7 @@ impl<'a> SchemaPath<'a> { /// - catalog (root catalog) /// - database catalog /// - schema catalog -/// - function catalog +/// - function catalog (i.e., user defined function) /// - table/sink/source/index/view catalog /// - column catalog pub struct Catalog { diff --git a/src/frontend/src/expr/user_defined_function.rs b/src/frontend/src/expr/user_defined_function.rs index abd39fdbbc0c4..165774d1acb4b 100644 --- a/src/frontend/src/expr/user_defined_function.rs +++ b/src/frontend/src/expr/user_defined_function.rs @@ -55,6 +55,8 @@ impl UserDefinedFunction { return_type, language: udf.get_language().clone(), identifier: udf.get_identifier().clone(), + // TODO: Ensure if we need `body` here + body: None, link: udf.get_link().clone(), }; diff --git a/src/frontend/src/handler/create_function.rs b/src/frontend/src/handler/create_function.rs index c623b67eecb60..4557b71223b98 100644 --- a/src/frontend/src/handler/create_function.rs +++ b/src/frontend/src/handler/create_function.rs @@ -44,6 +44,7 @@ pub async fn handle_create_function( if temporary { bail_not_implemented!("CREATE TEMPORARY FUNCTION"); } + // e.g., `language [ python / java / ...etc]` let language = match params.language { Some(lang) => { let lang = lang.real_value().to_lowercase(); @@ -161,6 +162,7 @@ pub async fn handle_create_function( return_type: Some(return_type.into()), language, identifier, + body: None, link, owner: session.user_id(), }; diff --git a/src/frontend/src/handler/create_sql_function.rs b/src/frontend/src/handler/create_sql_function.rs new file mode 100644 index 0000000000000..834e0bec3135d --- /dev/null +++ b/src/frontend/src/handler/create_sql_function.rs @@ -0,0 +1,181 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use itertools::Itertools; +use pgwire::pg_response::StatementType; +use risingwave_common::catalog::FunctionId; +use risingwave_common::types::DataType; +use risingwave_pb::catalog::function::{Kind, ScalarFunction, TableFunction}; +use risingwave_pb::catalog::Function; +use risingwave_sqlparser::ast::{ + CreateFunctionBody, FunctionDefinition, ObjectName, OperateFunctionArg, +}; +use risingwave_sqlparser::parser::{Parser, ParserError}; + +use super::*; +use crate::catalog::CatalogError; +use crate::{bind_data_type, Binder}; + +pub async fn handle_create_sql_function( + handler_args: HandlerArgs, + or_replace: bool, + temporary: bool, + name: ObjectName, + args: Option>, + returns: Option, + params: CreateFunctionBody, +) -> Result { + if or_replace { + bail_not_implemented!("CREATE OR REPLACE FUNCTION"); + } + + if temporary { + bail_not_implemented!("CREATE TEMPORARY FUNCTION"); + } + + let language = "sql".to_string(); + // Just a basic sanity check for language + if !matches!(params.language, Some(lang) if lang.real_value().to_lowercase() == "sql") { + return Err(ErrorCode::InvalidParameterValue( + "`language` for sql udf must be `sql`".to_string(), + ) + .into()); + } + + // SQL udf function supports both single quote (i.e., as 'select $1 + $2') + // and double dollar (i.e., as $$select $1 + $2$$) for as clause + let body = match ¶ms.as_ { + Some(FunctionDefinition::SingleQuotedDef(s)) => s.clone(), + Some(FunctionDefinition::DoubleDollarDef(s)) => s.clone(), + None => { + if params.return_.is_none() { + return Err(ErrorCode::InvalidParameterValue( + "AS or RETURN must be specified".to_string(), + ) + .into()); + } + // Otherwise this is a return expression + // Note: this is a current work around, and we are assuming return sql udf + // will NOT involve complex syntax, so just reuse the logic for select definition + format!("select {}", ¶ms.return_.unwrap().to_string()) + } + }; + + // We do NOT allow recursive calling inside sql udf + // Since there does not exist the base case for this definition + if body.contains(format!("{}(", name.real_value()).as_str()) { + return Err(ErrorCode::InvalidInputSyntax( + "recursive definition is forbidden, please recheck your function syntax".to_string(), + ) + .into()); + } + + // Sanity check for link, this must be none with sql udf function + if let Some(CreateFunctionUsing::Link(_)) = params.using { + return Err(ErrorCode::InvalidParameterValue( + "USING must NOT be specified with sql udf function".to_string(), + ) + .into()); + }; + + // Get return type for the current sql udf function + let return_type; + let kind = match returns { + Some(CreateFunctionReturns::Value(data_type)) => { + return_type = bind_data_type(&data_type)?; + Kind::Scalar(ScalarFunction {}) + } + Some(CreateFunctionReturns::Table(columns)) => { + if columns.len() == 1 { + // return type is the original type for single column + return_type = bind_data_type(&columns[0].data_type)?; + } else { + // return type is a struct for multiple columns + let datatypes = columns + .iter() + .map(|c| bind_data_type(&c.data_type)) + .collect::>>()?; + let names = columns + .iter() + .map(|c| c.name.real_value()) + .collect::>(); + return_type = DataType::new_struct(datatypes, names); + } + Kind::Table(TableFunction {}) + } + None => { + return Err(ErrorCode::InvalidParameterValue( + "return type must be specified".to_string(), + ) + .into()) + } + }; + + let mut arg_types = vec![]; + for arg in args.unwrap_or_default() { + arg_types.push(bind_data_type(&arg.data_type)?); + } + + // resolve database and schema id + let session = &handler_args.session; + let db_name = session.database(); + let (schema_name, function_name) = Binder::resolve_schema_qualified_name(db_name, name)?; + let (database_id, schema_id) = session.get_database_and_schema_id_for_create(schema_name)?; + + // check if function exists + if (session.env().catalog_reader().read_guard()) + .get_schema_by_id(&database_id, &schema_id)? + .get_function_by_name_args(&function_name, &arg_types) + .is_some() + { + let name = format!( + "{function_name}({})", + arg_types.iter().map(|t| t.to_string()).join(",") + ); + return Err(CatalogError::Duplicated("function", name).into()); + } + + // Parse function body here + // Note that the parsing here is just basic syntax / semantic check, the result will NOT be stored + // e.g., The provided function body contains invalid syntax, return type mismatch, ..., etc. + let parse_result = Parser::parse_sql(body.as_str()); + if let Err(ParserError::ParserError(err)) | Err(ParserError::TokenizerError(err)) = parse_result + { + // Here we just return the original parse error message + return Err(ErrorCode::InvalidInputSyntax(err).into()); + } else { + debug_assert!(parse_result.is_ok()); + } + + // Create the actual function, will be stored in function catalog + let function = Function { + id: FunctionId::placeholder().0, + schema_id, + database_id, + name: function_name, + kind: Some(kind), + arg_types: arg_types.into_iter().map(|t| t.into()).collect(), + return_type: Some(return_type.into()), + language, + identifier: "".to_string(), + body: Some(body), + link: "".to_string(), + owner: session.user_id(), + }; + + let catalog_writer = session.catalog_writer()?; + catalog_writer.create_function(function).await?; + + Ok(PgResponse::empty_result(StatementType::CREATE_FUNCTION)) +} diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index d2ec467df424f..495d9ebd3729a 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -53,6 +53,7 @@ pub mod create_mv; pub mod create_schema; pub mod create_sink; pub mod create_source; +pub mod create_sql_function; pub mod create_table; pub mod create_table_as; pub mod create_user; @@ -205,16 +206,39 @@ pub async fn handle( returns, params, } => { - create_function::handle_create_function( - handler_args, - or_replace, - temporary, - name, - args, - returns, - params, - ) - .await + // For general udf, `language` clause could be ignored + // refer: https://github.com/risingwavelabs/risingwave/pull/10608 + if params.language.is_none() + || !params + .language + .as_ref() + .unwrap() + .real_value() + .eq_ignore_ascii_case("sql") + { + // User defined function with external source (e.g., language [ python / java ]) + create_function::handle_create_function( + handler_args, + or_replace, + temporary, + name, + args, + returns, + params, + ) + .await + } else { + create_sql_function::handle_create_sql_function( + handler_args, + or_replace, + temporary, + name, + args, + returns, + params, + ) + .await + } } Statement::CreateTable { name, 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 5b3d55ab83bfb..bc9ce2b08c32b 100644 --- a/src/meta/model_v2/migration/src/m20230908_072257_init.rs +++ b/src/meta/model_v2/migration/src/m20230908_072257_init.rs @@ -708,6 +708,7 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Function::Language).string().not_null()) .col(ColumnDef::new(Function::Link).string().not_null()) .col(ColumnDef::new(Function::Identifier).string().not_null()) + .col(ColumnDef::new(Function::Body).string()) .col(ColumnDef::new(Function::Kind).string().not_null()) .foreign_key( &mut ForeignKey::create() @@ -1099,6 +1100,7 @@ enum Function { Language, Link, Identifier, + Body, Kind, } diff --git a/src/meta/model_v2/src/function.rs b/src/meta/model_v2/src/function.rs index 71391a3cc27b0..5976685893afb 100644 --- a/src/meta/model_v2/src/function.rs +++ b/src/meta/model_v2/src/function.rs @@ -41,6 +41,7 @@ pub struct Model { pub language: String, pub link: String, pub identifier: String, + pub body: Option, pub kind: FunctionKind, } @@ -94,6 +95,7 @@ impl From for ActiveModel { language: Set(function.language), link: Set(function.link), identifier: Set(function.identifier), + body: Set(function.body), kind: Set(function.kind.unwrap().into()), } } diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index d6d6891a12e8d..037f9e3417163 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -277,6 +277,7 @@ impl From> for PbFunction { language: value.0.language, link: value.0.link, identifier: value.0.identifier, + body: value.0.body, kind: Some(value.0.kind.into()), } } diff --git a/src/sqlparser/README.md b/src/sqlparser/README.md index 20a5ac0c8e6ab..2a829102710ba 100644 --- a/src/sqlparser/README.md +++ b/src/sqlparser/README.md @@ -4,5 +4,5 @@ This parser is a fork of . ## Add a new test case -1. Copy an item in the yaml file and edit the `input` to the sql you want to test -2. Run `./risedev do-apply-parser-test` to regenerate the `formatted_sql` whicih is the expected output \ No newline at end of file +1. Copy an item in the yaml file and edit the `input` to the sql you want to test. +2. Run `./risedev do-apply-parser-test` to regenerate the `formatted_sql` which is the expected output. \ No newline at end of file diff --git a/src/sqlparser/tests/sqlparser_postgres.rs b/src/sqlparser/tests/sqlparser_postgres.rs index 2d97834ad23b5..99e2c185fdcff 100644 --- a/src/sqlparser/tests/sqlparser_postgres.rs +++ b/src/sqlparser/tests/sqlparser_postgres.rs @@ -768,6 +768,53 @@ fn parse_create_function() { } ); + let sql = "CREATE FUNCTION sub(INT, INT) RETURNS INT LANGUAGE SQL AS $$select $1 - $2;$$"; + assert_eq!( + verified_stmt(sql), + Statement::CreateFunction { + or_replace: false, + temporary: false, + name: ObjectName(vec![Ident::new_unchecked("sub")]), + args: Some(vec![ + OperateFunctionArg::unnamed(DataType::Int), + OperateFunctionArg::unnamed(DataType::Int), + ]), + returns: Some(CreateFunctionReturns::Value(DataType::Int)), + params: CreateFunctionBody { + language: Some("SQL".into()), + as_: Some(FunctionDefinition::DoubleDollarDef( + "select $1 - $2;".into() + )), + ..Default::default() + } + }, + ); + + // Anonymous return sql udf parsing test + let sql = "CREATE FUNCTION return_test(INT, INT) RETURNS INT LANGUAGE SQL RETURN $1 + $2"; + assert_eq!( + verified_stmt(sql), + Statement::CreateFunction { + or_replace: false, + temporary: false, + name: ObjectName(vec![Ident::new_unchecked("return_test")]), + args: Some(vec![ + OperateFunctionArg::unnamed(DataType::Int), + OperateFunctionArg::unnamed(DataType::Int), + ]), + returns: Some(CreateFunctionReturns::Value(DataType::Int)), + params: CreateFunctionBody { + language: Some("SQL".into()), + return_: Some(Expr::BinaryOp { + left: Box::new(Expr::Parameter { index: 1 }), + op: BinaryOperator::Plus, + right: Box::new(Expr::Parameter { index: 2 }), + }), + ..Default::default() + } + }, + ); + let sql = "CREATE OR REPLACE FUNCTION add(a INT, IN b INT = 1) RETURNS INT LANGUAGE SQL IMMUTABLE RETURN a + b"; assert_eq!( verified_stmt(sql), From dc0402aefdd9b4a182617d3650326bb7c513d9a9 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Fri, 5 Jan 2024 17:27:20 +0800 Subject: [PATCH 10/12] chore(dashboard): cleanup dead code for old dashboard (#14386) Signed-off-by: Bugen Zhao --- dashboard/components/StatusLamp.js | 29 - dashboard/lib/color.js | 90 -- dashboard/lib/graaphEngine/canvasEngine.js | 598 ----------- dashboard/lib/graaphEngine/svgEngine.js | 198 ---- dashboard/lib/str.js | 24 - dashboard/lib/streamPlan/parser.js | 428 -------- dashboard/lib/streamPlan/streamChartHelper.js | 945 ------------------ dashboard/test/algo.test.js | 109 -- 8 files changed, 2421 deletions(-) delete mode 100644 dashboard/components/StatusLamp.js delete mode 100644 dashboard/lib/color.js delete mode 100644 dashboard/lib/graaphEngine/canvasEngine.js delete mode 100644 dashboard/lib/graaphEngine/svgEngine.js delete mode 100644 dashboard/lib/str.js delete mode 100644 dashboard/lib/streamPlan/parser.js delete mode 100644 dashboard/lib/streamPlan/streamChartHelper.js delete mode 100644 dashboard/test/algo.test.js diff --git a/dashboard/components/StatusLamp.js b/dashboard/components/StatusLamp.js deleted file mode 100644 index 6cadd9e2e7764..0000000000000 --- a/dashboard/components/StatusLamp.js +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Copyright 2024 RisingWave Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ -export default function StatusLamp(props) { - return ( -
- ) -} diff --git a/dashboard/lib/color.js b/dashboard/lib/color.js deleted file mode 100644 index 8b6d7d0fccd6a..0000000000000 --- a/dashboard/lib/color.js +++ /dev/null @@ -1,90 +0,0 @@ -/* - * Copyright 2024 RisingWave Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ -const two = [ - ["#6FE0D3", "#E09370"], - ["#75D0E0", "#E0A575"], - ["#75BAE0", "#E0B175"], - ["#77A5E0", "#E0BC77"], - ["#768CE0", "#E0C577"], - ["#7575E0", "#E0CD75"], - ["#A978E0", "#E0DF79"], - ["#C977E0", "#B9E077"], - ["#E072D7", "#92E072"], - ["#E069A4", "#69E069"], - ["#E06469", "#65E086"], - ["#E07860", "#60E0B2"], - ["#E08159", "#5AE0CE"], - ["#E09C5C", "#5CC5E0"], - ["#E0B763", "#6395E0"], - ["#E0CE5A", "#6B5AE0"], - ["#C8E051", "#AA51E0"], - ["#92E06F", "#E070DB"], - ["#79E085", "#E07998"], - ["#80E0B1", "#E08C80"], - ["#91DBE0", "#E0B292"], -] - -const twoGradient = [ - ["#1976d2", "#a6c9ff"], - ["#FFF38A", "#E0D463"], - ["#A3ACFF", "#7983DF"], - ["#A6C9FF", "#7BA3DF"], - ["#FFBE8C", "#E09B65"], - ["#FFD885", "#E0B65D"], - ["#9EE2FF", "#73BEDF"], - ["#DAFF8F", "#B8E066"], - ["#FFC885", "#E0A65D"], - ["#9EFCFF", "#74DCDF"], - ["#FBFF8C", "#DBE065"], - ["#9CFFDE", "#71DFBB"], - ["#FFAF91", "#E08869"], - ["#B699FF", "#9071E0"], - ["#9EFFB6", "#74DF8F"], - ["#FFA19C", "#E07872"], - ["#AEFF9C", "#85DF71"], - ["#FF96B9", "#E06D94"], - ["#FFE785", "#E0C75F"], - ["#FF94FB", "#E06BDC"], - ["#DA99FF", "#B66FE0"], - ["#8F93FF", "#666AE0"], -] - -const five = [ - ["#A8936C", "#F5D190", "#8B84F5", "#9AA84A", "#E1F578"], - ["#A87C6A", "#F5AB8E", "#82CBF5", "#A89348", "#F5D876"], - ["#A87490", "#F59DCB", "#90F5C7", "#A87752", "#F5B584"], - ["#856FA8", "#B995F5", "#BAF58A", "#A84D5B", "#F57D8E"], - ["#7783A8", "#A2B4F5", "#F5EE95", "#9C56A8", "#E589F5"], - ["#74A895", "#9DF5D4", "#F5BF91", "#526CA8", "#84A6F5"], - ["#74A878", "#9DF5A3", "#F5A290", "#5298A8", "#84DFF5"], - ["#94A877", "#D2F5A2", "#F596B6", "#56A88C", "#89F5D0"], - ["#A8A072", "#F5E79A", "#CD8DF5", "#5DA851", "#92F582"], - ["#A89176", "#F5CD9F", "#92A3F5", "#A8A554", "#F5F087"], - ["#A8726A", "#F59B8E", "#83ECF5", "#A88948", "#F5CB76"], -] -export function TwoColor(index) { - return two[index % two.length] -} - -export function FiveColor(index) { - return five[index % five.length] -} - -let s = Math.random() * 100 -export function TwoGradient(index) { - return twoGradient[(Math.round(s) + index) % two.length] -} diff --git a/dashboard/lib/graaphEngine/canvasEngine.js b/dashboard/lib/graaphEngine/canvasEngine.js deleted file mode 100644 index df661b53c5ec6..0000000000000 --- a/dashboard/lib/graaphEngine/canvasEngine.js +++ /dev/null @@ -1,598 +0,0 @@ -/* - * Copyright 2024 RisingWave Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ -import { fabric } from "fabric" - -// Disable cache to improve performance. -fabric.Object.prototype.objectCaching = false -fabric.Object.prototype.statefullCache = false -fabric.Object.prototype.noScaleCache = true -fabric.Object.prototype.needsItsOwnCache = () => false - -export class DrawElement { - /** - * @param {{svgElement: d3.Selection}} props - */ - constructor(props) { - /** - * @type {{svgElement: d3.Selection}} - */ - this.props = props - if (props.canvasElement) { - props.engine.canvas.add(props.canvasElement) - props.canvasElement.on("mouse:down", (e) => { - console.log(e) - }) - } - - this.eventHandler = new Map() - } - - // TODO: this method is for migrating from d3.js to fabric.js. - // This should be replaced by a more suitable way. - _attrMap(key, value) { - return [key, value] - } - - // TODO: this method is for migrating from d3.js to fabric.js. - // This should be replaced by a more suitable way. - attr(key, value) { - let setting = this._attrMap(key, value) - if (setting && setting.length === 2) { - this.props.canvasElement && - this.props.canvasElement.set(setting[0], setting[1]) - } - return this - } - - _afterPosition() { - let ele = this.props.canvasElement - ele && this.props.engine._addDrawElement(this) - } - - // TODO: this method is for migrating from d3.js to fabric.js. - // This should be replaced by a more suitable way. - position(x, y) { - this.props.canvasElement.set("left", x) - this.props.canvasElement.set("top", y) - this._afterPosition() - return this - } - - on(event, callback) { - this.eventHandler.set(event, callback) - return this - } - - getEventHandler(event) { - return this.eventHandler.get(event) - } - - // TODO: this method is for migrating from d3.js to fabric.js. - // This should be replaced by a more suitable way. - style(key, value) { - return this.attr(key, value) - } - - classed(clazz, flag) { - this.props.engine.classedElement(clazz, this, flag) - return this - } -} - -export class Group extends DrawElement { - /** - * @param {{engine: CanvasEngine}} props - */ - constructor(props) { - super(props) - - this.appendFunc = { - g: this._appendGroup, - circle: this._appendCircle, - rect: this._appendRect, - text: this._appendText, - path: this._appendPath, - polygon: this._appendPolygan, - } - - this.basicSetting = { - engine: props.engine, - } - } - - _appendGroup = () => { - return new Group(this.basicSetting) - } - - _appendCircle = () => { - return new Circle({ - ...this.basicSetting, - ...{ - canvasElement: new fabric.Circle({ - selectable: false, - hoverCursor: "pointer", - }), - }, - }) - } - - _appendRect = () => { - return new Rectangle({ - ...this.basicSetting, - ...{ - canvasElement: new fabric.Rect({ - selectable: false, - hoverCursor: "pointer", - }), - }, - }) - } - - _appendText = () => { - return (content) => - new Text({ - ...this.basicSetting, - ...{ - canvasElement: new fabric.Text(content || "undefined", { - selectable: false, - textAlign: "justify-center", - }), - }, - }) - } - - _appendPath = () => { - return (d) => - new Path({ - ...this.basicSetting, - ...{ - canvasElement: new fabric.Path(d, { selectable: false }), - }, - }) - } - - _appendPolygan = () => { - return new Polygan(this.basicSetting) - } - - append = (type) => { - return this.appendFunc[type]() - } -} - -export class Rectangle extends DrawElement { - /** - * @param {{g: fabric.Group}} props - */ - constructor(props) { - super(props) - this.props = props - } - - init(x, y, width, height) { - let ele = this.props.canvasElement - ele.set("left", x) - ele.set("top", y) - ele.set("width", width) - ele.set("height", height) - super._afterPosition() - return this - } - - _attrMap(key, value) { - if (key === "rx") { - this.props.canvasElement.set("rx", value) - this.props.canvasElement.set("ry", value) - return false - } - return [key, value] - } -} - -export class Circle extends DrawElement { - /** - * @param {{svgElement: d3.Selection}} props - */ - constructor(props) { - super(props) - this.props = props - this.radius = 0 - } - - init(x, y, r) { - this.props.canvasElement.set("left", x - r) - this.props.canvasElement.set("top", y - r) - this.props.canvasElement.set("radius", r) - super._afterPosition() - return this - } - - _attrMap(key, value) { - if (key === "r") { - this.radius = value - return ["radius", value] - } - if (key === "cx") { - return ["left", value - this.radius] - } - if (key === "cy") { - return ["top", value - this.radius] - } - return [key, value] - } -} - -export class Text extends DrawElement { - /** - * @param {{svgElement: d3.Selection, any, null, undefined>}} props - */ - constructor(props) { - super(props) - this.props = props - } - - position(x, y) { - let e = this.props.canvasElement - e.set("top", y) - e.set("left", x) - super._afterPosition() - return this - } - - _attrMap(key, value) { - if (key === "text-anchor") { - return ["textAlign", value] - } - if (key === "font-size") { - return ["fontSize", value] - } - return [key, value] - } - - text(content) { - return this - } - - getWidth() {} -} - -export class Polygan extends DrawElement { - constructor(props) { - super(props) - this.props = props - } -} - -export class Path extends DrawElement { - constructor(props) { - super(props) - this.props = props - this.strokeWidth = 1 - super._afterPosition() - } - - _attrMap(key, value) { - if (key === "fill") { - return ["fill", value === "none" ? false : value] - } - if (key === "stroke-width") { - this.props.canvasElement.set( - "top", - this.props.canvasElement.get("top") - value / 2 - ) - return ["strokeWidth", value] - } - if (key === "stroke-dasharray") { - return ["strokeDashArray", value.split(",")] - } - if (key === "layer") { - if (value === "back") { - this.props.canvasElement.canvas.sendToBack(this.props.canvasElement) - } - return false - } - return [key, value] - } -} - -// TODO: Use rbtree -class CordMapper { - constructor() { - this.map = new Map() - } - - rangeQuery(start, end) { - let rtn = new Set() - for (let [k, s] of this.map.entries()) { - if (start <= k && k <= end) { - s.forEach((v) => rtn.add(v)) - } - } - return rtn - } - - insert(k, v) { - if (this.map.has(k)) { - this.map.get(k).add(v) - } else { - this.map.set(k, new Set([v])) - } - } -} - -class GridMapper { - constructor() { - this.xMap = new CordMapper() - this.yMap = new CordMapper() - this.gs = 100 // grid size - } - - _getKey(value) { - return Math.round(value / this.gs) - } - - addObject(minX, maxX, minY, maxY, ele) { - for (let i = minX; i <= maxX + this.gs; i += this.gs) { - this.xMap.insert(this._getKey(i), ele) - } - for (let i = minY; i <= maxY + this.gs; i += this.gs) { - this.yMap.insert(this._getKey(i), ele) - } - } - - areaQuery(minX, maxX, minY, maxY) { - let xs = this.xMap.rangeQuery(this._getKey(minX), this._getKey(maxX)) - let ys = this.yMap.rangeQuery(this._getKey(minY), this._getKey(maxY)) - let rtn = new Set() - xs.forEach((e) => { - if (ys.has(e)) { - rtn.add(e) - } - }) - return rtn - } -} - -export class CanvasEngine { - /** - * @param {string} canvasId The DOM id of the canvas - * @param {number} height the height of the canvas - * @param {number} width the width of the canvas - */ - constructor(canvasId, height, width) { - let canvas = new fabric.Canvas(canvasId) - canvas.selection = false // improve performance - - this.height = height - this.width = width - this.canvas = canvas - this.clazzMap = new Map() - this.topGroup = new Group({ engine: this }) - this.gridMapper = new GridMapper() - this.canvasElementToDrawElement = new Map() - - let that = this - canvas.on("mouse:wheel", function (opt) { - var evt = opt.e - if (evt.ctrlKey === true) { - var delta = opt.e.deltaY - var zoom = canvas.getZoom() - zoom *= 0.999 ** delta - if (zoom > 10) zoom = 10 - if (zoom < 0.03) zoom = 0.03 - canvas.zoomToPoint({ x: opt.e.offsetX, y: opt.e.offsetY }, zoom) - that._refreshView() - evt.preventDefault() - evt.stopPropagation() - } else { - that.moveCamera(-evt.deltaX, -evt.deltaY) - evt.preventDefault() - evt.stopPropagation() - } - }) - - canvas.on("mouse:down", function (opt) { - var evt = opt.e - this.isDragging = true - this.selection = false - this.lastPosX = evt.clientX - this.lastPosY = evt.clientY - - that._handleClickEvent(opt.target) - }) - - canvas.on("mouse:move", function (opt) { - if (this.isDragging) { - var e = opt.e - that.moveCamera(e.clientX - this.lastPosX, e.clientY - this.lastPosY) - this.lastPosX = e.clientX - this.lastPosY = e.clientY - } - }) - canvas.on("mouse:up", function (opt) { - this.setViewportTransform(this.viewportTransform) - this.isDragging = false - this.selection = true - }) - } - - /** - * Move the current view point. - * @param {number} deltaX - * @param {number} deltaY - */ - async moveCamera(deltaX, deltaY) { - this.canvas.setZoom(this.canvas.getZoom()) // essential for rendering (seems like a bug) - let vpt = this.canvas.viewportTransform - vpt[4] += deltaX - vpt[5] += deltaY - this._refreshView() - } - - /** - * Invoke the click handler of an object. - * @param {fabric.Object} target - */ - async _handleClickEvent(target) { - if (target === null) { - return - } - let ele = this.canvasElementToDrawElement.get(target) - let func = ele.getEventHandler("click") - if (func) { - func() - } - } - - /** - * Set the objects in the current view point visible. - * And set other objects not visible. - */ - async _refreshView() { - const padding = 50 // Make the rendering area a little larger. - let vpt = this.canvas.viewportTransform - let zoom = this.canvas.getZoom() - let cameraWidth = this.width - let cameraHeight = this.height - let minX = -vpt[4] - padding - let maxX = -vpt[4] + cameraWidth + padding - let minY = -vpt[5] - padding - let maxY = -vpt[5] + cameraHeight + padding - let visibleSet = this.gridMapper.areaQuery( - minX / zoom, - maxX / zoom, - minY / zoom, - maxY / zoom - ) - - this.canvas.getObjects().forEach((e) => { - if (visibleSet.has(e)) { - e.visible = true - } else { - e.visible = false - } - }) - - this.canvas.requestRenderAll() - } - - /** - * Register an element to the engine. This should - * be called when a DrawElement instance is added - * to the canvas. - * @param {DrawElement} ele - */ - _addDrawElement(ele) { - let canvasElement = ele.props.canvasElement - this.canvasElementToDrawElement.set(canvasElement, ele) - this.gridMapper.addObject( - canvasElement.left, - canvasElement.left + canvasElement.width, - canvasElement.top, - canvasElement.top + canvasElement.height, - canvasElement - ) - } - - /** - * Assign a class to an object or remove a class from it. - * @param {string} clazz class name - * @param {DrawElement} element target object - * @param {boolean} flag true if the object is assigned, otherwise - * remove the class from the object - */ - classedElement(clazz, element, flag) { - if (!flag) { - this.clazzMap.has(clazz) && this.clazzMap.get(clazz).delete(element) - } else { - if (this.clazzMap.has(clazz)) { - this.clazzMap.get(clazz).add(element) - } else { - this.clazzMap.set(clazz, new Set([element])) - } - } - } - - /** - * Move current view point to the object specified by - * the selector. The selector is the class of the - * target object for now. - * @param {string} selector The class of the target object - */ - locateTo(selector) { - // - let selectorSet = this.clazzMap.get(selector) - if (selectorSet) { - let arr = Array.from(selectorSet) - if (arr.length > 0) { - let ele = arr[0] - let x = ele.props.canvasElement.get("left") - let y = ele.props.canvasElement.get("top") - let scale = 0.6 - this.canvas.setZoom(scale) - let vpt = this.canvas.viewportTransform - vpt[4] = (-x + this.width * 0.5) * scale - vpt[5] = (-y + this.height * 0.5) * scale - this.canvas.requestRenderAll() - this._refreshView() - } - } - } - - /** - * Move current view point to (0, 0) - */ - resetCamera() { - let zoom = this.canvas.getZoom() - zoom *= 0.999 - this.canvas.setZoom(zoom) - let vpt = this.canvas.viewportTransform - vpt[4] = 0 - vpt[5] = 0 - this.canvas.requestRenderAll() - this._refreshView() - } - - /** - * Dispose the current canvas. Remove all the objects to - * free memory. All objects in the canvas will be removed. - */ - cleanGraph() { - console.log("clean called") - this.canvas.dispose() - } - - /** - * Resize the canvas. This is called when the browser size - * is changed, such that the canvas can fix the current - * size of the browser. - * - * Note that the outer div box of the canvas will be set - * according to the parameters. However, the width and - * height of the canvas is double times of the parameters. - * This is the feature of fabric.js to keep the canvas - * in high resolution all the time. - * - * @param {number} width the width of the canvas - * @param {number} height the height of the canvas - */ - resize(width, height) { - this.width = width - this.height = height - this.canvas.setDimensions({ width: this.width, height: this.height }) - } -} diff --git a/dashboard/lib/graaphEngine/svgEngine.js b/dashboard/lib/graaphEngine/svgEngine.js deleted file mode 100644 index 8102b79df0a4b..0000000000000 --- a/dashboard/lib/graaphEngine/svgEngine.js +++ /dev/null @@ -1,198 +0,0 @@ -/* - * Copyright 2024 RisingWave Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ -import * as d3 from "d3" - -export class DrawElement { - /** - * @param {{svgElement: d3.Selection}} props - */ - constructor(props) { - /** - * @type {{svgElement: d3.Selection}} - */ - this.props = props - this.appendFunc = { - g: this._appendGroup, - circle: this._appendCircle, - rect: this._appendRect, - text: this._appendText, - path: this._appendPath, - polygon: this._appendPolygan, - } - } - - _appendGroup = () => { - return new Group({ svgElement: this.props.svgElement.append("g") }) - } - - _appendCircle = () => { - return new Circle({ svgElement: this.props.svgElement.append("circle") }) - } - - _appendRect = () => { - return new Rectangle({ svgElement: this.props.svgElement.append("rect") }) - } - - _appendText = () => { - return new Text({ svgElement: this.props.svgElement.append("text") }) - } - - _appendPath = () => { - return new Path({ svgElement: this.props.svgElement.append("path") }) - } - - _appendPolygan = () => { - return new Polygan({ svgElement: this.props.svgElement.append("polygon") }) - } - - on = (event, callback) => { - this.props.svgElement.on(event, callback) - return this - } - - style = (key, value) => { - this.props.svgElement.style(key, value) - return this - } - - classed = (clazz, flag) => { - this.props.svgElement.classed(clazz, flag) - return this - } - - attr = (key, value) => { - this.props.svgElement.attr(key, value) - return this - } - - append = (type) => { - return this.appendFunc[type]() - } -} - -export class Group extends DrawElement { - /** - * @param {{svgElement: d3.Selection}} props - */ - constructor(props) { - super(props) - } -} - -export class Rectangle extends DrawElement { - /** - * @param {{svgElement: d3.Selection}} props - */ - constructor(props) { - super(props) - } -} - -export class Circle extends DrawElement { - /** - * @param {{svgElement: d3.Selection}} props - */ - constructor(props) { - super(props) - } -} - -export class Text extends DrawElement { - /** - * @param {{svgElement: d3.Selection, any, null, undefined>}} props - */ - constructor(props) { - super(props) - this.props = props - } - - text(content) { - this.props.svgElement.text(content) - return this - } - - getWidth() { - return this.props.svgElement.node().getComputedTextLength() - } -} - -export class Polygan extends DrawElement { - constructor(props) { - super(props) - this.props = props - } -} - -export class Path extends DrawElement { - constructor(props) { - super(props) - } -} - -const originalZoom = new d3.ZoomTransform(0.5, 0, 0) - -export class SvgEngine { - /** - * @param {{g: d3.Selection}} props - */ - constructor(svgRef, height, width) { - this.height = height - this.width = width - this.svgRef = svgRef - - d3.select(svgRef).selectAll("*").remove() - this.svg = d3.select(svgRef).attr("viewBox", [0, 0, width, height]) - - this._g = this.svg.append("g").attr("class", "top") - this.topGroup = new Group({ svgElement: this._g }) - - this.transform - this.zoom = d3.zoom().on("zoom", (e) => { - this.transform = e.transform - this._g.attr("transform", e.transform) - }) - - this.svg.call(this.zoom).call(this.zoom.transform, originalZoom) - this.svg.on("pointermove", (event) => { - this.transform.invert(d3.pointer(event)) - }) - } - - locateTo(selector) { - let selection = d3.select(selector) - if (!selection.empty()) { - this.svg - .call(this.zoom) - .call( - this.zoom.transform, - new d3.ZoomTransform( - 0.7, - -0.7 * selection.attr("x"), - 0.7 * (-selection.attr("y") + this.height / 2) - ) - ) - } - } - - resetCamera() { - this.svg.call(this.zoom.transform, originalZoom) - } - - cleanGraph() { - d3.select(this.svgRef).selectAll("*").remove() - } -} diff --git a/dashboard/lib/str.js b/dashboard/lib/str.js deleted file mode 100644 index 52af7892f4a75..0000000000000 --- a/dashboard/lib/str.js +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Copyright 2024 RisingWave Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ -export function capitalize(sentence) { - let words = sentence.split(" ") - let s = "" - for (let word of words) { - s += word.charAt(0).toUpperCase() + word.slice(1, word.length) - } - return s -} diff --git a/dashboard/lib/streamPlan/parser.js b/dashboard/lib/streamPlan/parser.js deleted file mode 100644 index 050e0f05619a4..0000000000000 --- a/dashboard/lib/streamPlan/parser.js +++ /dev/null @@ -1,428 +0,0 @@ -/* - * Copyright 2024 RisingWave Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ -import { graphBfs } from "../algo" - -let cnt = 0 -function generateNewNodeId() { - return "g" + ++cnt -} - -function getNodeId(nodeProto, actorId) { - return ( - actorId + - ":" + - (nodeProto.operatorId === undefined - ? generateNewNodeId() - : "o" + nodeProto.operatorId) - ) -} - -class Node { - constructor(id, actorId, nodeProto) { - this.id = id - /** - * @type {any} - */ - this.nodeProto = nodeProto - /** - * @type {Array} - */ - this.output = output - /** - * @type {Node} - */ - this.rootNode = rootNode - /** - * @type {number} - */ - this.fragmentId = fragmentId - /** - * @type {string} - */ - this.computeNodeAddress = computeNodeAddress - /** - * @type {Array} - */ - this.representedActorList = null - /** - * @type {Set} - */ - this.representedWorkNodes = null - } -} - -export default class StreamPlanParser { - /** - * - * @param {[{node: any, actors: []}]} data raw response from the meta node - */ - constructor(data, shownActorList) { - this.actorId2Proto = new Map() - /** - * @type {Set} - * @private - */ - this.actorIdTomviewNodes = new Map() - this.shownActorSet = new Set(shownActorList) - - for (let computeNodeData of data) { - for (let singleActorProto of computeNodeData.actors) { - if ( - shownActorList && - !this.shownActorSet.has(singleActorProto.actorId) - ) { - continue - } - this.actorId2Proto.set(singleActorProto.actorId, { - computeNodeAddress: `${computeNodeData.node.host.host}:${computeNodeData.node.host.port}`, - ...singleActorProto, - }) - } - } - - this.parsedNodeMap = new Map() - this.parsedActorMap = new Map() - - for (let [_, singleActorProto] of this.actorId2Proto.entries()) { - this.parseActor(singleActorProto) - } - - this.parsedActorList = [] - for (let [_, actor] of this.parsedActorMap.entries()) { - this.parsedActorList.push(actor) - } - - /** @type {Set} */ - this.fragmentRepresentedActors = this._constructRepresentedActorList() - - /** @type {Map} */ - this.mvTableIdToSingleViewActorList = this._constructSingleViewMvList() - - /** @type {Map} */ - this.mvTableIdToChainViewActorList = this._constructChainViewMvList() - } - - /** - * Randomly select a actor to represent its - * fragment, and append a property named `representedActorList` - * to store all the other actors in the same fragment. - * - * Actors are degree of parallelism of a fragment, such that one of - * the actor in a fragment can represent all the other actor in - * the same fragment. - * - * @returns A Set containing actors representing its fragment. - */ - _constructRepresentedActorList() { - const fragmentId2actorList = new Map() - let fragmentRepresentedActors = new Set() - for (let actor of this.parsedActorList) { - if (!fragmentId2actorList.has(actor.fragmentId)) { - fragmentRepresentedActors.add(actor) - fragmentId2actorList.set(actor.fragmentId, [actor]) - } else { - fragmentId2actorList.get(actor.fragmentId).push(actor) - } - } - - for (let actor of fragmentRepresentedActors) { - actor.representedActorList = fragmentId2actorList - .get(actor.fragmentId) - .sort((x) => x.actorId) - actor.representedWorkNodes = new Set() - for (let representedActor of actor.representedActorList) { - representedActor.representedActorList = actor.representedActorList - actor.representedWorkNodes.add(representedActor.computeNodeAddress) - } - } - return fragmentRepresentedActors - } - - _constructChainViewMvList() { - let mvTableIdToChainViewActorList = new Map() - let shellNodes = new Map() - const getShellNode = (actorId) => { - if (shellNodes.has(actorId)) { - return shellNodes.get(actorId) - } - let shellNode = { - id: actorId, - parentNodes: [], - nextNodes: [], - } - for (let node of this.parsedActorMap.get(actorId).output) { - let nextNode = getShellNode(node.actorId) - nextNode.parentNodes.push(shellNode) - shellNode.nextNodes.push(nextNode) - } - shellNodes.set(actorId, shellNode) - return shellNode - } - - for (let actorId of this.actorId2Proto.keys()) { - getShellNode(actorId) - } - - for (let [actorId, mviewNode] of this.actorIdTomviewNodes.entries()) { - let list = new Set() - let shellNode = getShellNode(actorId) - graphBfs(shellNode, (n) => { - list.add(n.id) - }) - graphBfs( - shellNode, - (n) => { - list.add(n.id) - }, - "parentNodes" - ) - for (let actor of this.parsedActorMap.get(actorId).representedActorList) { - list.add(actor.actorId) - } - mvTableIdToChainViewActorList.set(mviewNode.typeInfo.tableId, [ - ...list.values(), - ]) - } - - return mvTableIdToChainViewActorList - } - - _constructSingleViewMvList() { - let mvTableIdToSingleViewActorList = new Map() - let shellNodes = new Map() - const getShellNode = (actorId) => { - if (shellNodes.has(actorId)) { - return shellNodes.get(actorId) - } - let shellNode = { - id: actorId, - parentNodes: [], - } - for (let node of this.parsedActorMap.get(actorId).output) { - getShellNode(node.actorId).parentNodes.push(shellNode) - } - shellNodes.set(actorId, shellNode) - return shellNode - } - for (let actor of this.parsedActorList) { - getShellNode(actor.actorId) - } - - for (let actorId of this.actorId2Proto.keys()) { - getShellNode(actorId) - } - - for (let [actorId, mviewNode] of this.actorIdTomviewNodes.entries()) { - let list = [] - let shellNode = getShellNode(actorId) - graphBfs( - shellNode, - (n) => { - list.push(n.id) - if (shellNode.id !== n.id && this.actorIdTomviewNodes.has(n.id)) { - return true // stop to traverse its next nodes - } - }, - "parentNodes" - ) - for (let actor of this.parsedActorMap.get(actorId).representedActorList) { - list.push(actor.actorId) - } - mvTableIdToSingleViewActorList.set(mviewNode.typeInfo.tableId, list) - } - - return mvTableIdToSingleViewActorList - } - - newDispatcher(actorId, type, downstreamActorId) { - return new Dispatcher(actorId, type, downstreamActorId, { - operatorId: 100000 + actorId, - }) - } - - /** - * Parse raw data from meta node to an actor - * @param {{ - * actorId: number, - * fragmentId: number, - * nodes: any, - * dispatcher?: {type: string}, - * downstreamActorId?: any - * }} actorProto - * @returns {Actor} - */ - parseActor(actorProto) { - let actorId = actorProto.actorId - if (this.parsedActorMap.has(actorId)) { - return this.parsedActorMap.get(actorId) - } - - let actor = new Actor( - actorId, - [], - null, - actorProto.fragmentId, - actorProto.computeNodeAddress - ) - - let rootNode - this.parsedActorMap.set(actorId, actor) - if (actorProto.dispatcher && actorProto.dispatcher[0].type) { - let nodeBeforeDispatcher = this.parseNode(actor.actorId, actorProto.nodes) - rootNode = this.newDispatcher( - actor.actorId, - actorProto.dispatcher[0].type, - actorProto.downstreamActorId - ) - rootNode.nextNodes = [nodeBeforeDispatcher] - } else { - rootNode = this.parseNode(actorId, actorProto.nodes) - } - actor.rootNode = rootNode - - return actor - } - - parseNode(actorId, nodeProto) { - let id = getNodeId(nodeProto, actorId) - if (this.parsedNodeMap.has(id)) { - return this.parsedNodeMap.get(id) - } - let newNode = new StreamNode(id, actorId, nodeProto) - this.parsedNodeMap.set(id, newNode) - - if (nodeProto.input !== undefined) { - for (let nextNodeProto of nodeProto.input) { - newNode.nextNodes.push(this.parseNode(actorId, nextNodeProto)) - } - } - - if (newNode.type === "merge" && newNode.typeInfo.upstreamActorId) { - for (let upStreamActorId of newNode.typeInfo.upstreamActorId) { - if (!this.actorId2Proto.has(upStreamActorId)) { - continue - } - this.parseActor(this.actorId2Proto.get(upStreamActorId)).output.push( - newNode - ) - } - } - - if (newNode.type === "streamScan" && newNode.typeInfo.upstreamActorIds) { - for (let upStreamActorId of newNode.typeInfo.upstreamActorIds) { - if (!this.actorId2Proto.has(upStreamActorId)) { - continue - } - this.parseActor(this.actorId2Proto.get(upStreamActorId)).output.push( - newNode - ) - } - } - - if (newNode.type === "materialize") { - this.actorIdTomviewNodes.set(actorId, newNode) - } - - return newNode - } - - getActor(actorId) { - return this.parsedActorMap.get(actorId) - } - - getOperator(operatorId) { - return this.parsedNodeMap.get(operatorId) - } - - /** - * @returns {Array} - */ - getParsedActorList() { - return this.parsedActorList - } -} diff --git a/dashboard/lib/streamPlan/streamChartHelper.js b/dashboard/lib/streamPlan/streamChartHelper.js deleted file mode 100644 index 5610273cda3de..0000000000000 --- a/dashboard/lib/streamPlan/streamChartHelper.js +++ /dev/null @@ -1,945 +0,0 @@ -/* - * Copyright 2024 RisingWave Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ -import * as d3 from "d3" -import { cloneDeep, max } from "lodash" -import { getConnectedComponent, treeBfs } from "../algo" -import * as color from "../color" -import { Group } from "../graaphEngine/canvasEngine" -import { newNumberArray } from "../util" -import StreamPlanParser, { Actor } from "./parser" -// Actor constant -// -// ======================================================= -// ^ -// | actorBoxPadding -// v -// --┌───────────┐ -// | │ node │ -// | │<--->radius│>───────\ -// | │ │ │ -// | └───────────┘ │ -// | │ -// | ┌───────────┐ │ ┌───────────┐ -// | │ node │ │ │ node │ -// widthUnit│<--->radius│>───────┼────────>│<--->radius│ -// | │ │ │ │ │ -// | └───────────┘ │ └───────────┘ -// | │ -// | ┌───────────┐ │ -// | │ node │ │ -// | │<--->radius│>───────/ -// | │ │ -// ---└───────────┘ -// |-----------------heightUnit---------------| -// - -const SCALE_FACTOR = 0.5 - -const operatorNodeRadius = 30 * SCALE_FACTOR // the radius of the tree nodes in an actor -const operatorNodeStrokeWidth = 5 * SCALE_FACTOR // the stroke width of the link of the tree nodes in an actor -const widthUnit = 230 * SCALE_FACTOR // the width of a tree node in an actor -const heightUnit = 250 * SCALE_FACTOR // the height of a tree layer in an actor -const actorBoxPadding = 100 * SCALE_FACTOR // box padding -const actorBoxStroke = 15 * SCALE_FACTOR // the width of the stroke of the box -const internalLinkStrokeWidth = 30 * SCALE_FACTOR // the width of the link between nodes -const actorBoxRadius = 20 * SCALE_FACTOR - -// Stream Plan constant -const gapBetweenRow = 100 * SCALE_FACTOR -const gapBetweenLayer = 300 * SCALE_FACTOR -const gapBetweenFlowChart = 500 * SCALE_FACTOR -const outgoingLinkStrokeWidth = 20 * SCALE_FACTOR -const outgoingLinkBgStrokeWidth = 40 * SCALE_FACTOR - -// Draw linking effect -const bendGap = 50 * SCALE_FACTOR // try example at: http://bl.ocks.org/d3indepth/b6d4845973089bc1012dec1674d3aff8 -const connectionGap = 20 * SCALE_FACTOR - -// Others -const fontSize = 30 * SCALE_FACTOR -const outGoingLinkBgColor = "#eee" - -/** - * Construct an id for a link in actor box. - * You may use this method to query and get the svg element - * of the link. - * @param {{id: number}} node1 a node (operator) in an actor box - * @param {{id: number}} node2 a node (operator) in an actor box - * @returns {string} The link id - */ -function constructInternalLinkId(node1, node2) { - return ( - "node-" + - (node1.id > node2.id - ? node1.id + "-" + node2.id - : node2.id + "-" + node1.id) - ) -} - -/** - * Construct an id for a node (operator) in an actor box. - * You may use this method to query and get the svg element - * of the link. - * @param {{id: number}} node a node (operator) in an actor box - * @returns {string} The node id - */ -function constructOperatorNodeId(node) { - return "node-" + node.id -} - -function hashIpv4Index(addr) { - let [ip, port] = addr.split(":") - let s = "" - ip.split(".").map((x) => (s += x)) - return Number(s + port) -} - -export function computeNodeAddrToSideColor(addr) { - return color.TwoGradient(hashIpv4Index(addr))[1] -} - -/** - * Work flow - * 1. Get the layout for actor boxes (Calculate the base coordination of each actor box) - * 2. Get the layout for operators in each actor box - * 3. Draw all actor boxes - * 4. Draw link between actor boxes - * - * - * Dependencies - * layoutActorBox <- dagLayout <- drawActorBox <- drawFlow - * [ The layout of the ] [ The layout of ] [ Draw an actor ] [ Draw many actors ] - * [ operators in an ] [ actors in a ] [ in specified ] [ and links between ] - * [ actor. ] [ stream plan ] [ place ] [ them. ] - * - */ -export class StreamChartHelper { - /** - * - * @param {Group} g The group element in canvas engine - * @param {*} data The raw response from the meta node - * @param {(e, node) => void} onNodeClick The callback function triggered when a node is click - * @param {(e, actor) => void} onActorClick - * @param {{type: string, node: {host: {host: string, port: number}}, id?: number}} selectedWokerNode - * @param {Array} shownActorIdList - */ - constructor( - g, - data, - onNodeClick, - onActorClick, - selectedWokerNode, - shownActorIdList - ) { - this.topGroup = g - this.streamPlan = new StreamPlanParser(data, shownActorIdList) - this.onNodeClick = onNodeClick - this.onActorClick = onActorClick - this.selectedWokerNode = selectedWokerNode - this.selectedWokerNodeStr = this.selectedWokerNode - ? selectedWokerNode.host.host + ":" + selectedWokerNode.host.port - : "Show All" - } - - getMvTableIdToSingleViewActorList() { - return this.streamPlan.mvTableIdToSingleViewActorList - } - - getMvTableIdToChainViewActorList() { - return this.streamPlan.mvTableIdToChainViewActorList - } - - /** - * @param {Actor} actor - * @returns - */ - isInSelectedActor(actor) { - if (this.selectedWokerNodeStr === "Show All") { - // show all - return true - } else { - return actor.representedWorkNodes.has(this.selectedWokerNodeStr) - } - } - - _mainColor(actor) { - let addr = actor.representedWorkNodes.has(this.selectedWokerNodeStr) - ? this.selectedWokerNodeStr - : actor.computeNodeAddress - return color.TwoGradient(hashIpv4Index(addr))[0] - } - - _sideColor(actor) { - let addr = actor.representedWorkNodes.has(this.selectedWokerNodeStr) - ? this.selectedWokerNodeStr - : actor.computeNodeAddress - return color.TwoGradient(hashIpv4Index(addr))[1] - } - - _operatorColor = (actor, operator) => { - return this.isInSelectedActor(actor) && operator.type === "mviewNode" - ? this._mainColor(actor) - : "#eee" - } - _actorBoxBackgroundColor = (actor) => { - return this.isInSelectedActor(actor) ? this._sideColor(actor) : "#eee" - } - _actorOutgoinglinkColor = (actor) => { - return this.isInSelectedActor(actor) ? this._mainColor(actor) : "#fff" - } - - // - // A simple DAG layout algorithm. - // The layout is built based on two rules. - // 1. The link should have at two turnning points. - // 2. The turnning point of a link should be placed - // at the margin after the layer of its starting point. - // ------------------------------------------------------- - // Example 1: (X)-(Z) and (Y)-(Z) is valid. - // Row 0 (X)---------------->(Z) - // | - // Row 1 | - // | - // Row 2 (Y)---/ - // Layer 1 | Layer 2 | Layer 3 - // ------------------------------------------------------- - // Example 2: (A)-(B) is not valid. - // Row 0 (X) /---------\ (Z) - // | | - // Row 1 (A)---/ (Y) |-->(B) - // - // Layer 1 | Layer 2 | Layer 3 - // ------------------------------------------------------- - // Example 3: (C)-(Z) is not valid - // Row 0 (X) /-->(Z) - // | - // Row 1 (C)-------------/ - // - // Layer 1 | Layer 2 | Layer 3 - // ------------------------------------------------------- - // Note that the layer of each node can be different - // For example: - // Row 0 ( 1) ( 3) ( 5) ( 2) ( 9) - // Row 1 ( 4) ( 6) (10) - // Row 2 ( 7) ( 8) - // Layer 0 | Layer 1 | Layer 2 | Layer 3 | Layer 4 | - // - // Row 0 ( 1) ( 3) ( 5) ( 2) ( 9) - // Row 1 ( 4) ( 6) (10) - // Row 2 ( 7) ( 8) - // Layer 0 | Layer 1 | Layer 2 | Layer 3 | Layer 4 | - /** - * Topological sort - * @param {Array} nodes An array of node: {nextNodes: [...]} - * @returns {Map} position of each node - */ - dagLayout(nodes) { - let sorted = [] - let _nodes = [] - let node2dagNode = new Map() - const visit = (n) => { - if (n.temp) { - throw Error("This is not a DAG") - } - if (!n.perm) { - n.temp = true - let maxG = -1 - for (let nextNode of n.node.nextNodes) { - node2dagNode.get(nextNode).isInput = false - n.isOutput = false - let g = visit(node2dagNode.get(nextNode)) - if (g > maxG) { - maxG = g - } - } - n.temp = false - n.perm = true - n.g = maxG + 1 - sorted.unshift(n.node) - } - return n.g - } - for (let node of nodes) { - let dagNode = { - node: node, - temp: false, - perm: false, - isInput: true, - isOutput: true, - } - node2dagNode.set(node, dagNode) - _nodes.push(dagNode) - } - let maxLayer = 0 - for (let node of _nodes) { - let g = visit(node) - if (g > maxLayer) { - maxLayer = g - } - } - // use the bottom up strategy to construct generation number - // makes the generation number of root node the samllest - // to make the computation easier, need to flip it back. - for (let node of _nodes) { - // node.g = node.isInput ? 0 : (maxLayer - node.g); // TODO: determine which is more suitable - node.g = maxLayer - node.g - } - - let layers = [] - for (let i = 0; i < maxLayer + 1; ++i) { - layers.push({ - nodes: [], - occupyRow: new Set(), - }) - } - let node2Layer = new Map() - let node2Row = new Map() - for (let node of _nodes) { - layers[node.g].nodes.push(node.node) - node2Layer.set(node.node, node.g) - } - - // layers to rtn - let rtn = new Map() - - const putNodeInPosition = (node, row) => { - node2Row.set(node, row) - layers[node2Layer.get(node)].occupyRow.add(row) - } - - const occupyLine = (ls, le, r) => { - // layer start, layer end, row - for (let i = ls; i <= le; ++i) { - layers[i].occupyRow.add(r) - } - } - - const hasOccupied = (layer, row) => layers[layer].occupyRow.has(row) - - const isStraightLineOccupied = (ls, le, r) => { - // layer start, layer end, row - if (r < 0) { - return false - } - for (let i = ls; i <= le; ++i) { - if (hasOccupied(i, r)) { - return true - } - } - return false - } - - for (let node of nodes) { - node.nextNodes.sort((a, b) => node2Layer.get(b) - node2Layer.get(a)) - } - - for (let layer of layers) { - for (let node of layer.nodes) { - if (!node2Row.has(node)) { - // checking node is not placed. - for (let nextNode of node.nextNodes) { - if (node2Row.has(nextNode)) { - continue - } - let r = -1 - while ( - isStraightLineOccupied( - node2Layer.get(node), - node2Layer.get(nextNode), - ++r - ) - ) {} - putNodeInPosition(node, r) - putNodeInPosition(nextNode, r) - occupyLine( - node2Layer.get(node) + 1, - node2Layer.get(nextNode) - 1, - r - ) - break - } - if (!node2Row.has(node)) { - let r = -1 - while (hasOccupied(node2Layer.get(node), ++r)) {} - putNodeInPosition(node, r) - } - } - // checking node is placed in some position - for (let nextNode of node.nextNodes) { - if (node2Row.has(nextNode)) { - continue - } - // check straight line position first - let r = node2Row.get(node) - if ( - !isStraightLineOccupied( - node2Layer.get(node) + 1, - node2Layer.get(nextNode), - r - ) - ) { - putNodeInPosition(nextNode, r) - occupyLine( - node2Layer.get(node) + 1, - node2Layer.get(nextNode) - 1, - r - ) - continue - } - // check lowest available position - r = -1 - while ( - isStraightLineOccupied( - node2Layer.get(node) + 1, - node2Layer.get(nextNode), - ++r - ) - ) {} - putNodeInPosition(nextNode, r) - occupyLine(node2Layer.get(node) + 1, node2Layer.get(nextNode) - 1, r) - } - } - } - for (let node of nodes) { - rtn.set(node.id, [node2Layer.get(node), node2Row.get(node)]) - } - - return rtn - } - - /** - * Calculate the position of each node in the actor box. - * @param {{id: any, nextNodes: [], x: number, y: number}} rootNode The root node of an actor box (dispatcher) - * @returns {[width, height]} The size of the actor box - */ - calculateActorBoxSize(rootNode) { - let rootNodeCopy = cloneDeep(rootNode) - return this.layoutActorBox(rootNodeCopy, 0, 0) - } - - /** - * Calculate the position of each node (operator) in the actor box. - * This will change the node's position - * @param {{id: any, nextNodes: [], x: number, y: number}} rootNode The root node of an actor box (dispatcher) - * @param {number} baseX The x coordination of the top-left corner of the actor box - * @param {number} baseY The y coordination of the top-left corner of the actor box - * @returns {[width, height]} The size of the actor box - */ - layoutActorBox(rootNode, baseX, baseY) { - // calculate nodes' required width - let maxLayer = 0 - const getRequiredWidth = (node, layer) => { - if (node.width !== undefined) { - return node.width - } - - if (layer > maxLayer) { - maxLayer = layer - } - - node.layer = layer - - let requiredWidth = 0 - for (let nextNode of node.nextNodes) { - requiredWidth += getRequiredWidth(nextNode, layer + 1) - } - - node.isLeaf = requiredWidth === 0 - - node.width = requiredWidth > 0 ? requiredWidth : widthUnit - - return node.width - } - - getRequiredWidth(rootNode, 0) - - // calculate nodes' position - rootNode.x = baseX || 0 - rootNode.y = baseY || 0 - let leafY = rootNode.x - heightUnit * maxLayer - treeBfs(rootNode, (c) => { - let tmpY = c.y - c.width / 2 - for (let nextNode of c.nextNodes) { - nextNode.x = nextNode.isLeaf ? leafY : c.x - heightUnit - nextNode.y = tmpY + nextNode.width / 2 - tmpY += nextNode.width - } - }) - - // calculate box size - let minX = Infinity - let maxX = -Infinity - let minY = Infinity - let maxY = -Infinity - treeBfs(rootNode, (node) => { - if (node.x > maxX) { - maxX = node.x - } - if (node.x < minX) { - minX = node.x - } - if (node.y > maxY) { - maxY = node.y - } - if (node.y < minY) { - minY = node.y - } - }) - let boxWidth = maxX - minX - let boxHeight = maxY - minY - return [boxWidth + actorBoxPadding * 2, boxHeight + actorBoxPadding * 2] - } - - /** - * @param {{ - * g: Group, - * rootNode: {id: any, nextNodes: []}, - * nodeColor: string, - * strokeColor?: string, - * baseX?: number, - * baseY?: number - * }} props - * @param {Group} props.g The group element in canvas engine - * @param {{id: any, nextNodes: []}} props.rootNode The root node of the tree in the actor - * @param {string} props.nodeColor [optional] The filled color of nodes. - * @param {string} props.strokeColor [optional] The color of the stroke. - * @param {number} props.baseX [optional] The x coordination of the lef-top corner. default: 0 - * @param {number} props.baseY [optional] The y coordination of the lef-top corner. default: 0 - * @returns {Group} The group element of this tree - */ - drawActorBox(props) { - if (props.g === undefined) { - throw Error("Invalid Argument: Target group cannot be undefined.") - } - - const actor = props.actor - const group = props.g.append("g") - const rootNode = props.rootNode || [] - const baseX = props.x === undefined ? 0 : props.x - const baseY = props.y === undefined ? 0 : props.y - const strokeColor = props.strokeColor || "white" - const linkColor = props.linkColor || "gray" - - group.attr("class", actor.computeNodeAddress) - - const [boxWidth, boxHeight] = this.calculateActorBoxSize(rootNode) - this.layoutActorBox( - rootNode, - baseX + boxWidth - actorBoxPadding, - baseY + boxHeight / 2 - ) - - const onNodeClicked = (e, node, actor) => { - this.onNodeClick && this.onNodeClick(e, node, actor) - } - - const onActorClick = (e, actor) => { - this.onActorClick && this.onActorClick(e, actor) - } - - /** - * @param {Group} g actor box group - * @param {number} x top-right corner of the label - * @param {number} y top-right corner of the label - * @param {Array} actorIds - * @param {string} color - * @returns {number} width of this label - */ - const drawActorIdLabel = (g, x, y, actorIds, color) => { - y = y - actorBoxStroke - let actorStr = actorIds.toString() - let padding = 15 - // let height = fontSize + 2 * padding; - let gap = 30 - // let polygon = g.append("polygon"); - let textEle = g - .append("text")(actorStr) - .attr("font-size", fontSize) - .position(x - padding - 5, y + padding) - let width = textEle.getWidth() + 2 * padding - // polygon.attr("points", `${x},${y} ${x - width - gap},${y}, ${x - width},${y + height}, ${x},${y + height}`) - // .attr("fill", color); - return width + gap - } - - // draw box - group.attr("id", "actor-" + actor.actorId) - let actorRect = group.append("rect") - for (let representedActor of actor.representedActorList) { - actorRect.classed("actor-" + representedActor.actorId, true) - } - actorRect.classed("fragment-" + actor.fragmentId, true) - actorRect - .init(baseX, baseY, boxWidth, boxHeight) - .attr("fill", this._actorBoxBackgroundColor(actor)) - .attr("rx", actorBoxRadius) - .attr("stroke-width", actorBoxStroke) - .on("click", (e) => onActorClick(e, actor)) - - group - .append("text")(`Fragment ${actor.fragmentId}`) - .position(baseX, baseY - actorBoxStroke - fontSize) - .attr("font-size", fontSize) - - // draw compute node label - let computeNodeToActorIds = new Map() - for (let representedActor of actor.representedActorList) { - if (computeNodeToActorIds.has(representedActor.computeNodeAddress)) { - computeNodeToActorIds - .get(representedActor.computeNodeAddress) - .push(representedActor.actorId) - } else { - computeNodeToActorIds.set(representedActor.computeNodeAddress, [ - representedActor.actorId, - ]) - } - } - let labelStartX = baseX + actorBoxStroke - for (let [addr, actorIds] of computeNodeToActorIds.entries()) { - let w = drawActorIdLabel( - group, - labelStartX, - baseY + boxHeight, - actorIds, - color.TwoGradient(hashIpv4Index(addr))[1] - ) - labelStartX -= w - } - - // draw links - const linkData = [] - treeBfs(rootNode, (c) => { - for (let nextNode of c.nextNodes) { - linkData.push({ - sourceNode: c, - nextNode: nextNode, - source: [c.x, c.y], - target: [nextNode.x, nextNode.y], - }) - } - }) - const linkGen = d3.linkHorizontal() - for (let link of linkData) { - group - .append("path")(linkGen(link)) - .attr( - "stroke-dasharray", - `${internalLinkStrokeWidth / 2},${internalLinkStrokeWidth / 2}` - ) - // .attr("d", linkGen(link)) - .attr("fill", "none") - .attr("class", "actor-" + actor.actorId) - .classed("internal-link", true) - .attr("id", constructInternalLinkId(link.sourceNode, link.nextNode)) - .style("stroke-width", internalLinkStrokeWidth) - .attr("stroke", linkColor) - } - - // draw nodes - treeBfs(rootNode, (node) => { - node.d3Selection = group - .append("circle") - .init(node.x, node.y, operatorNodeRadius) - .attr("id", constructOperatorNodeId(node)) - .attr("stroke", strokeColor) - .attr("fill", this._operatorColor(actor, node)) - .style("cursor", "pointer") - .style("stroke-width", operatorNodeStrokeWidth) - .on("click", (e) => onNodeClicked(e, node, actor)) - group - .append("text")(node.type ? node.type : node.dispatcherType) - .position(node.x, node.y + operatorNodeRadius + 10) - .attr("font-size", fontSize) - }) - - return { - g: group, - x: baseX - boxWidth - actorBoxPadding, - y: baseY - boxHeight / 2 - actorBoxPadding, - width: boxWidth + actorBoxPadding * 2, - height: boxHeight + actorBoxPadding * 2, - } - } - /** - * - * @param {{ - * g: Group, - * actorDagList: Array, - * baseX?: number, - * baseY?: number - * }} props - * @param {Group} props.g The target group contains this group. - * @param {Array} props.actorDagList A list of dag nodes constructed from actors - * { id: actor.actorId, nextNodes: [], actor: actor } - * @param {number} props.baseX [optional] The x coordination of left-top corner. default: 0. - * @param {number} props.baseY [optional] The y coordination of left-top corner. default: 0. - * @returns {{group: Group, width: number, height: number}} The size of the flow - */ - drawFlow(props) { - if (props.g === undefined) { - throw Error("Invalid Argument: Target group cannot be undefined.") - } - - const g = props.g - const actorDagList = props.actorDagList || [] - const baseX = props.baseX || 0 - const baseY = props.baseY || 0 - - let layoutPositionMapper = this.dagLayout(actorDagList) - const actors = [] - for (let actorDag of actorDagList) { - actors.push(actorDag.actor) - } - - // calculate actor box size - for (let actor of actors) { - ;[actor.boxWidth, actor.boxHeight] = this.calculateActorBoxSize( - actor.rootNode - ) - ;[actor.layer, actor.row] = layoutPositionMapper.get(actor.actorId) - } - - // calculate the minimum required width of each layer and row - let maxRow = 0 - let maxLayer = 0 - for (let actor of actors) { - maxLayer = max([actor.layer, maxLayer]) - maxRow = max([actor.row, maxRow]) - } - let rowGap = newNumberArray(maxRow + 1) - let layerGap = newNumberArray(maxLayer + 1) - for (let actor of actors) { - layerGap[actor.layer] = max([layerGap[actor.layer], actor.boxWidth]) - rowGap[actor.row] = max([rowGap[actor.row], actor.boxHeight]) - } - let row2y = newNumberArray(maxRow + 1) - let layer2x = newNumberArray(maxLayer + 1) - row2y = row2y.map((_, r) => { - if (r === 0) { - return 0 - } - let rtn = 0 - for (let i = 0; i < r; ++i) { - rtn += rowGap[i] + gapBetweenRow - } - return rtn - }) - layer2x = layer2x.map((_, l) => { - if (l === 0) { - return 0 - } - let rtn = 0 - for (let i = 0; i < l; ++i) { - rtn += layerGap[i] + gapBetweenLayer - } - return rtn - }) - - // Draw fragment (represent by one actor) - const group = g.append("g") - const linkLayerBackground = group.append("g") - const linkLayer = group.append("g") - const fragmentLayer = group.append("g") - linkLayerBackground.attr("class", "linkLayerBackground") - linkLayer.attr("class", "linkLayer") - fragmentLayer.attr("class", "fragmentLayer") - - let actorBoxList = [] - for (let actor of actors) { - let actorBox = this.drawActorBox({ - actor: actor, - g: fragmentLayer, - rootNode: actor.rootNode, - x: baseX + layer2x[actor.layer], - y: baseY + row2y[actor.row], - strokeColor: "white", - linkColor: "white", - }) - actorBoxList.push(actorBox) - } - - // Draw link between (represent by one actor) - const getLinkBetweenPathStr = (start, end, compensation) => { - const lineGen = d3.line().curve(d3.curveBasis) - let pathStr = lineGen([ - end, - [ - start[0] + - compensation + - actorBoxPadding + - connectionGap + - bendGap * 2, - end[1], - ], - [ - start[0] + compensation + actorBoxPadding + connectionGap + bendGap, - end[1], - ], - [ - start[0] + compensation + actorBoxPadding + connectionGap + bendGap, - start[1], - ], - [start[0] + compensation + actorBoxPadding + connectionGap, start[1]], - start, - ]) - return pathStr - } - - let linkData = [] - for (let actor of actors) { - for (let outputNode of actor.output) { - linkData.push({ - actor: actor, - d: getLinkBetweenPathStr( - [actor.rootNode.x, actor.rootNode.y], - [outputNode.x, outputNode.y], - layerGap[actor.layer] - actor.boxWidth - ), - }) - } - } - - for (let s of linkData) { - linkLayer - .append("path")(s.d) - .attr( - "stroke-dasharray", - `${outgoingLinkStrokeWidth},${outgoingLinkStrokeWidth}` - ) - .attr("fill", "none") - .attr("class", "actor-" + s.actor.actorId) - .classed("outgoing-link", true) - .style("stroke-width", outgoingLinkStrokeWidth) - .attr("stroke", this._actorOutgoinglinkColor(s.actor)) - .attr("layer", "back") - } - - for (let s of linkData) { - linkLayerBackground - .append("path")(s.d) - .attr("fill", "none") - .style("stroke-width", outgoingLinkBgStrokeWidth) - .attr("class", "actor-" + s.actor.actorId) - .classed("outgoing-link-bg", true) - .attr("stroke", outGoingLinkBgColor) - .attr("layer", "back") - } - - // calculate box size - let width = 0 - let height = 0 - for (let actorBox of actorBoxList) { - let biggestX = actorBox.x - baseX + actorBox.width - let biggestY = actorBox.y - baseY + actorBox.height - width = max([biggestX, width]) - height = max([biggestY, height]) - } - - group.attr("class", "flowchart") - return { - g: group, - width: width, - height: height, - } - } - - /** - * A flow is an extracted connected component of actors of - * the raw response from the meta node. This method will first - * merge actors in the same fragment using some identifier - * (currently it is the id of the operator before the dispatcher). - * And then use `drawFlow()` to draw each connected component. - */ - drawManyFlow() { - const g = this.topGroup - const baseX = 0 - const baseY = 0 - - g.attr("id", "") - - let fragmentRepresentedActors = this.streamPlan.fragmentRepresentedActors - // get dag layout of these actors - let dagNodeMap = new Map() - for (let actor of fragmentRepresentedActors) { - actor.rootNode.actorId = actor.actorId - treeBfs(actor.rootNode, (node) => { - node.actorId = actor.actorId - }) - dagNodeMap.set(actor.actorId, { - id: actor.actorId, - nextNodes: [], - actor: actor, - }) - } - for (let actor of fragmentRepresentedActors) { - for (let outputActorNode of actor.output) { - let outputDagNode = dagNodeMap.get(outputActorNode.actorId) - if (outputDagNode) { - // the output actor node is in a represented actor - dagNodeMap.get(actor.actorId).nextNodes.push(outputDagNode) - } - } - } - let actorDagNodes = [] - for (let id of dagNodeMap.keys()) { - actorDagNodes.push(dagNodeMap.get(id)) - } - - let actorsList = getConnectedComponent(actorDagNodes) - - let y = baseY - for (let actorDagList of actorsList) { - let flowChart = this.drawFlow({ - g: g, - baseX: baseX, - baseY: y, - actorDagList: actorDagList, - }) - y += flowChart.height + gapBetweenFlowChart - } - } -} - -/** - * create a graph view based on raw input from the meta node, - * and append the svg component to the giving svg group. - * @param {Group} g The parent group contain the graph. - * @param {any} data Raw response from the meta node. e.g. [{node: {...}, actors: {...}}, ...] - * @param {(clickEvent, node, actor) => void} onNodeClick callback when a node (operator) is clicked. - * @param {{type: string, node: {host: {host: string, port: number}}, id?: number}} selectedWokerNode - * @returns {StreamChartHelper} - */ -export default function createView( - engine, - data, - onNodeClick, - onActorClick, - selectedWokerNode, - shownActorIdList -) { - console.log(shownActorIdList, "shownActorList") - let streamChartHelper = new StreamChartHelper( - engine.topGroup, - data, - onNodeClick, - onActorClick, - selectedWokerNode, - shownActorIdList - ) - streamChartHelper.drawManyFlow() - return streamChartHelper -} diff --git a/dashboard/test/algo.test.js b/dashboard/test/algo.test.js deleted file mode 100644 index 9fe47669fed6e..0000000000000 --- a/dashboard/test/algo.test.js +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Copyright 2024 RisingWave Labs - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - * - */ -import { Node } from "../lib/algo" -import { StreamChartHelper } from "../lib/streamPlan/streamChartHelper" - -describe("Algo", () => { - it("should generate right dag layout", () => { - // fake data - let nodes = [] - for (let i = 0; i < 10; ++i) { - nodes.push(new Node([], i + 1)) - } - const n = (i) => nodes[i - 1] - n(1).nextNodes = [n(2), n(3)] - n(2).nextNodes = [n(9)] - n(3).nextNodes = [n(5), n(10)] - n(4).nextNodes = [n(5)] - n(5).nextNodes = [n(6), n(7)] - n(6).nextNodes = [n(9), n(10)] - n(7).nextNodes = [n(8)] - - let dagPositionMapper = new StreamChartHelper().dagLayout(nodes) - - // construct map - let maxLayer = 0 - let maxRow = 0 - for (let node of dagPositionMapper.keys()) { - let pos = dagPositionMapper.get(node) - maxLayer = pos[0] > maxLayer ? pos[0] : maxLayer - maxRow = pos[1] > maxRow ? pos[1] : maxRow - } - let m = [] - for (let i = 0; i < maxLayer + 1; ++i) { - m.push([]) - for (let r = 0; r < maxRow + 1; ++r) { - m[i].push([]) - } - } - for (let node of dagPositionMapper.keys()) { - let pos = dagPositionMapper.get(node) - m[pos[0]][pos[1]] = node - } - - // search - const _search = (l, r, d) => { - // Layer, Row - if (l > maxLayer || r > maxRow || r < 0) { - return false - } - if (m[l][r].id !== undefined) { - return m[l][r].id === d - } - return _search(l + 1, r, d) - } - - const canReach = (node, nextNode) => { - let pos = dagPositionMapper.get(node) - for (let r = 0; r <= maxRow; ++r) { - if (_search(pos[0] + 1, r, nextNode.id)) { - return true - } - } - return false - } - - //check all links - let ok = true - for (let node of nodes) { - for (let nextNode of node.nextNodes) { - if (!canReach(node, nextNode)) { - console.error( - `Failed to connect node ${node.id} to node ${nextNode.id}` - ) - ok = false - break - } - } - if (!ok) { - break - } - } - - // visualization - // let s = ""; - // for(let r = maxRow; r >= 0; --r){ - // for(let l = 0; l <= maxLayer; ++l){ - // s += `\t${m[l][r].id ? m[l][r].id : " "}` - // } - // s += "\n" - // } - // console.log(s); - - expect(ok).toEqual(true) - }) -}) From ed4101f86b33b18ad6d40cdbf4a3f8367c0a273c Mon Sep 17 00:00:00 2001 From: Wallace Date: Fri, 5 Jan 2024 18:08:23 +0800 Subject: [PATCH 11/12] feat(storage): reduce size of compaction task (#14160) Signed-off-by: Little-Wallace --- .../compaction/selector/level_selector.rs | 2 +- .../compactor/fast_compactor_runner.rs | 1 - src/storage/src/hummock/compactor/mod.rs | 1 - src/storage/src/hummock/sstable/builder.rs | 4 - .../src/hummock/sstable/multi_builder.rs | 75 +++++-------------- 5 files changed, 21 insertions(+), 62 deletions(-) diff --git a/src/meta/src/hummock/compaction/selector/level_selector.rs b/src/meta/src/hummock/compaction/selector/level_selector.rs index 736c10fac69c6..e2b1cc65f3c2a 100644 --- a/src/meta/src/hummock/compaction/selector/level_selector.rs +++ b/src/meta/src/hummock/compaction/selector/level_selector.rs @@ -120,7 +120,7 @@ impl DynamicLevelSelectorCore { Box::new(MinOverlappingPicker::new( picker_info.select_level, picker_info.target_level, - self.config.max_bytes_for_level_base, + self.config.max_bytes_for_level_base / 2, overlap_strategy, )) } diff --git a/src/storage/src/hummock/compactor/fast_compactor_runner.rs b/src/storage/src/hummock/compactor/fast_compactor_runner.rs index 1391414b91bdd..653a00f21b7c2 100644 --- a/src/storage/src/hummock/compactor/fast_compactor_runner.rs +++ b/src/storage/src/hummock/compactor/fast_compactor_runner.rs @@ -317,7 +317,6 @@ impl CompactorRunner { builder_factory, context.compactor_metrics.clone(), Some(task_progress.clone()), - task_config.is_target_l0_or_lbase, task_config.table_vnode_partition.clone(), ); assert_eq!( diff --git a/src/storage/src/hummock/compactor/mod.rs b/src/storage/src/hummock/compactor/mod.rs index a663c7c3819a6..d82172650e19c 100644 --- a/src/storage/src/hummock/compactor/mod.rs +++ b/src/storage/src/hummock/compactor/mod.rs @@ -265,7 +265,6 @@ impl Compactor { builder_factory, self.context.compactor_metrics.clone(), task_progress.clone(), - self.task_config.is_target_l0_or_lbase, self.task_config.table_vnode_partition.clone(), ); let compaction_statistics = compact_and_build_sst( diff --git a/src/storage/src/hummock/sstable/builder.rs b/src/storage/src/hummock/sstable/builder.rs index b246d15b03fcf..9202b3ec28788 100644 --- a/src/storage/src/hummock/sstable/builder.rs +++ b/src/storage/src/hummock/sstable/builder.rs @@ -717,10 +717,6 @@ impl SstableBuilder { self.approximate_len() >= self.options.capacity } - pub fn reach_max_sst_size(&self) -> bool { - self.approximate_len() as u64 >= self.options.max_sst_size - } - fn finalize_last_table_stats(&mut self) { if self.table_ids.is_empty() || self.last_table_id.is_none() { return; diff --git a/src/storage/src/hummock/sstable/multi_builder.rs b/src/storage/src/hummock/sstable/multi_builder.rs index a1946f56a33de..42a19866fc467 100644 --- a/src/storage/src/hummock/sstable/multi_builder.rs +++ b/src/storage/src/hummock/sstable/multi_builder.rs @@ -72,13 +72,11 @@ where task_progress: Option>, last_table_id: u32, - is_target_level_l0_or_lbase: bool, table_partition_vnode: BTreeMap, split_weight_by_vnode: u32, /// When vnode of the coming key is greater than `largest_vnode_in_current_partition`, we will /// switch SST. largest_vnode_in_current_partition: usize, - last_vnode: usize, } impl CapacitySplitTableBuilder @@ -91,7 +89,6 @@ where builder_factory: F, compactor_metrics: Arc, task_progress: Option>, - is_target_level_l0_or_lbase: bool, table_partition_vnode: BTreeMap, ) -> Self { Self { @@ -101,11 +98,9 @@ where compactor_metrics, task_progress, last_table_id: 0, - is_target_level_l0_or_lbase, table_partition_vnode, split_weight_by_vnode: 0, largest_vnode_in_current_partition: VirtualNode::MAX.to_index(), - last_vnode: 0, } } @@ -117,11 +112,9 @@ where compactor_metrics: Arc::new(CompactorMetrics::unused()), task_progress: None, last_table_id: 0, - is_target_level_l0_or_lbase: false, table_partition_vnode: BTreeMap::default(), split_weight_by_vnode: 0, largest_vnode_in_current_partition: VirtualNode::MAX.to_index(), - last_vnode: 0, } } @@ -181,7 +174,7 @@ where value: HummockValue<&[u8]>, is_new_user_key: bool, ) -> HummockResult<()> { - let (switch_builder, vnode_changed) = self.check_table_and_vnode_change(&full_key.user_key); + let switch_builder = self.check_switch_builder(&full_key.user_key); // We use this `need_seal_current` flag to store whether we need to call `seal_current` and // then call `seal_current` later outside the `if let` instead of calling @@ -195,15 +188,7 @@ where let mut last_range_tombstone_epoch = HummockEpoch::MAX; if let Some(builder) = self.current_builder.as_mut() { if is_new_user_key { - if switch_builder { - need_seal_current = true; - } else if builder.reach_capacity() { - if !self.is_target_level_l0_or_lbase || builder.reach_max_sst_size() { - need_seal_current = true; - } else { - need_seal_current = self.is_target_level_l0_or_lbase && vnode_changed; - } - } + need_seal_current = switch_builder || builder.reach_capacity(); } if need_seal_current && let Some(event) = builder.last_range_tombstone() @@ -253,9 +238,8 @@ where builder.add(full_key, value).await } - pub fn check_table_and_vnode_change(&mut self, user_key: &UserKey<&[u8]>) -> (bool, bool) { + pub fn check_switch_builder(&mut self, user_key: &UserKey<&[u8]>) -> bool { let mut switch_builder = false; - let mut vnode_changed = false; if user_key.table_id.table_id != self.last_table_id { let new_vnode_partition_count = self.table_partition_vnode.get(&user_key.table_id.table_id); @@ -272,8 +256,6 @@ where // table_id change self.last_table_id = user_key.table_id.table_id; switch_builder = true; - self.last_vnode = 0; - vnode_changed = true; if self.split_weight_by_vnode > 1 { self.largest_vnode_in_current_partition = VirtualNode::COUNT / (self.split_weight_by_vnode as usize) - 1; @@ -285,10 +267,6 @@ where } if self.largest_vnode_in_current_partition != VirtualNode::MAX.to_index() { let key_vnode = user_key.get_vnode_id(); - if key_vnode != self.last_vnode { - self.last_vnode = key_vnode; - vnode_changed = true; - } if key_vnode > self.largest_vnode_in_current_partition { // vnode partition change switch_builder = true; @@ -303,11 +281,10 @@ where ((key_vnode - small_segments_area) / (basic + 1) + 1) * (basic + 1) + small_segments_area }) - 1; - self.last_vnode = key_vnode; debug_assert!(key_vnode <= self.largest_vnode_in_current_partition); } } - (switch_builder, vnode_changed) + switch_builder } pub fn need_flush(&self) -> bool { @@ -616,7 +593,6 @@ mod tests { LocalTableBuilderFactory::new(1001, mock_sstable_store(), opts), Arc::new(CompactorMetrics::unused()), None, - false, BTreeMap::default(), ); let full_key = FullKey::for_test( @@ -713,7 +689,6 @@ mod tests { LocalTableBuilderFactory::new(1001, mock_sstable_store(), opts), Arc::new(CompactorMetrics::unused()), None, - false, BTreeMap::default(), ); del_iter.rewind().await.unwrap(); @@ -750,7 +725,6 @@ mod tests { LocalTableBuilderFactory::new(1001, mock_sstable_store(), opts), Arc::new(CompactorMetrics::unused()), None, - false, BTreeMap::default(), ); builder @@ -870,56 +844,47 @@ mod tests { LocalTableBuilderFactory::new(1001, mock_sstable_store(), opts), Arc::new(CompactorMetrics::unused()), None, - false, table_partition_vnode, ); let mut table_key = VirtualNode::from_index(0).to_be_bytes().to_vec(); table_key.extend_from_slice("a".as_bytes()); - let (switch_builder, vnode_changed) = - builder.check_table_and_vnode_change(&UserKey::for_test(TableId::from(1), &table_key)); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(1), &table_key)); assert!(switch_builder); - assert!(vnode_changed); { let mut table_key = VirtualNode::from_index(62).to_be_bytes().to_vec(); table_key.extend_from_slice("a".as_bytes()); - let (switch_builder, vnode_changed) = builder - .check_table_and_vnode_change(&UserKey::for_test(TableId::from(1), &table_key)); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(1), &table_key)); assert!(!switch_builder); - assert!(vnode_changed); let mut table_key = VirtualNode::from_index(63).to_be_bytes().to_vec(); table_key.extend_from_slice("a".as_bytes()); - let (switch_builder, vnode_changed) = builder - .check_table_and_vnode_change(&UserKey::for_test(TableId::from(1), &table_key)); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(1), &table_key)); assert!(!switch_builder); - assert!(vnode_changed); let mut table_key = VirtualNode::from_index(64).to_be_bytes().to_vec(); table_key.extend_from_slice("a".as_bytes()); - let (switch_builder, vnode_changed) = builder - .check_table_and_vnode_change(&UserKey::for_test(TableId::from(1), &table_key)); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(1), &table_key)); assert!(switch_builder); - assert!(vnode_changed); } - let (switch_builder, vnode_changed) = - builder.check_table_and_vnode_change(&UserKey::for_test(TableId::from(2), &table_key)); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(2), &table_key)); assert!(switch_builder); - assert!(vnode_changed); - let (switch_builder, vnode_changed) = - builder.check_table_and_vnode_change(&UserKey::for_test(TableId::from(3), &table_key)); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(3), &table_key)); assert!(switch_builder); - assert!(vnode_changed); - let (switch_builder, vnode_changed) = - builder.check_table_and_vnode_change(&UserKey::for_test(TableId::from(4), &table_key)); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(4), &table_key)); assert!(switch_builder); - assert!(vnode_changed); - let (switch_builder, vnode_changed) = - builder.check_table_and_vnode_change(&UserKey::for_test(TableId::from(5), &table_key)); + let switch_builder = + builder.check_switch_builder(&UserKey::for_test(TableId::from(5), &table_key)); assert!(!switch_builder); - assert!(!vnode_changed); } } From 7954da3e7ea628d4d27d840287c31bd5148d2c9a Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Fri, 5 Jan 2024 18:14:55 +0800 Subject: [PATCH 12/12] feat: support multiple temporal filter with or (#14382) --- .../tests/testdata/input/temporal_filter.yaml | 6 + .../testdata/output/temporal_filter.yaml | 123 +++++++++++------- .../src/optimizer/logical_optimization.rs | 6 +- src/frontend/src/optimizer/rule/mod.rs | 2 + .../stream/filter_with_now_to_join_rule.rs | 14 -- src/frontend/src/optimizer/rule/stream/mod.rs | 1 + .../rule/stream/split_now_and_rule.rs | 84 ++++++++++++ 7 files changed, 177 insertions(+), 59 deletions(-) create mode 100644 src/frontend/src/optimizer/rule/stream/split_now_and_rule.rs diff --git a/src/frontend/planner_test/tests/testdata/input/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/input/temporal_filter.yaml index 8df9d78869f04..6bd62c1ce4d61 100644 --- a/src/frontend/planner_test/tests/testdata/input/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/input/temporal_filter.yaml @@ -109,5 +109,11 @@ sql: | create table t1 (ts timestamp with time zone); select * from t1 where ts + interval '1 hour' > now() or ts > ' 2023-12-18 00:00:00+00:00' or ts is null; + expected_outputs: + - stream_plan +- name: Many Temporal filter with or predicate + sql: | + create table t (t timestamp with time zone, a int); + select * from t where (t > NOW() - INTERVAL '1 hour' OR t is NULL OR a < 1) AND (t < NOW() - INTERVAL '1 hour' OR a > 1); expected_outputs: - stream_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml index a8799aba8a4f4..1f5934ce76378 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_filter.yaml @@ -218,56 +218,43 @@ create table t1 (ts timestamp with time zone); select * from t1 where ts < now() - interval '1 hour' and ts >= now() - interval '2 hour'; stream_plan: |- - StreamMaterialize { columns: [ts, t1._row_id(hidden)], stream_key: [t1._row_id], pk_columns: [t1._row_id], pk_conflict: NoCheck } - └─StreamDynamicFilter { predicate: (t1.ts < $expr2), output: [t1.ts, t1._row_id], condition_always_relax: true } - ├─StreamDynamicFilter { predicate: (t1.ts >= $expr1), output_watermarks: [t1.ts], output: [t1.ts, t1._row_id], cleaned_by_watermark: true } + StreamMaterialize { columns: [ts, t1._row_id(hidden)], stream_key: [t1._row_id], pk_columns: [t1._row_id], pk_conflict: NoCheck, watermark_columns: [ts] } + └─StreamDynamicFilter { predicate: (t1.ts >= $expr2), output_watermarks: [t1.ts], output: [t1.ts, t1._row_id], cleaned_by_watermark: true } + ├─StreamDynamicFilter { predicate: (t1.ts < $expr1), output: [t1.ts, t1._row_id], condition_always_relax: true } │ ├─StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } │ └─StreamNow { output: [now] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [ts, t1._row_id(hidden)], stream_key: [t1._row_id], pk_columns: [t1._row_id], pk_conflict: NoCheck } + StreamMaterialize { columns: [ts, t1._row_id(hidden)], stream_key: [t1._row_id], pk_columns: [t1._row_id], pk_conflict: NoCheck, watermark_columns: [ts] } ├── materialized table: 4294967294 - └── StreamDynamicFilter { predicate: (t1.ts < $expr2), output: [t1.ts, t1._row_id], condition_always_relax: true } + └── StreamDynamicFilter { predicate: (t1.ts >= $expr2), output_watermarks: [t1.ts], output: [t1.ts, t1._row_id], cleaned_by_watermark: true } ├── left table: 0 ├── right table: 1 - ├── StreamDynamicFilter { predicate: (t1.ts >= $expr1), output_watermarks: [t1.ts], output: [t1.ts, t1._row_id], cleaned_by_watermark: true } - │ ├── left table: 2 - │ ├── right table: 3 - │ ├── StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - │ │ ├── state table: 4 + ├── StreamDynamicFilter { predicate: (t1.ts < $expr1), output: [t1.ts, t1._row_id], condition_always_relax: true } { left table: 2, right table: 3 } + │ ├── StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 4 } │ │ ├── Upstream │ │ └── BatchPlanNode │ └── StreamExchange Broadcast from 1 └── StreamExchange Broadcast from 2 Fragment 1 - StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } └── StreamNow { output: [now] } { state table: 5 } Fragment 2 - StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └── StreamNow { output: [now] } { state table: 6 } - Table 0 - ├── columns: [ t1_ts, t1__row_id ] - ├── primary key: [ $0 ASC, $1 ASC ] - ├── value indices: [ 0, 1 ] - ├── distribution key: [ 1 ] - └── read pk prefix len hint: 1 + Table 0 { columns: [ t1_ts, t1__row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } Table 1 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 2 - ├── columns: [ t1_ts, t1__row_id ] - ├── primary key: [ $0 ASC, $1 ASC ] - ├── value indices: [ 0, 1 ] - ├── distribution key: [ 1 ] - └── read pk prefix len hint: 1 + Table 2 { columns: [ t1_ts, t1__row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } Table 3 { columns: [ $expr1 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } @@ -283,12 +270,7 @@ Table 6 { columns: [ now ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 - ├── columns: [ ts, t1._row_id ] - ├── primary key: [ $1 ASC ] - ├── value indices: [ 0, 1 ] - ├── distribution key: [ 1 ] - └── read pk prefix len hint: 1 + Table 4294967294 { columns: [ ts, t1._row_id ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - name: Temporal filter in on clause for inner join's left side sql: | @@ -300,14 +282,14 @@ └─StreamExchange { dist: HashShard(t1.a, t1._row_id, t2._row_id) } └─StreamHashJoin { type: Inner, predicate: t1.a = t2.b, output: [t1.a, t1.ta, t2.b, t2.tb, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.a) } - │ └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id], condition_always_relax: true } - │ ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } + │ └─StreamDynamicFilter { predicate: (t1.ta >= $expr2), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } + │ ├─StreamDynamicFilter { predicate: (t1.ta < $expr1), output: [t1.a, t1.ta, t1._row_id], condition_always_relax: true } │ │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } │ │ └─StreamExchange { dist: Broadcast } - │ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } │ │ └─StreamNow { output: [now] } │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } │ └─StreamNow { output: [now] } └─StreamExchange { dist: HashShard(t2.b) } └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } @@ -331,14 +313,14 @@ ├─StreamExchange { dist: HashShard(t2.b) } │ └─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } └─StreamExchange { dist: HashShard(t1.a) } - └─StreamDynamicFilter { predicate: (t1.ta < $expr2), output: [t1.a, t1.ta, t1._row_id], condition_always_relax: true } - ├─StreamDynamicFilter { predicate: (t1.ta >= $expr1), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } + └─StreamDynamicFilter { predicate: (t1.ta >= $expr2), output_watermarks: [t1.ta], output: [t1.a, t1.ta, t1._row_id], cleaned_by_watermark: true } + ├─StreamDynamicFilter { predicate: (t1.ta < $expr1), output: [t1.a, t1.ta, t1._row_id], condition_always_relax: true } │ ├─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } │ └─StreamNow { output: [now] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └─StreamNow { output: [now] } - name: Temporal filter in on clause for full join's left side sql: | @@ -360,14 +342,14 @@ ├─StreamExchange { dist: HashShard(t1.a) } │ └─StreamTableScan { table: t1, columns: [t1.a, t1.ta, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(t2.b) } - └─StreamDynamicFilter { predicate: (t2.tb < $expr2), output: [t2.b, t2.tb, t2._row_id], condition_always_relax: true } - ├─StreamDynamicFilter { predicate: (t2.tb >= $expr1), output_watermarks: [t2.tb], output: [t2.b, t2.tb, t2._row_id], cleaned_by_watermark: true } + └─StreamDynamicFilter { predicate: (t2.tb >= $expr2), output_watermarks: [t2.tb], output: [t2.b, t2.tb, t2._row_id], cleaned_by_watermark: true } + ├─StreamDynamicFilter { predicate: (t2.tb < $expr1), output: [t2.b, t2.tb, t2._row_id], condition_always_relax: true } │ ├─StreamTableScan { table: t2, columns: [t2.b, t2.tb, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } │ └─StreamNow { output: [now] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '02:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } └─StreamNow { output: [now] } - name: Temporal filter in on clause for right join's right side sql: | @@ -462,3 +444,56 @@ └─StreamShare { id: 2 } └─StreamFilter { predicate: (((Not((t1.ts > '2023-12-18 00:00:00+00:00':Timestamptz)) AND Not(IsNull(t1.ts))) OR (t1.ts > '2023-12-18 00:00:00+00:00':Timestamptz)) OR IsNull(t1.ts)) } └─StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } +- name: Many Temporal filter with or predicate + sql: | + create table t (t timestamp with time zone, a int); + select * from t where (t > NOW() - INTERVAL '1 hour' OR t is NULL OR a < 1) AND (t < NOW() - INTERVAL '1 hour' OR a > 1); + stream_plan: |- + StreamMaterialize { columns: [t, a, $src(hidden), t._row_id(hidden), $src#1(hidden)], stream_key: [t._row_id, $src, $src#1], pk_columns: [t._row_id, $src, $src#1], pk_conflict: NoCheck } + └─StreamUnion { all: true } + ├─StreamExchange { dist: HashShard(t._row_id, $src, 0:Int32) } + │ └─StreamProject { exprs: [t.t, t.a, $src, t._row_id, 0:Int32] } + │ └─StreamDynamicFilter { predicate: (t.t < $expr2), output: [t.t, t.a, t._row_id, $src], condition_always_relax: true } + │ ├─StreamFilter { predicate: Not((t.a > 1:Int32)) } + │ │ └─StreamShare { id: 13 } + │ │ └─StreamUnion { all: true } + │ │ ├─StreamExchange { dist: HashShard(t._row_id, 0:Int32) } + │ │ │ └─StreamProject { exprs: [t.t, t.a, t._row_id, 0:Int32], output_watermarks: [t.t] } + │ │ │ └─StreamDynamicFilter { predicate: (t.t > $expr1), output_watermarks: [t.t], output: [t.t, t.a, t._row_id], cleaned_by_watermark: true } + │ │ │ ├─StreamFilter { predicate: (Not((t.a > 1:Int32)) OR (t.a > 1:Int32)) AND Not(IsNull(t.t)) AND Not((t.a < 1:Int32)) } + │ │ │ │ └─StreamShare { id: 2 } + │ │ │ │ └─StreamFilter { predicate: (Not((t.a > 1:Int32)) OR (t.a > 1:Int32)) AND (((Not(IsNull(t.t)) AND Not((t.a < 1:Int32))) OR IsNull(t.t)) OR (t.a < 1:Int32)) } + │ │ │ │ └─StreamTableScan { table: t, columns: [t.t, t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + │ │ │ └─StreamExchange { dist: Broadcast } + │ │ │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ │ │ └─StreamNow { output: [now] } + │ │ └─StreamExchange { dist: HashShard(t._row_id, 1:Int32) } + │ │ └─StreamProject { exprs: [t.t, t.a, t._row_id, 1:Int32] } + │ │ └─StreamFilter { predicate: (Not((t.a > 1:Int32)) OR (t.a > 1:Int32)) AND (IsNull(t.t) OR (t.a < 1:Int32)) } + │ │ └─StreamShare { id: 2 } + │ │ └─StreamFilter { predicate: (Not((t.a > 1:Int32)) OR (t.a > 1:Int32)) AND (((Not(IsNull(t.t)) AND Not((t.a < 1:Int32))) OR IsNull(t.t)) OR (t.a < 1:Int32)) } + │ │ └─StreamTableScan { table: t, columns: [t.t, t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t._row_id, $src, 1:Int32) } + └─StreamProject { exprs: [t.t, t.a, $src, t._row_id, 1:Int32] } + └─StreamFilter { predicate: (t.a > 1:Int32) } + └─StreamShare { id: 13 } + └─StreamUnion { all: true } + ├─StreamExchange { dist: HashShard(t._row_id, 0:Int32) } + │ └─StreamProject { exprs: [t.t, t.a, t._row_id, 0:Int32], output_watermarks: [t.t] } + │ └─StreamDynamicFilter { predicate: (t.t > $expr1), output_watermarks: [t.t], output: [t.t, t.a, t._row_id], cleaned_by_watermark: true } + │ ├─StreamFilter { predicate: (Not((t.a > 1:Int32)) OR (t.a > 1:Int32)) AND Not(IsNull(t.t)) AND Not((t.a < 1:Int32)) } + │ │ └─StreamShare { id: 2 } + │ │ └─StreamFilter { predicate: (Not((t.a > 1:Int32)) OR (t.a > 1:Int32)) AND (((Not(IsNull(t.t)) AND Not((t.a < 1:Int32))) OR IsNull(t.t)) OR (t.a < 1:Int32)) } + │ │ └─StreamTableScan { table: t, columns: [t.t, t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '01:00:00':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + │ └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(t._row_id, 1:Int32) } + └─StreamProject { exprs: [t.t, t.a, t._row_id, 1:Int32] } + └─StreamFilter { predicate: (Not((t.a > 1:Int32)) OR (t.a > 1:Int32)) AND (IsNull(t.t) OR (t.a < 1:Int32)) } + └─StreamShare { id: 2 } + └─StreamFilter { predicate: (Not((t.a > 1:Int32)) OR (t.a > 1:Int32)) AND (((Not(IsNull(t.t)) AND Not((t.a < 1:Int32))) OR IsNull(t.t)) OR (t.a < 1:Int32)) } + └─StreamTableScan { table: t, columns: [t.t, t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index b513d4904669c..db5dc8ceca7d2 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -240,7 +240,11 @@ static BUSHY_TREE_JOIN_ORDERING: LazyLock = LazyLock::new(|| static FILTER_WITH_NOW_TO_JOIN: LazyLock = LazyLock::new(|| { OptimizationStage::new( "Push down filter with now into a left semijoin", - vec![SplitNowOrRule::create(), FilterWithNowToJoinRule::create()], + vec![ + SplitNowAndRule::create(), + SplitNowOrRule::create(), + FilterWithNowToJoinRule::create(), + ], ApplyOrder::TopDown, ) }); diff --git a/src/frontend/src/optimizer/rule/mod.rs b/src/frontend/src/optimizer/rule/mod.rs index d59bde580b1d1..acde2f7b72eb6 100644 --- a/src/frontend/src/optimizer/rule/mod.rs +++ b/src/frontend/src/optimizer/rule/mod.rs @@ -90,6 +90,7 @@ pub use top_n_on_index_rule::*; mod stream; pub use stream::bushy_tree_join_ordering_rule::*; pub use stream::filter_with_now_to_join_rule::*; +pub use stream::split_now_and_rule::*; pub use stream::split_now_or_rule::*; pub use stream::stream_project_merge_rule::*; mod trivial_project_to_values_rule; @@ -190,6 +191,7 @@ macro_rules! for_all_rules { , { AggProjectMergeRule } , { UnionMergeRule } , { DagToTreeRule } + , { SplitNowAndRule } , { SplitNowOrRule } , { FilterWithNowToJoinRule } , { TopNOnIndexRule } diff --git a/src/frontend/src/optimizer/rule/stream/filter_with_now_to_join_rule.rs b/src/frontend/src/optimizer/rule/stream/filter_with_now_to_join_rule.rs index b5f1d46d51743..498696589c81b 100644 --- a/src/frontend/src/optimizer/rule/stream/filter_with_now_to_join_rule.rs +++ b/src/frontend/src/optimizer/rule/stream/filter_with_now_to_join_rule.rs @@ -13,7 +13,6 @@ // limitations under the License. use risingwave_common::types::DataType; -use risingwave_pb::expr::expr_node::Type; use risingwave_pb::plan_common::JoinType; use crate::expr::{ @@ -55,11 +54,6 @@ impl Rule for FilterWithNowToJoinRule { } }); - // We want to put `input_expr >/>= now_expr` before `input_expr u8 { - match cmp { - Type::GreaterThan | Type::GreaterThanOrEqual => 0, - Type::LessThan | Type::LessThanOrEqual => 1, - _ => 2, - } -} - struct NowAsInputRef { index: usize, } diff --git a/src/frontend/src/optimizer/rule/stream/mod.rs b/src/frontend/src/optimizer/rule/stream/mod.rs index 3b088d93d64ec..cc86298e766e8 100644 --- a/src/frontend/src/optimizer/rule/stream/mod.rs +++ b/src/frontend/src/optimizer/rule/stream/mod.rs @@ -14,5 +14,6 @@ pub(crate) mod bushy_tree_join_ordering_rule; pub(crate) mod filter_with_now_to_join_rule; +pub(crate) mod split_now_and_rule; pub(crate) mod split_now_or_rule; pub(crate) mod stream_project_merge_rule; diff --git a/src/frontend/src/optimizer/rule/stream/split_now_and_rule.rs b/src/frontend/src/optimizer/rule/stream/split_now_and_rule.rs new file mode 100644 index 0000000000000..f82e4a8fdd304 --- /dev/null +++ b/src/frontend/src/optimizer/rule/stream/split_now_and_rule.rs @@ -0,0 +1,84 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use crate::optimizer::plan_node::{LogicalFilter, PlanTreeNodeUnary}; +use crate::optimizer::rule::{BoxedRule, Rule}; +use crate::optimizer::PlanRef; +use crate::utils::Condition; + +/// Split `LogicalFilter` with many AND conjunctions with now into multiple `LogicalFilter`, prepared for `SplitNowOrRule` +/// +/// Before: +/// ```text +/// `LogicalFilter` +/// (now() or c11 or c12 ..) and (now() or c21 or c22 ...) and .. and other exprs +/// | +/// Input +/// ``` +/// +/// After: +/// ```text +/// `LogicalFilter`(now() or c11 or c12 ..) +/// | +/// `LogicalFilter`(now() or c21 or c22 ...) +/// | +/// ...... +/// | +/// `LogicalFilter` other exprs +/// | +/// Input +/// ``` +pub struct SplitNowAndRule {} +impl Rule for SplitNowAndRule { + fn apply(&self, plan: PlanRef) -> Option { + let filter: &LogicalFilter = plan.as_logical_filter()?; + let input = filter.input(); + if filter.predicate().conjunctions.len() == 1 { + return None; + } + + if filter + .predicate() + .conjunctions + .iter() + .all(|e| e.count_nows() == 0) + { + return None; + } + + let [with_now, others] = + filter + .predicate() + .clone() + .group_by::<_, 2>(|e| if e.count_nows() > 0 { 0 } else { 1 }); + + let mut plan = LogicalFilter::create(input, others); + for e in with_now { + plan = LogicalFilter::new( + plan, + Condition { + conjunctions: vec![e], + }, + ) + .into(); + } + Some(plan) + } +} + +impl SplitNowAndRule { + pub fn create() -> BoxedRule { + Box::new(SplitNowAndRule {}) + } +}