From 03870476d7481e7b198522f21faa9c9d0abcd1a0 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Wed, 15 Nov 2023 15:13:16 +0800 Subject: [PATCH 01/36] fix(object store): make lifecycle expiration rule more reasonable (#13436) --- src/object_store/src/object/s3.rs | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 899d1dc128ee2..3cca6cd2aef9c 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -723,14 +723,15 @@ impl S3ObjectStore { if let Ok(config) = &get_config_result { for rule in config.rules() { - // When both of the conditions are met, it is considered that there is a risk of data deletion. - // - // 1. expiration status rule is enabled - // 2. (a) prefix filter is not set - // or (b) prefix filter is set to the data directory of RisingWave. - // - // P.S. 1 && (2a || 2b) - is_expiration_configured |= rule.status == ExpirationStatus::Enabled // 1 + if rule.expiration().is_some() { + // When both of the conditions are met, it is considered that there is a risk of data deletion. + // + // 1. expiration status rule is enabled + // 2. (a) prefix filter is not set + // or (b) prefix filter is set to the data directory of RisingWave. + // + // P.S. 1 && (2a || 2b) + is_expiration_configured |= rule.status == ExpirationStatus::Enabled // 1 && match rule.filter().as_ref() { // 2a None => true, @@ -743,10 +744,11 @@ impl S3ObjectStore { _ => false, }; - if matches!(rule.status(), ExpirationStatus::Enabled) - && rule.abort_incomplete_multipart_upload().is_some() - { - configured_rules.push(rule); + if matches!(rule.status(), ExpirationStatus::Enabled) + && rule.abort_incomplete_multipart_upload().is_some() + { + configured_rules.push(rule); + } } } } From 1327bb237b079967da35603b684aae9b857a8629 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 15 Nov 2023 15:37:22 +0800 Subject: [PATCH 02/36] chore(main-cron): add test owners, notify on test failure (#13410) --- ci/scripts/notify.py | 109 +++++++++++++++++++++++++++++++++++++ ci/workflows/main-cron.yml | 26 +++++++++ 2 files changed, 135 insertions(+) create mode 100755 ci/scripts/notify.py diff --git a/ci/scripts/notify.py b/ci/scripts/notify.py new file mode 100755 index 0000000000000..5ca526115810c --- /dev/null +++ b/ci/scripts/notify.py @@ -0,0 +1,109 @@ +#!/usr/bin/env python3 + +import subprocess + +# Add new test keys here. +# Add their corresponding owners (by slack username) here. +# NOTE(kwannoel): we may have to migrate to use `slack_user_id`. +# I use `slack_username` since it is more readable, but not officially supported in the docs. +TEST_MAP = { + "test-notify": ["noelkwan", "noelkwan"], + "backfill-tests": ["noelkwan"], + "e2e-iceberg-sink-tests": ["renjie"], + "e2e-java-binding-tests": ["yiming"], + "e2e-clickhouse-sink-tests": ["bohan"], + "e2e-pulsar-sink-tests": ["renjie"], + "s3-source-test-for-opendal-fs-engine": ["congyi"], + "pulsar-source-tests": ["renjie"], + "connector-node-integration-test": ["siyuan"], +} + +def get_failed_tests(get_test_status, test_map): + failed_test_map = {} + for test in test_map.keys(): + test_status = get_test_status(test) + if test_status == "hard_failed" or test_status == "soft_failed": + failed_test_map[test] = test_map[test] + return failed_test_map + +def generate_test_status_message(failed_test_map): + messages = [] + for test, users in failed_test_map.items(): + users = " ".join(map(lambda user: f"<@{user}>", users)) + messages.append(f"Test {test} failed {users}") + message = "\\n".join(messages) + return message + +def get_buildkite_test_status(test): + result = subprocess.run(f"buildkite-agent step get \"outcome\" --step \"{test}\"", capture_output = True, text = True, shell=True) + outcome = result.stdout.strip() + return outcome + +def get_mock_test_status(test): + mock_test_map = { + "test-notify": "hard_failed", + "e2e-iceberg-sink-tests": "passed", + "e2e-java-binding-tests": "soft_failed", + "e2e-clickhouse-sink-tests": "", + "e2e-pulsar-sink-tests": "", + "s3-source-test-for-opendal-fs-engine": "", + "pulsar-source-tests": "", + "connector-node-integration-test": "" + } + return mock_test_map[test] + +def get_mock_test_status_all_pass(test): + mock_test_map = { + "test-notify": "hard_failed", + "e2e-iceberg-sink-tests": "passed", + "e2e-java-binding-tests": "soft_failed", + "e2e-clickhouse-sink-tests": "", + "e2e-pulsar-sink-tests": "", + "s3-source-test-for-opendal-fs-engine": "", + "pulsar-source-tests": "", + "connector-node-integration-test": "" + } + return mock_test_map[test] + +def format_cmd(messages): + cmd=f""" +cat <<- YAML | buildkite-agent pipeline upload +steps: + - label: "Test" + command: echo "running failed test notification" && exit 1 + notify: + - slack: + channels: + - "#notification-buildkite" + message: {messages} +YAML + """ + return cmd + +def run_test_1(): + failed_test_map = get_failed_tests(get_mock_test_status, TEST_MAP) + message = generate_test_status_message(failed_test_map) + if message == "": + print("All tests passed, no need to notify") + return + else: + print("Some tests failed, notify users") + print(message) + cmd = format_cmd(message) + print(cmd) + +def main(): + failed_test_map = get_failed_tests(get_buildkite_test_status, TEST_MAP) + message = generate_test_status_message(failed_test_map) + if message == "": + print("All tests passed, no need to notify") + return + else: + print("Some tests failed, notify users") + print(message) + cmd = format_cmd(message) + print(cmd) + subprocess.run(cmd, shell=True) + print("notification sent") + +main() \ No newline at end of file diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 6c7ad6f8e042a..42fccac95af20 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -268,11 +268,13 @@ steps: retry: *auto-retry - label: "end-to-end iceberg sink test (release)" + key: "e2e-iceberg-sink-tests" command: "ci/scripts/e2e-iceberg-sink-test.sh -p ci-release" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests") depends_on: - "build" - "build-other" + plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -298,6 +300,7 @@ steps: retry: *auto-retry - label: "e2e java-binding test (release)" + key: "e2e-java-binding-tests" command: "ci/scripts/java-binding-test.sh -p ci-release" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-java-binding-tests") depends_on: @@ -405,6 +408,7 @@ steps: 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" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") depends_on: build @@ -423,6 +427,7 @@ steps: retry: *auto-retry - label: "pulsar source check" + key: "pulsar-source-tests" command: "ci/scripts/pulsar-source-test.sh -p ci-release" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-pulsar-source-tests") depends_on: @@ -504,6 +509,7 @@ steps: soft_fail: true - label: "Backfill tests" + key: "backfill-tests" command: "ci/scripts/backfill-test.sh -p ci-release" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-backfill-tests") depends_on: @@ -549,6 +555,7 @@ steps: retry: *auto-retry - label: "end-to-end clickhouse sink test" + key: "e2e-clickhouse-sink-tests" command: "ci/scripts/e2e-clickhouse-sink-test.sh -p ci-release" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-clickhouse-sink-tests") depends_on: @@ -564,6 +571,7 @@ steps: retry: *auto-retry - label: "end-to-end pulsar sink test" + key: "e2e-pulsar-sink-tests" command: "ci/scripts/e2e-pulsar-sink-test.sh -p ci-release" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-pulsar-sink-tests") depends_on: @@ -579,6 +587,7 @@ steps: retry: *auto-retry - label: "connector node integration test Java {{matrix.java_version}}" + key: "connector-node-integration-test" command: "ci/scripts/connector-node-integration-test.sh -p ci-release -v {{matrix.java_version}}" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-connector-node-integration-tests") depends_on: @@ -660,3 +669,20 @@ steps: DOCKER_TOKEN: docker-token timeout_in_minutes: 10 retry: *auto-retry + + # Notification test. + - key: "test-notify" + if: build.pull_request.labels includes "ci/main-cron/test-notify" + command: | + bash -c 'echo test && exit -1' + + - wait: true + continue_on_failure: true + allow_dependency_failure: true + + # Notifies on test failure for certain tests. + # You may update `notify.py` to add tests and people to notify. + # This should be the LAST part of the main-cron file. + - label: "trigger failed test notification" + if: build.pull_request.labels includes "ci/main-cron/test-notify" || build.branch == "main" + command: "ci/scripts/notify.py" \ No newline at end of file From 36f0bd4f2831b90b1d4e1a98afab32d85d39ec32 Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 15 Nov 2023 16:00:15 +0800 Subject: [PATCH 03/36] feat(frontend): support kill command (#13434) --- src/common/src/error.rs | 2 ++ src/frontend/src/handler/kill_process.rs | 37 ++++++++++++++++++++++ src/frontend/src/handler/mod.rs | 3 ++ src/frontend/src/handler/show.rs | 3 +- src/frontend/src/session.rs | 40 +++++++++++++++++------- src/sqlparser/src/ast/mod.rs | 7 +++++ src/sqlparser/src/keywords.rs | 1 + src/sqlparser/src/parser.rs | 6 ++++ src/utils/pgwire/src/pg_response.rs | 1 + src/utils/pgwire/src/pg_server.rs | 4 ++- 10 files changed, 90 insertions(+), 14 deletions(-) create mode 100644 src/frontend/src/handler/kill_process.rs diff --git a/src/common/src/error.rs b/src/common/src/error.rs index 1c59473063b7d..0de3ded976388 100644 --- a/src/common/src/error.rs +++ b/src/common/src/error.rs @@ -156,6 +156,8 @@ pub enum ErrorCode { ), #[error("Task not found")] TaskNotFound, + #[error("Session not found")] + SessionNotFound, #[error("Item not found: {0}")] ItemNotFound(String), #[error("Invalid input syntax: {0}")] diff --git a/src/frontend/src/handler/kill_process.rs b/src/frontend/src/handler/kill_process.rs new file mode 100644 index 0000000000000..a1506bc80c3c9 --- /dev/null +++ b/src/frontend/src/handler/kill_process.rs @@ -0,0 +1,37 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::error::{ErrorCode, Result}; + +use crate::handler::{HandlerArgs, RwPgResponse}; + +pub(super) async fn handle_kill( + handler_args: HandlerArgs, + process_id: i32, +) -> Result { + // Process id and secret key in session id are the same in RisingWave. + let session_id = (process_id, process_id); + tracing::trace!("kill query in session: {:?}", session_id); + let session = handler_args.session; + // TODO: cancel queries with await. + let mut session_exists = session.env().cancel_queries_in_session(session_id); + session_exists |= session.env().cancel_creating_jobs_in_session(session_id); + + if session_exists { + Ok(PgResponse::empty_result(StatementType::KILL)) + } else { + Err(ErrorCode::SessionNotFound.into()) + } +} diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index de8f048660f4a..cc7b242e07056 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -29,6 +29,7 @@ use self::util::DataChunkToRowSetAdapter; use self::variable::handle_set_time_zone; use crate::catalog::table_catalog::TableType; use crate::handler::cancel_job::handle_cancel; +use crate::handler::kill_process::handle_kill; use crate::scheduler::{DistributedQueryStream, LocalQueryStream}; use crate::session::SessionImpl; use crate::utils::WithOptions; @@ -69,6 +70,7 @@ pub mod explain; pub mod extended_handle; mod flush; pub mod handle_privilege; +mod kill_process; pub mod privilege; pub mod query; mod show; @@ -611,6 +613,7 @@ pub async fn handle( session, } => transaction::handle_set(handler_args, modes, snapshot, session).await, Statement::CancelJobs(jobs) => handle_cancel(handler_args, jobs).await, + Statement::Kill(process_id) => handle_kill(handler_args, process_id).await, Statement::Comment { object_type, object_name, diff --git a/src/frontend/src/handler/show.rs b/src/frontend/src/handler/show.rs index 977d2cb08a8bd..555162d42ed64 100644 --- a/src/frontend/src/handler/show.rs +++ b/src/frontend/src/handler/show.rs @@ -277,7 +277,8 @@ pub async fn handle_show_object( .values() .map(|s| { Row::new(vec![ - Some(format!("{}-{}", s.id().0, s.id().1).into()), + // Since process id and the secret id in the session id are the same in RisingWave, just display the process id. + Some(format!("{}", s.id().0).into()), Some(s.user_name().to_owned().into()), Some(format!("{}", s.peer_addr()).into()), Some(s.database().to_owned().into()), diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index db3c5a49ba357..69af0cab87e20 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -440,6 +440,32 @@ impl FrontendEnv { .unwrap(), )) } + + /// Cancel queries (i.e. batch queries) in session. + /// If the session exists return true, otherwise, return false. + pub fn cancel_queries_in_session(&self, session_id: SessionId) -> bool { + let guard = self.sessions_map.read(); + if let Some(session) = guard.get(&session_id) { + session.cancel_current_query(); + true + } else { + info!("Current session finished, ignoring cancel query request"); + false + } + } + + /// Cancel creating jobs (i.e. streaming queries) in session. + /// If the session exists return true, otherwise, return false. + pub fn cancel_creating_jobs_in_session(&self, session_id: SessionId) -> bool { + let guard = self.sessions_map.read(); + if let Some(session) = guard.get(&session_id) { + session.cancel_current_creating_job(); + true + } else { + info!("Current session finished, ignoring cancel creating request"); + false + } + } } pub struct AuthContext { @@ -920,21 +946,11 @@ impl SessionManager for SessionManagerImpl { /// Used when cancel request happened. fn cancel_queries_in_session(&self, session_id: SessionId) { - let guard = self.env.sessions_map.read(); - if let Some(session) = guard.get(&session_id) { - session.cancel_current_query() - } else { - info!("Current session finished, ignoring cancel query request") - } + self.env.cancel_queries_in_session(session_id); } fn cancel_creating_jobs_in_session(&self, session_id: SessionId) { - let guard = self.env.sessions_map.read(); - if let Some(session) = guard.get(&session_id) { - session.cancel_current_creating_job() - } else { - info!("Current session finished, ignoring cancel creating request") - } + self.env.cancel_creating_jobs_in_session(session_id); } fn end_session(&self, session: &Self::Session) { diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 560403ac6e918..3f253343be274 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1197,6 +1197,9 @@ pub enum Statement { }, /// CANCEL JOBS COMMAND CancelJobs(JobIdents), + /// KILL COMMAND + /// Kill process in the show processlist. + Kill(i32), /// DROP Drop(DropStatement), /// DROP Function @@ -1836,6 +1839,10 @@ impl fmt::Display for Statement { write!(f, "CANCEL JOBS {}", display_comma_separated(&jobs.0))?; Ok(()) } + Statement::Kill(process_id) => { + write!(f, "KILL {}", process_id)?; + Ok(()) + } } } } diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index 4efdce19605cd..ec549320a3b62 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -282,6 +282,7 @@ define_keywords!( JSON, KEY, KEYS, + KILL, LANGUAGE, LARGE, LAST, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 85932fb65e5f3..c4c70c8a938ce 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -240,6 +240,7 @@ impl Parser { } } Keyword::CANCEL => Ok(self.parse_cancel_job()?), + Keyword::KILL => Ok(self.parse_kill_process()?), Keyword::DESCRIBE => Ok(Statement::Describe { name: self.parse_object_name()?, }), @@ -4117,6 +4118,11 @@ impl Parser { Ok(Statement::CancelJobs(JobIdents(job_ids))) } + pub fn parse_kill_process(&mut self) -> Result { + let process_id = self.parse_literal_uint()? as i32; + Ok(Statement::Kill(process_id)) + } + /// Parser `from schema` after `show tables` and `show materialized views`, if not conclude /// `from` then use default schema name. pub fn parse_from_and_identifier(&mut self) -> Result, ParserError> { diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 048919076db83..d6a35f35e5e1d 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -96,6 +96,7 @@ pub enum StatementType { SET_TRANSACTION, CANCEL_COMMAND, WAIT, + KILL, } impl std::fmt::Display for StatementType { diff --git a/src/utils/pgwire/src/pg_server.rs b/src/utils/pgwire/src/pg_server.rs index f561540797b1a..8743f387d6646 100644 --- a/src/utils/pgwire/src/pg_server.rs +++ b/src/utils/pgwire/src/pg_server.rs @@ -31,7 +31,9 @@ use crate::pg_response::{PgResponse, ValuesStream}; use crate::types::Format; pub type BoxedError = Box; -pub type SessionId = (i32, i32); +type ProcessId = i32; +type SecretKey = i32; +pub type SessionId = (ProcessId, SecretKey); /// The interface for a database system behind pgwire protocol. /// We can mock it for testing purpose. From b20059fec3013d39bc34c460a1b95611a61ca92c Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 15 Nov 2023 16:09:33 +0800 Subject: [PATCH 04/36] refactor(stream): use `ExecutorParams::info` in remaining executors (#13435) Signed-off-by: Richard Chien --- src/stream/src/executor/barrier_recv.rs | 26 ++++-- .../src/executor/dedup/append_only_dedup.rs | 32 ++++--- src/stream/src/executor/expand.rs | 46 +++++----- src/stream/src/executor/filter.rs | 20 +++-- src/stream/src/executor/hop_window.rs | 11 +-- src/stream/src/executor/integration_tests.rs | 2 +- src/stream/src/executor/lookup_union.rs | 14 ++-- src/stream/src/executor/merge.rs | 40 +++++---- src/stream/src/executor/mview/materialize.rs | 2 +- src/stream/src/executor/no_op.rs | 16 ++-- src/stream/src/executor/now.rs | 74 ++++++++--------- src/stream/src/executor/receiver.rs | 34 ++++---- src/stream/src/executor/sink.rs | 83 +++++++++++-------- src/stream/src/executor/sort.rs | 27 +++--- .../src/executor/stateless_simple_agg.rs | 10 +-- src/stream/src/executor/temporal_join.rs | 35 +++----- src/stream/src/executor/union.rs | 13 +-- src/stream/src/executor/utils.rs | 10 +-- src/stream/src/executor/values.rs | 41 ++++----- src/stream/src/executor/watermark_filter.rs | 29 +++---- .../src/from_proto/append_only_dedup.rs | 9 +- src/stream/src/from_proto/barrier_recv.rs | 5 +- src/stream/src/from_proto/batch_query.rs | 17 ++-- src/stream/src/from_proto/expand.rs | 3 +- src/stream/src/from_proto/filter.rs | 8 +- src/stream/src/from_proto/hash_agg.rs | 6 +- src/stream/src/from_proto/hop_window.rs | 28 +------ src/stream/src/from_proto/lookup_union.rs | 2 +- src/stream/src/from_proto/merge.rs | 17 ++-- src/stream/src/from_proto/mview.rs | 14 +--- src/stream/src/from_proto/no_op.rs | 2 +- src/stream/src/from_proto/now.rs | 2 +- src/stream/src/from_proto/simple_agg.rs | 6 +- src/stream/src/from_proto/sink.rs | 11 ++- src/stream/src/from_proto/sort.rs | 5 +- src/stream/src/from_proto/source/fs_fetch.rs | 2 +- .../src/from_proto/source/trad_source.rs | 2 +- .../src/from_proto/stateless_simple_agg.rs | 15 +--- src/stream/src/from_proto/temporal_join.rs | 11 +-- src/stream/src/from_proto/union.rs | 2 +- src/stream/src/from_proto/values.rs | 5 +- src/stream/src/from_proto/watermark_filter.rs | 4 +- src/stream/src/task/stream_manager.rs | 10 --- .../tests/integration_tests/hop_window.rs | 4 +- 44 files changed, 327 insertions(+), 428 deletions(-) diff --git a/src/stream/src/executor/barrier_recv.rs b/src/stream/src/executor/barrier_recv.rs index 5bf01c139d168..0fd009f555f42 100644 --- a/src/stream/src/executor/barrier_recv.rs +++ b/src/stream/src/executor/barrier_recv.rs @@ -18,15 +18,15 @@ use tokio::sync::mpsc::UnboundedReceiver; use tokio_stream::wrappers::UnboundedReceiverStream; use super::{ - ActorContext, ActorContextRef, Barrier, BoxedMessageStream, Executor, Message, PkIndicesRef, - StreamExecutorError, + ActorContext, ActorContextRef, Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, + PkIndices, PkIndicesRef, StreamExecutorError, }; /// The executor only for receiving barrier from the meta service. It always resides in the leaves /// of the streaming graph. pub struct BarrierRecvExecutor { _ctx: ActorContextRef, - identity: String, + info: ExecutorInfo, /// The barrier receiver registered in the local barrier manager. barrier_receiver: UnboundedReceiver, @@ -35,18 +35,26 @@ pub struct BarrierRecvExecutor { impl BarrierRecvExecutor { pub fn new( ctx: ActorContextRef, + info: ExecutorInfo, barrier_receiver: UnboundedReceiver, - executor_id: u64, ) -> Self { Self { _ctx: ctx, - identity: format!("BarrierRecvExecutor {:X}", executor_id), + info, barrier_receiver, } } pub fn for_test(barrier_receiver: UnboundedReceiver) -> Self { - Self::new(ActorContext::create(0), barrier_receiver, 0) + Self::new( + ActorContext::create(0), + ExecutorInfo { + schema: Schema::empty().clone(), + pk_indices: PkIndices::new(), + identity: "BarrierRecvExecutor".to_string(), + }, + barrier_receiver, + ) } } @@ -63,15 +71,15 @@ impl Executor for BarrierRecvExecutor { } fn schema(&self) -> &Schema { - Schema::empty() + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &[] + &self.info.pk_indices } fn identity(&self) -> &str { - &self.identity + &self.info.identity } } diff --git a/src/stream/src/executor/dedup/append_only_dedup.rs b/src/stream/src/executor/dedup/append_only_dedup.rs index 898f78290bbb0..3a486c02b12d6 100644 --- a/src/stream/src/executor/dedup/append_only_dedup.rs +++ b/src/stream/src/executor/dedup/append_only_dedup.rs @@ -29,8 +29,8 @@ use crate::common::table::state_table::StateTable; use crate::executor::error::StreamExecutorError; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, Message, - PkIndices, PkIndicesRef, StreamExecutorResult, + expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, + ExecutorInfo, Message, PkIndicesRef, StreamExecutorResult, }; use crate::task::AtomicU64Ref; @@ -41,9 +41,7 @@ pub struct AppendOnlyDedupExecutor { state_table: StateTable, cache: DedupCache, - pk_indices: PkIndices, - identity: String, - schema: Schema, + info: ExecutorInfo, ctx: ActorContextRef, } @@ -51,22 +49,18 @@ impl AppendOnlyDedupExecutor { pub fn new( input: BoxedExecutor, state_table: StateTable, - pk_indices: PkIndices, - executor_id: u64, + info: ExecutorInfo, ctx: ActorContextRef, watermark_epoch: AtomicU64Ref, metrics: Arc, ) -> Self { - let schema = input.schema().clone(); let metrics_info = MetricsInfo::new(metrics, state_table.table_id(), ctx.id, "AppendOnly Dedup"); Self { input: Some(input), state_table, cache: DedupCache::new(watermark_epoch, metrics_info), - pk_indices, - identity: format!("AppendOnlyDedupExecutor {:X}", executor_id), - schema, + info, ctx, } } @@ -206,15 +200,15 @@ impl Executor for AppendOnlyDedupExecutor { } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - &self.identity + &self.info.identity } } @@ -258,12 +252,16 @@ mod tests { ) .await; - let (mut tx, input) = MockSource::channel(schema, pk_indices.clone()); + let (mut tx, input) = MockSource::channel(schema.clone(), pk_indices.clone()); + let info = ExecutorInfo { + schema, + pk_indices, + identity: "AppendOnlyDedupExecutor".to_string(), + }; let mut dedup_executor = Box::new(AppendOnlyDedupExecutor::new( Box::new(input), state_table, - pk_indices, - 1, + info, ActorContext::create(123), Arc::new(AtomicU64::new(0)), Arc::new(StreamingMetrics::unused()), diff --git a/src/stream/src/executor/expand.rs b/src/stream/src/executor/expand.rs index 1aed4e15c0463..0ebae0a4de830 100644 --- a/src/stream/src/executor/expand.rs +++ b/src/stream/src/executor/expand.rs @@ -17,35 +17,26 @@ use std::fmt::Debug; use futures::StreamExt; use futures_async_stream::try_stream; use risingwave_common::array::{Array, I64Array}; -use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::types::DataType; +use risingwave_common::catalog::Schema; use super::error::StreamExecutorError; use super::*; pub struct ExpandExecutor { + info: ExecutorInfo, input: BoxedExecutor, - schema: Schema, - pk_indices: PkIndices, column_subsets: Vec>, } impl ExpandExecutor { pub fn new( + info: ExecutorInfo, input: Box, - pk_indices: PkIndices, column_subsets: Vec>, ) -> Self { - let schema = { - let mut fields = input.schema().clone().into_fields(); - fields.extend(fields.clone()); - fields.push(Field::with_name(DataType::Int64, "flag")); - Schema::new(fields) - }; Self { + info, input, - schema, - pk_indices, column_subsets, } } @@ -84,15 +75,15 @@ impl Debug for ExpandExecutor { impl Executor for ExpandExecutor { fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - "ExpandExecutor" + &self.info.identity } fn execute(self: Box) -> BoxedMessageStream { @@ -109,7 +100,7 @@ mod tests { use super::ExpandExecutor; use crate::executor::test_utils::MockSource; - use crate::executor::{Executor, PkIndices}; + use crate::executor::{Executor, ExecutorInfo, PkIndices}; #[tokio::test] async fn test_expand() { @@ -120,21 +111,28 @@ mod tests { + 6 6 3 - 7 5 4", ); - let schema = Schema { + let input_schema = Schema { fields: vec![ Field::unnamed(DataType::Int64), Field::unnamed(DataType::Int64), Field::unnamed(DataType::Int64), ], }; - let source = MockSource::with_chunks(schema, PkIndices::new(), vec![chunk1]); + let source = MockSource::with_chunks(input_schema.clone(), PkIndices::new(), vec![chunk1]); + let schema = { + let mut fields = input_schema.into_fields(); + fields.extend(fields.clone()); + fields.push(Field::with_name(DataType::Int64, "flag")); + Schema::new(fields) + }; + let info = ExecutorInfo { + schema, + pk_indices: vec![], + identity: "ExpandExecutor".to_string(), + }; let column_subsets = vec![vec![0, 1], vec![1, 2]]; - let expand = Box::new(ExpandExecutor::new( - Box::new(source), - PkIndices::new(), - column_subsets, - )); + let expand = Box::new(ExpandExecutor::new(info, Box::new(source), column_subsets)); let mut expand = expand.execute(); let chunk = expand.next().await.unwrap().unwrap().into_chunk().unwrap(); diff --git a/src/stream/src/executor/filter.rs b/src/stream/src/executor/filter.rs index 1a1e645e44e6d..5f713f6981718 100644 --- a/src/stream/src/executor/filter.rs +++ b/src/stream/src/executor/filter.rs @@ -40,19 +40,14 @@ pub struct FilterExecutor { impl FilterExecutor { pub fn new( ctx: ActorContextRef, + info: ExecutorInfo, input: Box, expr: NonStrictExpression, - executor_id: u64, ) -> Self { - let input_info = input.info(); Self { _ctx: ctx, + info, input, - info: ExecutorInfo { - schema: input_info.schema, - pk_indices: input_info.pk_indices, - identity: format!("FilterExecutor {:X}", executor_id), - }, expr, } } @@ -222,15 +217,22 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let source = MockSource::with_chunks(schema, PkIndices::new(), vec![chunk1, chunk2]); + let pk_indices = PkIndices::new(); + let source = + MockSource::with_chunks(schema.clone(), pk_indices.clone(), vec![chunk1, chunk2]); + let info = ExecutorInfo { + schema, + pk_indices, + identity: "FilterExecutor".to_string(), + }; let test_expr = build_from_pretty("(greater_than:boolean $0:int8 $1:int8)"); let filter = Box::new(FilterExecutor::new( ActorContext::create(123), + info, Box::new(source), test_expr, - 1, )); let mut filter = filter.execute(); diff --git a/src/stream/src/executor/hop_window.rs b/src/stream/src/executor/hop_window.rs index 42d13d790da88..f7c61fdbc315a 100644 --- a/src/stream/src/executor/hop_window.rs +++ b/src/stream/src/executor/hop_window.rs @@ -28,8 +28,8 @@ use crate::common::StreamChunkBuilder; pub struct HopWindowExecutor { _ctx: ActorContextRef, - pub input: BoxedExecutor, pub info: ExecutorInfo, + pub input: BoxedExecutor, pub time_col_idx: usize, pub window_slide: Interval, pub window_size: Interval, @@ -43,8 +43,8 @@ impl HopWindowExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - input: BoxedExecutor, info: ExecutorInfo, + input: BoxedExecutor, time_col_idx: usize, window_slide: Interval, window_size: Interval, @@ -55,8 +55,8 @@ impl HopWindowExecutor { ) -> Self { HopWindowExecutor { _ctx: ctx, - input, info, + input, time_col_idx, window_slide, window_size, @@ -293,13 +293,13 @@ mod tests { super::HopWindowExecutor::new( ActorContext::create(123), - input, ExecutorInfo { // TODO: the schema is incorrect, but it seems useless here. schema, pk_indices, - identity: "test".to_string(), + identity: "HopWindowExecutor".to_string(), }, + input, 2, window_slide, window_size, @@ -316,6 +316,7 @@ mod tests { ) .boxed() } + #[tokio::test] async fn test_execute() { let default_indices: Vec<_> = (0..5).collect(); diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index 675cd37957fa0..dd2407d2efece 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -57,12 +57,12 @@ async fn test_merger_sum_aggr() { // for the local aggregator, we need two states: row count and sum let aggregator = StatelessSimpleAggExecutor::new( actor_ctx.clone(), - input.boxed(), ExecutorInfo { schema, pk_indices: vec![], identity: format!("StatelessSimpleAggExecutor {:X}", 1), }, + input.boxed(), agg_calls, ) .unwrap(); diff --git a/src/stream/src/executor/lookup_union.rs b/src/stream/src/executor/lookup_union.rs index 422a4f2ba6072..f178c3d72d3cd 100644 --- a/src/stream/src/executor/lookup_union.rs +++ b/src/stream/src/executor/lookup_union.rs @@ -44,13 +44,9 @@ impl std::fmt::Debug for LookupUnionExecutor { } impl LookupUnionExecutor { - pub fn new(pk_indices: PkIndices, inputs: Vec, order: Vec) -> Self { + pub fn new(info: ExecutorInfo, inputs: Vec, order: Vec) -> Self { Self { - info: ExecutorInfo { - schema: inputs[0].schema().clone(), - pk_indices, - identity: "LookupUnionExecutor".to_string(), - }, + info, inputs, order: order.iter().map(|x| *x as _).collect(), } @@ -195,7 +191,11 @@ mod tests { .stop_on_finish(false); let executor = Box::new(LookupUnionExecutor::new( - vec![0], + ExecutorInfo { + schema: source0.schema().clone(), + pk_indices: vec![0], + identity: "LookupUnionExecutor".to_string(), + }, vec![Box::new(source0), Box::new(source1), Box::new(source2)], vec![2, 1, 0], )) diff --git a/src/stream/src/executor/merge.rs b/src/stream/src/executor/merge.rs index f2f7d84ca2a3d..07d6d6857df93 100644 --- a/src/stream/src/executor/merge.rs +++ b/src/stream/src/executor/merge.rs @@ -34,21 +34,21 @@ use crate::task::{FragmentId, SharedContext}; /// `MergeExecutor` merges data from multiple channels. Dataflow from one channel /// will be stopped on barrier. pub struct MergeExecutor { - /// Upstream channels. - upstreams: Vec, - /// The context of the actor. actor_context: ActorContextRef, + /// Logical Operator Info + info: ExecutorInfo, + + /// Upstream channels. + upstreams: Vec, + /// Belonged fragment id. fragment_id: FragmentId, /// Upstream fragment id. upstream_fragment_id: FragmentId, - /// Logical Operator Info - info: ExecutorInfo, - /// Shared context of the stream manager. context: Arc, @@ -59,27 +59,21 @@ pub struct MergeExecutor { impl MergeExecutor { #[allow(clippy::too_many_arguments)] pub fn new( - schema: Schema, - pk_indices: PkIndices, ctx: ActorContextRef, + info: ExecutorInfo, fragment_id: FragmentId, upstream_fragment_id: FragmentId, - executor_id: u64, inputs: Vec, context: Arc, _receiver_id: u64, metrics: Arc, ) -> Self { Self { - upstreams: inputs, actor_context: ctx, + info, + upstreams: inputs, fragment_id, upstream_fragment_id, - info: ExecutorInfo { - schema, - pk_indices, - identity: format!("MergeExecutor {:X}", executor_id), - }, context, metrics, } @@ -91,12 +85,14 @@ impl MergeExecutor { use crate::executor::exchange::input::Input; Self::new( - schema, - vec![], ActorContext::create(114), + ExecutorInfo { + schema, + pk_indices: vec![], + identity: "MergeExecutor".to_string(), + }, 514, 1919, - 1024, inputs .into_iter() .enumerate() @@ -590,12 +586,14 @@ mod tests { .unwrap(); let merge = MergeExecutor::new( - schema, - vec![], ActorContext::create(actor_id), + ExecutorInfo { + schema, + pk_indices: vec![], + identity: "MergeExecutor".to_string(), + }, fragment_id, upstream_fragment_id, - 1024, inputs, ctx.clone(), 233, diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index 59bef6ae7df0a..a3ec5c36a5eb1 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -404,7 +404,7 @@ impl Executor for MaterializeExecutor { } fn identity(&self) -> &str { - self.info.identity.as_str() + &self.info.identity } fn info(&self) -> ExecutorInfo { diff --git a/src/stream/src/executor/no_op.rs b/src/stream/src/executor/no_op.rs index f323f95f5adc5..02f6c5544bdd8 100644 --- a/src/stream/src/executor/no_op.rs +++ b/src/stream/src/executor/no_op.rs @@ -14,21 +14,23 @@ use risingwave_common::catalog::Schema; -use super::{ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, PkIndicesRef}; +use super::{ + ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, PkIndicesRef, +}; /// No-op executor directly forwards the input stream. Currently used to break the multiple edges in /// the fragment graph. pub struct NoOpExecutor { _ctx: ActorContextRef, - identity: String, + info: ExecutorInfo, input: BoxedExecutor, } impl NoOpExecutor { - pub fn new(ctx: ActorContextRef, input: BoxedExecutor, executor_id: u64) -> Self { + pub fn new(ctx: ActorContextRef, info: ExecutorInfo, input: BoxedExecutor) -> Self { Self { _ctx: ctx, - identity: format!("NoOpExecutor {:X}", executor_id), + info, input, } } @@ -40,14 +42,14 @@ impl Executor for NoOpExecutor { } fn schema(&self) -> &Schema { - self.input.schema() + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - self.input.pk_indices() + &self.info.pk_indices } fn identity(&self) -> &str { - &self.identity + &self.info.identity } } diff --git a/src/stream/src/executor/now.rs b/src/stream/src/executor/now.rs index 2ee5468ff5ad9..8acf4806d0f6a 100644 --- a/src/stream/src/executor/now.rs +++ b/src/stream/src/executor/now.rs @@ -18,46 +18,36 @@ use std::ops::Bound::Unbounded; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::catalog::Schema; use risingwave_common::row::{self, OwnedRow}; use risingwave_common::types::{DataType, Datum}; use risingwave_storage::StateStore; use tokio::sync::mpsc::UnboundedReceiver; use super::{ - Barrier, BoxedMessageStream, Executor, Message, Mutation, PkIndices, PkIndicesRef, + Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, Mutation, PkIndicesRef, StreamExecutorError, Watermark, }; use crate::common::table::state_table::StateTable; pub struct NowExecutor { + info: ExecutorInfo, + /// Receiver of barrier channel. barrier_receiver: UnboundedReceiver, - pk_indices: PkIndices, - identity: String, - schema: Schema, state_table: StateTable, } impl NowExecutor { pub fn new( + info: ExecutorInfo, barrier_receiver: UnboundedReceiver, - executor_id: u64, state_table: StateTable, ) -> Self { - let schema = Schema::new(vec![Field { - data_type: DataType::Timestamptz, - name: String::from("now"), - sub_fields: vec![], - type_name: String::default(), - }]); - Self { + info, barrier_receiver, - pk_indices: vec![], - identity: format!("NowExecutor {:X}", executor_id), - schema, state_table, } } @@ -67,7 +57,7 @@ impl NowExecutor { let Self { mut barrier_receiver, mut state_table, - schema, + info, .. } = self; @@ -130,13 +120,13 @@ impl NowExecutor { StreamChunk::from_rows( &[(Op::Delete, last_row), (Op::Insert, row)], - &schema.data_types(), + &info.schema.data_types(), ) } else { let row = row::once(×tamp); state_table.insert(row); - StreamChunk::from_rows(&[(Op::Insert, row)], &schema.data_types()) + StreamChunk::from_rows(&[(Op::Insert, row)], &info.schema.data_types()) }; yield Message::Chunk(stream_chunk); @@ -158,25 +148,24 @@ impl Executor for NowExecutor { } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - self.identity.as_str() + &self.info.identity } } #[cfg(test)] mod tests { use risingwave_common::array::StreamChunk; - use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId}; + use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema, TableId}; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::{DataType, ScalarImpl}; - use risingwave_common::util::sort_util::OrderType; use risingwave_storage::memory::MemoryStateStore; use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; @@ -184,7 +173,8 @@ mod tests { use crate::common::table::state_table::StateTable; use crate::executor::test_utils::StreamExecutorTestExt; use crate::executor::{ - Barrier, BoxedMessageStream, Executor, Mutation, PkIndices, StreamExecutorResult, Watermark, + Barrier, BoxedMessageStream, Executor, ExecutorInfo, Mutation, StreamExecutorResult, + Watermark, }; #[tokio::test] @@ -387,16 +377,6 @@ mod tests { Ok(()) } - #[inline] - fn create_pk_indices() -> PkIndices { - vec![] - } - - #[inline] - fn create_order_types() -> Vec { - vec![] - } - fn create_state_store() -> MemoryStateStore { MemoryStateStore::new() } @@ -406,19 +386,33 @@ mod tests { ) -> (UnboundedSender, BoxedMessageStream) { let table_id = TableId::new(1); let column_descs = vec![ColumnDesc::unnamed(ColumnId::new(0), DataType::Timestamptz)]; - let order_types = create_order_types(); - let pk_indices = create_pk_indices(); let state_table = StateTable::new_without_distribution( state_store.clone(), table_id, column_descs, - order_types, - pk_indices, + vec![], + vec![], ) .await; let (sender, barrier_receiver) = unbounded_channel(); - let now_executor = NowExecutor::new(barrier_receiver, 1, state_table); + + let schema = Schema::new(vec![Field { + data_type: DataType::Timestamptz, + name: String::from("now"), + sub_fields: vec![], + type_name: String::default(), + }]); + + let now_executor = NowExecutor::new( + ExecutorInfo { + schema, + pk_indices: vec![], + identity: "NowExecutor".to_string(), + }, + barrier_receiver, + state_table, + ); (sender, Box::new(now_executor).execute()) } } diff --git a/src/stream/src/executor/receiver.rs b/src/stream/src/executor/receiver.rs index 5b96cf6f9f8d8..5920cc57d2ae3 100644 --- a/src/stream/src/executor/receiver.rs +++ b/src/stream/src/executor/receiver.rs @@ -25,20 +25,19 @@ use super::ActorContextRef; use crate::executor::exchange::input::new_input; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, PkIndices, - PkIndicesRef, + expect_first_barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, PkIndicesRef, }; use crate::task::{FragmentId, SharedContext}; /// `ReceiverExecutor` is used along with a channel. After creating a mpsc channel, /// there should be a `ReceiverExecutor` running in the background, so as to push /// messages down to the executors. pub struct ReceiverExecutor { - /// Input from upstream. - input: BoxedInput, - /// Logical Operator Info info: ExecutorInfo, + /// Input from upstream. + input: BoxedInput, + /// The context of the actor. actor_context: ActorContextRef, @@ -67,9 +66,8 @@ impl std::fmt::Debug for ReceiverExecutor { impl ReceiverExecutor { #[allow(clippy::too_many_arguments)] pub fn new( - schema: Schema, - pk_indices: PkIndices, ctx: ActorContextRef, + info: ExecutorInfo, fragment_id: FragmentId, upstream_fragment_id: FragmentId, input: BoxedInput, @@ -79,11 +77,7 @@ impl ReceiverExecutor { ) -> Self { Self { input, - info: ExecutorInfo { - schema, - pk_indices, - identity: "ReceiverExecutor".to_string(), - }, + info, actor_context: ctx, upstream_fragment_id, metrics, @@ -99,9 +93,12 @@ impl ReceiverExecutor { use crate::executor::ActorContext; Self::new( - Schema::default(), - vec![], ActorContext::create(114), + ExecutorInfo { + schema: Schema::default(), + pk_indices: vec![], + identity: "ReceiverExecutor".to_string(), + }, 514, 1919, LocalInput::new(input, 0).boxed_input(), @@ -272,10 +269,15 @@ mod tests { ) .unwrap(); - let receiver = ReceiverExecutor::new( + let info = ExecutorInfo { schema, - vec![], + pk_indices: vec![], + identity: "ReceiverExecutor".to_string(), + }; + + let receiver = ReceiverExecutor::new( ActorContext::create(actor_id), + info, fragment_id, upstream_fragment_id, input, diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index 29e4647c90c41..f68da97fb4ed3 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -33,18 +33,16 @@ use risingwave_connector::sink::{ }; use super::error::{StreamExecutorError, StreamExecutorResult}; -use super::{BoxedExecutor, Executor, Message, PkIndices}; +use super::{BoxedExecutor, Executor, ExecutorInfo, Message, PkIndices}; use crate::executor::{expect_first_barrier, ActorContextRef, BoxedMessageStream}; pub struct SinkExecutor { + actor_context: ActorContextRef, + info: ExecutorInfo, input: BoxedExecutor, sink: SinkImpl, - identity: String, - pk_indices: PkIndices, input_columns: Vec, - input_schema: Schema, sink_param: SinkParam, - actor_context: ActorContextRef, log_reader: F::Reader, log_writer: F::Writer, sink_writer_param: SinkWriterParam, @@ -79,30 +77,30 @@ fn force_delete_only(c: StreamChunk) -> StreamChunk { impl SinkExecutor { #[allow(clippy::too_many_arguments)] pub async fn new( + actor_context: ActorContextRef, + info: ExecutorInfo, input: BoxedExecutor, sink_writer_param: SinkWriterParam, sink_param: SinkParam, columns: Vec, - actor_context: ActorContextRef, log_store_factory: F, - pk_indices: PkIndices, ) -> StreamExecutorResult { let (log_reader, log_writer) = log_store_factory.build().await; let sink = build_sink(sink_param.clone())?; - let input_schema = columns + let input_schema: Schema = columns .iter() .map(|column| Field::from(&column.column_desc)) .collect(); + assert_eq!(input_schema.data_types(), info.schema.data_types()); + Ok(Self { + actor_context, + info, input, sink, - identity: format!("SinkExecutor {:X?}", sink_writer_param.executor_id), - pk_indices, input_columns: columns, - input_schema, sink_param, - actor_context, log_reader, log_writer, sink_writer_param, @@ -110,7 +108,7 @@ impl SinkExecutor { } fn execute_inner(self) -> BoxedMessageStream { - let stream_key = self.pk_indices; + let stream_key = self.info.pk_indices.clone(); let stream_key_sink_pk_mismatch = { stream_key @@ -136,6 +134,7 @@ impl SinkExecutor { self.input_columns, self.sink_writer_param, self.actor_context, + self.info, ); select(consume_log_stream.into_stream(), write_log_stream).boxed() }) @@ -293,9 +292,9 @@ impl SinkExecutor { columns: Vec, sink_writer_param: SinkWriterParam, actor_context: ActorContextRef, + info: ExecutorInfo, ) -> StreamExecutorResult { let metrics = sink_writer_param.sink_metrics.clone(); - let identity = format!("SinkExecutor {:X?}", sink_writer_param.executor_id); let log_sinker = sink.new_log_sinker(sink_writer_param).await?; let visible_columns = columns @@ -330,7 +329,7 @@ impl SinkExecutor { } GLOBAL_ERROR_METRICS.user_sink_error.report([ S::SINK_NAME.to_owned(), - identity, + info.identity, err_str, ]); return Err(e.into()); @@ -345,15 +344,15 @@ impl Executor for SinkExecutor { } fn schema(&self) -> &Schema { - &self.input_schema + &self.info.schema } fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - &self.identity + &self.info.identity } } @@ -400,11 +399,11 @@ mod test { .iter() .map(|column| Field::from(column.column_desc.clone())) .collect(); - let pk = vec![0]; + let pk_indices = vec![0]; let mock = MockSource::with_messages( - schema, - pk.clone(), + schema.clone(), + pk_indices.clone(), vec![ Message::Barrier(Barrier::new_test_barrier(1)), Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( @@ -433,21 +432,27 @@ mod test { .filter(|col| !col.is_hidden) .map(|col| col.column_desc.clone()) .collect(), - downstream_pk: pk.clone(), + downstream_pk: pk_indices.clone(), sink_type: SinkType::ForceAppendOnly, format_desc: None, db_name: "test".into(), sink_from_name: "test".into(), }; + let info = ExecutorInfo { + schema, + pk_indices, + identity: "SinkExecutor".to_string(), + }; + let sink_executor = SinkExecutor::new( + ActorContext::create(0), + info, Box::new(mock), SinkWriterParam::for_test(), sink_param, columns.clone(), - ActorContext::create(0), BoundedInMemLogStoreFactory::new(1), - pk, ) .await .unwrap(); @@ -520,7 +525,7 @@ mod test { .collect(); let mock = MockSource::with_messages( - schema, + schema.clone(), vec![0, 1], vec![ Message::Barrier(Barrier::new_test_barrier(1)), @@ -561,14 +566,20 @@ mod test { sink_from_name: "test".into(), }; + let info = ExecutorInfo { + schema, + pk_indices: vec![0, 1], + identity: "SinkExecutor".to_string(), + }; + let sink_executor = SinkExecutor::new( + ActorContext::create(0), + info, Box::new(mock), SinkWriterParam::for_test(), sink_param, columns.clone(), - ActorContext::create(0), BoundedInMemLogStoreFactory::new(1), - vec![0, 1], ) .await .unwrap(); @@ -652,11 +663,11 @@ mod test { .iter() .map(|column| Field::from(column.column_desc.clone())) .collect(); - let pk = vec![0]; + let pk_indices = vec![0]; let mock = MockSource::with_messages( - schema, - pk.clone(), + schema.clone(), + pk_indices.clone(), vec![ Message::Barrier(Barrier::new_test_barrier(1)), Message::Barrier(Barrier::new_test_barrier(2)), @@ -672,21 +683,27 @@ mod test { .filter(|col| !col.is_hidden) .map(|col| col.column_desc.clone()) .collect(), - downstream_pk: pk.clone(), + downstream_pk: pk_indices.clone(), sink_type: SinkType::ForceAppendOnly, format_desc: None, db_name: "test".into(), sink_from_name: "test".into(), }; + let info = ExecutorInfo { + schema, + pk_indices, + identity: "SinkExecutor".to_string(), + }; + let sink_executor = SinkExecutor::new( + ActorContext::create(0), + info, Box::new(mock), SinkWriterParam::for_test(), sink_param, columns, - ActorContext::create(0), BoundedInMemLogStoreFactory::new(1), - pk, ) .await .unwrap(); diff --git a/src/stream/src/executor/sort.rs b/src/stream/src/executor/sort.rs index 1d2a2003d4e0a..9960d3c83b0f8 100644 --- a/src/stream/src/executor/sort.rs +++ b/src/stream/src/executor/sort.rs @@ -21,7 +21,7 @@ use risingwave_storage::StateStore; use super::sort_buffer::SortBuffer; use super::{ expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndices, PkIndicesRef, StreamExecutorError, Watermark, + ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, Watermark, }; use crate::common::table::state_table::StateTable; use crate::common::StreamChunkBuilder; @@ -32,11 +32,10 @@ pub struct SortExecutor { } pub struct SortExecutorArgs { - pub input: BoxedExecutor, - pub actor_ctx: ActorContextRef, - pub pk_indices: PkIndices, - pub executor_id: u64, + pub info: ExecutorInfo, + + pub input: BoxedExecutor, pub buffer_table: StateTable, pub chunk_size: usize, @@ -77,16 +76,11 @@ impl Executor for SortExecutor { impl SortExecutor { pub fn new(args: SortExecutorArgs) -> Self { - let schema = args.input.schema().clone(); Self { input: args.input, inner: ExecutorInner { actor_ctx: args.actor_ctx, - info: ExecutorInfo { - identity: format!("SortExecutor {:X}", args.executor_id), - schema, - pk_indices: args.pk_indices, - }, + info: args.info, buffer_table: args.buffer_table, chunk_size: args.chunk_size, sort_column_index: args.sort_column_index, @@ -216,12 +210,15 @@ mod tests { ) .await; - let (tx, source) = MockSource::channel(input_schema, input_pk_indices.clone()); + let (tx, source) = MockSource::channel(input_schema, input_pk_indices); let sort_executor = SortExecutor::new(SortExecutorArgs { - input: source.boxed(), actor_ctx: ActorContext::create(123), - pk_indices: input_pk_indices, - executor_id: 1, + info: ExecutorInfo { + schema: source.schema().clone(), + pk_indices: source.pk_indices().to_vec(), + identity: "SortExecutor".to_string(), + }, + input: source.boxed(), buffer_table, chunk_size: 1024, sort_column_index, diff --git a/src/stream/src/executor/stateless_simple_agg.rs b/src/stream/src/executor/stateless_simple_agg.rs index d0d00a6961371..321d862343e01 100644 --- a/src/stream/src/executor/stateless_simple_agg.rs +++ b/src/stream/src/executor/stateless_simple_agg.rs @@ -29,8 +29,8 @@ use crate::error::StreamResult; pub struct StatelessSimpleAggExecutor { _ctx: ActorContextRef, - pub(super) input: Box, pub(super) info: ExecutorInfo, + pub(super) input: Box, pub(super) aggs: Vec, pub(super) agg_calls: Vec, } @@ -124,15 +124,15 @@ impl StatelessSimpleAggExecutor { impl StatelessSimpleAggExecutor { pub fn new( ctx: ActorContextRef, - input: Box, info: ExecutorInfo, + input: Box, agg_calls: Vec, ) -> StreamResult { let aggs = agg_calls.iter().map(build_retractable).try_collect()?; Ok(StatelessSimpleAggExecutor { _ctx: ctx, - input, info, + input, aggs, agg_calls, }) @@ -171,8 +171,8 @@ mod tests { let simple_agg = Box::new( StatelessSimpleAggExecutor::new( ActorContext::create(123), - Box::new(source), info, + Box::new(source), agg_calls, ) .unwrap(), @@ -229,8 +229,8 @@ mod tests { let simple_agg = Box::new( StatelessSimpleAggExecutor::new( ActorContext::create(123), - Box::new(source), info, + Box::new(source), agg_calls, ) .unwrap(), diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs index 6cb32e8cdadd3..7734b97ff3daf 100644 --- a/src/stream/src/executor/temporal_join.rs +++ b/src/stream/src/executor/temporal_join.rs @@ -39,18 +39,20 @@ use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::TableIter; use risingwave_storage::StateStore; -use super::{Barrier, Executor, Message, MessageStream, StreamExecutorError, StreamExecutorResult}; +use super::{ + Barrier, Executor, ExecutorInfo, Message, MessageStream, StreamExecutorError, + StreamExecutorResult, +}; use crate::cache::{cache_may_stale, new_with_hasher_in, ManagedLruCache}; use crate::common::metrics::MetricsInfo; use crate::common::JoinStreamChunkBuilder; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ - ActorContextRef, BoxedExecutor, JoinType, JoinTypePrimitive, PkIndices, Watermark, -}; +use crate::executor::{ActorContextRef, BoxedExecutor, JoinType, JoinTypePrimitive, Watermark}; use crate::task::AtomicU64Ref; pub struct TemporalJoinExecutor { ctx: ActorContextRef, + info: ExecutorInfo, left: BoxedExecutor, right: BoxedExecutor, right_table: TemporalSide, @@ -59,10 +61,7 @@ pub struct TemporalJoinExecutor, condition: Option, output_indices: Vec, - pk_indices: PkIndices, - schema: Schema, chunk_size: usize, - identity: String, // TODO: update metrics #[allow(dead_code)] metrics: Arc, @@ -332,6 +331,7 @@ impl TemporalJoinExecutor #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, + info: ExecutorInfo, left: BoxedExecutor, right: BoxedExecutor, table: StorageTable, @@ -339,23 +339,14 @@ impl TemporalJoinExecutor right_join_keys: Vec, null_safe: Vec, condition: Option, - pk_indices: PkIndices, output_indices: Vec, table_output_indices: Vec, table_stream_key_indices: Vec, - executor_id: u64, watermark_epoch: AtomicU64Ref, metrics: Arc, chunk_size: usize, join_key_data_types: Vec, ) -> Self { - let schema_fields = [left.schema().fields.clone(), right.schema().fields.clone()].concat(); - - let schema: Schema = output_indices - .iter() - .map(|&idx| schema_fields[idx].clone()) - .collect(); - let alloc = StatsAlloc::new(Global).shared(); let metrics_info = MetricsInfo::new( @@ -374,6 +365,7 @@ impl TemporalJoinExecutor Self { ctx: ctx.clone(), + info, left, right, right_table: TemporalSide { @@ -389,10 +381,7 @@ impl TemporalJoinExecutor null_safe, condition, output_indices, - schema, chunk_size, - pk_indices, - identity: format!("TemporalJoinExecutor {:X}", executor_id), metrics, } } @@ -432,7 +421,7 @@ impl TemporalJoinExecutor InternalMessage::Chunk(chunk) => { let mut builder = JoinStreamChunkBuilder::new( self.chunk_size, - self.schema.data_types(), + self.info.schema.data_types(), left_map.clone(), right_map.clone(), ); @@ -507,14 +496,14 @@ impl Executor } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - self.identity.as_str() + &self.info.identity } } diff --git a/src/stream/src/executor/union.rs b/src/stream/src/executor/union.rs index f0a6d9230a10b..ec1e6dbc9edb6 100644 --- a/src/stream/src/executor/union.rs +++ b/src/stream/src/executor/union.rs @@ -28,8 +28,8 @@ use crate::executor::{BoxedMessageStream, ExecutorInfo}; /// `UnionExecutor` merges data from multiple inputs. pub struct UnionExecutor { - inputs: Vec, info: ExecutorInfo, + inputs: Vec, } impl std::fmt::Debug for UnionExecutor { @@ -42,15 +42,8 @@ impl std::fmt::Debug for UnionExecutor { } impl UnionExecutor { - pub fn new(pk_indices: PkIndices, inputs: Vec) -> Self { - Self { - info: ExecutorInfo { - schema: inputs[0].schema().clone(), - pk_indices, - identity: "UnionExecutor".to_string(), - }, - inputs, - } + pub fn new(info: ExecutorInfo, inputs: Vec) -> Self { + Self { info, inputs } } } diff --git a/src/stream/src/executor/utils.rs b/src/stream/src/executor/utils.rs index fe5f6b044a115..03a815d292c13 100644 --- a/src/stream/src/executor/utils.rs +++ b/src/stream/src/executor/utils.rs @@ -23,14 +23,8 @@ pub struct DummyExecutor { } impl DummyExecutor { - pub fn new() -> Self { - Self { - info: ExecutorInfo { - schema: Schema::empty().clone(), - pk_indices: vec![], - identity: "DummyExecutor".to_string(), - }, - } + pub fn new(info: ExecutorInfo) -> Self { + Self { info } } } diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 8c09b56aa3551..868e5ea2bb7c7 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -25,7 +25,7 @@ use risingwave_expr::expr::NonStrictExpression; use tokio::sync::mpsc::UnboundedReceiver; use super::{ - ActorContextRef, Barrier, BoxedMessageStream, Executor, Message, PkIndices, PkIndicesRef, + ActorContextRef, Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, }; use crate::task::CreateMviewProgress; @@ -36,43 +36,39 @@ const DEFAULT_CHUNK_SIZE: usize = 1024; /// May refractor with `BarrierRecvExecutor` in the near future. pub struct ValuesExecutor { ctx: ActorContextRef, + info: ExecutorInfo, + // Receiver of barrier channel. barrier_receiver: UnboundedReceiver, progress: CreateMviewProgress, rows: vec::IntoIter>, - pk_indices: PkIndices, - identity: String, - schema: Schema, } impl ValuesExecutor { /// Currently hard-code the `pk_indices` as the last column. pub fn new( ctx: ActorContextRef, + info: ExecutorInfo, progress: CreateMviewProgress, rows: Vec>, - schema: Schema, barrier_receiver: UnboundedReceiver, - executor_id: u64, ) -> Self { Self { ctx, + info, progress, barrier_receiver, rows: rows.into_iter(), - pk_indices: vec![schema.len() - 1], // the last one column is pk - identity: format!("ValuesExecutor {:X}", executor_id), - schema, } } #[try_stream(ok = Message, error = StreamExecutorError)] async fn into_stream(self) { let Self { + info, mut progress, mut barrier_receiver, - schema, mut rows, .. } = self; @@ -100,7 +96,7 @@ impl ValuesExecutor { } } - let cardinality = schema.len(); + let cardinality = info.schema.len(); ensure!(cardinality > 0); while !rows.is_empty() { // We need a one row chunk rather than an empty chunk because constant @@ -109,7 +105,7 @@ impl ValuesExecutor { let one_row_chunk = DataChunk::new_dummy(1); let chunk_size = DEFAULT_CHUNK_SIZE.min(rows.len()); - let mut array_builders = schema.create_array_builders(chunk_size); + let mut array_builders = info.schema.create_array_builders(chunk_size); for row in rows.by_ref().take(chunk_size) { for (expr, builder) in row.into_iter().zip_eq_fast(&mut array_builders) { let out = expr.eval_infallible(&one_row_chunk).await; @@ -145,15 +141,15 @@ impl Executor for ValuesExecutor { } fn schema(&self) -> &Schema { - &self.schema + &self.info.schema } fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices + &self.info.pk_indices } fn identity(&self) -> &str { - self.identity.as_str() + &self.info.identity } } @@ -172,7 +168,7 @@ mod tests { use super::ValuesExecutor; use crate::executor::test_utils::StreamExecutorTestExt; - use crate::executor::{ActorContext, Barrier, Executor, Mutation}; + use crate::executor::{ActorContext, Barrier, Executor, ExecutorInfo, Mutation}; use crate::task::{CreateMviewProgress, LocalBarrierManager}; #[tokio::test] @@ -208,20 +204,25 @@ mod tests { Some(ScalarImpl::Int64(0)), )), ]; - let fields = exprs + let schema = exprs .iter() // for each column .map(|col| Field::unnamed(col.return_type())) - .collect::>(); + .collect::(); + let pk_indices = vec![schema.len() - 1]; + let info = ExecutorInfo { + schema, + pk_indices, + identity: "ValuesExecutor".to_string(), + }; let values_executor_struct = ValuesExecutor::new( ActorContext::create(actor_id), + info, progress, vec![exprs .into_iter() .map(NonStrictExpression::for_test) .collect()], - Schema { fields }, barrier_receiver, - 10005, ); let mut values_executor = Box::new(values_executor_struct).execute(); diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index c8899553ac46a..a66a78df61aca 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -43,37 +43,32 @@ use crate::task::ActorEvalErrorReport; /// This will also guarantee all later rows with event time **less than** the watermark will be /// filtered. pub struct WatermarkFilterExecutor { + ctx: ActorContextRef, + info: ExecutorInfo, + input: BoxedExecutor, /// The expression used to calculate the watermark value. watermark_expr: NonStrictExpression, /// The column we should generate watermark and filter on. event_time_col_idx: usize, - ctx: ActorContextRef, - info: ExecutorInfo, table: StateTable, } impl WatermarkFilterExecutor { pub fn new( + ctx: ActorContextRef, + info: ExecutorInfo, input: BoxedExecutor, watermark_expr: NonStrictExpression, event_time_col_idx: usize, - ctx: ActorContextRef, table: StateTable, - executor_id: u64, ) -> Self { - let info = ExecutorInfo { - schema: input.info().schema, - pk_indices: input.info().pk_indices, - identity: format!("WatermarkFilterExecutor {:X}", executor_id), - }; - Self { + ctx, + info, input, watermark_expr, event_time_col_idx, - ctx, - info, table, } } @@ -417,14 +412,20 @@ mod tests { let (tx, source) = MockSource::channel(schema, vec![0]); + let info = ExecutorInfo { + schema: source.schema().clone(), + pk_indices: source.pk_indices().to_vec(), + identity: "WatermarkFilterExecutor".to_string(), + }; + ( WatermarkFilterExecutor::new( + ActorContext::create(123), + info, source.boxed(), watermark_expr, 1, - ActorContext::create(123), table, - 0, ) .boxed(), tx, diff --git a/src/stream/src/from_proto/append_only_dedup.rs b/src/stream/src/from_proto/append_only_dedup.rs index ac44fcb87ed9e..c42618dfe8390 100644 --- a/src/stream/src/from_proto/append_only_dedup.rs +++ b/src/stream/src/from_proto/append_only_dedup.rs @@ -14,7 +14,6 @@ use std::sync::Arc; -use itertools::Itertools; use risingwave_pb::stream_plan::DedupNode; use risingwave_storage::StateStore; @@ -39,16 +38,10 @@ impl ExecutorBuilder for AppendOnlyDedupExecutorBuilder { let table = node.get_state_table()?; let vnodes = params.vnode_bitmap.map(Arc::new); let state_table = StateTable::from_table_catalog(table, store, vnodes).await; - let pk_indices = node - .dedup_column_indices - .iter() - .map(|idx| *idx as _) - .collect_vec(); Ok(Box::new(AppendOnlyDedupExecutor::new( input, state_table, - pk_indices, - params.executor_id, + params.info, params.actor_context, stream.get_watermark_epoch(), stream.streaming_metrics.clone(), diff --git a/src/stream/src/from_proto/barrier_recv.rs b/src/stream/src/from_proto/barrier_recv.rs index 42efc234ebb8c..90133802393d5 100644 --- a/src/stream/src/from_proto/barrier_recv.rs +++ b/src/stream/src/from_proto/barrier_recv.rs @@ -40,9 +40,6 @@ impl ExecutorBuilder for BarrierRecvExecutorBuilder { .lock_barrier_manager() .register_sender(params.actor_context.id, sender); - Ok( - BarrierRecvExecutor::new(params.actor_context, barrier_receiver, params.executor_id) - .boxed(), - ) + Ok(BarrierRecvExecutor::new(params.actor_context, params.info, barrier_receiver).boxed()) } } diff --git a/src/stream/src/from_proto/batch_query.rs b/src/stream/src/from_proto/batch_query.rs index 2913ae3cd7262..e3616c6274e15 100644 --- a/src/stream/src/from_proto/batch_query.rs +++ b/src/stream/src/from_proto/batch_query.rs @@ -38,7 +38,9 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { ) -> StreamResult { if node.table_desc.is_none() { // used in sharing cdc source backfill as a dummy batch plan node - return Ok(Box::new(DummyExecutor::new())); + let mut info = params.info; + info.identity = "DummyBatchQueryExecutor".to_string(); + return Ok(Box::new(DummyExecutor::new(info))); } let table_desc: &StorageTableDesc = node @@ -113,17 +115,10 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { prefix_hint_len, versioned, ); + assert_eq!(table.schema().data_types(), params.info.schema.data_types()); - let schema = table.schema().clone(); - let executor = BatchQueryExecutor::new( - table, - stream.config.developer.chunk_size, - ExecutorInfo { - schema, - pk_indices: params.pk_indices, - identity: "BatchQuery".to_owned(), - }, - ); + let executor = + BatchQueryExecutor::new(table, stream.config.developer.chunk_size, params.info); Ok(executor.boxed()) } diff --git a/src/stream/src/from_proto/expand.rs b/src/stream/src/from_proto/expand.rs index e88e944396203..0760918f54217 100644 --- a/src/stream/src/from_proto/expand.rs +++ b/src/stream/src/from_proto/expand.rs @@ -29,7 +29,6 @@ impl ExecutorBuilder for ExpandExecutorBuilder { _stream: &mut LocalStreamManagerCore, ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); - let pk_indices = params.pk_indices; let column_subsets = node .column_subsets .iter() @@ -41,6 +40,6 @@ impl ExecutorBuilder for ExpandExecutorBuilder { .collect_vec() }) .collect_vec(); - Ok(ExpandExecutor::new(input, pk_indices, column_subsets).boxed()) + Ok(ExpandExecutor::new(params.info, input, column_subsets).boxed()) } } diff --git a/src/stream/src/from_proto/filter.rs b/src/stream/src/from_proto/filter.rs index f703153fa4edd..b8b34dde1eec7 100644 --- a/src/stream/src/from_proto/filter.rs +++ b/src/stream/src/from_proto/filter.rs @@ -33,12 +33,6 @@ impl ExecutorBuilder for FilterExecutorBuilder { let search_condition = build_non_strict_from_prost(node.get_search_condition()?, params.eval_error_report)?; - Ok(FilterExecutor::new( - params.actor_context, - input, - search_condition, - params.executor_id, - ) - .boxed()) + Ok(FilterExecutor::new(params.actor_context, params.info, input, search_condition).boxed()) } } diff --git a/src/stream/src/from_proto/hash_agg.rs b/src/stream/src/from_proto/hash_agg.rs index 153b6aea6a7d9..7b4c70a592417 100644 --- a/src/stream/src/from_proto/hash_agg.rs +++ b/src/stream/src/from_proto/hash_agg.rs @@ -100,11 +100,7 @@ impl ExecutorBuilder for HashAggExecutorBuilder { input, actor_ctx: params.actor_context, - info: ExecutorInfo { - schema: params.schema, - pk_indices: params.pk_indices, - identity: params.identity, - }, + info: params.info, extreme_cache_size: stream.config.developer.unsafe_extreme_cache_size, diff --git a/src/stream/src/from_proto/hop_window.rs b/src/stream/src/from_proto/hop_window.rs index 8cc61888c3880..d32acfad66840 100644 --- a/src/stream/src/from_proto/hop_window.rs +++ b/src/stream/src/from_proto/hop_window.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::{Field, Schema}; -use risingwave_common::types::DataType; use risingwave_expr::expr::build_non_strict_from_prost; use risingwave_pb::stream_plan::HopWindowNode; @@ -33,9 +31,8 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { ) -> StreamResult { let ExecutorParams { actor_context, + info, input, - pk_indices, - executor_id, env, .. } = params; @@ -60,27 +57,6 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { .try_collect()?; let time_col = node.get_time_col() as usize; - let time_col_data_type = input.schema().fields()[time_col].data_type(); - let output_type = DataType::window_of(&time_col_data_type).unwrap(); - let original_schema: Schema = input - .schema() - .clone() - .into_fields() - .into_iter() - .chain([ - Field::with_name(output_type.clone(), "window_start"), - Field::with_name(output_type, "window_end"), - ]) - .collect(); - let actual_schema: Schema = output_indices - .iter() - .map(|&idx| original_schema[idx].clone()) - .collect(); - let info = ExecutorInfo { - schema: actual_schema, - identity: format!("HopWindowExecutor {:X}", executor_id), - pk_indices, - }; let window_slide = node.get_window_slide()?.into(); let window_size = node.get_window_size()?.into(); @@ -88,8 +64,8 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { Ok(HopWindowExecutor::new( actor_context, - input, info, + input, time_col, window_slide, window_size, diff --git a/src/stream/src/from_proto/lookup_union.rs b/src/stream/src/from_proto/lookup_union.rs index 582b5fb745996..3c523f11a5c54 100644 --- a/src/stream/src/from_proto/lookup_union.rs +++ b/src/stream/src/from_proto/lookup_union.rs @@ -28,6 +28,6 @@ impl ExecutorBuilder for LookupUnionExecutorBuilder { _store: impl StateStore, _stream: &mut LocalStreamManagerCore, ) -> StreamResult { - Ok(LookupUnionExecutor::new(params.pk_indices, params.input, node.order.clone()).boxed()) + Ok(LookupUnionExecutor::new(params.info, params.input, node.order.clone()).boxed()) } } diff --git a/src/stream/src/from_proto/merge.rs b/src/stream/src/from_proto/merge.rs index bc963f8d2351f..175109bd09649 100644 --- a/src/stream/src/from_proto/merge.rs +++ b/src/stream/src/from_proto/merge.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::{Field, Schema}; use risingwave_pb::stream_plan::{DispatcherType, MergeNode}; use super::*; @@ -32,9 +31,6 @@ impl ExecutorBuilder for MergeExecutorBuilder { ) -> StreamResult { let upstreams = node.get_upstream_actor_id(); let upstream_fragment_id = node.get_upstream_fragment_id(); - let fields = node.fields.iter().map(Field::from).collect(); - let schema = Schema::new(fields); - let actor_context = params.actor_context; let inputs: Vec<_> = upstreams .iter() @@ -42,7 +38,7 @@ impl ExecutorBuilder for MergeExecutorBuilder { new_input( &stream.context, stream.streaming_metrics.clone(), - actor_context.id, + params.actor_context.id, params.fragment_id, upstream_actor_id, upstream_fragment_id, @@ -63,9 +59,8 @@ impl ExecutorBuilder for MergeExecutorBuilder { if always_single_input { Ok(ReceiverExecutor::new( - schema, - params.pk_indices, - actor_context, + params.actor_context, + params.info, params.fragment_id, upstream_fragment_id, inputs.into_iter().exactly_one().unwrap(), @@ -76,12 +71,10 @@ impl ExecutorBuilder for MergeExecutorBuilder { .boxed()) } else { Ok(MergeExecutor::new( - schema, - params.pk_indices, - actor_context, + params.actor_context, + params.info, params.fragment_id, upstream_fragment_id, - params.executor_id, inputs, stream.context.clone(), params.operator_id, diff --git a/src/stream/src/from_proto/mview.rs b/src/stream/src/from_proto/mview.rs index e960beedc3691..7d71ebd335423 100644 --- a/src/stream/src/from_proto/mview.rs +++ b/src/stream/src/from_proto/mview.rs @@ -47,17 +47,12 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { let conflict_behavior = ConflictBehavior::from_protobuf(&table.handle_pk_conflict_behavior()); - let info = ExecutorInfo { - schema: params.schema, - pk_indices: params.pk_indices, - identity: params.identity, - }; macro_rules! new_executor { ($SD:ident) => { MaterializeExecutor::<_, $SD>::new( input, - info, + params.info, store, order_key, params.actor_context, @@ -109,14 +104,9 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { let vnodes = params.vnode_bitmap.map(Arc::new); let conflict_behavior = ConflictBehavior::from_protobuf(&table.handle_pk_conflict_behavior()); - let info = ExecutorInfo { - schema: params.schema, - pk_indices: params.pk_indices, - identity: params.identity, - }; let executor = MaterializeExecutor::<_, BasicSerde>::new( input, - info, + params.info, store, keys, params.actor_context, diff --git a/src/stream/src/from_proto/no_op.rs b/src/stream/src/from_proto/no_op.rs index 07afc3fba9d6a..16c05b61047c2 100644 --- a/src/stream/src/from_proto/no_op.rs +++ b/src/stream/src/from_proto/no_op.rs @@ -32,6 +32,6 @@ impl ExecutorBuilder for NoOpExecutorBuilder { _stream: &mut LocalStreamManagerCore, ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); - Ok(NoOpExecutor::new(params.actor_context, input, params.executor_id).boxed()) + Ok(NoOpExecutor::new(params.actor_context, params.info, input).boxed()) } } diff --git a/src/stream/src/from_proto/now.rs b/src/stream/src/from_proto/now.rs index c763ff952f3d9..da6dc44f99158 100644 --- a/src/stream/src/from_proto/now.rs +++ b/src/stream/src/from_proto/now.rs @@ -43,8 +43,8 @@ impl ExecutorBuilder for NowExecutorBuilder { StateTable::from_table_catalog(node.get_state_table()?, store, None).await; Ok(Box::new(NowExecutor::new( + params.info, barrier_receiver, - params.executor_id, state_table, ))) } diff --git a/src/stream/src/from_proto/simple_agg.rs b/src/stream/src/from_proto/simple_agg.rs index c0c5da2609042..a61cf375ae50b 100644 --- a/src/stream/src/from_proto/simple_agg.rs +++ b/src/stream/src/from_proto/simple_agg.rs @@ -61,11 +61,7 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder { input, actor_ctx: params.actor_context, - info: ExecutorInfo { - schema: params.schema, - pk_indices: params.pk_indices, - identity: params.identity, - }, + info: params.info, extreme_cache_size: stream.config.developer.unsafe_extreme_cache_size, diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index 9a2ca590b1c5a..245e7028194dc 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -103,11 +103,10 @@ impl ExecutorBuilder for SinkExecutorBuilder { sink_from_name, }; - let identity = format!("SinkExecutor {:X?}", params.executor_id); let sink_id_str = format!("{}", sink_id.sink_id); let sink_metrics = stream.streaming_metrics.new_sink_metrics( - identity.as_str(), + ¶ms.info.identity, sink_id_str.as_str(), connector, ); @@ -127,13 +126,13 @@ impl ExecutorBuilder for SinkExecutorBuilder { let factory = BoundedInMemLogStoreFactory::new(1); Ok(Box::new( SinkExecutor::new( + params.actor_context, + params.info, input_executor, sink_write_param, sink_param, columns, - params.actor_context, factory, - params.pk_indices, ) .await?, )) @@ -157,13 +156,13 @@ impl ExecutorBuilder for SinkExecutorBuilder { Ok(Box::new( SinkExecutor::new( + params.actor_context, + params.info, input_executor, sink_write_param, sink_param, columns, - params.actor_context, factory, - params.pk_indices, ) .await?, )) diff --git a/src/stream/src/from_proto/sort.rs b/src/stream/src/from_proto/sort.rs index 2e15e041842cb..35fd633098017 100644 --- a/src/stream/src/from_proto/sort.rs +++ b/src/stream/src/from_proto/sort.rs @@ -36,10 +36,9 @@ impl ExecutorBuilder for SortExecutorBuilder { let state_table = StateTable::from_table_catalog(node.get_state_table()?, store, Some(vnodes)).await; Ok(Box::new(SortExecutor::new(SortExecutorArgs { - input, actor_ctx: params.actor_context, - pk_indices: params.pk_indices, - executor_id: params.executor_id, + info: params.info, + input, buffer_table: state_table, chunk_size: params.env.config().developer.chunk_size, sort_column_index: node.sort_column_index as _, diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index a380d5989eace..aad6914d4bcef 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -55,7 +55,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { source_info.clone(), params.env.connector_params(), params.env.config().developer.connector_message_buffer_size, - params.pk_indices.clone(), + params.info.pk_indices.clone(), ); let source_ctrl_opts = SourceCtrlOpts { diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index e78f3cd79c88e..3232c49d8cbb3 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -70,7 +70,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { // TODO: use the correct information to fill in pk_dicies. // We should consdier add back the "pk_column_ids" field removed by #8841 in // StreamSource - params.pk_indices.clone(), + params.info.pk_indices.clone(), ); let source_ctrl_opts = SourceCtrlOpts { diff --git a/src/stream/src/from_proto/stateless_simple_agg.rs b/src/stream/src/from_proto/stateless_simple_agg.rs index e33098490c39e..016372e3c23c8 100644 --- a/src/stream/src/from_proto/stateless_simple_agg.rs +++ b/src/stream/src/from_proto/stateless_simple_agg.rs @@ -36,16 +36,9 @@ impl ExecutorBuilder for StatelessSimpleAggExecutorBuilder { .map(AggCall::from_protobuf) .try_collect()?; - Ok(StatelessSimpleAggExecutor::new( - params.actor_context, - input, - ExecutorInfo { - schema: params.schema, - pk_indices: params.pk_indices, - identity: params.identity, - }, - agg_calls, - )? - .boxed()) + Ok( + StatelessSimpleAggExecutor::new(params.actor_context, params.info, input, agg_calls)? + .boxed(), + ) } } diff --git a/src/stream/src/from_proto/temporal_join.rs b/src/stream/src/from_proto/temporal_join.rs index 32d89af3926de..e64bcf8ab3972 100644 --- a/src/stream/src/from_proto/temporal_join.rs +++ b/src/stream/src/from_proto/temporal_join.rs @@ -25,7 +25,7 @@ use risingwave_storage::table::Distribution; use super::*; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ActorContextRef, JoinType, PkIndices, TemporalJoinExecutor}; +use crate::executor::{ActorContextRef, JoinType, TemporalJoinExecutor}; use crate::task::AtomicU64Ref; pub struct TemporalJoinExecutorBuilder; @@ -158,6 +158,7 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { let dispatcher_args = TemporalJoinExecutorDispatcherArgs { ctx: params.actor_context, + info: params.info, left: source_l, right: source_r, right_table: table, @@ -165,11 +166,9 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { right_join_keys, null_safe, condition, - pk_indices: params.pk_indices, output_indices, table_output_indices, table_stream_key_indices, - executor_id: params.executor_id, watermark_epoch: stream.get_watermark_epoch(), chunk_size: params.env.config().developer.chunk_size, metrics: params.executor_stats, @@ -183,6 +182,7 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { struct TemporalJoinExecutorDispatcherArgs { ctx: ActorContextRef, + info: ExecutorInfo, left: BoxedExecutor, right: BoxedExecutor, right_table: StorageTable, @@ -190,11 +190,9 @@ struct TemporalJoinExecutorDispatcherArgs { right_join_keys: Vec, null_safe: Vec, condition: Option, - pk_indices: PkIndices, output_indices: Vec, table_output_indices: Vec, table_stream_key_indices: Vec, - executor_id: u64, watermark_epoch: AtomicU64Ref, chunk_size: usize, metrics: Arc, @@ -215,6 +213,7 @@ impl HashKeyDispatcher for TemporalJoinExecutorDispatcherArgs { JoinType::$join_type }, >::new( self.ctx, + self.info, self.left, self.right, self.right_table, @@ -222,11 +221,9 @@ impl HashKeyDispatcher for TemporalJoinExecutorDispatcherArgs self.right_join_keys, self.null_safe, self.condition, - self.pk_indices, self.output_indices, self.table_output_indices, self.table_stream_key_indices, - self.executor_id, self.watermark_epoch, self.metrics, self.chunk_size, diff --git a/src/stream/src/from_proto/union.rs b/src/stream/src/from_proto/union.rs index 077ee5a0c41a0..884203e4ba967 100644 --- a/src/stream/src/from_proto/union.rs +++ b/src/stream/src/from_proto/union.rs @@ -28,6 +28,6 @@ impl ExecutorBuilder for UnionExecutorBuilder { _store: impl StateStore, _stream: &mut LocalStreamManagerCore, ) -> StreamResult { - Ok(UnionExecutor::new(params.pk_indices, params.input).boxed()) + Ok(UnionExecutor::new(params.info, params.input).boxed()) } } diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs index 75e49741ddc0c..13d29ea794252 100644 --- a/src/stream/src/from_proto/values.rs +++ b/src/stream/src/from_proto/values.rs @@ -13,7 +13,6 @@ // limitations under the License. use itertools::Itertools; -use risingwave_common::catalog::{Field, Schema}; use risingwave_expr::expr::build_non_strict_from_prost; use risingwave_pb::stream_plan::ValuesNode; use risingwave_storage::StateStore; @@ -58,14 +57,12 @@ impl ExecutorBuilder for ValuesExecutorBuilder { .collect_vec() }) .collect_vec(); - let schema = Schema::new(node.get_fields().iter().map(Field::from).collect_vec()); Ok(Box::new(ValuesExecutor::new( params.actor_context, + params.info, progress, rows, - schema, barrier_receiver, - params.executor_id, ))) } } diff --git a/src/stream/src/from_proto/watermark_filter.rs b/src/stream/src/from_proto/watermark_filter.rs index fa7856cb3c9a9..765b7892bea1a 100644 --- a/src/stream/src/from_proto/watermark_filter.rs +++ b/src/stream/src/from_proto/watermark_filter.rs @@ -50,12 +50,12 @@ impl ExecutorBuilder for WatermarkFilterBuilder { StateTable::from_table_catalog_inconsistent_op(&table, store, Some(vnodes)).await; Ok(WatermarkFilterExecutor::new( + params.actor_context, + params.info, input, watermark_expr, event_time_col_idx, - params.actor_context, table, - params.executor_id, ) .boxed()) } diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index a237e790588ed..22a99c2099b5a 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -126,11 +126,6 @@ pub struct ExecutorParams { /// Basic information about the executor. pub info: ExecutorInfo, - // TODO(rc): The following three fields will be removed. Executor builders should use `info` instead. - pub schema: Schema, - pub pk_indices: PkIndices, - pub identity: String, - /// Executor id, unique across all actors. pub executor_id: u64, @@ -570,11 +565,6 @@ impl LocalStreamManagerCore { identity: identity.clone(), }, - // TODO(rc): The following three fields will be removed. Executor builders should use `info` instead. - schema: schema.clone(), - pk_indices: pk_indices.clone(), - identity: identity.clone(), - executor_id, operator_id, op_info, diff --git a/src/stream/tests/integration_tests/hop_window.rs b/src/stream/tests/integration_tests/hop_window.rs index 9d6d879240fc0..b2978aa2aefc1 100644 --- a/src/stream/tests/integration_tests/hop_window.rs +++ b/src/stream/tests/integration_tests/hop_window.rs @@ -47,12 +47,12 @@ fn create_executor(output_indices: Vec) -> (MessageSender, BoxedMessageSt tx, HopWindowExecutor::new( ActorContext::create(123), - Box::new(source), ExecutorInfo { schema, pk_indices, - identity: "test".to_string(), + identity: "HopWindowExecutor".to_string(), }, + Box::new(source), TIME_COL_IDX, window_slide, window_size, From 48bf62e6b59056b1efc5aefa9717e6c18724f546 Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Wed, 15 Nov 2023 16:11:44 +0800 Subject: [PATCH 05/36] feat: new table stream graph (#12240) Signed-off-by: Shanicky Chen --- e2e_test/batch/aggregate/jsonb_agg.slt.part | 94 +++---- .../basic/table_with_default_columns.slt.part | 4 +- e2e_test/batch/basic/to_jsonb.slt.part | 32 +-- .../batch/functions/array_transform.slt.part | 2 +- e2e_test/ddl/table/generated_columns.slt.part | 6 +- e2e_test/streaming/distinct_on.slt | 4 +- src/common/src/hash/consistent_hash/vnode.rs | 16 +- .../tests/testdata/output/ch_benchmark.yaml | 258 +++++++++--------- .../tests/testdata/output/create_source.yaml | 31 ++- .../tests/testdata/output/explain.yaml | 20 +- .../testdata/output/generated_columns.yaml | 30 +- .../tests/testdata/output/union.yaml | 2 +- .../tests/testdata/output/watermark.yaml | 14 +- src/frontend/src/handler/create_table.rs | 6 +- src/frontend/src/optimizer/mod.rs | 198 +++++++++++--- .../src/optimizer/plan_node/logical_source.rs | 3 +- .../optimizer/plan_node/stream_row_id_gen.rs | 17 +- .../src/optimizer/plan_node/stream_union.rs | 11 +- src/meta/service/src/ddl_service.rs | 5 + .../integration_tests/scale/no_shuffle.rs | 7 +- 20 files changed, 473 insertions(+), 287 deletions(-) diff --git a/e2e_test/batch/aggregate/jsonb_agg.slt.part b/e2e_test/batch/aggregate/jsonb_agg.slt.part index fad23dda14c68..97eb4462b8414 100644 --- a/e2e_test/batch/aggregate/jsonb_agg.slt.part +++ b/e2e_test/batch/aggregate/jsonb_agg.slt.part @@ -12,79 +12,79 @@ insert into t values (true, 3, 'aaa', '{}', '2021-01-01 03:00:00', '3 days'); query T -select jsonb_agg(v1) from t; +select jsonb_agg(v1 order by v1) from t; ---- -[null, false, true, true] +[false, true, true, null] query T -select jsonb_agg(v2::int2) from t; +select jsonb_agg(v2::int2 order by v2) from t; ---- -[null, 1, 2, 3] +[1, 2, 3, null] query T -select jsonb_agg(v2::int4) from t; +select jsonb_agg(v2::int4 order by v2) from t; ---- -[null, 1, 2, 3] +[1, 2, 3, null] query T -select jsonb_agg(v2::int8) from t; +select jsonb_agg(v2::int8 order by v2) from t; ---- -[null, 1, 2, 3] +[1, 2, 3, null] query T -select jsonb_agg(v2::float4) from t; +select jsonb_agg(v2::float4 order by v2) from t; ---- -[null, 1.0, 2.0, 3.0] +[1.0, 2.0, 3.0, null] query T -select jsonb_agg(v2::float8) from t; +select jsonb_agg(v2::float8 order by v2) from t; ---- -[null, 1.0, 2.0, 3.0] +[1.0, 2.0, 3.0, null] query T -select jsonb_agg(v2::decimal) from t; +select jsonb_agg(v2::decimal order by v2) from t; ---- -[null, 1.0, 2.0, 3.0] +[1.0, 2.0, 3.0, null] query T -select jsonb_agg(v3) from t; +select jsonb_agg(v3 order by v3) from t; ---- -[null, "ccc", "bbb", "aaa"] +["aaa", "bbb", "ccc", null] query T -select jsonb_agg(v3::bytea) from t; +select jsonb_agg(v3::bytea order by v3) from t; ---- -[null, "\\x636363", "\\x626262", "\\x616161"] +["\\x616161", "\\x626262", "\\x636363", null] query T -select jsonb_agg(v4) from t; +select jsonb_agg(v4 order by v4) from t; ---- -[null, null, false, {}] +[false, null, {}, null] query T -select jsonb_agg(v5::date) from t; +select jsonb_agg(v5::date order by v5) from t; ---- -[null, "2019-01-01", "2020-01-01", "2021-01-01"] +["2019-01-01", "2020-01-01", "2021-01-01", null] query T -select jsonb_agg(v5::time) from t; +select jsonb_agg(v5::time order by v5) from t; ---- -[null, "01:00:00", "02:00:00", "03:00:00"] +["01:00:00", "02:00:00", "03:00:00", null] query T -select jsonb_agg(v5::timestamp) from t; +select jsonb_agg(v5::timestamp order by v5) from t; ---- -[null, "2019-01-01T01:00:00", "2020-01-01T02:00:00", "2021-01-01T03:00:00"] +["2019-01-01T01:00:00", "2020-01-01T02:00:00", "2021-01-01T03:00:00", null] query T -select jsonb_agg(v5::timestamptz) from t; +select jsonb_agg(v5::timestamptz order by v5) from t; ---- -[null, "2019-01-01T01:00:00+00:00", "2020-01-01T02:00:00+00:00", "2021-01-01T03:00:00+00:00"] +["2019-01-01T01:00:00+00:00", "2020-01-01T02:00:00+00:00", "2021-01-01T03:00:00+00:00", null] query T -select jsonb_agg(v6) from t; +select jsonb_agg(v6 order by v6) from t; ---- -[null, "1 day", "2 days", "3 days"] +["1 day", "2 days", "3 days", null] # query T # select jsonb_agg(distinct v1) from t; @@ -97,85 +97,85 @@ select jsonb_agg(v2 order by v3 desc) from t; [null, 1, 2, 3] query T -select jsonb_agg(v2) filter (where v3 >= 'bbb') from t; +select jsonb_agg(v2 order by v3) filter (where v3 >= 'bbb') from t; ---- -[1, 2] +[2, 1] statement error field name must not be null select jsonb_object_agg(v3, v1) from t; query T -select jsonb_object_agg(v3, v1) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v1 order by v3) filter (where v3 is not null) from t; ---- {"aaa": true, "bbb": true, "ccc": false} query T -select jsonb_object_agg(v3, v2::int2) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::int2 order by v3) filter (where v3 is not null) from t; ---- {"aaa": 3, "bbb": 2, "ccc": 1} query T -select jsonb_object_agg(v3, v2::int4) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::int4 order by v3) filter (where v3 is not null) from t; ---- {"aaa": 3, "bbb": 2, "ccc": 1} query T -select jsonb_object_agg(v3, v2::int8) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::int8 order by v3) filter (where v3 is not null) from t; ---- {"aaa": 3, "bbb": 2, "ccc": 1} query T -select jsonb_object_agg(v3, v2::float4) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::float4 order by v3) filter (where v3 is not null) from t; ---- {"aaa": 3.0, "bbb": 2.0, "ccc": 1.0} query T -select jsonb_object_agg(v3, v2::float8) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::float8 order by v3) filter (where v3 is not null) from t; ---- {"aaa": 3.0, "bbb": 2.0, "ccc": 1.0} query T -select jsonb_object_agg(v3, v2::decimal) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::decimal order by v3) filter (where v3 is not null) from t; ---- {"aaa": 3.0, "bbb": 2.0, "ccc": 1.0} query T -select jsonb_object_agg(v3, v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v3 order by v3) filter (where v3 is not null) from t; ---- {"aaa": "aaa", "bbb": "bbb", "ccc": "ccc"} query T -select jsonb_object_agg(v3, v3::bytea) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v3::bytea order by v3) filter (where v3 is not null) from t; ---- {"aaa": "\\x616161", "bbb": "\\x626262", "ccc": "\\x636363"} query T -select jsonb_object_agg(v3, v4) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v4 order by v3) filter (where v3 is not null) from t; ---- {"aaa": {}, "bbb": false, "ccc": null} query T -select jsonb_object_agg(v3, v5::date) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v5::date order by v3) filter (where v3 is not null) from t; ---- {"aaa": "2021-01-01", "bbb": "2020-01-01", "ccc": "2019-01-01"} query T -select jsonb_object_agg(v3, v5::time) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v5::time order by v3) filter (where v3 is not null) from t; ---- {"aaa": "03:00:00", "bbb": "02:00:00", "ccc": "01:00:00"} query T -select jsonb_object_agg(v3, v5::timestamp) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v5::timestamp order by v3) filter (where v3 is not null) from t; ---- {"aaa": "2021-01-01T03:00:00", "bbb": "2020-01-01T02:00:00", "ccc": "2019-01-01T01:00:00"} query T -select jsonb_object_agg(v3, v5::timestamptz) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v5::timestamptz order by v3) filter (where v3 is not null) from t; ---- {"aaa": "2021-01-01T03:00:00+00:00", "bbb": "2020-01-01T02:00:00+00:00", "ccc": "2019-01-01T01:00:00+00:00"} query T -select jsonb_object_agg(v3, v6) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v6 order by v3) filter (where v3 is not null) from t; ---- {"aaa": "3 days", "bbb": "2 days", "ccc": "1 day"} diff --git a/e2e_test/batch/basic/table_with_default_columns.slt.part b/e2e_test/batch/basic/table_with_default_columns.slt.part index da37cd7787de1..117850683f6b9 100644 --- a/e2e_test/batch/basic/table_with_default_columns.slt.part +++ b/e2e_test/batch/basic/table_with_default_columns.slt.part @@ -66,7 +66,7 @@ create table t2 (v1 int, v2 int default 1.5); statement ok insert into t2 values (1), (2); -query II +query II rowsort select * from t2; ---- 1 2 @@ -76,7 +76,7 @@ select * from t2; statement ok alter table t2 add column v3 timestamp with time zone default now(); -query IT +query IT rowsort select v1, v3 >= date '2021-01-01' as later_than_2021 from t2; ---- 1 t diff --git a/e2e_test/batch/basic/to_jsonb.slt.part b/e2e_test/batch/basic/to_jsonb.slt.part index 9cad572daf8d0..471a4de9f41ae 100644 --- a/e2e_test/batch/basic/to_jsonb.slt.part +++ b/e2e_test/batch/basic/to_jsonb.slt.part @@ -230,7 +230,7 @@ statement ok INSERT INTO structtype VALUES (row(6, 'v1')), (row(7, 'v2')); query T -SELECT to_jsonb(st) from structtype; +SELECT to_jsonb(st) from structtype order by (st).key; ---- {"key": 6, "val": "v1"} {"key": 7, "val": "v2"} @@ -252,37 +252,37 @@ INSERT INTO alltypes VALUES (false, (SMALLINT '972'), (INT '-235825836'), (BIGINT '842'), (REAL '27'), (FLOAT '675'), (0), DATE '2023-08-25', 'uwAFEeex9Y', TIME '22:12:37', TIMESTAMP '2023-09-01 22:11:40', '2023-09-01 22:12:37Z'::TIMESTAMPTZ, (INTERVAL '28'), CAST(NULL AS STRUCT), ARRAY[(INT '355')], ARRAY['xan6o2VHID', 'MTSy3lzImo', 'UZqnEMW60w'], BYTEA'\xAABBCCDDEEFF', '"value3"'::JSONB); query T -SELECT to_jsonb(row(c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18)) from alltypes; +SELECT to_jsonb(row(c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18)) from alltypes order by c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18; ---- -{"f1": true, "f10": "22:12:54", "f11": "2023-09-01T22:12:42", "f12": "2023-09-01T22:12:42+00:00", "f13": "-00:01:18", "f14": null, "f15": [354, 627], "f16": ["yRMgX7pFXW", "r7PAN6KB2b", "NQJbRQoVib"], "f17": "\\xdeadbeef", "f18": {"key1": "value1"}, "f2": 786, "f3": 1409922817, "f4": 925, "f5": 536.0, "f6": 782.0, "f7": 487.0, "f8": "2023-09-01", "f9": "IwfwuseZmg"} -{"f1": true, "f10": "22:12:25", "f11": "2023-09-01T21:12:54", "f12": "2023-09-01T21:12:54+00:00", "f13": "24:00:00", "f14": null, "f15": [0], "f16": ["3NE5ewEx4T"], "f17": "\\xdeadbeef", "f18": {"key2": "value2"}, "f2": 82, "f3": 216, "f4": 732, "f5": 337.0, "f6": 772.0, "f7": 378.0, "f8": "2023-09-01", "f9": "6nNf6LL2C1"} -{"f1": false, "f10": "22:11:17", "f11": "2023-09-01T22:11:50", "f12": "2023-09-01T21:12:54+00:00", "f13": "-00:01:31", "f14": null, "f15": [77, 718, 401, 874], "f16": ["k6N5rUX8p1", "sFRQ1u2ihF"], "f17": "\\x0123456789abcdef", "f18": null, "f2": 761, "f3": 966, "f4": 153, "f5": 1023789440.0, "f6": 752.0, "f7": 630.0, "f8": "2023-09-01", "f9": "ySrgeBXDuc"} {"f1": false, "f10": "22:11:54", "f11": "2023-09-01T22:12:54", "f12": "2023-09-01T21:12:54+00:00", "f13": "-00:01:24", "f14": null, "f15": [246, 120, 154], "f16": ["Rau1Iezv50", "uWtqX1jIP0", "N356wachUq", "tDVFlmtDNk"], "f17": "\\x00ff00", "f18": {"key3": ["value3", "value55"]}, "f2": 255, "f3": 1, "f4": 9223372036854775807, "f5": 0.0, "f6": 775.0, "f7": -2147483648.0, "f8": "2023-09-01", "f9": "2498VN2txc"} +{"f1": false, "f10": "22:11:17", "f11": "2023-09-01T22:11:50", "f12": "2023-09-01T21:12:54+00:00", "f13": "-00:01:31", "f14": null, "f15": [77, 718, 401, 874], "f16": ["k6N5rUX8p1", "sFRQ1u2ihF"], "f17": "\\x0123456789abcdef", "f18": null, "f2": 761, "f3": 966, "f4": 153, "f5": 1023789440.0, "f6": 752.0, "f7": 630.0, "f8": "2023-09-01", "f9": "ySrgeBXDuc"} +{"f1": false, "f10": "22:12:37", "f11": "2023-09-01T22:11:40", "f12": "2023-09-01T22:12:37+00:00", "f13": "00:00:28", "f14": null, "f15": [355], "f16": ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "f17": "\\xaabbccddeeff", "f18": "value3", "f2": 972, "f3": -235825836, "f4": 842, "f5": 27.0, "f6": 675.0, "f7": 0.0, "f8": "2023-08-25", "f9": "uwAFEeex9Y"} +{"f1": true, "f10": "22:12:25", "f11": "2023-09-01T21:12:54", "f12": "2023-09-01T21:12:54+00:00", "f13": "24:00:00", "f14": null, "f15": [0], "f16": ["3NE5ewEx4T"], "f17": "\\xdeadbeef", "f18": {"key2": "value2"}, "f2": 82, "f3": 216, "f4": 732, "f5": 337.0, "f6": 772.0, "f7": 378.0, "f8": "2023-09-01", "f9": "6nNf6LL2C1"} +{"f1": true, "f10": "22:12:54", "f11": "2023-09-01T22:12:42", "f12": "2023-09-01T22:12:42+00:00", "f13": "-00:01:18", "f14": null, "f15": [354, 627], "f16": ["yRMgX7pFXW", "r7PAN6KB2b", "NQJbRQoVib"], "f17": "\\xdeadbeef", "f18": {"key1": "value1"}, "f2": 786, "f3": 1409922817, "f4": 925, "f5": 536.0, "f6": 782.0, "f7": 487.0, "f8": "2023-09-01", "f9": "IwfwuseZmg"} {"f1": true, "f10": "22:12:54", "f11": "2023-09-01T22:12:02", "f12": "2023-09-01T22:12:02+00:00", "f13": "01:00:00", "f14": null, "f15": [329, 577, 255, 70], "f16": ["1HQloIk7oW", "ixxNgP8vaq", "9CSOsftyRA", "jiqocRdrUC"], "f17": "\\x00ff00", "f18": {"key4": {"inner_key": "value4"}}, "f2": 933, "f3": 915, "f4": 433, "f5": 734.0, "f6": 438.0, "f7": 998512901.0, "f8": "2023-09-01", "f9": "Qgfzps4qkX"} {"f1": true, "f10": "22:12:40", "f11": "2023-09-01T22:11:27", "f12": "2023-09-01T22:12:02+00:00", "f13": "-00:01:00", "f14": null, "f15": [252, 137, 110, 574], "f16": ["t5073iSwvs"], "f17": "\\x80", "f18": null, "f2": 7096, "f3": -1627323193, "f4": 191, "f5": 483.0, "f6": 85.0, "f7": -2147483648.0, "f8": "2023-09-01", "f9": "sLgs9Am1iP"} -{"f1": false, "f10": "22:12:37", "f11": "2023-09-01T22:11:40", "f12": "2023-09-01T22:12:37+00:00", "f13": "00:00:28", "f14": null, "f15": [355], "f16": ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "f17": "\\xaabbccddeeff", "f18": "value3", "f2": 972, "f3": -235825836, "f4": 842, "f5": 27.0, "f6": 675.0, "f7": 0.0, "f8": "2023-08-25", "f9": "uwAFEeex9Y"} query T -SELECT jsonb_build_array(c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18) from alltypes; +SELECT jsonb_build_array(c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18) from alltypes order by c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18; ---- -[true, 786, 1409922817, 925, 536.0, 782.0, 487.0, "2023-09-01", "IwfwuseZmg", "22:12:54", "2023-09-01T22:12:42", "2023-09-01T22:12:42+00:00", "-00:01:18", null, [354, 627], ["yRMgX7pFXW", "r7PAN6KB2b", "NQJbRQoVib"], "\\xdeadbeef", {"key1": "value1"}] -[true, 82, 216, 732, 337.0, 772.0, 378.0, "2023-09-01", "6nNf6LL2C1", "22:12:25", "2023-09-01T21:12:54", "2023-09-01T21:12:54+00:00", "24:00:00", null, [0], ["3NE5ewEx4T"], "\\xdeadbeef", {"key2": "value2"}] -[false, 761, 966, 153, 1023789440.0, 752.0, 630.0, "2023-09-01", "ySrgeBXDuc", "22:11:17", "2023-09-01T22:11:50", "2023-09-01T21:12:54+00:00", "-00:01:31", null, [77, 718, 401, 874], ["k6N5rUX8p1", "sFRQ1u2ihF"], "\\x0123456789abcdef", null] [false, 255, 1, 9223372036854775807, 0.0, 775.0, -2147483648.0, "2023-09-01", "2498VN2txc", "22:11:54", "2023-09-01T22:12:54", "2023-09-01T21:12:54+00:00", "-00:01:24", null, [246, 120, 154], ["Rau1Iezv50", "uWtqX1jIP0", "N356wachUq", "tDVFlmtDNk"], "\\x00ff00", {"key3": ["value3", "value55"]}] +[false, 761, 966, 153, 1023789440.0, 752.0, 630.0, "2023-09-01", "ySrgeBXDuc", "22:11:17", "2023-09-01T22:11:50", "2023-09-01T21:12:54+00:00", "-00:01:31", null, [77, 718, 401, 874], ["k6N5rUX8p1", "sFRQ1u2ihF"], "\\x0123456789abcdef", null] +[false, 972, -235825836, 842, 27.0, 675.0, 0.0, "2023-08-25", "uwAFEeex9Y", "22:12:37", "2023-09-01T22:11:40", "2023-09-01T22:12:37+00:00", "00:00:28", null, [355], ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "\\xaabbccddeeff", "value3"] +[true, 82, 216, 732, 337.0, 772.0, 378.0, "2023-09-01", "6nNf6LL2C1", "22:12:25", "2023-09-01T21:12:54", "2023-09-01T21:12:54+00:00", "24:00:00", null, [0], ["3NE5ewEx4T"], "\\xdeadbeef", {"key2": "value2"}] +[true, 786, 1409922817, 925, 536.0, 782.0, 487.0, "2023-09-01", "IwfwuseZmg", "22:12:54", "2023-09-01T22:12:42", "2023-09-01T22:12:42+00:00", "-00:01:18", null, [354, 627], ["yRMgX7pFXW", "r7PAN6KB2b", "NQJbRQoVib"], "\\xdeadbeef", {"key1": "value1"}] [true, 933, 915, 433, 734.0, 438.0, 998512901.0, "2023-09-01", "Qgfzps4qkX", "22:12:54", "2023-09-01T22:12:02", "2023-09-01T22:12:02+00:00", "01:00:00", null, [329, 577, 255, 70], ["1HQloIk7oW", "ixxNgP8vaq", "9CSOsftyRA", "jiqocRdrUC"], "\\x00ff00", {"key4": {"inner_key": "value4"}}] [true, 7096, -1627323193, 191, 483.0, 85.0, -2147483648.0, "2023-09-01", "sLgs9Am1iP", "22:12:40", "2023-09-01T22:11:27", "2023-09-01T22:12:02+00:00", "-00:01:00", null, [252, 137, 110, 574], ["t5073iSwvs"], "\\x80", null] -[false, 972, -235825836, 842, 27.0, 675.0, 0.0, "2023-08-25", "uwAFEeex9Y", "22:12:37", "2023-09-01T22:11:40", "2023-09-01T22:12:37+00:00", "00:00:28", null, [355], ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "\\xaabbccddeeff", "value3"] query T -SELECT jsonb_build_object(c1, c1, c2, c2, c3, c3, c4, c4, c5, c5, c6, c6, c7, c7, c8, c8, c9, c9, c10, c10, c11, c11, c12, c12, c13, c13, 'f14', c14, 'f15', c15, 'f16', c16, c17, c17, 'f18', c18) from alltypes; +SELECT jsonb_build_object(c1, c1, c2, c2, c3, c3, c4, c4, c5, c5, c6, c6, c7, c7, c8, c8, c9, c9, c10, c10, c11, c11, c12, c12, c13, c13, 'f14', c14, 'f15', c15, 'f16', c16, c17, c17, 'f18', c18) from alltypes order by c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18; ---- -{"-00:01:18": "-00:01:18", "1409922817": 1409922817, "2023-09-01": "2023-09-01", "2023-09-01 22:12:42": "2023-09-01T22:12:42", "2023-09-01 22:12:42+00:00": "2023-09-01T22:12:42+00:00", "22:12:54": "22:12:54", "487": 487.0, "536": 536.0, "782": 782.0, "786": 786, "925": 925, "IwfwuseZmg": "IwfwuseZmg", "\\xdeadbeef": "\\xdeadbeef", "f14": null, "f15": [354, 627], "f16": ["yRMgX7pFXW", "r7PAN6KB2b", "NQJbRQoVib"], "f18": {"key1": "value1"}, "true": true} -{"2023-09-01": "2023-09-01", "2023-09-01 21:12:54": "2023-09-01T21:12:54", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "216": 216, "22:12:25": "22:12:25", "24:00:00": "24:00:00", "337": 337.0, "378": 378.0, "6nNf6LL2C1": "6nNf6LL2C1", "732": 732, "772": 772.0, "82": 82, "\\xdeadbeef": "\\xdeadbeef", "f14": null, "f15": [0], "f16": ["3NE5ewEx4T"], "f18": {"key2": "value2"}, "true": true} -{"-00:01:31": "-00:01:31", "1023789440": 1023789440.0, "153": 153, "2023-09-01": "2023-09-01", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "2023-09-01 22:11:50": "2023-09-01T22:11:50", "22:11:17": "22:11:17", "630": 630.0, "752": 752.0, "761": 761, "966": 966, "\\x0123456789abcdef": "\\x0123456789abcdef", "f14": null, "f15": [77, 718, 401, 874], "f16": ["k6N5rUX8p1", "sFRQ1u2ihF"], "f18": null, "false": false, "ySrgeBXDuc": "ySrgeBXDuc"} {"-00:01:24": "-00:01:24", "-2147483648": -2147483648.0, "0": 0.0, "1": 1, "2023-09-01": "2023-09-01", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "2023-09-01 22:12:54": "2023-09-01T22:12:54", "22:11:54": "22:11:54", "2498VN2txc": "2498VN2txc", "255": 255, "775": 775.0, "9223372036854775807": 9223372036854775807, "\\x00ff00": "\\x00ff00", "f14": null, "f15": [246, 120, 154], "f16": ["Rau1Iezv50", "uWtqX1jIP0", "N356wachUq", "tDVFlmtDNk"], "f18": {"key3": ["value3", "value55"]}, "false": false} +{"-00:01:31": "-00:01:31", "1023789440": 1023789440.0, "153": 153, "2023-09-01": "2023-09-01", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "2023-09-01 22:11:50": "2023-09-01T22:11:50", "22:11:17": "22:11:17", "630": 630.0, "752": 752.0, "761": 761, "966": 966, "\\x0123456789abcdef": "\\x0123456789abcdef", "f14": null, "f15": [77, 718, 401, 874], "f16": ["k6N5rUX8p1", "sFRQ1u2ihF"], "f18": null, "false": false, "ySrgeBXDuc": "ySrgeBXDuc"} +{"-235825836": -235825836, "0": 0.0, "00:00:28": "00:00:28", "2023-08-25": "2023-08-25", "2023-09-01 22:11:40": "2023-09-01T22:11:40", "2023-09-01 22:12:37+00:00": "2023-09-01T22:12:37+00:00", "22:12:37": "22:12:37", "27": 27.0, "675": 675.0, "842": 842, "972": 972, "\\xaabbccddeeff": "\\xaabbccddeeff", "f14": null, "f15": [355], "f16": ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "f18": "value3", "false": false, "uwAFEeex9Y": "uwAFEeex9Y"} +{"2023-09-01": "2023-09-01", "2023-09-01 21:12:54": "2023-09-01T21:12:54", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "216": 216, "22:12:25": "22:12:25", "24:00:00": "24:00:00", "337": 337.0, "378": 378.0, "6nNf6LL2C1": "6nNf6LL2C1", "732": 732, "772": 772.0, "82": 82, "\\xdeadbeef": "\\xdeadbeef", "f14": null, "f15": [0], "f16": ["3NE5ewEx4T"], "f18": {"key2": "value2"}, "true": true} +{"-00:01:18": "-00:01:18", "1409922817": 1409922817, "2023-09-01": "2023-09-01", "2023-09-01 22:12:42": "2023-09-01T22:12:42", "2023-09-01 22:12:42+00:00": "2023-09-01T22:12:42+00:00", "22:12:54": "22:12:54", "487": 487.0, "536": 536.0, "782": 782.0, "786": 786, "925": 925, "IwfwuseZmg": "IwfwuseZmg", "\\xdeadbeef": "\\xdeadbeef", "f14": null, "f15": [354, 627], "f16": ["yRMgX7pFXW", "r7PAN6KB2b", "NQJbRQoVib"], "f18": {"key1": "value1"}, "true": true} {"01:00:00": "01:00:00", "2023-09-01": "2023-09-01", "2023-09-01 22:12:02": "2023-09-01T22:12:02", "2023-09-01 22:12:02+00:00": "2023-09-01T22:12:02+00:00", "22:12:54": "22:12:54", "433": 433, "438": 438.0, "734": 734.0, "915": 915, "933": 933, "998512901": 998512901.0, "Qgfzps4qkX": "Qgfzps4qkX", "\\x00ff00": "\\x00ff00", "f14": null, "f15": [329, 577, 255, 70], "f16": ["1HQloIk7oW", "ixxNgP8vaq", "9CSOsftyRA", "jiqocRdrUC"], "f18": {"key4": {"inner_key": "value4"}}, "true": true} {"-00:01:00": "-00:01:00", "-1627323193": -1627323193, "-2147483648": -2147483648.0, "191": 191, "2023-09-01": "2023-09-01", "2023-09-01 22:11:27": "2023-09-01T22:11:27", "2023-09-01 22:12:02+00:00": "2023-09-01T22:12:02+00:00", "22:12:40": "22:12:40", "483": 483.0, "7096": 7096, "85": 85.0, "\\x80": "\\x80", "f14": null, "f15": [252, 137, 110, 574], "f16": ["t5073iSwvs"], "f18": null, "sLgs9Am1iP": "sLgs9Am1iP", "true": true} -{"-235825836": -235825836, "0": 0.0, "00:00:28": "00:00:28", "2023-08-25": "2023-08-25", "2023-09-01 22:11:40": "2023-09-01T22:11:40", "2023-09-01 22:12:37+00:00": "2023-09-01T22:12:37+00:00", "22:12:37": "22:12:37", "27": 27.0, "675": 675.0, "842": 842, "972": 972, "\\xaabbccddeeff": "\\xaabbccddeeff", "f14": null, "f15": [355], "f16": ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "f18": "value3", "false": false, "uwAFEeex9Y": "uwAFEeex9Y"} query T SELECT jsonb_build_array(); diff --git a/e2e_test/batch/functions/array_transform.slt.part b/e2e_test/batch/functions/array_transform.slt.part index d07a7e5701429..231e27a5e0e32 100644 --- a/e2e_test/batch/functions/array_transform.slt.part +++ b/e2e_test/batch/functions/array_transform.slt.part @@ -43,7 +43,7 @@ insert into t values (4, '{1,2,3}'), (5, '{4,5,6,8}'); # this makes sure `x + 1` is not extracted as common sub-expression by accident. See #11766 query TT -select array_transform(arr, |x| x + 1), array_transform(arr, |x| x + 1 + 2) from t; +select array_transform(arr, |x| x + 1), array_transform(arr, |x| x + 1 + 2) from t order by v; ---- {2,3,4} {4,5,6} {5,6,7,9} {7,8,9,11} diff --git a/e2e_test/ddl/table/generated_columns.slt.part b/e2e_test/ddl/table/generated_columns.slt.part index 88294172ac206..2271522a47fd9 100644 --- a/e2e_test/ddl/table/generated_columns.slt.part +++ b/e2e_test/ddl/table/generated_columns.slt.part @@ -8,7 +8,7 @@ insert into t1 (v2) values (1), (2); statement ok flush; -query IIIR +query IIIR rowsort select * from t1; ---- 0 1 2 2.02 @@ -52,7 +52,7 @@ insert into t2 values (1), (2); statement ok flush; -query II +query II rowsort select * from t2; ---- 1 2 @@ -124,7 +124,7 @@ sleep 2s statement ok flush; -query TT +query TT rowsort select v, t >= date '2021-01-01' as later_than_2021 from mv; ---- 1 t diff --git a/e2e_test/streaming/distinct_on.slt b/e2e_test/streaming/distinct_on.slt index f00e43eef3ff6..3312ce71e97ac 100644 --- a/e2e_test/streaming/distinct_on.slt +++ b/e2e_test/streaming/distinct_on.slt @@ -5,7 +5,7 @@ statement ok create table t1 (k int, v int) append only; statement ok -create materialized view mv1 as select distinct on (k) k + v as sum from t1; +create materialized view mv1 as select distinct on (k) k + v as sum from t1 order by k, v; statement ok insert into t1 values (1,11), (2,22), (3,33), (1,111); @@ -21,7 +21,7 @@ statement ok create table t2 (k int, v int); statement ok -create materialized view mv2 as select distinct on (k) k + v as sum from t2; +create materialized view mv2 as select distinct on (k) k + v as sum from t2 order by k, v; statement ok insert into t2 values (1,11), (2,22), (3,33), (1,111); diff --git a/src/common/src/hash/consistent_hash/vnode.rs b/src/common/src/hash/consistent_hash/vnode.rs index 0ee3596c82d8e..9c8fe989aaf39 100644 --- a/src/common/src/hash/consistent_hash/vnode.rs +++ b/src/common/src/hash/consistent_hash/vnode.rs @@ -119,10 +119,18 @@ impl VirtualNode { if let Ok(idx) = keys.iter().exactly_one() && let ArrayImpl::Serial(serial_array) = &**data_chunk.column_at(*idx) { - return serial_array - .iter() - .map(|serial| extract_vnode_id_from_row_id(serial.unwrap().as_row_id())) - .collect(); + return serial_array.iter().enumerate().map(|(idx, serial)| { + if let Some(serial) = serial { + extract_vnode_id_from_row_id(serial.as_row_id()) + } else { + // NOTE: here it will hash the entire row when the `_row_id` is missing, + // which could result in rows from the same chunk being allocated to different chunks. + // This process doesn’t guarantee the order of rows, producing indeterminate results in some cases, + // such as when `distinct on` is used without an `order by`. + let (row, _) = data_chunk.row_at(idx); + row.hash(Crc32FastBuilder).into() + } + } ).collect(); } data_chunk diff --git a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml index 97e64b0867b8c..026f3de33eebc 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -31,7 +31,7 @@ └─StreamExchange { dist: HashShard(order_line.ol_number) } └─StreamProject { exprs: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id] } └─StreamFilter { predicate: (order_line.ol_delivery_d > '2007-01-02 00:00:00':Timestamp) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [ol_number, sum_qty, sum_amount, avg_qty, avg_amount, count_order], stream_key: [ol_number], pk_columns: [ol_number], pk_conflict: NoCheck } { materialized table: 4294967294 } @@ -42,7 +42,7 @@ Fragment 1 StreamProject { exprs: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id] } └── StreamFilter { predicate: (order_line.ol_delivery_d > '2007-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -119,7 +119,7 @@ │ │ └─BatchFilter { predicate: Like(item.i_data, '%b':Varchar) } │ │ └─BatchScan { table: item, columns: [item.i_id, item.i_name, item.i_data], distribution: UpstreamHashShard(item.i_id) } │ └─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } - │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } └─BatchHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity)] } └─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } └─BatchHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity] } @@ -131,7 +131,7 @@ │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } └─BatchExchange { order: [], dist: HashShard($expr1) } └─BatchProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } - └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey) } @@ -158,14 +158,14 @@ │ │ └─StreamExchange { dist: HashShard($expr1) } │ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } │ │ └─StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } │ └─StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } │ ├─StreamExchange { dist: HashShard(item.i_id) } │ │ └─StreamProject { exprs: [item.i_id, item.i_name] } │ │ └─StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } │ │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } └─StreamExchange { dist: HashShard(supplier.s_suppkey) } └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } ├─StreamExchange { dist: HashShard(region.r_regionkey) } @@ -234,7 +234,7 @@ Fragment 9 StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } └── StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } - └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 25 } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 25 } ├── Upstream └── BatchPlanNode @@ -246,7 +246,7 @@ └── BatchPlanNode Fragment 11 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 31 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 31 } ├── Upstream └── BatchPlanNode @@ -400,15 +400,15 @@ └─BatchHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, orders.o_entry_d, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = new_order.no_w_id AND orders.o_d_id = new_order.no_d_id AND orders.o_id = new_order.no_o_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], lookup table: new_order } - │ └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } │ └─BatchHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } │ ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } │ │ └─BatchProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } │ │ └─BatchFilter { predicate: (customer.c_state >= 'a':Varchar) AND (customer.c_state < 'b':Varchar) } - │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ └─BatchExchange { order: [], dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ └─BatchFilter { predicate: (orders.o_entry_d > '2007-01-02 00:00:00':Timestamp) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount], distribution: SomeShard } stream_plan: |- @@ -422,17 +422,17 @@ │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id) } │ │ └─StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } │ │ └─StreamFilter { predicate: (customer.c_w_id = customer.c_w_id) AND (customer.c_d_id = customer.c_d_id) AND Like(customer.c_state, 'a%':Varchar) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ └─StreamExchange { dist: HashShard(new_order.no_d_id, new_order.no_w_id) } │ └─StreamFilter { predicate: (new_order.no_w_id = new_order.no_w_id) AND (new_order.no_d_id = new_order.no_d_id) AND (new_order.no_o_id = new_order.no_o_id) } - │ └─StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], pk: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], dist: UpstreamHashShard(new_order.no_o_id, new_order.no_d_id, new_order.no_w_id) } + │ └─StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], pk: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], dist: UpstreamHashShard(new_order.no_w_id, new_order.no_d_id, new_order.no_o_id) } └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: all } ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─StreamFilter { predicate: (orders.o_d_id = orders.o_d_id) AND (orders.o_w_id = orders.o_w_id) AND (orders.o_entry_d > '2007-01-02 00:00:00':Timestamp) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─StreamFilter { predicate: (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_w_id = order_line.ol_w_id) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [ol_o_id, ol_w_id, ol_d_id, revenue, o_entry_d], stream_key: [o_entry_d, ol_o_id, ol_d_id, ol_w_id], pk_columns: [revenue, o_entry_d, ol_o_id, ol_d_id, ol_w_id], pk_conflict: NoCheck } @@ -463,25 +463,25 @@ Fragment 3 StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } └── StreamFilter { predicate: (customer.c_w_id = customer.c_w_id) AND (customer.c_d_id = customer.c_d_id) AND Like(customer.c_state, 'a%':Varchar) } - └── StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 9 } + └── StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 9 } ├── Upstream └── BatchPlanNode Fragment 4 StreamFilter { predicate: (new_order.no_w_id = new_order.no_w_id) AND (new_order.no_d_id = new_order.no_d_id) AND (new_order.no_o_id = new_order.no_o_id) } - └── StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], pk: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], dist: UpstreamHashShard(new_order.no_o_id, new_order.no_d_id, new_order.no_w_id) } { state table: 10 } + └── StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], pk: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], dist: UpstreamHashShard(new_order.no_w_id, new_order.no_d_id, new_order.no_o_id) } { state table: 10 } ├── Upstream └── BatchPlanNode Fragment 5 StreamFilter { predicate: (orders.o_d_id = orders.o_d_id) AND (orders.o_w_id = orders.o_w_id) AND (orders.o_entry_d > '2007-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 15 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 15 } ├── Upstream └── BatchPlanNode Fragment 6 StreamFilter { predicate: (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_w_id = order_line.ol_w_id) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 16 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 16 } ├── Upstream └── BatchPlanNode @@ -549,7 +549,7 @@ └─BatchHashJoin { type: LeftSemi, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND (order_line.ol_delivery_d >= orders.o_entry_d), output: [orders.o_ol_cnt] } ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_w_id, orders.o_d_id) } │ └─BatchFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d < '2032-01-02 00:00:00':Timestamp) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } └─BatchProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d] } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d], distribution: SomeShard } @@ -560,10 +560,10 @@ └─StreamHashJoin { type: LeftSemi, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND (order_line.ol_delivery_d >= orders.o_entry_d), output: [orders.o_ol_cnt, orders.o_w_id, orders.o_d_id, orders.o_id] } ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_w_id, orders.o_d_id) } │ └─StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d < '2032-01-02 00:00:00':Timestamp) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [o_ol_cnt, order_count], stream_key: [o_ol_cnt], pk_columns: [o_ol_cnt], pk_conflict: NoCheck } @@ -585,13 +585,13 @@ Fragment 2 StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d < '2032-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 5 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -662,16 +662,16 @@ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2] } └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [customer.c_state, order_line.ol_amount, stock.s_i_id, stock.s_w_id], lookup table: stock } - └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id, order_line.ol_w_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_i_id) } └─BatchHashJoin { type: Inner, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id, output: [customer.c_state, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─BatchHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_state, orders.o_id, orders.o_d_id, orders.o_w_id] } │ ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ └─BatchExchange { order: [], dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ └─BatchProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } │ └─BatchFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount], distribution: SomeShard } stream_plan: |- @@ -686,18 +686,18 @@ │ ├─StreamExchange { dist: HashShard(orders.o_id, customer.c_d_id, customer.c_w_id, orders.o_w_id) } │ │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_d_id, customer.c_w_id, customer.c_state, orders.o_id, orders.o_d_id, orders.o_w_id, customer.c_id] } │ │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ │ └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ │ └─StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } │ │ └─StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } │ └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, stock.s_w_id) } │ └─StreamHashJoin { type: Inner, predicate: order_line.ol_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, stock.s_i_id, stock.s_w_id, order_line.ol_number, order_line.ol_i_id] } │ ├─StreamExchange { dist: HashShard(order_line.ol_w_id, order_line.ol_i_id) } │ │ └─StreamFilter { predicate: (order_line.ol_d_id = order_line.ol_d_id) } - │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } │ └─StreamExchange { dist: HashShard(stock.s_w_id, stock.s_i_id) } - │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } └─StreamExchange { dist: HashShard(supplier.s_suppkey, supplier.s_nationkey) } └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_nationkey, nation.n_name, region.r_regionkey] } ├─StreamExchange { dist: HashShard(region.r_regionkey) } @@ -746,14 +746,14 @@ └── StreamExchange Hash([1, 2, 3]) from 5 Fragment 4 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 13 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 13 } ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └── StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 14 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 14 } ├── Upstream └── BatchPlanNode @@ -764,12 +764,12 @@ Fragment 7 StreamFilter { predicate: (order_line.ol_d_id = order_line.ol_d_id) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 19 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 19 } ├── Upstream └── BatchPlanNode Fragment 8 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 20 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 20 } ├── Upstream └── BatchPlanNode @@ -894,7 +894,7 @@ └─StreamStatelessSimpleAgg { aggs: [sum(order_line.ol_amount)] } └─StreamProject { exprs: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └─StreamFilter { predicate: (order_line.ol_delivery_d >= '1997-01-01 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) AND (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 100000:Int32) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } @@ -910,7 +910,7 @@ StreamStatelessSimpleAgg { aggs: [sum(order_line.ol_amount)] } └── StreamProject { exprs: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d >= '1997-01-01 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) AND (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 100000:Int32) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -980,12 +980,12 @@ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id AND orders.o_c_id = customer.c_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state], lookup table: customer } - └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_c_id, orders.o_d_id, orders.o_w_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_c_id) } └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], lookup table: orders } - └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id) } └─BatchHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_supply_w_id AND stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(stock.s_i_id, stock.s_w_id) } - │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } └─BatchProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount] } └─BatchFilter { predicate: (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d <= '2032-01-02 00:00:00':Timestamp) } @@ -1007,17 +1007,17 @@ │ │ ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_w_id) } │ │ │ └─StreamHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_supply_w_id AND stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number] } │ │ │ ├─StreamExchange { dist: HashShard(stock.s_i_id, stock.s_w_id) } - │ │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } │ │ │ └─StreamExchange { dist: HashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } │ │ │ └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number] } │ │ │ └─StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d <= '2032-01-02 00:00:00':Timestamp) } - │ │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + │ │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } │ │ └─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id, customer.c_d_id, customer.c_w_id) } │ │ └─StreamHashJoin { type: Inner, predicate: orders.o_c_id = customer.c_id AND orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, customer.c_d_id, customer.c_w_id, customer.c_state, orders.o_c_id, customer.c_id] } │ │ ├─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } - │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } │ │ └─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } └─StreamExchange { dist: HashShard(supplier.s_suppkey) } @@ -1067,14 +1067,14 @@ └── StreamExchange Hash([3, 4]) from 6 Fragment 5 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 17 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 17 } ├── Upstream └── BatchPlanNode Fragment 6 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d <= '2032-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 18 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 18 } ├── Upstream └── BatchPlanNode @@ -1084,12 +1084,12 @@ └── StreamExchange Hash([1, 2, 0]) from 9 Fragment 8 - StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 23 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 23 } ├── Upstream └── BatchPlanNode Fragment 9 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 24 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 24 } ├── Upstream └── BatchPlanNode @@ -1246,9 +1246,9 @@ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id AND orders.o_c_id = customer.c_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state], lookup table: customer } - └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_c_id, orders.o_d_id, orders.o_w_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_c_id) } └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d <= '2032-01-02 00:00:00':Timestamp), output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], lookup table: orders } - └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id) } └─BatchHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = order_line.ol_supply_w_id AND item.i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(item.i_id, stock.s_w_id) } │ └─BatchHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } @@ -1256,7 +1256,7 @@ │ │ └─BatchScan { table: item, columns: [item.i_id], scan_ranges: [item.i_id < Int32(1000)], distribution: UpstreamHashShard(item.i_id) } │ └─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } │ └─BatchFilter { predicate: (stock.s_i_id < 1000:Int32) } - │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } └─BatchFilter { predicate: (order_line.ol_i_id = order_line.ol_i_id) AND (order_line.ol_i_id < 1000:Int32) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount], distribution: SomeShard } @@ -1277,9 +1277,9 @@ │ │ │ └─StreamHashJoin { type: Inner, predicate: orders.o_c_id = customer.c_id AND orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, customer.c_d_id, customer.c_w_id, customer.c_state, orders.o_c_id, customer.c_id] } │ │ │ ├─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ │ │ │ └─StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d <= '2032-01-02 00:00:00':Timestamp) } - │ │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } │ │ │ └─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ │ └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } │ │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = order_line.ol_supply_w_id AND item.i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, item.i_id, order_line.ol_number] } │ │ ├─StreamExchange { dist: HashShard(item.i_id, stock.s_w_id) } @@ -1289,10 +1289,10 @@ │ │ │ │ └─StreamTableScan { table: item, columns: [item.i_id], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } │ │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) } │ │ │ └─StreamFilter { predicate: (stock.s_i_id < 1000:Int32) } - │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } │ │ └─StreamExchange { dist: HashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } │ │ └─StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_i_id = order_line.ol_i_id) AND (order_line.ol_i_id < 1000:Int32) } - │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_name, supplier.s_nationkey, nation.n_nationkey] } │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } @@ -1355,12 +1355,12 @@ Fragment 5 StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d <= '2032-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 17 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 17 } ├── Upstream └── BatchPlanNode Fragment 6 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 18 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 18 } ├── Upstream └── BatchPlanNode @@ -1382,13 +1382,13 @@ Fragment 10 StreamFilter { predicate: (stock.s_i_id < 1000:Int32) } - └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 28 } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 28 } ├── Upstream └── BatchPlanNode Fragment 11 StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_i_id = order_line.ol_i_id) AND (order_line.ol_i_id < 1000:Int32) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 29 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 29 } ├── Upstream └── BatchPlanNode @@ -1562,9 +1562,9 @@ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id, output: [order_line.ol_amount, stock.s_i_id, stock.s_w_id, orders.o_entry_d], lookup table: orders } - └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id) } └─BatchLookupJoin { type: Inner, predicate: order_line.ol_supply_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, stock.s_i_id, stock.s_w_id], lookup table: stock } - └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_supply_w_id, order_line.ol_i_id) } └─BatchHashJoin { type: Inner, predicate: item.i_id = order_line.ol_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(item.i_id) } │ └─BatchProject { exprs: [item.i_id] } @@ -1589,14 +1589,14 @@ │ │ │ └─StreamFilter { predicate: Like(item.i_data, '%BB':Varchar) } │ │ │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } │ └─StreamExchange { dist: HashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } │ └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: [orders.o_entry_d, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } │ ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } │ └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } │ └─StreamFilter { predicate: (order_line.ol_i_id = order_line.ol_i_id) } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } └─StreamExchange { dist: HashShard(supplier.s_suppkey) } └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_name, supplier.s_nationkey, nation.n_nationkey] } ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } @@ -1647,7 +1647,7 @@ └── BatchPlanNode Fragment 5 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 14 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 14 } ├── Upstream └── BatchPlanNode @@ -1661,13 +1661,13 @@ └── StreamExchange Hash([0, 1, 2]) from 8 Fragment 7 - StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 19 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 19 } ├── Upstream └── BatchPlanNode Fragment 8 StreamFilter { predicate: (order_line.ol_i_id = order_line.ol_i_id) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── state table: 20 ├── Upstream └── BatchPlanNode @@ -1788,10 +1788,10 @@ ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─BatchHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } │ ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ └─BatchExchange { order: [], dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ └─BatchFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount], distribution: SomeShard } stream_plan: |- @@ -1806,14 +1806,14 @@ │ └─StreamHashJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND order_line.ol_d_id = customer.c_d_id AND order_line.ol_w_id = customer.c_w_id, output: all } │ ├─StreamExchange { dist: HashShard(order_line.ol_d_id, order_line.ol_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } │ │ └─StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) } - │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } │ └─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ └─StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─StreamExchange { dist: HashShard(nation.n_nationkey) } └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } stream_dist_plan: |+ @@ -1849,7 +1849,7 @@ Fragment 3 StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── state table: 9 ├── Upstream └── BatchPlanNode @@ -1864,13 +1864,13 @@ └── StreamExchange Hash([1, 2, 3]) from 6 Fragment 5 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 14 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 14 } ├── Upstream └── BatchPlanNode Fragment 6 StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 15 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 15 } ├── Upstream └── BatchPlanNode @@ -1977,7 +1977,7 @@ │ │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } │ └─BatchExchange { order: [], dist: HashShard($expr1) } │ └─BatchProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } - │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } └─BatchProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr4] } └─BatchSimpleAgg { aggs: [sum(sum(stock.s_order_cnt))] } └─BatchExchange { order: [], dist: Single } @@ -1989,7 +1989,7 @@ │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } └─BatchExchange { order: [], dist: HashShard($expr3) } └─BatchProject { exprs: [stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr3] } - └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } stream_plan: |- StreamMaterialize { columns: [s_i_id, ordercount], stream_key: [s_i_id], pk_columns: [ordercount, s_i_id], pk_conflict: NoCheck } └─StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt)] } @@ -2009,7 +2009,7 @@ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } │ └─StreamExchange { dist: HashShard($expr1) } │ └─StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr3] } └─StreamSimpleAgg { aggs: [sum(sum(stock.s_order_cnt)), count] } @@ -2027,7 +2027,7 @@ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_i_id, ordercount], stream_key: [s_i_id], pk_columns: [ordercount, s_i_id], pk_conflict: NoCheck } @@ -2082,7 +2082,7 @@ Fragment 6 StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } ├── state table: 13 ├── Upstream └── BatchPlanNode @@ -2202,7 +2202,7 @@ └─BatchProject { exprs: [orders.o_ol_cnt, Case(((orders.o_carrier_id = 1:Int32) OR (orders.o_carrier_id = 2:Int32)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_carrier_id <> 1:Int32) AND (orders.o_carrier_id <> 2:Int32)), 1:Int32, 0:Int32) as $expr2] } └─BatchHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id AND (orders.o_entry_d <= order_line.ol_delivery_d), output: [orders.o_carrier_id, orders.o_ol_cnt] } ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchFilter { predicate: (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d], distribution: SomeShard } @@ -2215,10 +2215,10 @@ └─StreamFilter { predicate: (orders.o_entry_d <= order_line.ol_delivery_d) } └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: all } ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─StreamFilter { predicate: (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [o_ol_cnt, high_line_count, low_line_count], stream_key: [o_ol_cnt], pk_columns: [o_ol_cnt], pk_conflict: NoCheck } @@ -2238,13 +2238,13 @@ └── StreamExchange Hash([0, 1, 2]) from 3 Fragment 2 - StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 5 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 StreamFilter { predicate: (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -2294,11 +2294,11 @@ └─BatchExchange { order: [], dist: HashShard(customer.c_id) } └─BatchHashJoin { type: LeftOuter, predicate: customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id AND customer.c_id = orders.o_c_id, output: [customer.c_id, orders.o_id] } ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } └─BatchExchange { order: [], dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } └─BatchProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └─BatchFilter { predicate: (orders.o_carrier_id > 8:Int32) } - └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } stream_plan: |- StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [custdist, c_count], pk_conflict: NoCheck } └─StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] } @@ -2308,11 +2308,11 @@ └─StreamExchange { dist: HashShard(customer.c_id) } └─StreamHashJoin { type: LeftOuter, predicate: customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id AND customer.c_id = orders.o_c_id, output: [customer.c_id, orders.o_id, customer.c_w_id, customer.c_d_id, orders.o_w_id, orders.o_d_id] } ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } └─StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └─StreamFilter { predicate: (orders.o_carrier_id > 8:Int32) } - └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [custdist, c_count], pk_conflict: NoCheck } @@ -2341,14 +2341,14 @@ └── StreamExchange Hash([1, 2, 3]) from 4 Fragment 3 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 6 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 6 } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └── StreamFilter { predicate: (orders.o_carrier_id > 8:Int32) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } ├── state table: 7 ├── Upstream └── BatchPlanNode @@ -2412,7 +2412,7 @@ ├─StreamExchange { dist: HashShard(order_line.ol_i_id) } │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } │ └─StreamFilter { predicate: (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-02 00:00:00':Timestamp) } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } └─StreamExchange { dist: HashShard(item.i_id) } └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } stream_dist_plan: |+ @@ -2440,7 +2440,7 @@ Fragment 2 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── state table: 5 ├── Upstream └── BatchPlanNode @@ -2630,7 +2630,7 @@ └─BatchProject { exprs: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } └─BatchLookupJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND (Not((item.i_data >= 'zz':Varchar)) OR Not((item.i_data < 'z{':Varchar))), output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data], lookup table: item } └─BatchExchange { order: [], dist: UpstreamHashShard(stock.s_i_id) } - └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } stream_plan: |- StreamMaterialize { columns: [i_name, brand, i_price, supplier_cnt], stream_key: [i_name, brand, i_price], pk_columns: [supplier_cnt, i_name, brand, i_price], pk_conflict: NoCheck } └─StreamProject { exprs: [item.i_name, $expr2, item.i_price, count(distinct $expr3)] } @@ -2642,7 +2642,7 @@ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id, output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, item.i_id] } │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } │ └─StreamExchange { dist: HashShard(item.i_id) } │ └─StreamFilter { predicate: Not(Like(item.i_data, 'zz%':Varchar)) } │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_price, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } @@ -2678,7 +2678,7 @@ └── StreamExchange Hash([0]) from 4 Fragment 3 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 10 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 10 } ├── Upstream └── BatchPlanNode @@ -2819,7 +2819,7 @@ └─StreamHashJoin { type: Inner, predicate: order_line.ol_i_id = item.i_id, output: all } ├─StreamExchange { dist: HashShard(order_line.ol_i_id) } │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_quantity::Decimal as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } └─StreamProject { exprs: [item.i_id, (sum(order_line.ol_quantity)::Decimal / count(order_line.ol_quantity)::Decimal) as $expr2] } └─StreamHashAgg { group_key: [item.i_id], aggs: [sum(order_line.ol_quantity), count(order_line.ol_quantity), count] } └─StreamHashJoin { type: Inner, predicate: item.i_id = order_line.ol_i_id, output: [item.i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } @@ -2828,7 +2828,7 @@ │ └─StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } └─StreamExchange { dist: HashShard(order_line.ol_i_id) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } @@ -2858,7 +2858,7 @@ Fragment 2 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_quantity::Decimal as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── state table: 5 ├── Upstream └── BatchPlanNode @@ -2871,7 +2871,7 @@ └── BatchPlanNode Fragment 4 - StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 12 } + StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 12 } ├── Upstream └── BatchPlanNode @@ -2939,9 +2939,9 @@ ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─BatchHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt] } │ ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ └─BatchExchange { order: [], dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount], distribution: SomeShard } stream_plan: |- @@ -2954,11 +2954,11 @@ ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, customer.c_w_id, customer.c_d_id] } │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_last, o_id, o_entry_d, o_ol_cnt, sum, sum(order_line.ol_amount)(hidden), orders.o_id(hidden), orders.o_d_id(hidden), orders.o_w_id(hidden)], stream_key: [o_id, c_last, orders.o_id, orders.o_d_id, orders.o_w_id, o_entry_d, o_ol_cnt], pk_columns: [sum(order_line.ol_amount), o_entry_d, o_id, c_last, orders.o_id, orders.o_d_id, orders.o_w_id, o_ol_cnt], pk_conflict: NoCheck } @@ -2981,17 +2981,17 @@ └── StreamExchange Hash([1, 2, 3]) from 3 Fragment 2 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 9 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 9 } ├── Upstream └── BatchPlanNode Fragment 3 - StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 10 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 10 } ├── Upstream └── BatchPlanNode Fragment 4 - StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 11 } + StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 11 } ├── Upstream └── BatchPlanNode @@ -3059,7 +3059,7 @@ ├─StreamExchange { dist: HashShard(order_line.ol_i_id) } │ └─StreamProject { exprs: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } │ └─StreamFilter { predicate: (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 10:Int32) } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } └─StreamExchange { dist: HashShard(item.i_id) } └─StreamProject { exprs: [item.i_id, item.i_data] } └─StreamFilter { predicate: (item.i_price >= 1:Decimal) AND (item.i_price <= 400000:Decimal) } @@ -3086,7 +3086,7 @@ Fragment 2 StreamProject { exprs: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 10:Int32) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── state table: 5 ├── Upstream └── BatchPlanNode @@ -3166,7 +3166,7 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(stock.s_i_id) } └─BatchHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity] } ├─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } - │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_i_id) } └─BatchProject { exprs: [order_line.ol_i_id, order_line.ol_quantity] } └─BatchFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } @@ -3191,11 +3191,11 @@ └─StreamHashJoin { type: LeftSemi, predicate: stock.s_i_id = item.i_id, output: all } ├─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } │ └─StreamExchange { dist: HashShard(order_line.ol_i_id) } │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } │ └─StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } └─StreamExchange { dist: HashShard(item.i_id) } └─StreamProject { exprs: [item.i_id] } └─StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) } @@ -3244,14 +3244,14 @@ └── StreamExchange Hash([0]) from 8 Fragment 6 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 19 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 19 } ├── Upstream └── BatchPlanNode Fragment 7 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── state table: 20 ├── Upstream └── BatchPlanNode @@ -3359,9 +3359,9 @@ │ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } │ └─BatchProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } │ └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, stock.s_i_id, stock.s_w_id], lookup table: stock } - │ └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id, order_line.ol_w_id) } + │ └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_i_id) } │ └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND (order_line.ol_delivery_d > orders.o_entry_d), output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d], lookup table: orders } - │ └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } + │ └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id) } │ └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d], distribution: SomeShard } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } └─BatchProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d] } @@ -3378,15 +3378,15 @@ │ │ └─StreamHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_w_id AND stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = orders.o_w_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number] } │ │ ├─StreamExchange { dist: HashShard(stock.s_w_id, stock.s_i_id, stock.s_w_id) } │ │ │ └─StreamFilter { predicate: (stock.s_w_id = stock.s_w_id) } - │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } + │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } │ │ └─StreamExchange { dist: HashShard(order_line.ol_w_id, order_line.ol_i_id, orders.o_w_id) } │ │ └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, orders.o_w_id, order_line.ol_number, orders.o_d_id, orders.o_id] } │ │ └─StreamFilter { predicate: (order_line.ol_delivery_d > orders.o_entry_d) } │ │ └─StreamHashJoin { type: Inner, predicate: order_line.ol_o_id = orders.o_id AND order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id, output: all } │ │ ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - │ │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + │ │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } │ │ └─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } @@ -3397,7 +3397,7 @@ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } @@ -3430,7 +3430,7 @@ Fragment 4 StreamFilter { predicate: (stock.s_w_id = stock.s_w_id) } - └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 13 } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 13 } ├── Upstream └── BatchPlanNode @@ -3442,12 +3442,12 @@ └── StreamExchange Hash([0, 1, 2]) from 7 Fragment 6 - StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 18 } + StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 18 } ├── Upstream └── BatchPlanNode Fragment 7 - StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 19 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 19 } ├── Upstream └── BatchPlanNode @@ -3470,7 +3470,7 @@ Fragment 11 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 26 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 26 } ├── Upstream └── BatchPlanNode @@ -3582,7 +3582,7 @@ │ ├─BatchExchange { order: [], dist: HashShard(customer.c_id, customer.c_w_id, customer.c_d_id) } │ │ └─BatchProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance] } │ │ └─BatchFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 1:Int32), '1':Varchar, '2':Varchar, '3':Varchar, '4':Varchar, '5':Varchar, '6':Varchar, '7':Varchar) } - │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance, customer.c_phone], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance, customer.c_phone], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ └─BatchExchange { order: [], dist: HashShard(orders.o_c_id, orders.o_w_id, orders.o_d_id) } │ └─BatchProject { exprs: [orders.o_c_id, orders.o_w_id, orders.o_d_id] } │ └─BatchScan { table: orders, columns: [orders.o_d_id, orders.o_w_id, orders.o_c_id], distribution: SomeShard } @@ -3604,10 +3604,10 @@ │ ├─StreamExchange { dist: HashShard(customer.c_id, customer.c_w_id, customer.c_d_id) } │ │ └─StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance] } │ │ └─StreamFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 1:Int32), '1':Varchar, '2':Varchar, '3':Varchar, '4':Varchar, '5':Varchar, '6':Varchar, '7':Varchar) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } │ └─StreamExchange { dist: HashShard(orders.o_c_id, orders.o_w_id, orders.o_d_id) } │ └─StreamProject { exprs: [orders.o_c_id, orders.o_w_id, orders.o_d_id, orders.o_id] } - │ └─StreamTableScan { table: orders, columns: [orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [(sum(sum(customer.c_balance)) / sum0(count(customer.c_balance))::Decimal) as $expr1] } └─StreamSimpleAgg { aggs: [sum(sum(customer.c_balance)), sum0(count(customer.c_balance)), count] } @@ -3615,4 +3615,4 @@ └─StreamStatelessSimpleAgg { aggs: [sum(customer.c_balance), count(customer.c_balance)] } └─StreamProject { exprs: [customer.c_balance, customer.c_w_id, customer.c_d_id, customer.c_id] } └─StreamFilter { predicate: (customer.c_balance > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 1:Int32), '1':Varchar, '2':Varchar, '3':Varchar, '4':Varchar, '5':Varchar, '6':Varchar, '7':Varchar) } - └─StreamTableScan { table: customer, columns: [customer.c_balance, customer.c_w_id, customer.c_d_id, customer.c_id, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } + └─StreamTableScan { table: customer, columns: [customer.c_balance, customer.c_w_id, customer.c_d_id, customer.c_id, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index 1801fa33d75ea..c8082fb3b7138 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -21,10 +21,13 @@ ) FORMAT PLAIN ENCODE CSV (delimiter = ',', without_header = true); explain_output: | StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource { source: s0, columns: [v1, v2, _row_id] } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamUnion { all: true } + ├─StreamExchange [no_shuffle] { dist: SomeShard } + │ └─StreamSource { source: s0, columns: [v1, v2, _row_id] } + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource - id: csv_delimiter_tab sql: | explain create table s0 (v1 int, v2 varchar) with ( @@ -35,10 +38,13 @@ ) FORMAT PLAIN ENCODE CSV (delimiter = E'\t', without_header = true); explain_output: | StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource { source: s0, columns: [v1, v2, _row_id] } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamUnion { all: true } + ├─StreamExchange [no_shuffle] { dist: SomeShard } + │ └─StreamSource { source: s0, columns: [v1, v2, _row_id] } + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource - id: create_source_with_cdc_backfill sql: | create source mysql_mydb with ( @@ -57,8 +63,11 @@ ) from mysql_mydb table 'mydb.t1'; explain_output: | StreamMaterialize { columns: [v1, v2], stream_key: [v1], pk_columns: [v1], pk_conflict: Overwrite } - └─StreamExchange { dist: HashShard(mydb.t1.v1) } - └─StreamDml { columns: [v1, v2] } - └─StreamCdcTableScan { table: mydb.t1, columns: [v1, v2] } + └─StreamUnion { all: true } + ├─StreamExchange { dist: HashShard(mydb.t1.v1) } + │ └─StreamCdcTableScan { table: mydb.t1, columns: [v1, v2] } + └─StreamExchange { dist: HashShard(v1) } + └─StreamDml { columns: [v1, v2] } + └─StreamSource with_config_map: CDC_BACKFILL: 'true' diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index 63899bc26c0e4..f416e9155a023 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -191,15 +191,19 @@ explain create table t (v1 int, v2 varchar); explain_output: | StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource + └─StreamRowIdGen { row_id_index: 2 } + └─StreamUnion { all: true } + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource - sql: | explain create table t (v1 int, v2 varchar) with ( connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest' ) FORMAT PLAIN ENCODE JSON; explain_output: | StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource { source: t, columns: [v1, v2, _row_id] } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamUnion { all: true } + ├─StreamExchange [no_shuffle] { dist: SomeShard } + │ └─StreamSource { source: t, columns: [v1, v2, _row_id] } + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource diff --git a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml index a3acaf174d237..e88df797de0c8 100644 --- a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml +++ b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml @@ -4,11 +4,12 @@ explain create table t1 (v1 int as v2-1, v2 int, v3 int as v2+1); explain_output: | StreamMaterialize { columns: [v1, v2, v3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamProject { exprs: [(v2 - 1:Int32) as $expr1, v2, (v2 + 1:Int32) as $expr2, _row_id] } - └─StreamDml { columns: [v2, _row_id] } - └─StreamSource + └─StreamRowIdGen { row_id_index: 3 } + └─StreamUnion { all: true } + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamProject { exprs: [(v2 - 1:Int32) as $expr1, v2, (v2 + 1:Int32) as $expr2, _row_id] } + └─StreamDml { columns: [v2, _row_id] } + └─StreamSource - name: source with generated columns sql: | create source s1 (v1 int as v2-1, v2 int, v3 int as v2+1) with (connector = 'kinesis') FORMAT PLAIN ENCODE JSON; @@ -30,11 +31,12 @@ explain create table t1 (proc_time TIMESTAMP AS proctime()); explain_output: | StreamMaterialize { columns: [proc_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite, watermark_columns: [proc_time] } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamRowIdGen { row_id_index: 1 } - └─StreamProject { exprs: [AtTimeZone(Proctime, 'UTC':Varchar) as $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamDml { columns: [_row_id] } - └─StreamSource + └─StreamRowIdGen { row_id_index: 1 } + └─StreamUnion { all: true, output_watermarks: [$expr1] } + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamProject { exprs: [AtTimeZone(Proctime, 'UTC':Varchar) as $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamDml { columns: [_row_id] } + └─StreamSource - name: watermark on generated column sql: | explain create table t (v int, w int as v+1, watermark for w as w) append only @@ -42,6 +44,8 @@ StreamMaterialize { columns: [v, w, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [w] } └─StreamRowIdGen { row_id_index: 2 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: $expr1 }], output_watermarks: [$expr1] } - └─StreamProject { exprs: [v, (v + 1:Int32) as $expr1, _row_id] } - └─StreamDml { columns: [v, _row_id] } - └─StreamSource + └─StreamUnion { all: true } + └─StreamExchange [no_shuffle] { dist: SomeShard } + └─StreamProject { exprs: [v, (v + 1:Int32) as $expr1, _row_id] } + └─StreamDml { columns: [v, _row_id] } + └─StreamSource diff --git a/src/frontend/planner_test/tests/testdata/output/union.yaml b/src/frontend/planner_test/tests/testdata/output/union.yaml index 725aab00d5a32..48f45722494ac 100644 --- a/src/frontend/planner_test/tests/testdata/output/union.yaml +++ b/src/frontend/planner_test/tests/testdata/output/union.yaml @@ -596,7 +596,7 @@ Fragment 4 StreamProject { exprs: [t4.a, t4.b, t4.c, 3:Int32] } - └── StreamTableScan { table: t4, columns: [t4.a, t4.b, t4.c], pk: [t4.b, t4.a], dist: UpstreamHashShard(t4.a, t4.b) } + └── StreamTableScan { table: t4, columns: [t4.a, t4.b, t4.c], pk: [t4.b, t4.a], dist: UpstreamHashShard(t4.b, t4.a) } ├── state table: 3 ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml index d57d41fa76bc3..cc54951beec4a 100644 --- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml @@ -51,8 +51,12 @@ StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - '00:00:01':Interval) }], output_watermarks: [v1] } - └─StreamDml { columns: [v1, _row_id] } - └─StreamSource { source: t, columns: [v1, _row_id] } + └─StreamUnion { all: true } + ├─StreamExchange [no_shuffle] { dist: SomeShard } + │ └─StreamSource { source: t, columns: [v1, _row_id] } + └─StreamExchange [no_shuffle] { dist: SomeShard } + └─StreamDml { columns: [v1, _row_id] } + └─StreamSource - name: watermark on append only table without source sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only; @@ -60,8 +64,10 @@ StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - '00:00:01':Interval) }], output_watermarks: [v1] } - └─StreamDml { columns: [v1, _row_id] } - └─StreamSource + └─StreamUnion { all: true } + └─StreamExchange [no_shuffle] { dist: SomeShard } + └─StreamDml { columns: [v1, _row_id] } + └─StreamSource - name: hash agg sql: | create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 2ba36099569e3..c9b96065f2971 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -698,6 +698,8 @@ fn gen_table_plan_inner( let connection_id = resolve_privatelink_in_with_option(&mut with_options, &schema_name, &session)?; + let is_external_source = source_info.is_some(); + let source = source_info.map(|source_info| PbSource { id: TableId::placeholder().table_id, schema_id, @@ -776,6 +778,7 @@ fn gen_table_plan_inner( append_only, watermark_descs, version, + is_external_source, )?; let mut table = materialize.table().to_prost(schema_id, database_id); @@ -884,8 +887,9 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( pk_column_ids, None, append_only, - vec![], // no watermarks + vec![], Some(col_id_gen.into_version()), + true, )?; let mut table = materialize.table().to_prost(schema_id, database_id); diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index dd38a2f6f089a..eb77732339558 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -17,7 +17,6 @@ use std::ops::DerefMut; pub mod plan_node; pub use plan_node::{Explain, PlanRef}; - pub mod property; mod delta_join_solver; @@ -39,7 +38,9 @@ pub use logical_optimization::*; pub use optimizer_context::*; use plan_expr_rewriter::ConstEvalRewriter; use property::Order; -use risingwave_common::catalog::{ColumnCatalog, ColumnId, ConflictBehavior, Field, Schema}; +use risingwave_common::catalog::{ + ColumnCatalog, ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, +}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::iter_util::ZipEqDebug; @@ -59,8 +60,10 @@ use self::plan_visitor::{has_batch_exchange, CardinalityVisitor}; use self::property::{Cardinality, RequiredDist}; use self::rule::*; use crate::catalog::table_catalog::{TableType, TableVersion}; +use crate::optimizer::plan_node::generic::Union; use crate::optimizer::plan_node::{ - BatchExchange, PlanNodeType, PlanTreeNode, RewriteExprsRecursive, + BatchExchange, PlanNodeType, PlanTreeNode, RewriteExprsRecursive, StreamExchange, StreamUnion, + ToStream, }; use crate::optimizer::plan_visitor::TemporalJoinValidator; use crate::optimizer::property::Distribution; @@ -431,29 +434,156 @@ impl PlanRoot { append_only: bool, watermark_descs: Vec, version: Option, + with_external_source: bool, ) -> Result { - let mut stream_plan = self.gen_optimized_stream_plan(false)?; + let stream_plan = self.gen_optimized_stream_plan(false)?; - // Add DML node. - stream_plan = StreamDml::new( - stream_plan, - append_only, - columns + assert!(!pk_column_ids.is_empty() || row_id_index.is_some()); + + let pk_column_indices = { + let mut id_to_idx = HashMap::new(); + + columns.iter().enumerate().for_each(|(idx, c)| { + id_to_idx.insert(c.column_id(), idx); + }); + pk_column_ids .iter() - .filter(|&c| (!c.is_generated())) - .map(|c| c.column_desc.clone()) - .collect(), - ) - .into(); + .map(|c| id_to_idx.get(c).copied().unwrap()) // pk column id must exist in table columns. + .collect_vec() + }; + + fn inject_project_if_needed(columns: &[ColumnCatalog], node: PlanRef) -> Result { + let exprs = LogicalSource::derive_output_exprs_from_generated_columns(columns)?; + if let Some(exprs) = exprs { + let logical_project = generic::Project::new(exprs, node); + return Ok(StreamProject::new(logical_project).into()); + } + Ok(node) + } - // Add generated columns. - let exprs = LogicalSource::derive_output_exprs_from_generated_columns(&columns)?; - if let Some(exprs) = exprs { - let logical_project = generic::Project::new(exprs, stream_plan); - // The project node merges a chunk if it has an ungenerated row id as stream key. - stream_plan = StreamProject::new(logical_project).into(); + fn inject_dml_node( + columns: &[ColumnCatalog], + append_only: bool, + stream_plan: PlanRef, + pk_column_indices: &[usize], + kind: PrimaryKeyKind, + column_descs: Vec, + ) -> Result { + let mut dml_node = StreamDml::new(stream_plan, append_only, column_descs).into(); + + // Add generated columns. + dml_node = inject_project_if_needed(columns, dml_node)?; + + dml_node = match kind { + PrimaryKeyKind::UserDefinedPrimaryKey | PrimaryKeyKind::RowIdAsPrimaryKey => { + RequiredDist::hash_shard(pk_column_indices) + .enforce_if_not_satisfies(dml_node, &Order::any())? + } + PrimaryKeyKind::AppendOnly => StreamExchange::new_no_shuffle(dml_node).into(), + }; + + Ok(dml_node) + } + + #[derive(PartialEq, Debug, Copy, Clone)] + enum PrimaryKeyKind { + UserDefinedPrimaryKey, + RowIdAsPrimaryKey, + AppendOnly, } + let kind = if append_only { + assert!(row_id_index.is_some()); + PrimaryKeyKind::AppendOnly + } else if let Some(row_id_index) = row_id_index { + assert_eq!( + pk_column_indices.iter().exactly_one().copied().unwrap(), + row_id_index + ); + PrimaryKeyKind::RowIdAsPrimaryKey + } else { + PrimaryKeyKind::UserDefinedPrimaryKey + }; + + let column_descs = columns + .iter() + .filter(|&c| (!c.is_generated())) + .map(|c| c.column_desc.clone()) + .collect(); + + let union_inputs = if with_external_source { + let mut external_source_node = stream_plan; + external_source_node = inject_project_if_needed(&columns, external_source_node)?; + external_source_node = match kind { + PrimaryKeyKind::UserDefinedPrimaryKey => { + RequiredDist::hash_shard(&pk_column_indices) + .enforce_if_not_satisfies(external_source_node, &Order::any())? + } + PrimaryKeyKind::RowIdAsPrimaryKey | PrimaryKeyKind::AppendOnly => { + StreamExchange::new_no_shuffle(external_source_node).into() + } + }; + + let dummy_source_node = LogicalSource::new( + None, + columns.clone(), + row_id_index, + false, + true, + context.clone(), + ) + .and_then(|s| s.to_stream(&mut ToStreamContext::new(false)))?; + + let dml_node = inject_dml_node( + &columns, + append_only, + dummy_source_node, + &pk_column_indices, + kind, + column_descs, + )?; + + vec![external_source_node, dml_node] + } else { + let dml_node = inject_dml_node( + &columns, + append_only, + stream_plan, + &pk_column_indices, + kind, + column_descs, + )?; + + vec![dml_node] + }; + + let dists = union_inputs + .iter() + .map(|input| input.distribution()) + .unique() + .collect_vec(); + + let dist = match &dists[..] { + &[Distribution::SomeShard, Distribution::HashShard(_)] + | &[Distribution::HashShard(_), Distribution::SomeShard] => Distribution::SomeShard, + &[dist @ Distribution::SomeShard] | &[dist @ Distribution::HashShard(_)] => { + dist.clone() + } + _ => { + unreachable!() + } + }; + + let mut stream_plan = StreamUnion::new_with_dist( + Union { + all: true, + inputs: union_inputs, + source_col: None, + }, + dist.clone(), + ) + .into(); + // Add WatermarkFilter node. if !watermark_descs.is_empty() { stream_plan = StreamWatermarkFilter::new(stream_plan, watermark_descs).into(); @@ -461,7 +591,19 @@ impl PlanRoot { // Add RowIDGen node if needed. if let Some(row_id_index) = row_id_index { - stream_plan = StreamRowIdGen::new(stream_plan, row_id_index).into(); + match kind { + PrimaryKeyKind::UserDefinedPrimaryKey => { + unreachable!() + } + PrimaryKeyKind::RowIdAsPrimaryKey | PrimaryKeyKind::AppendOnly => { + stream_plan = StreamRowIdGen::new_with_dist( + stream_plan, + row_id_index, + Distribution::HashShard(vec![row_id_index]), + ) + .into(); + } + } } let conflict_behavior = match append_only { @@ -469,18 +611,6 @@ impl PlanRoot { false => ConflictBehavior::Overwrite, }; - let pk_column_indices = { - let mut id_to_idx = HashMap::new(); - - columns.iter().enumerate().for_each(|(idx, c)| { - id_to_idx.insert(c.column_id(), idx); - }); - pk_column_ids - .iter() - .map(|c| id_to_idx.get(c).copied().unwrap()) // pk column id must exist in table columns. - .collect_vec() - }; - let table_required_dist = { let mut bitset = FixedBitSet::with_capacity(columns.len()); for idx in &pk_column_indices { @@ -702,7 +832,7 @@ mod tests { let subplan = root.into_subplan(); assert_eq!( subplan.schema(), - &Schema::new(vec![Field::with_name(DataType::Int32, "v1"),]) + &Schema::new(vec![Field::with_name(DataType::Int32, "v1")]) ); } } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 542178a830b73..30c4ee19002ad 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -45,6 +45,7 @@ use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamDedup, ToStreamContext, }; +use crate::optimizer::property::Distribution::HashShard; use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::utils::{ColIndexMapping, Condition, IndexRewriter}; @@ -574,7 +575,7 @@ impl ToStream for LogicalSource { if let Some(row_id_index) = self.core.row_id_index && self.core.gen_row_id { - plan = StreamRowIdGen::new(plan, row_id_index).into(); + plan = StreamRowIdGen::new_with_dist(plan, row_id_index, HashShard(vec![row_id_index])).into(); } Ok(plan) } diff --git a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs index 203bf8951493f..b7634c652a34d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs @@ -30,13 +30,16 @@ pub struct StreamRowIdGen { impl StreamRowIdGen { pub fn new(input: PlanRef, row_id_index: usize) -> Self { - let distribution = if input.append_only() { - // remove exchange for append only source - Distribution::HashShard(vec![row_id_index]) - } else { - input.distribution().clone() - }; + let distribution = input.distribution().clone(); + Self::new_with_dist(input, row_id_index, distribution) + } + /// Create a new `StreamRowIdGen` with a custom distribution. + pub fn new_with_dist( + input: PlanRef, + row_id_index: usize, + distribution: Distribution, + ) -> StreamRowIdGen { let base = PlanBase::new_stream( input.ctx(), input.schema().clone(), @@ -68,7 +71,7 @@ impl PlanTreeNodeUnary for StreamRowIdGen { } fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new(input, self.row_id_index) + Self::new_with_dist(input, self.row_id_index, self.distribution().clone()) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index 425cdc6914564..c67884ac10b27 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -26,6 +26,7 @@ use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanRef}; use crate::optimizer::plan_node::generic::GenericPlanNode; use crate::optimizer::plan_node::{PlanBase, PlanTreeNode, StreamNode}; +use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamUnion` implements [`super::LogicalUnion`] @@ -40,6 +41,12 @@ impl StreamUnion { let inputs = &core.inputs; let dist = inputs[0].distribution().clone(); assert!(inputs.iter().all(|input| *input.distribution() == dist)); + Self::new_with_dist(core, dist) + } + + pub fn new_with_dist(core: generic::Union, dist: Distribution) -> Self { + let inputs = &core.inputs; + let watermark_columns = inputs.iter().fold( { let mut bitset = FixedBitSet::with_capacity(core.schema().len()); @@ -56,6 +63,7 @@ impl StreamUnion { inputs.iter().all(|x| x.emit_on_window_close()), watermark_columns, ); + StreamUnion { base, core } } } @@ -78,7 +86,8 @@ impl PlanTreeNode for StreamUnion { fn clone_with_inputs(&self, inputs: &[crate::optimizer::PlanRef]) -> PlanRef { let mut new = self.core.clone(); new.inputs = inputs.to_vec(); - Self::new(new).into() + let dist = self.distribution().clone(); + Self::new_with_dist(new, dist).into() } } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index abc9e690ceeb9..10d4524db5370 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -868,6 +868,11 @@ fn fill_table_stream_graph_info( for fragment in fragment_graph.fragments.values_mut() { visit_fragment(fragment, |node_body| { if let NodeBody::Source(source_node) = node_body { + if source_node.source_inner.is_none() { + // skip empty source for dml node + return; + } + // If we're creating a table with connector, we should additionally fill its ID first. if let Some(&mut (ref mut source, source_id)) = source_info.as_mut() { source.id = source_id; diff --git a/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs b/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs index 6b04503ab2bc5..ed65c8d5894e3 100644 --- a/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs +++ b/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs @@ -15,7 +15,7 @@ use anyhow::Result; use itertools::Itertools; use risingwave_simulation::cluster::{Cluster, Configuration}; -use risingwave_simulation::ctl_ext::predicate::identity_contains; +use risingwave_simulation::ctl_ext::predicate::{identity_contains, no_identity_contains}; use risingwave_simulation::utils::AssertResult; #[tokio::test] @@ -48,7 +48,10 @@ async fn test_delta_join() -> Result<()> { .await?; assert_eq!(lookup_fragments.len(), 2, "failed to plan delta join"); let union_fragment = cluster - .locate_one_fragment([identity_contains("union")]) + .locate_one_fragment([ + identity_contains("union"), + no_identity_contains("materialize"), // skip union for table + ]) .await?; let mut test_times = 0; From 2628460bc5d28d0b5add8d5c7dd2ce8eb7705e1c Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Wed, 15 Nov 2023 16:44:21 +0800 Subject: [PATCH 06/36] fix(type): fix parsing array literal and printing struct value (#13229) Signed-off-by: Runji Wang --- e2e_test/batch/basic/array.slt.part | 5 +- .../test_multi_column_reference.slt.part | 2 +- .../batch/order/test_order_struct.slt.part | 12 +- e2e_test/batch/types/jsonb.slt.part | 10 +- .../types/struct/nested_structs.slt.part | 12 +- e2e_test/batch/types/struct/struct.slt.part | 44 ++- .../batch/types/struct/struct_case.slt.part | 2 +- .../batch/types/struct/struct_cast.slt.part | 18 +- .../struct/struct_cross_product.slt.part | 50 ++-- e2e_test/ddl/alter_rename_relation.slt | 4 +- e2e_test/sink/kafka/avro.slt | 4 +- e2e_test/sink/kafka/protobuf.slt | 2 +- e2e_test/source/basic/kafka.slt | 6 +- e2e_test/source/basic/kafka_batch.slt | 2 +- .../basic/old_row_format_syntax/kafka.slt | 6 +- .../old_row_format_syntax/kafka_batch.slt | 2 +- e2e_test/streaming/struct_table.slt | 8 +- e2e_test/udf/udf.slt | 14 +- src/common/src/array/list_array.rs | 282 +++++++++++++++++- src/common/src/array/struct_array.rs | 56 +++- src/common/src/types/mod.rs | 7 +- src/expr/impl/benches/expr.rs | 5 + src/expr/impl/src/scalar/cast.rs | 83 +----- .../tests/testdata/input/array.yaml | 2 +- .../tests/testdata/output/array.yaml | 10 +- .../tests/testdata/output/expr.yaml | 2 +- src/tests/regress/data/expected/arrays.out | 12 +- src/tests/regress/data/sql/arrays.sql | 16 +- 28 files changed, 476 insertions(+), 202 deletions(-) diff --git a/e2e_test/batch/basic/array.slt.part b/e2e_test/batch/basic/array.slt.part index 3229118b6d001..c02ec89312c8c 100644 --- a/e2e_test/batch/basic/array.slt.part +++ b/e2e_test/batch/basic/array.slt.part @@ -157,14 +157,13 @@ select pg_typeof((v1::integer[][])[2][1]), (v1::integer[][])[2][1] from (values integer 2 # Test multiple castings of the same input. -query TTI +query TI select - (arr::varchar[][])[1][2] as double_varchar, (arr::varchar[][][])[1][2][3] as triple_varchar, (arr::integer[][][])[1][2][3] as triple_integer from (values ('{{{1, 2, 3}, {44, 55, 66}}}')) as t(arr); ---- -{44, 55, 66} 66 66 +66 66 # Test cast from array to string query T diff --git a/e2e_test/batch/duckdb/select/test_multi_column_reference.slt.part b/e2e_test/batch/duckdb/select/test_multi_column_reference.slt.part index 23b252b33f32d..8b3e5b0e63d39 100644 --- a/e2e_test/batch/duckdb/select/test_multi_column_reference.slt.part +++ b/e2e_test/batch/duckdb/select/test_multi_column_reference.slt.part @@ -129,7 +129,7 @@ INSERT INTO t.t VALUES (ROW(ROW(ROW(42)))); query I SELECT (t.t.t).t FROM t.t; ---- -((42)) +("(42)") query I SELECT (t.t.t).t.t FROM t.t; diff --git a/e2e_test/batch/order/test_order_struct.slt.part b/e2e_test/batch/order/test_order_struct.slt.part index 0ceedf969251e..de55d2f964fc7 100644 --- a/e2e_test/batch/order/test_order_struct.slt.part +++ b/e2e_test/batch/order/test_order_struct.slt.part @@ -12,10 +12,10 @@ select * from t order by 1; ---- (1,2) (1,3) -(1,NULL) +(1,) (2,1) (2,2) -(NULL,NULL) +(,) statement ok drop table t; @@ -29,10 +29,10 @@ insert into t values (('abc',('bcd',2)),1), (('abc',('bcd',2)),2), (('a',('bcbcd query TTII select * from t order by 1 DESC, 2 ASC; ---- -(b,(a,2)) 2 -(abc,(bcd,2)) 1 -(abc,(bcd,2)) 2 -(a,(bcbcd,2)) 2 +(b,"(a,2)") 2 +(abc,"(bcd,2)") 1 +(abc,"(bcd,2)") 2 +(a,"(bcbcd,2)") 2 statement ok drop table t; diff --git a/e2e_test/batch/types/jsonb.slt.part b/e2e_test/batch/types/jsonb.slt.part index 9812f254ff0cc..949b18af315c0 100644 --- a/e2e_test/batch/types/jsonb.slt.part +++ b/e2e_test/batch/types/jsonb.slt.part @@ -227,14 +227,8 @@ b "bar" query T select jsonb_each('{"a":"foo", "b":"bar"}'::jsonb); ---- -(a,"foo") -(b,"bar") - -# FIXME: the output format is inconsistent with pg -# https://github.com/risingwavelabs/risingwave/issues/4769 -# -# (a,"""foo""") -# (b,"""bar""") +(a,"""foo""") +(b,"""bar""") statement error cannot deconstruct select * from jsonb_each('null'::jsonb) diff --git a/e2e_test/batch/types/struct/nested_structs.slt.part b/e2e_test/batch/types/struct/nested_structs.slt.part index 9ba4b0f2718b6..cff09a75e3623 100644 --- a/e2e_test/batch/types/struct/nested_structs.slt.part +++ b/e2e_test/batch/types/struct/nested_structs.slt.part @@ -13,7 +13,7 @@ INSERT INTO a VALUES (ROW(ROW(3), 4)) query I SELECT * FROM a ---- -((3),4) +("(3)",4) query I SELECT ((c).i).a FROM a @@ -26,7 +26,7 @@ INSERT INTO a VALUES (NULL) query I rowsort SELECT * FROM a ---- -((3),4) +("(3)",4) NULL query I rowsort @@ -48,10 +48,10 @@ INSERT INTO a VALUES (ROW(NULL, 1)) query I rowsort SELECT * FROM a ---- -((1),NULL) -((3),4) -((NULL),1) -(NULL,1) +("()",1) +("(1)",) +("(3)",4) +(,1) NULL # nested struct mismatch on insertion diff --git a/e2e_test/batch/types/struct/struct.slt.part b/e2e_test/batch/types/struct/struct.slt.part index 58c56bc81fb97..3829838c4e1fe 100644 --- a/e2e_test/batch/types/struct/struct.slt.part +++ b/e2e_test/batch/types/struct/struct.slt.part @@ -25,8 +25,8 @@ select (v2).* from st; query II rowsort select * from st; ---- -1 (1,(1,2)) -1 (1,(1,3)) +1 (1,"(1,2)") +1 (1,"(1,3)") query I select (v2).v2.v1 from st; @@ -43,22 +43,22 @@ select (v2).v2 from st; query II select * from st where v2 = (1,(1,3)); ---- -1 (1,(1,3)) +1 (1,"(1,3)") query II select * from st where v2 != (1,(1,3)); ---- -1 (1,(1,2)) +1 (1,"(1,2)") query II select * from st where (v2).v2 < (1,3); ---- -1 (1,(1,2)) +1 (1,"(1,2)") query II select * from st where (v2).v2 > (1,2); ---- -1 (1,(1,3)) +1 (1,"(1,3)") query I select max((v2).v2) from st; @@ -82,7 +82,7 @@ insert into st values(1,(1,(1,null))); query II select * from st; ---- -1 (1,(1,NULL)) +1 (1,"(1,)") statement ok drop table st; @@ -90,7 +90,7 @@ drop table st; query T select Row('foo', 'bar', null); ---- -(foo,bar,NULL) +(foo,bar,) query T select Row(); @@ -100,7 +100,7 @@ select Row(); query T select Row(null); ---- -(NULL) +() statement ok create table t (v1 int); @@ -136,3 +136,29 @@ select * from t where Row(1,v1*2) > Row(1,2); statement ok drop table t; + +# row to text +query TTTT +select Row('a'), Row(''), Row('"'), Row(' a '), Row('a b'); +---- +(a) ("") ("""") (" a ") ("a b") + +query TTT +select Row('{}'), Row('[]'), Row('()'), Row(','); +---- +({}) ([]) ("()") (",") + +query TTT +select Row(NULL), Row(NULL, NULL), Row('null'); +---- +() (,) (null) + +query TTT +select Row(Array[] :: varchar[]), Row(Array[1] :: varchar[]), Row(Array[1,2] :: varchar[]); +---- +({}) ({1}) ("{1,2}") + +query T +select Row(Array['"'] :: varchar[]); +---- +("{""\\""""}") diff --git a/e2e_test/batch/types/struct/struct_case.slt.part b/e2e_test/batch/types/struct/struct_case.slt.part index 3200d7b05b35d..88b3a4c8ebd2b 100644 --- a/e2e_test/batch/types/struct/struct_case.slt.part +++ b/e2e_test/batch/types/struct/struct_case.slt.part @@ -24,7 +24,7 @@ NULL query I SELECT CASE WHEN 1=0 THEN NULL ELSE ROW(NULL) END ---- -(NULL) +() # now with a table query II diff --git a/e2e_test/batch/types/struct/struct_cast.slt.part b/e2e_test/batch/types/struct/struct_cast.slt.part index 77ac022dd9e09..fb24123c6441d 100644 --- a/e2e_test/batch/types/struct/struct_cast.slt.part +++ b/e2e_test/batch/types/struct/struct_cast.slt.part @@ -13,12 +13,12 @@ SELECT ROW(1,2)::STRUCT; query I SELECT (NULL, 'hello')::STRUCT; ---- -(NULL,hello) +(,hello) query I SELECT (NULL, NULL)::STRUCT; ---- -(NULL,NULL) +(,) query I SELECT NULL::STRUCT; @@ -29,7 +29,7 @@ NULL query I SELECT (NULL, NULL)::STRUCT; ---- -(NULL,NULL) +(,) query I SELECT ((NULL, NULL)::STRUCT).i; @@ -45,22 +45,22 @@ NULL query I SELECT (1, (2, 3))::STRUCT>; ---- -(1,(2,3)) +(1,"(2,3)") query I SELECT (1, (NULL, 3))::STRUCT>; ---- -(1,(NULL,3)) +(1,"(,3)") query I SELECT (1, (2, NULL))::STRUCT>; ---- -(1,(2,NULL)) +(1,"(2,)") query I SELECT (1, NULL)::STRUCT>; ---- -(1,NULL) +(1,) # cast and extract query I @@ -83,9 +83,9 @@ INSERT INTO structs VALUES ((1,2)), ((null, 2)), ((1, null)), (NULL) query I rowsort SELECT s FROM structs; ---- +(,2) +(1,) (1,2) -(1,NULL) -(NULL,2) NULL statement ok diff --git a/e2e_test/batch/types/struct/struct_cross_product.slt.part b/e2e_test/batch/types/struct/struct_cross_product.slt.part index 2c45edf96785a..7046f8cbec65a 100644 --- a/e2e_test/batch/types/struct/struct_cross_product.slt.part +++ b/e2e_test/batch/types/struct/struct_cross_product.slt.part @@ -14,45 +14,45 @@ insert into t values (1, ROW(ROW(3, 7), ARRAY[1, 2, 3])), (2, NULL), (3, ROW(NUL query IIII SELECT * FROM t AS v, t AS w ORDER BY v.v1, w.v1; ---- -1 ((3,7),{1,2,3}) 1 ((3,7),{1,2,3}) -1 ((3,7),{1,2,3}) 2 NULL -1 ((3,7),{1,2,3}) 3 (NULL,{4,5,NULL}) -2 NULL 1 ((3,7),{1,2,3}) +1 ("(3,7)","{1,2,3}") 1 ("(3,7)","{1,2,3}") +1 ("(3,7)","{1,2,3}") 2 NULL +1 ("(3,7)","{1,2,3}") 3 (,"{4,5,NULL}") +2 NULL 1 ("(3,7)","{1,2,3}") 2 NULL 2 NULL -2 NULL 3 (NULL,{4,5,NULL}) -3 (NULL,{4,5,NULL}) 1 ((3,7),{1,2,3}) -3 (NULL,{4,5,NULL}) 2 NULL -3 (NULL,{4,5,NULL}) 3 (NULL,{4,5,NULL}) +2 NULL 3 (,"{4,5,NULL}") +3 (,"{4,5,NULL}") 1 ("(3,7)","{1,2,3}") +3 (,"{4,5,NULL}") 2 NULL +3 (,"{4,5,NULL}") 3 (,"{4,5,NULL}") query IIII SELECT * FROM t v, t w WHERE v.v1 >= w.v1 ORDER BY v.v1, w.v1; ---- -1 ((3,7),{1,2,3}) 1 ((3,7),{1,2,3}) -2 NULL 1 ((3,7),{1,2,3}) +1 ("(3,7)","{1,2,3}") 1 ("(3,7)","{1,2,3}") +2 NULL 1 ("(3,7)","{1,2,3}") 2 NULL 2 NULL -3 (NULL,{4,5,NULL}) 1 ((3,7),{1,2,3}) -3 (NULL,{4,5,NULL}) 2 NULL -3 (NULL,{4,5,NULL}) 3 (NULL,{4,5,NULL}) +3 (,"{4,5,NULL}") 1 ("(3,7)","{1,2,3}") +3 (,"{4,5,NULL}") 2 NULL +3 (,"{4,5,NULL}") 3 (,"{4,5,NULL}") query IIII SELECT * FROM t v, t w WHERE v.v1 <> w.v1 ORDER BY v.v1, w.v1; ---- -1 ((3,7),{1,2,3}) 2 NULL -1 ((3,7),{1,2,3}) 3 (NULL,{4,5,NULL}) -2 NULL 1 ((3,7),{1,2,3}) -2 NULL 3 (NULL,{4,5,NULL}) -3 (NULL,{4,5,NULL}) 1 ((3,7),{1,2,3}) -3 (NULL,{4,5,NULL}) 2 NULL +1 ("(3,7)","{1,2,3}") 2 NULL +1 ("(3,7)","{1,2,3}") 3 (,"{4,5,NULL}") +2 NULL 1 ("(3,7)","{1,2,3}") +2 NULL 3 (,"{4,5,NULL}") +3 (,"{4,5,NULL}") 1 ("(3,7)","{1,2,3}") +3 (,"{4,5,NULL}") 2 NULL query IIII SELECT * FROM t v, t w WHERE v.v1 <> w.v1 OR v.v1 > w.v1 ORDER BY v.v1, w.v1; ---- -1 ((3,7),{1,2,3}) 2 NULL -1 ((3,7),{1,2,3}) 3 (NULL,{4,5,NULL}) -2 NULL 1 ((3,7),{1,2,3}) -2 NULL 3 (NULL,{4,5,NULL}) -3 (NULL,{4,5,NULL}) 1 ((3,7),{1,2,3}) -3 (NULL,{4,5,NULL}) 2 NULL +1 ("(3,7)","{1,2,3}") 2 NULL +1 ("(3,7)","{1,2,3}") 3 (,"{4,5,NULL}") +2 NULL 1 ("(3,7)","{1,2,3}") +2 NULL 3 (,"{4,5,NULL}") +3 (,"{4,5,NULL}") 1 ("(3,7)","{1,2,3}") +3 (,"{4,5,NULL}") 2 NULL statement ok drop table t; diff --git a/e2e_test/ddl/alter_rename_relation.slt b/e2e_test/ddl/alter_rename_relation.slt index df085a273b0a9..3e19d08084d2f 100644 --- a/e2e_test/ddl/alter_rename_relation.slt +++ b/e2e_test/ddl/alter_rename_relation.slt @@ -158,8 +158,8 @@ SELECT * from v3 query IIII rowsort SELECT * from v4 ---- -1 (1,(1,2)) 1 (1,(1,2)) -2 (2,(2,4)) 2 (2,(2,4)) +1 (1,"(1,2)") 1 (1,"(1,2)") +2 (2,"(2,4)") 2 (2,"(2,4)") statement ok CREATE MATERIALIZED VIEW mv4 AS SELECT * FROM src; diff --git a/e2e_test/sink/kafka/avro.slt b/e2e_test/sink/kafka/avro.slt index 88909135b9e53..a9040c93f793f 100644 --- a/e2e_test/sink/kafka/avro.slt +++ b/e2e_test/sink/kafka/avro.slt @@ -59,8 +59,8 @@ select time_micros_field, time_millis_field from from_kafka order by string_field; ---- -t Rising \x6130 3.5 4.25 22 23 NULL {{NULL,3},NULL,{7,NULL,2}} 2006-01-02 22:04:05+00:00 NULL NULL 12:34:56.123456 NULL -f Wave \x5a4446 1.5 NULL 11 12 (NULL,foo) NULL NULL 2006-01-02 22:04:05+00:00 2021-04-01 NULL 23:45:16.654 +t Rising \x6130 3.5 4.25 22 23 NULL {{NULL,3},NULL,{7,NULL,2}} 2006-01-02 22:04:05+00:00 NULL NULL 12:34:56.123456 NULL +f Wave \x5a4446 1.5 NULL 11 12 (,foo) NULL NULL 2006-01-02 22:04:05+00:00 2021-04-01 NULL 23:45:16.654 statement error SchemaFetchError create sink sink_err from into_kafka with ( diff --git a/e2e_test/sink/kafka/protobuf.slt b/e2e_test/sink/kafka/protobuf.slt index 9510c6e7fbf8b..83e92fd431a0d 100644 --- a/e2e_test/sink/kafka/protobuf.slt +++ b/e2e_test/sink/kafka/protobuf.slt @@ -63,7 +63,7 @@ select timestamp_field, oneof_int32 from from_kafka order by string_field; ---- -t Rising \x6130 3.5 4.25 22 23 24 0 26 27 (1,) {4,0,4} (1136239445,0) 42 +t Rising \x6130 3.5 4.25 22 23 24 0 26 27 (1,"") {4,0,4} (1136239445,0) 42 f Wave \x5a4446 1.5 0 11 12 13 14 15 16 (4,foo) {} (0,0) 0 statement error failed to read file diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index ff1178cb9c156..941d6ae98c0ae 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -629,17 +629,17 @@ select id, sequence_id, name, score, avg_score, is_lasted, entrance_date, birthd query ITITT select id, code, timestamp, xfas, contacts, sex from s10; ---- -100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ({1xxx,2xxx},{1xxx,2xxx}) MALE +100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ("{1xxx,2xxx}","{1xxx,2xxx}") MALE query ITITT select id, code, timestamp, xfas, contacts, sex from s11; ---- -0 abc 1473305798 {"(0,200,127.0.0.1)","(1,400,127.0.0.2)"} ({1xxx,2xxx},{1xxx,2xxx}) MALE +0 abc 1473305798 {"(0,200,127.0.0.1)","(1,400,127.0.0.2)"} ("{1xxx,2xxx}","{1xxx,2xxx}") MALE query ITITT select id, code, timestamp, xfas, contacts, jsonb from s12; ---- -100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ({1xxx,2xxx},{1xxx,2xxx}) {"blockNumber": 16938734} +100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ("{1xxx,2xxx}","{1xxx,2xxx}") {"blockNumber": 16938734} query ITIT select * from s13 order by id; diff --git a/e2e_test/source/basic/kafka_batch.slt b/e2e_test/source/basic/kafka_batch.slt index a1b7690a36c45..525031684166c 100644 --- a/e2e_test/source/basic/kafka_batch.slt +++ b/e2e_test/source/basic/kafka_batch.slt @@ -203,7 +203,7 @@ drop source s6 query ITITT select id, code, timestamp, xfas, contacts from s7; ---- -100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ({1xxx,2xxx},{1xxx,2xxx}) +100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ("{1xxx,2xxx}","{1xxx,2xxx}") statement ok drop source s7 diff --git a/e2e_test/source/basic/old_row_format_syntax/kafka.slt b/e2e_test/source/basic/old_row_format_syntax/kafka.slt index aeba9b8667d78..3d2e4719d744d 100644 --- a/e2e_test/source/basic/old_row_format_syntax/kafka.slt +++ b/e2e_test/source/basic/old_row_format_syntax/kafka.slt @@ -583,17 +583,17 @@ select id, sequence_id, name, score, avg_score, is_lasted, entrance_date, birthd query ITITT select id, code, timestamp, xfas, contacts, sex from s10; ---- -100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ({1xxx,2xxx},{1xxx,2xxx}) MALE +100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ("{1xxx,2xxx}","{1xxx,2xxx}") MALE query ITITT select id, code, timestamp, xfas, contacts, sex from s11; ---- -0 abc 1473305798 {"(0,200,127.0.0.1)","(1,400,127.0.0.2)"} ({1xxx,2xxx},{1xxx,2xxx}) MALE +0 abc 1473305798 {"(0,200,127.0.0.1)","(1,400,127.0.0.2)"} ("{1xxx,2xxx}","{1xxx,2xxx}") MALE query ITITT select id, code, timestamp, xfas, contacts, jsonb from s12; ---- -100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ({1xxx,2xxx},{1xxx,2xxx}) {"blockNumber": 16938734} +100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ("{1xxx,2xxx}","{1xxx,2xxx}") {"blockNumber": 16938734} query ITIT select * from s13 order by id; diff --git a/e2e_test/source/basic/old_row_format_syntax/kafka_batch.slt b/e2e_test/source/basic/old_row_format_syntax/kafka_batch.slt index 5ab2f2dbce15f..7a1495470f253 100644 --- a/e2e_test/source/basic/old_row_format_syntax/kafka_batch.slt +++ b/e2e_test/source/basic/old_row_format_syntax/kafka_batch.slt @@ -181,7 +181,7 @@ drop source s6 query ITITT select id, code, timestamp, xfas, contacts from s7; ---- -100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ({1xxx,2xxx},{1xxx,2xxx}) +100 abc 1473305798 {"(0,200,10.0.0.1)","(1,400,10.0.0.2)"} ("{1xxx,2xxx}","{1xxx,2xxx}") statement ok drop source s7 diff --git a/e2e_test/streaming/struct_table.slt b/e2e_test/streaming/struct_table.slt index 462818456b909..e254d85366678 100644 --- a/e2e_test/streaming/struct_table.slt +++ b/e2e_test/streaming/struct_table.slt @@ -41,10 +41,10 @@ flush; query II select * from t1 order by v1; ---- -1 (2,(1,0)) -2 (5,(4,1)) -3 (6,(3,4)) -4 (3,(2,2)) +1 (2,"(1,0)") +2 (5,"(4,1)") +3 (6,"(3,4)") +4 (3,"(2,2)") statement ok create materialized view mv3 as select * from t1 order by (v2).v3; diff --git a/e2e_test/udf/udf.slt b/e2e_test/udf/udf.slt index 7ebdf49d0b20d..62979196e74ca 100644 --- a/e2e_test/udf/udf.slt +++ b/e2e_test/udf/udf.slt @@ -132,10 +132,10 @@ select jsonb_array_identity(ARRAY[null, '1'::jsonb, '"str"'::jsonb, '{}'::jsonb] query T select jsonb_array_struct_identity(ROW(ARRAY[null, '1'::jsonb, '"str"'::jsonb, '{}'::jsonb], 4)::struct); ---- -({NULL,1,"\"str\"","{}"},4) +("{NULL,1,""\\""str\\"""",""{}""}",4) query T -select return_all( +select (return_all( true, 1 ::smallint, 1 ::int, @@ -150,12 +150,12 @@ select return_all( 'string', 'bytes'::bytea, '{"key":1}'::jsonb -); +)).*; ---- -(t,1,1,1,1,1,1234567890123456789012345678,2023-06-01,01:02:03.456789,2023-06-01 01:02:03.456789,1 mon 2 days 00:00:03,string,\x6279746573,{"key": 1}) +t 1 1 1 1 1 1234567890123456789012345678 2023-06-01 01:02:03.456789 2023-06-01 01:02:03.456789 1 mon 2 days 00:00:03 string \x6279746573 {"key": 1} query T -select return_all_arrays( +select (return_all_arrays( array[null, true], array[null, 1 ::smallint], array[null, 1 ::int], @@ -170,9 +170,9 @@ select return_all_arrays( array[null, 'string'], array[null, 'bytes'::bytea], array[null, '{"key":1}'::jsonb] -); +)).*; ---- -({NULL,t},{NULL,1},{NULL,1},{NULL,1},{NULL,1},{NULL,1},{NULL,1234567890123456789012345678},{NULL,2023-06-01},{NULL,01:02:03.456789},{NULL,"2023-06-01 01:02:03.456789"},{NULL,"1 mon 2 days 00:00:03"},{NULL,string},{NULL,"\\x6279746573"},{NULL,"{\"key\": 1}"}) +{NULL,t} {NULL,1} {NULL,1} {NULL,1} {NULL,1} {NULL,1} {NULL,1234567890123456789012345678} {NULL,2023-06-01} {NULL,01:02:03.456789} {NULL,"2023-06-01 01:02:03.456789"} {NULL,"1 mon 2 days 00:00:03"} {NULL,string} {NULL,"\\x6279746573"} {NULL,"{\"key\": 1}"} query I select series(5); diff --git a/src/common/src/array/list_array.rs b/src/common/src/array/list_array.rs index 7eaaffff98534..2bc7772e19ee3 100644 --- a/src/common/src/array/list_array.rs +++ b/src/common/src/array/list_array.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::borrow::Cow; use std::cmp::Ordering; use std::fmt; use std::fmt::Debug; @@ -31,7 +32,8 @@ use crate::buffer::{Bitmap, BitmapBuilder}; use crate::estimate_size::EstimateSize; use crate::row::Row; use crate::types::{ - hash_datum, DataType, Datum, DatumRef, DefaultOrd, Scalar, ScalarRefImpl, ToDatumRef, ToText, + hash_datum, DataType, Datum, DatumRef, DefaultOrd, Scalar, ScalarImpl, ScalarRefImpl, + ToDatumRef, ToText, }; use crate::util::memcmp_encoding; use crate::util::value_encoding::estimate_serialize_datum_size; @@ -624,6 +626,235 @@ impl<'a> From<&'a ListValue> for ListRef<'a> { } } +impl ListValue { + /// Construct an array from literal string. + pub fn from_str(input: &str, data_type: &DataType) -> Result { + struct Parser<'a> { + input: &'a str, + data_type: &'a DataType, + } + + impl Parser<'_> { + /// Parse a datum. + fn parse(&mut self) -> Result { + self.skip_whitespace(); + if self.data_type.is_array() { + if self.try_parse_null() { + return Ok(None); + } + Ok(Some(self.parse_array()?.into())) + } else { + self.parse_value() + } + } + + /// Parse an array. + fn parse_array(&mut self) -> Result { + self.skip_whitespace(); + if !self.try_consume('{') { + return Err("Array value must start with \"{\"".to_string()); + } + self.skip_whitespace(); + if self.try_consume('}') { + return Ok(ListValue::new(vec![])); + } + let mut elems = Vec::new(); + loop { + let mut parser = Self { + input: self.input, + data_type: self.data_type.as_list(), + }; + elems.push(parser.parse()?); + self.input = parser.input; + + // expect ',' or '}' + self.skip_whitespace(); + match self.peek() { + Some(',') => { + self.try_consume(','); + } + Some('}') => { + self.try_consume('}'); + break; + } + None => return Err(Self::eoi()), + _ => return Err("Unexpected array element.".to_string()), + } + } + Ok(ListValue::new(elems)) + } + + /// Parse a non-array value. + fn parse_value(&mut self) -> Result { + if self.peek() == Some('"') { + return Ok(Some(self.parse_quoted()?)); + } + // peek until the next unescaped ',' or '}' + let mut chars = self.input.char_indices(); + let mut has_escape = false; + let s = loop { + match chars.next().ok_or_else(Self::eoi)? { + (_, '\\') => { + has_escape = true; + chars.next().ok_or_else(Self::eoi)?; + } + (i, c @ ',' | c @ '}') => { + let s = &self.input[..i]; + // consume the value and leave the ',' or '}' for parent + self.input = &self.input[i..]; + + break if has_escape { + Cow::Owned(Self::unescape_trim_end(s)) + } else { + let trimmed = s.trim_end(); + if trimmed.is_empty() { + return Err(format!("Unexpected \"{c}\" character.")); + } + if trimmed.eq_ignore_ascii_case("null") { + return Ok(None); + } + Cow::Borrowed(trimmed) + }; + } + (_, '{') => return Err("Unexpected \"{\" character.".to_string()), + (_, '"') => return Err("Unexpected array element.".to_string()), + _ => {} + } + }; + Ok(Some(ScalarImpl::from_literal(&s, self.data_type)?)) + } + + /// Parse a double quoted non-array value. + fn parse_quoted(&mut self) -> Result { + assert!(self.try_consume('"')); + // peek until the next unescaped '"' + let mut chars = self.input.char_indices(); + let mut has_escape = false; + let s = loop { + match chars.next().ok_or_else(Self::eoi)? { + (_, '\\') => { + has_escape = true; + chars.next().ok_or_else(Self::eoi)?; + } + (i, '"') => { + let s = &self.input[..i]; + self.input = &self.input[i + 1..]; + break if has_escape { + Cow::Owned(Self::unescape(s)) + } else { + Cow::Borrowed(s) + }; + } + _ => {} + } + }; + ScalarImpl::from_literal(&s, self.data_type) + } + + /// Unescape a string. + fn unescape(s: &str) -> String { + let mut unescaped = String::with_capacity(s.len()); + let mut chars = s.chars(); + while let Some(mut c) = chars.next() { + if c == '\\' { + c = chars.next().unwrap(); + } + unescaped.push(c); + } + unescaped + } + + /// Unescape a string and trim the trailing whitespaces. + /// + /// Example: `"\ " -> " "` + fn unescape_trim_end(s: &str) -> String { + let mut unescaped = String::with_capacity(s.len()); + let mut chars = s.chars(); + let mut len_after_last_escaped_char = 0; + while let Some(mut c) = chars.next() { + if c == '\\' { + c = chars.next().unwrap(); + unescaped.push(c); + len_after_last_escaped_char = unescaped.len(); + } else { + unescaped.push(c); + } + } + let l = unescaped[len_after_last_escaped_char..].trim_end().len(); + unescaped.truncate(len_after_last_escaped_char + l); + unescaped + } + + /// Consume the next 4 characters if it matches "null". + /// + /// Note: We don't use this function when parsing non-array values. + /// Because we can't decide whether it is a null value or a string starts with "null". + /// Consider this case: `{null value}` => `["null value"]` + fn try_parse_null(&mut self) -> bool { + if let Some(s) = self.input.get(..4) && s.eq_ignore_ascii_case("null") { + let next_char = self.input[4..].chars().next(); + match next_char { + None | Some(',' | '}') => {} + Some(c) if c.is_ascii_whitespace() => {} + // following normal characters + _ => return false, + } + self.input = &self.input[4..]; + true + } else { + false + } + } + + /// Consume the next character if it matches `c`. + fn try_consume(&mut self, c: char) -> bool { + if self.peek() == Some(c) { + self.input = &self.input[c.len_utf8()..]; + true + } else { + false + } + } + + /// Expect end of input. + fn expect_end(&mut self) -> Result<(), String> { + self.skip_whitespace(); + match self.peek() { + Some(_) => Err("Junk after closing right brace.".to_string()), + None => Ok(()), + } + } + + /// Skip whitespaces. + fn skip_whitespace(&mut self) { + self.input = match self + .input + .char_indices() + .find(|(_, c)| !c.is_ascii_whitespace()) + { + Some((i, _)) => &self.input[i..], + None => "", + }; + } + + /// Peek the next character. + fn peek(&self) -> Option { + self.input.chars().next() + } + + /// Return the error message for unexpected end of input. + fn eoi() -> String { + "Unexpected end of input.".into() + } + } + + let mut parser = Parser { input, data_type }; + let array = parser.parse_array()?; + parser.expect_end()?; + Ok(array) + } +} + #[cfg(test)] mod tests { use more_asserts::{assert_gt, assert_lt}; @@ -1035,4 +1266,53 @@ mod tests { let scalar = list_ref.get(1).unwrap(); assert_eq!(scalar, Some(types::ScalarRefImpl::Int32(5))); } + + #[test] + fn test_from_to_literal() { + #[track_caller] + fn test(typestr: &str, input: &str, output: Option<&str>) { + let datatype: DataType = typestr.parse().unwrap(); + let list = ListValue::from_str(input, &datatype).unwrap(); + let actual = list.as_scalar_ref().to_text(); + let output = output.unwrap_or(input); + assert_eq!(actual, output); + } + + #[track_caller] + fn test_err(typestr: &str, input: &str, err: &str) { + let datatype: DataType = typestr.parse().unwrap(); + let actual_err = ListValue::from_str(input, &datatype).unwrap_err(); + assert_eq!(actual_err, err); + } + + test("varchar[]", "{}", None); + test("varchar[]", "{1 2}", Some(r#"{"1 2"}"#)); + test("varchar[]", "{🥵,🤡}", None); + test("varchar[]", r#"{aa\\bb}"#, Some(r#"{"aa\\bb"}"#)); + test("int[]", "{1,2,3}", None); + test("varchar[]", r#"{"1,2"}"#, None); + test("varchar[]", r#"{1, ""}"#, Some(r#"{1,""}"#)); + test("varchar[]", r#"{"\""}"#, None); + test("varchar[]", r#"{\ }"#, Some(r#"{" "}"#)); + test("varchar[]", r#"{\\ }"#, Some(r#"{"\\"}"#)); + test("varchar[]", "{nulla}", None); + test("varchar[]", "{null a}", Some(r#"{"null a"}"#)); + test( + "varchar[]", + r#"{"null", "NULL", null, NuLL}"#, + Some(r#"{"null","NULL",NULL,NULL}"#), + ); + test("varchar[][]", "{{1, 2, 3}, null }", Some("{{1,2,3},NULL}")); + test( + "varchar[][][]", + "{{{1, 2, 3}}, {{4, 5, 6}}}", + Some("{{{1,2,3}},{{4,5,6}}}"), + ); + test_err("varchar[]", "()", r#"Array value must start with "{""#); + test_err("varchar[]", "{1,", r#"Unexpected end of input."#); + test_err("varchar[]", "{1,}", r#"Unexpected "}" character."#); + test_err("varchar[]", "{1,,3}", r#"Unexpected "," character."#); + test_err("varchar[]", r#"{"a""b"}"#, r#"Unexpected array element."#); + test_err("varchar[]", r#"{}{"#, r#"Junk after closing right brace."#); + } } diff --git a/src/common/src/array/struct_array.rs b/src/common/src/array/struct_array.rs index 27c5d8c0dc237..492a7ca3aed1d 100644 --- a/src/common/src/array/struct_array.rs +++ b/src/common/src/array/struct_array.rs @@ -12,9 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use core::fmt; use std::cmp::Ordering; -use std::fmt::Debug; +use std::fmt::{self, Debug, Write}; use std::hash::Hash; use std::sync::Arc; @@ -411,6 +410,7 @@ impl Debug for StructRef<'_> { impl ToText for StructRef<'_> { fn write(&self, f: &mut W) -> std::fmt::Result { + let mut raw_text = String::new(); iter_fields_ref!(*self, it, { write!(f, "(")?; let mut is_first = true; @@ -420,7 +420,12 @@ impl ToText for StructRef<'_> { } else { write!(f, ",")?; } - ToText::write(&x, f)?; + // print nothing for null + if x.is_some() { + raw_text.clear(); + x.write(&mut raw_text)?; + quote_if_need(&raw_text, f)?; + } } write!(f, ")") }) @@ -434,6 +439,32 @@ impl ToText for StructRef<'_> { } } +/// Double quote a string if it contains any special characters. +fn quote_if_need(input: &str, writer: &mut impl Write) -> std::fmt::Result { + if !input.is_empty() // non-empty + && !input.contains([ + '"', '\\', '(', ')', ',', + // PostgreSQL `array_isspace` includes '\x0B' but rust + // [`char::is_ascii_whitespace`] does not. + ' ', '\t', '\n', '\r', '\x0B', '\x0C', + ]) + { + return writer.write_str(input); + } + + writer.write_char('"')?; + + for ch in input.chars() { + match ch { + '"' => writer.write_str("\"\"")?, + '\\' => writer.write_str("\\\\")?, + _ => writer.write_char(ch)?, + } + } + + writer.write_char('"') +} + #[cfg(test)] mod tests { use more_asserts::assert_gt; @@ -711,4 +742,23 @@ mod tests { assert_eq!(lhs_serialized.cmp(&rhs_serialized), order); } } + + #[test] + fn test_quote() { + #[track_caller] + fn test(input: &str, quoted: &str) { + let mut actual = String::new(); + quote_if_need(input, &mut actual).unwrap(); + assert_eq!(quoted, actual); + } + test("abc", "abc"); + test("", r#""""#); + test(" x ", r#"" x ""#); + test("a b", r#""a b""#); + test(r#"a"bc"#, r#""a""bc""#); + test(r#"a\bc"#, r#""a\\bc""#); + test("{1}", "{1}"); + test("{1,2}", r#""{1,2}""#); + test(r#"{"f": 1}"#, r#""{""f"": 1}""#); + } } diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 3d62780c19b91..98ea7ee061bd6 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -997,11 +997,10 @@ impl ScalarImpl { DataType::Timestamptz => Timestamptz::from_str(s).map_err(|e| e.to_string())?.into(), DataType::Time => Time::from_str(s).map_err(|e| e.to_string())?.into(), DataType::Interval => Interval::from_str(s).map_err(|e| e.to_string())?.into(), - // Not processing list or struct literal right now. Leave it for later phase (normal backend - // evaluation). - DataType::List { .. } => return Err("not supported".into()), + DataType::List { .. } => ListValue::from_str(s, t)?.into(), + // Not processing struct literal right now. Leave it for later phase (normal backend evaluation). DataType::Struct(_) => return Err("not supported".into()), - DataType::Jsonb => return Err("not supported".into()), + DataType::Jsonb => JsonbVal::from_str(s).map_err(|e| e.to_string())?.into(), DataType::Bytea => str_to_bytea(s)?.into(), }) } diff --git a/src/expr/impl/benches/expr.rs b/src/expr/impl/benches/expr.rs index 685b5de98c1e8..1d8ab36472fb6 100644 --- a/src/expr/impl/benches/expr.rs +++ b/src/expr/impl/benches/expr.rs @@ -354,6 +354,11 @@ fn bench_expr(c: &mut Criterion) { }); } + c.bench_function("cast(character varying) -> int8[]", |bencher| { + let expr = build_from_pretty(r#"(cast:int8[] {1,"2"}:varchar)"#); + bencher.to_async(FuturesExecutor).iter(|| expr.eval(&input)) + }); + let sigs = FUNCTION_REGISTRY .iter_aggregates() .sorted_by_cached_key(|sig| format!("{sig:?}")); diff --git a/src/expr/impl/src/scalar/cast.rs b/src/expr/impl/src/scalar/cast.rs index ae2ed50472b86..0f69e4cdad5e6 100644 --- a/src/expr/impl/src/scalar/cast.rs +++ b/src/expr/impl/src/scalar/cast.rs @@ -20,7 +20,7 @@ use itertools::Itertools; use risingwave_common::array::{ListRef, ListValue, StructRef, StructValue}; use risingwave_common::cast; use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, Int256, IntoOrdered, JsonbRef, ToText, F64}; +use risingwave_common::types::{Int256, IntoOrdered, JsonbRef, ToText, F64}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::{ build_func, Context, Expression, ExpressionBoxExt, InputRefExpression, @@ -189,63 +189,9 @@ pub fn str_to_bytea(elem: &str) -> Result> { cast::str_to_bytea(elem).map_err(|err| ExprError::Parse(err.into())) } -// TODO(nanderstabel): optimize for multidimensional List. Depth can be given as a parameter to this -// function. -/// Takes a string input in the form of a comma-separated list enclosed in braces, and returns a -/// vector of strings containing the list items. -/// -/// # Examples -/// - "{1, 2, 3}" => ["1", "2", "3"] -/// - "{1, {2, 3}}" => ["1", "{2, 3}"] -fn unnest(input: &str) -> Result> { - let trimmed = input.trim(); - if !trimmed.starts_with('{') || !trimmed.ends_with('}') { - return Err(ExprError::Parse("Input must be braced".into())); - } - let trimmed = &trimmed[1..trimmed.len() - 1]; - - let mut items = Vec::new(); - let mut depth = 0; - let mut start = 0; - for (i, c) in trimmed.chars().enumerate() { - match c { - '{' => depth += 1, - '}' => depth -= 1, - ',' if depth == 0 => { - let item = trimmed[start..i].trim(); - items.push(item); - start = i + 1; - } - _ => {} - } - } - if depth != 0 { - return Err(ExprError::Parse("Unbalanced braces".into())); - } - let last = trimmed[start..].trim(); - if !last.is_empty() { - items.push(last); - } - Ok(items) -} - #[function("cast(varchar) -> anyarray", type_infer = "panic")] fn str_to_list(input: &str, ctx: &Context) -> Result { - let cast = build_func( - PbType::Cast, - ctx.return_type.as_list().clone(), - vec![InputRefExpression::new(DataType::Varchar, 0).boxed()], - ) - .unwrap(); - let mut values = vec![]; - for item in unnest(input)? { - let v = cast - .eval_row(&OwnedRow::new(vec![Some(item.to_string().into())])) // TODO: optimize - .now_or_never() - .unwrap()?; - values.push(v); - } - Ok(ListValue::new(values)) + ListValue::from_str(input, &ctx.return_type).map_err(|err| ExprError::Parse(err.into())) } /// Cast array with `source_elem_type` into array with `target_elem_type` by casting each element. @@ -352,31 +298,6 @@ mod tests { test!(general_to_text(Decimal::NaN), "NaN"); } - #[test] - fn test_unnest() { - assert_eq!(unnest("{ }").unwrap(), vec![] as Vec); - assert_eq!( - unnest("{1, 2, 3}").unwrap(), - vec!["1".to_string(), "2".to_string(), "3".to_string()] - ); - assert_eq!( - unnest("{{1, 2, 3}, {4, 5, 6}}").unwrap(), - vec!["{1, 2, 3}".to_string(), "{4, 5, 6}".to_string()] - ); - assert_eq!( - unnest("{{{1, 2, 3}}, {{4, 5, 6}}}").unwrap(), - vec!["{{1, 2, 3}}".to_string(), "{{4, 5, 6}}".to_string()] - ); - assert_eq!( - unnest("{{{1, 2, 3}, {4, 5, 6}}}").unwrap(), - vec!["{{1, 2, 3}, {4, 5, 6}}".to_string()] - ); - assert_eq!( - unnest("{{{aa, bb, cc}, {dd, ee, ff}}}").unwrap(), - vec!["{{aa, bb, cc}, {dd, ee, ff}}".to_string()] - ); - } - #[test] fn test_str_to_list() { // Empty List diff --git a/src/frontend/planner_test/tests/testdata/input/array.yaml b/src/frontend/planner_test/tests/testdata/input/array.yaml index bab19465d46e0..c08133a99251f 100644 --- a/src/frontend/planner_test/tests/testdata/input/array.yaml +++ b/src/frontend/planner_test/tests/testdata/input/array.yaml @@ -84,7 +84,7 @@ expected_outputs: - batch_plan - sql: | - select array_cat('{a}', '{{b}}') = array['a', '{b}']; + select array_cat('{a}', '{"{b}"}') = array['a', '{b}']; name: array_cat(unknown as text[], unknown as text[]) -> text[] expected_outputs: - batch_plan diff --git a/src/frontend/planner_test/tests/testdata/output/array.yaml b/src/frontend/planner_test/tests/testdata/output/array.yaml index f43fba600f885..e578ddac53071 100644 --- a/src/frontend/planner_test/tests/testdata/output/array.yaml +++ b/src/frontend/planner_test/tests/testdata/output/array.yaml @@ -115,7 +115,7 @@ batch_plan: 'BatchValues { rows: [[ARRAY[a, b]:List(Varchar)]] }' - name: array_cat(unknown as text[], unknown as text[]) -> text[] sql: | - select array_cat('{a}', '{{b}}') = array['a', '{b}']; + select array_cat('{a}', '{"{b}"}') = array['a', '{b}']; batch_plan: 'BatchValues { rows: [[true:Boolean]] }' - name: array_cat(unknown as int[], int[]) -> int[] sql: | @@ -238,19 +238,19 @@ - name: unknown to varchar[] in implicit context sql: | values (array['a', 'b']), ('{c,d}'); - logical_plan: 'LogicalValues { rows: [[Array(''a'':Varchar, ''b'':Varchar)], [''{c,d}'':Varchar::List(Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List(Varchar)] } }' + logical_plan: 'LogicalValues { rows: [[Array(''a'':Varchar, ''b'':Varchar)], [ARRAY[c, d]:List(Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List(Varchar)] } }' - name: unknown to varchar[] in assign context sql: | create table t (v1 varchar[]); insert into t values ('{c,d}'); logical_plan: |- LogicalInsert { table: t, mapping: [0:0] } - └─LogicalValues { rows: [['{c,d}':Varchar::List(Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List(Varchar)] } } + └─LogicalValues { rows: [[ARRAY[c, d]:List(Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List(Varchar)] } } - name: unknown to varchar[] in explicit context sql: | select ('{c,d}')::varchar[]; logical_plan: |- - LogicalProject { exprs: ['{c,d}':Varchar::List(Varchar) as $expr1] } + LogicalProject { exprs: [ARRAY[c, d]:List(Varchar)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: varchar[] to string in assign context sql: | @@ -287,7 +287,7 @@ sql: | select array[1] = '{1}'; logical_plan: |- - LogicalProject { exprs: [(Array(1:Int32) = '{1}':Varchar::List(Int32)) as $expr1] } + LogicalProject { exprs: [(Array(1:Int32) = ARRAY[1]:List(Int32)) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - name: compare with different type sql: | diff --git a/src/frontend/planner_test/tests/testdata/output/expr.yaml b/src/frontend/planner_test/tests/testdata/output/expr.yaml index 2074a0410b431..65df2522c81dd 100644 --- a/src/frontend/planner_test/tests/testdata/output/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/expr.yaml @@ -393,7 +393,7 @@ - sql: | select 1 < ALL('{2,3}'); logical_plan: |- - LogicalProject { exprs: [All((1:Int32 < '{2,3}':Varchar::List(Int32))) as $expr1] } + LogicalProject { exprs: [All((1:Int32 < ARRAY[2, 3]:List(Int32))) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - sql: | select 1 < SOME(null::integer[]); diff --git a/src/tests/regress/data/expected/arrays.out b/src/tests/regress/data/expected/arrays.out index 0c30298435324..678a5087b85b5 100644 --- a/src/tests/regress/data/expected/arrays.out +++ b/src/tests/regress/data/expected/arrays.out @@ -1464,25 +1464,25 @@ select '{}'::text[]; {} (1 row) -select '{{{1,2,3,4},{2,3,4,5}},{{3,4,5,6},{4,5,6,7}}}'::text[]; +select '{{{1,2,3,4},{2,3,4,5}},{{3,4,5,6},{4,5,6,7}}}'::text[][][]; text ----------------------------------------------- {{{1,2,3,4},{2,3,4,5}},{{3,4,5,6},{4,5,6,7}}} (1 row) select '{0 second ,0 second}'::interval[]; - interval ---------------- - {"@ 0","@ 0"} + interval +--------------------- + {00:00:00,00:00:00} (1 row) -select '{ { "," } , { 3 } }'::text[]; +select '{ { "," } , { 3 } }'::text[][]; text ------------- {{","},{3}} (1 row) -select ' { { " 0 second " , 0 second } }'::text[]; +select ' { { " 0 second " , 0 second } }'::text[][]; text ------------------------------- {{" 0 second ","0 second"}} diff --git a/src/tests/regress/data/sql/arrays.sql b/src/tests/regress/data/sql/arrays.sql index 30c2155742d86..a97e9e3005d07 100644 --- a/src/tests/regress/data/sql/arrays.sql +++ b/src/tests/regress/data/sql/arrays.sql @@ -251,7 +251,7 @@ SELECT ARRAY[ARRAY['hello'],ARRAY['world']]; --@ SELECT ARRAY(select f2 from arrtest_f order by f2) AS "ARRAY"; -- with nulls ---@ SELECT '{1,null,3}'::int[]; +SELECT '{1,null,3}'::int[]; SELECT ARRAY[1,NULL,3]; -- functions @@ -372,12 +372,12 @@ select 33 * any (44); -- nulls select 33 = any (null::int[]); select null::int = any ('{1,2,3}'); ---@ select 33 = any ('{1,null,3}'); ---@ select 33 = any ('{1,null,33}'); +select 33 = any ('{1,null,3}'); +select 33 = any ('{1,null,33}'); select 33 = all (null::int[]); select null::int = all ('{1,2,3}'); ---@ select 33 = all ('{1,null,3}'); ---@ select 33 = all ('{33,null,33}'); +select 33 = all ('{1,null,3}'); +select 33 = all ('{33,null,33}'); -- nulls later in the bitmap --@ SELECT -1 != ALL(ARRAY(SELECT NULLIF(g.i, 900) FROM generate_series(1,1000) g(i))); @@ -439,10 +439,10 @@ select array[]; -- all of the following should be accepted select '{}'::text[]; ---@ select '{{{1,2,3,4},{2,3,4,5}},{{3,4,5,6},{4,5,6,7}}}'::text[]; +select '{{{1,2,3,4},{2,3,4,5}},{{3,4,5,6},{4,5,6,7}}}'::text[][][]; --@ select '{0 second ,0 second}'::interval[]; ---@ select '{ { "," } , { 3 } }'::text[]; ---@ select ' { { " 0 second " , 0 second } }'::text[]; +select '{ { "," } , { 3 } }'::text[][]; +select ' { { " 0 second " , 0 second } }'::text[][]; --@ select '{ --@ 0 second, --@ @ 1 hour @ 42 minutes @ 20 seconds From 1b1950eaa51a0c89965a25bec829b07e197a291e Mon Sep 17 00:00:00 2001 From: August Date: Wed, 15 Nov 2023 17:07:35 +0800 Subject: [PATCH 07/36] chore: add some unit test for alter relation rename in catalog controller (#13438) --- src/meta/src/controller/catalog.rs | 84 ++++++++++++++++++++++++++++++ 1 file changed, 84 insertions(+) diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index cd69cdb630f40..f7eb35b10f7b1 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -1020,6 +1020,8 @@ impl CatalogController { #[cfg(test)] #[cfg(not(madsim))] mod tests { + use risingwave_meta_model_v2::ViewId; + use super::*; const TEST_DATABASE_ID: DatabaseId = 1; @@ -1115,4 +1117,86 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_alter_rename() -> MetaResult<()> { + let mgr = CatalogController::new(MetaSrvEnv::for_test().await)?; + let pb_source = PbSource { + schema_id: TEST_SCHEMA_ID as _, + database_id: TEST_DATABASE_ID as _, + name: "s1".to_string(), + owner: TEST_OWNER_ID as _, + definition: r#"CREATE SOURCE s1 (v1 int) with ( + connector = 'kafka', + topic = 'kafka_alter', + properties.bootstrap.server = 'message_queue:29092', + scan.startup.mode = 'earliest' +) FORMAT PLAIN ENCODE JSON"# + .to_string(), + ..Default::default() + }; + mgr.create_source(pb_source).await?; + let source_id: SourceId = Source::find() + .select_only() + .column(source::Column::SourceId) + .filter(source::Column::Name.eq("s1")) + .into_tuple() + .one(&mgr.inner.read().await.db) + .await? + .unwrap(); + + let pb_view = PbView { + schema_id: TEST_SCHEMA_ID as _, + database_id: TEST_DATABASE_ID as _, + name: "view_1".to_string(), + owner: TEST_OWNER_ID as _, + sql: "CREATE VIEW view_1 AS SELECT v1 FROM s1".to_string(), + dependent_relations: vec![source_id as _], + ..Default::default() + }; + mgr.create_view(pb_view).await?; + let view_id: ViewId = View::find() + .select_only() + .column(view::Column::ViewId) + .filter(view::Column::Name.eq("view_1")) + .into_tuple() + .one(&mgr.inner.read().await.db) + .await? + .unwrap(); + + mgr.alter_relation_name(ObjectType::Source, source_id, "s2") + .await?; + let source = Source::find_by_id(source_id) + .one(&mgr.inner.read().await.db) + .await? + .unwrap(); + assert_eq!(source.name, "s2"); + assert_eq!( + source.definition, + "CREATE SOURCE s2 (v1 INT) WITH (\ + connector = 'kafka', \ + topic = 'kafka_alter', \ + properties.bootstrap.server = 'message_queue:29092', \ + scan.startup.mode = 'earliest'\ +) FORMAT PLAIN ENCODE JSON" + ); + + let view = View::find_by_id(view_id) + .one(&mgr.inner.read().await.db) + .await? + .unwrap(); + assert_eq!( + view.definition, + "CREATE VIEW view_1 AS SELECT v1 FROM s2 AS s1" + ); + + mgr.drop_relation(ObjectType::Source, source_id, DropMode::Cascade) + .await?; + assert!(View::find_by_id(view_id) + .one(&mgr.inner.read().await.db) + .await? + .is_none()); + + Ok(()) + } } From a62493618c19abe7c6d43ba765aa8c613a25e5e1 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed, 15 Nov 2023 18:08:07 +0800 Subject: [PATCH 08/36] refactor(datagen): generalize `TimestampField` to `ChronoField` (#13439) --- src/common/src/field_generator/mod.rs | 6 +- src/common/src/field_generator/timestamp.rs | 78 +++++++++++-------- .../src/source/datagen/source/reader.rs | 2 +- 3 files changed, 48 insertions(+), 38 deletions(-) diff --git a/src/common/src/field_generator/mod.rs b/src/common/src/field_generator/mod.rs index 9d61c01e41861..02a958aa0ebec 100644 --- a/src/common/src/field_generator/mod.rs +++ b/src/common/src/field_generator/mod.rs @@ -26,7 +26,7 @@ pub use timestamp::*; pub use varchar::*; use crate::array::{ListValue, StructValue}; -use crate::types::{DataType, Datum, ScalarImpl}; +use crate::types::{DataType, Datum, ScalarImpl, Timestamp}; pub const DEFAULT_MIN: i16 = i16::MIN; pub const DEFAULT_MAX: i16 = i16::MAX; @@ -95,7 +95,7 @@ pub enum FieldGeneratorImpl { VarcharRandomVariableLength(VarcharRandomVariableLengthField), VarcharRandomFixedLength(VarcharRandomFixedLengthField), VarcharConstant, - Timestamp(TimestampField), + Timestamp(ChronoField), Struct(Vec<(String, FieldGeneratorImpl)>), List(Box, usize), } @@ -181,7 +181,7 @@ impl FieldGeneratorImpl { max_past_mode: Option, seed: u64, ) -> Result { - Ok(FieldGeneratorImpl::Timestamp(TimestampField::new( + Ok(FieldGeneratorImpl::Timestamp(ChronoField::new( base, max_past, max_past_mode, diff --git a/src/common/src/field_generator/timestamp.rs b/src/common/src/field_generator/timestamp.rs index 000f806c66d50..54c55de273965 100644 --- a/src/common/src/field_generator/timestamp.rs +++ b/src/common/src/field_generator/timestamp.rs @@ -18,26 +18,19 @@ use chrono::{Duration, DurationRound}; use humantime::parse_duration; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; -use serde_json::{json, Value}; +use serde_json::Value; use tracing::debug; use super::DEFAULT_MAX_PAST; use crate::types::{Datum, Scalar, Timestamp}; -#[derive(Debug)] -enum LocalNow { - Relative, - Absolute(NaiveDateTime), -} - -pub struct TimestampField { - base: Option>, +pub struct ChronoField { max_past: Duration, - local_now: LocalNow, + absolute_base: Option, seed: u64, } -impl TimestampField { +impl ChronoField { pub fn new( base: Option>, max_past_option: Option, @@ -45,14 +38,8 @@ impl TimestampField { seed: u64, ) -> Result { let local_now = match max_past_mode.as_deref() { - Some("relative") => LocalNow::Relative, - _ => { - LocalNow::Absolute( - Local::now() - .naive_local() - .duration_round(Duration::microseconds(1))?, - ) // round to 1 us std duration - } + Some("relative") => None, + _ => Some(T::from_now()), }; let max_past = if let Some(max_past_option) = max_past_option { @@ -63,36 +50,59 @@ impl TimestampField { }; debug!(?local_now, ?max_past, "parse timestamp field option"); Ok(Self { - base, // convert to chrono::Duration max_past: chrono::Duration::from_std(max_past)?, - local_now, + absolute_base: base.map(T::from_base).or(local_now), seed, }) } - fn generate_data(&mut self, offset: u64) -> NaiveDateTime { + fn generate_data(&mut self, offset: u64) -> T { let milliseconds = self.max_past.num_milliseconds(); let mut rng = StdRng::seed_from_u64(offset ^ self.seed); let max_milliseconds = rng.gen_range(0..=milliseconds); - let now = match self.base { - Some(base) => base.naive_local(), - None => match self.local_now { - LocalNow::Relative => Local::now() - .naive_local() - .duration_round(Duration::microseconds(1)) - .unwrap(), - LocalNow::Absolute(now) => now, - }, + let base = match self.absolute_base { + Some(base) => base, + None => T::from_now(), }; - now - Duration::milliseconds(max_milliseconds) + base.minus(Duration::milliseconds(max_milliseconds)) } pub fn generate(&mut self, offset: u64) -> Value { - json!(self.generate_data(offset).to_string()) + self.generate_data(offset).to_json() } pub fn generate_datum(&mut self, offset: u64) -> Datum { - Some(Timestamp::new(self.generate_data(offset)).to_scalar_value()) + Some(self.generate_data(offset).to_scalar_value()) + } +} + +pub trait ChronoFieldInner: std::fmt::Debug + Copy + Scalar { + fn from_now() -> Self; + fn from_base(base: DateTime) -> Self; + fn minus(&self, duration: Duration) -> Self; + fn to_json(&self) -> Value; +} + +impl ChronoFieldInner for Timestamp { + fn from_now() -> Self { + Timestamp::new( + Local::now() + .naive_local() + .duration_round(Duration::microseconds(1)) + .unwrap(), + ) + } + + fn from_base(base: DateTime) -> Self { + Timestamp::new(base.naive_local()) + } + + fn minus(&self, duration: Duration) -> Self { + Timestamp::new(self.0 - duration) + } + + fn to_json(&self) -> Value { + Value::String(self.0.to_string()) } } diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index bd9f74ee3aa9a..11cb9db08c48a 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -215,7 +215,7 @@ fn generator_from_data_type( match data_type { DataType::Timestamp => { let max_past_key = format!("fields.{}.max_past", name); - let max_past_value = fields_option_map.get(&max_past_key).map(|s| s.to_string()); + let max_past_value = fields_option_map.get(&max_past_key).cloned(); let max_past_mode_key = format!("fields.{}.max_past_mode", name); let max_past_mode_value = fields_option_map .get(&max_past_mode_key) From 4e16834d7ea5df3b8c6d16cc03f0431a34e29fbf Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 15 Nov 2023 20:01:57 +0800 Subject: [PATCH 09/36] chore(dependabot): add opentelemetry group (#13303) --- .github/dependabot.yml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index d0cf53d2b2f8d..c4689affbb32b 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -24,6 +24,10 @@ updates: patterns: - "tonic*" - "prost*" + opentelemetry: + patterns: + - "opentelemetry*" + - "tracing-opentelemetry" mysql: patterns: - "mysql_common" From 04aca78fcb5b843e82a0e0b0e6928dd5fac3a0c8 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Thu, 16 Nov 2023 13:13:38 +0800 Subject: [PATCH 10/36] fix: add assertion to check stream executor schema consistency (#13288) Signed-off-by: Richard Chien --- src/stream/src/task/stream_manager.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 22a99c2099b5a..35ebb2581dc40 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -583,6 +583,12 @@ impl LocalStreamManagerCore { "`pk_indices` of {} not consistent with what derived by optimizer", executor.identity() ); + assert_eq!( + executor.schema(), + &schema, + "`schema` of {} not consistent with what derived by optimizer", + executor.identity() + ); // Wrap the executor for debug purpose. let executor = WrapperExecutor::new( From f901182c17018b16039ca315efc8850ea5607692 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 16 Nov 2023 13:34:04 +0800 Subject: [PATCH 11/36] fix(test): missing rowsort in udf.slt (#13453) --- e2e_test/udf/udf.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/udf/udf.slt b/e2e_test/udf/udf.slt index 62979196e74ca..b6d3161b7d3fc 100644 --- a/e2e_test/udf/udf.slt +++ b/e2e_test/udf/udf.slt @@ -239,7 +239,7 @@ insert into t values (1), (2), (3); statement ok flush; -query II +query II rowsort select * from mv; ---- 1 0 From fb7fa70f0a351b098b5f341746657acc65ba27cb Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 16 Nov 2023 13:36:47 +0800 Subject: [PATCH 12/36] chore: turn off more test targets (#13427) --- src/ctl/Cargo.toml | 3 +++ src/tests/compaction_test/Cargo.toml | 2 ++ src/tests/sqlsmith/Cargo.toml | 2 ++ 3 files changed, 7 insertions(+) diff --git a/src/ctl/Cargo.toml b/src/ctl/Cargo.toml index 4c9fe916d29b9..f8bd5d9d4d27d 100644 --- a/src/ctl/Cargo.toml +++ b/src/ctl/Cargo.toml @@ -58,3 +58,6 @@ risingwave_hummock_sdk = { workspace = true, features = ["enable_test_epoch"] } [lints] workspace = true + +[lib] +test = false diff --git a/src/tests/compaction_test/Cargo.toml b/src/tests/compaction_test/Cargo.toml index 98fe4ce6eedd2..786ddb269fbd6 100644 --- a/src/tests/compaction_test/Cargo.toml +++ b/src/tests/compaction_test/Cargo.toml @@ -53,10 +53,12 @@ workspace-hack = { path = "../../workspace-hack" } [[bin]] name = "compaction-test" path = "src/bin/compaction.rs" +test = false [[bin]] name = "delete-range-test" path = "src/bin/delete_range.rs" +test = false [lints] workspace = true diff --git a/src/tests/sqlsmith/Cargo.toml b/src/tests/sqlsmith/Cargo.toml index 402c6119cd1cf..726b8d15ea1ec 100644 --- a/src/tests/sqlsmith/Cargo.toml +++ b/src/tests/sqlsmith/Cargo.toml @@ -43,10 +43,12 @@ libtest-mimic = "0.6" [[bin]] name = "sqlsmith" path = "src/bin/main.rs" +test = false [[bin]] name = "sqlsmith-reducer" path = "src/bin/reducer.rs" +test = false [features] enable_sqlsmith_unit_test = [] From 8bcae118e4c09cd345cfa093f5ca0664c626dfa9 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 16 Nov 2023 15:12:16 +0800 Subject: [PATCH 13/36] refactor(risedev): refine error reporting (#13455) Signed-off-by: Bugen Zhao --- src/risedevtool/src/bin/risedev-dev.rs | 15 ++-- src/risedevtool/src/task.rs | 72 +++++++++---------- .../src/task/task_etcd_ready_check.rs | 2 +- .../src/task/task_kafka_ready_check.rs | 6 +- src/risedevtool/src/wait.rs | 20 ++++-- 5 files changed, 64 insertions(+), 51 deletions(-) diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index 474e8dd0cbd15..3d922b161bda4 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -349,6 +349,10 @@ fn task_main( } fn main() -> Result<()> { + // Intentionally disable backtrace to provide more compact error message for `risedev dev`. + // Backtraces for RisingWave components are enabled in `Task::execute`. + std::env::set_var("RUST_BACKTRACE", "0"); + preflight_check()?; let task_name = std::env::args() @@ -440,11 +444,11 @@ fn main() -> Result<()> { } Err(err) => { println!( - "{} - Failed to start: {}\nCaused by:\n\t{}", + "{} - Failed to start: {:?}", // with `Caused by` style("ERROR").red().bold(), err, - err.root_cause().to_string().trim(), ); + println!(); println!( "* Use `{}` to enable new components, if they are missing.", style("./risedev configure").blue().bold(), @@ -464,9 +468,12 @@ fn main() -> Result<()> { ); println!("---"); println!(); - println!(); - Err(err) + // As we have already printed the error above, we don't need to print that error again. + // However, to return with a proper exit code, still return an error here. + Err(anyhow!( + "Failed to start all services. See details and instructions above." + )) } } } diff --git a/src/risedevtool/src/task.rs b/src/risedevtool/src/task.rs index a2e4ec9bf46dc..32d4886544890 100644 --- a/src/risedevtool/src/task.rs +++ b/src/risedevtool/src/task.rs @@ -42,9 +42,9 @@ use std::process::{Command, Output}; use std::sync::Arc; use std::time::Duration; -use anyhow::{anyhow, Result}; +use anyhow::{anyhow, Context, Result}; use indicatif::ProgressBar; -use reqwest::blocking::Client; +use reqwest::blocking::{Client, Response}; use tempfile::TempDir; pub use utils::*; @@ -172,7 +172,9 @@ where let addr = server.as_ref().parse()?; wait( || { - TcpStream::connect_timeout(&addr, Duration::from_secs(1))?; + TcpStream::connect_timeout(&addr, Duration::from_secs(1)).with_context(|| { + format!("failed to establish tcp connection to {}", server.as_ref()) + })?; Ok(()) }, &mut self.log, @@ -184,7 +186,11 @@ where Ok(()) } - pub fn wait_http(&mut self, server: impl AsRef) -> anyhow::Result<()> { + fn wait_http_with_response_cb( + &mut self, + server: impl AsRef, + cb: impl Fn(Response) -> anyhow::Result<()>, + ) -> anyhow::Result<()> { let server = server.as_ref(); wait( || { @@ -192,12 +198,13 @@ where .get(server) .timeout(Duration::from_secs(1)) .body("") - .send()?; - if resp.status().is_success() { - Ok(()) - } else { - Err(anyhow!("http failed with status: {}", resp.status())) - } + .send()? + .error_for_status() + .with_context(|| { + format!("failed to establish http connection to {}", server) + })?; + + cb(resp) }, &mut self.log, self.status_file.as_ref().unwrap(), @@ -207,39 +214,26 @@ where ) } - pub fn wait_http_with_cb( + pub fn wait_http(&mut self, server: impl AsRef) -> anyhow::Result<()> { + self.wait_http_with_response_cb(server, |_| Ok(())) + } + + pub fn wait_http_with_text_cb( &mut self, server: impl AsRef, cb: impl Fn(&str) -> bool, ) -> anyhow::Result<()> { - let server = server.as_ref(); - wait( - || { - let resp = Client::new() - .get(server) - .timeout(Duration::from_secs(1)) - .body("") - .send()?; - if resp.status().is_success() { - let data = resp.text()?; - if cb(&data) { - Ok(()) - } else { - Err(anyhow!( - "http health check callback failed with body: {:?}", - data - )) - } - } else { - Err(anyhow!("http failed with status: {}", resp.status())) - } - }, - &mut self.log, - self.status_file.as_ref().unwrap(), - self.id.as_ref().unwrap(), - Some(Duration::from_secs(30)), - true, - ) + self.wait_http_with_response_cb(server, |resp| { + let data = resp.text()?; + if cb(&data) { + Ok(()) + } else { + Err(anyhow!( + "http health check callback failed with body: {:?}", + data + )) + } + }) } pub fn wait(&mut self, wait_func: impl FnMut() -> Result<()>) -> anyhow::Result<()> { diff --git a/src/risedevtool/src/task/task_etcd_ready_check.rs b/src/risedevtool/src/task/task_etcd_ready_check.rs index ffc62e3707022..9c4bea769ddd1 100644 --- a/src/risedevtool/src/task/task_etcd_ready_check.rs +++ b/src/risedevtool/src/task/task_etcd_ready_check.rs @@ -44,7 +44,7 @@ impl Task for EtcdReadyCheckTask { response.health == "true" }; - ctx.wait_http_with_cb(health_check_addr, online_cb)?; + ctx.wait_http_with_text_cb(health_check_addr, online_cb)?; ctx.pb .set_message(format!("api {}:{}", self.config.address, self.config.port)); ctx.complete_spin(); diff --git a/src/risedevtool/src/task/task_kafka_ready_check.rs b/src/risedevtool/src/task/task_kafka_ready_check.rs index 1b53c96473acf..2e7b8cf436b7a 100644 --- a/src/risedevtool/src/task/task_kafka_ready_check.rs +++ b/src/risedevtool/src/task/task_kafka_ready_check.rs @@ -14,7 +14,7 @@ use std::time::Duration; -use anyhow::{anyhow, Result}; +use anyhow::{Context, Result}; use rdkafka::config::FromClientConfig; use rdkafka::consumer::{BaseConsumer, Consumer}; use rdkafka::ClientConfig; @@ -48,7 +48,7 @@ impl Task for KafkaReadyCheckTask { let consumer = rt.block_on(async { BaseConsumer::from_config(&config) .await - .map_err(|e| anyhow!("{}", e)) + .context("failed to create consumer") })?; ctx.wait(|| { @@ -56,7 +56,7 @@ impl Task for KafkaReadyCheckTask { let _metadata = consumer .fetch_metadata(None, Duration::from_secs(1)) .await - .map_err(|e| anyhow!("{}", e))?; + .context("failed to fetch metadata")?; Ok(()) }) })?; diff --git a/src/risedevtool/src/wait.rs b/src/risedevtool/src/wait.rs index 3a47c7a4b6432..e5c0671b0c3e3 100644 --- a/src/risedevtool/src/wait.rs +++ b/src/risedevtool/src/wait.rs @@ -48,7 +48,13 @@ pub fn wait( if let Some(ref timeout) = timeout { if std::time::Instant::now() - start_time >= *timeout { - return Err(anyhow!("failed to connect, last error: {:?}", last_error)); + let context = "timeout when trying to connect"; + + return Err(if let Some(last_error) = last_error { + last_error.context(context) + } else { + anyhow!(context) + }); } } @@ -56,11 +62,17 @@ pub fn wait( let mut buf = String::new(); fs_err::File::open(p)?.read_to_string(&mut buf)?; - return Err(anyhow!( + let context = format!( "{} exited while waiting for connection: {}", style(id).red().bold(), - buf, - )); + buf.trim(), + ); + + return Err(if let Some(last_error) = last_error { + last_error.context(context) + } else { + anyhow!(context) + }); } sleep(Duration::from_millis(30)); From ef53d65e640f96e86c8c7ef208d9e5051408986f Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 16 Nov 2023 15:39:26 +0800 Subject: [PATCH 14/36] fix(optimizer): skip subquery optimization if no apply (#13460) --- .../tests/testdata/output/cse_expr.yaml | 2 +- .../tests/testdata/output/explain.yaml | 14 +------------- src/frontend/src/optimizer/logical_optimization.rs | 10 ++++++++-- 3 files changed, 10 insertions(+), 16 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml index eba0edf5b1f04..5057fa13473c3 100644 --- a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml @@ -84,5 +84,5 @@ sql: | with t(v, arr) as (select 1, array[2, 3]) select v < all(arr), v < some(arr) from t; batch_plan: |- - BatchProject { exprs: [All((1:Int32 < $expr10063)) as $expr1, Some((1:Int32 < $expr10063)) as $expr2] } + BatchProject { exprs: [All((1:Int32 < $expr10048)) as $expr1, Some((1:Int32 < $expr10048)) as $expr2] } └─BatchValues { rows: [[1:Int32, ARRAY[2, 3]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index f416e9155a023..eced1aaf718bc 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -26,11 +26,6 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Predicate Push Down: - - LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Convert Over Window: apply TrivialProjectToValuesRule 1 time(s) @@ -70,7 +65,7 @@ "stages": { "0": { "root": { - "plan_node_id": 10037, + "plan_node_id": 10031, "plan_node_type": "BatchValues", "schema": [ { @@ -133,13 +128,6 @@ Predicate Push Down: - LogicalProject { exprs: [t1.v1, t2.v2] } - └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1, _row_id] } - └─LogicalScan { table: t2, columns: [v2, _row_id] } - - Predicate Push Down: - LogicalProject { exprs: [t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } ├─LogicalScan { table: t1, columns: [v1, _row_id] } diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index f8d9e6f78c737..887f393e81c6a 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -426,6 +426,10 @@ impl LogicalOptimizer { explain_trace: bool, ctx: &OptimizerContextRef, ) -> Result { + // Bail our if no apply operators. + if !has_logical_apply(plan.clone()) { + return Ok(plan); + } // Simple Unnesting. plan = plan.optimize_by_rules(&SIMPLE_UNNESTING); if HasMaxOneRowApply().visit(plan.clone()) { @@ -437,8 +441,6 @@ impl LogicalOptimizer { // Predicate push down before translate apply, because we need to calculate the domain // and predicate push down can reduce the size of domain. plan = Self::predicate_pushdown(plan, explain_trace, ctx); - // In order to unnest a table function, we need to convert it into a `project_set` first. - plan = plan.optimize_by_rules(&TABLE_FUNCTION_TO_PROJECT_SET); // In order to unnest values with correlated input ref, we need to extract project first. plan = plan.optimize_by_rules(&VALUES_EXTRACT_PROJECT); // General Unnesting. @@ -535,6 +537,8 @@ impl LogicalOptimizer { } plan = plan.optimize_by_rules(&SET_OPERATION_MERGE); plan = plan.optimize_by_rules(&SET_OPERATION_TO_JOIN); + // In order to unnest a table function, we need to convert it into a `project_set` first. + plan = plan.optimize_by_rules(&TABLE_FUNCTION_TO_PROJECT_SET); plan = Self::subquery_unnesting(plan, enable_share_plan, explain_trace, &ctx)?; @@ -628,6 +632,8 @@ impl LogicalOptimizer { plan = plan.optimize_by_rules(&SET_OPERATION_MERGE); plan = plan.optimize_by_rules(&SET_OPERATION_TO_JOIN); plan = plan.optimize_by_rules(&ALWAYS_FALSE_FILTER); + // In order to unnest a table function, we need to convert it into a `project_set` first. + plan = plan.optimize_by_rules(&TABLE_FUNCTION_TO_PROJECT_SET); plan = Self::subquery_unnesting(plan, false, explain_trace, &ctx)?; From be3d41807165286a4cb236f491f724612e9699a8 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 16 Nov 2023 15:49:10 +0800 Subject: [PATCH 15/36] refactor(frontend): separate sys_scan and stream_scan (#13452) --- .../tests/testdata/output/pg_catalog.yaml | 8 +- .../tests/testdata/output/subquery.yaml | 22 +- src/frontend/src/optimizer/mod.rs | 8 +- .../src/optimizer/plan_node/batch_seq_scan.rs | 97 ++--- .../optimizer/plan_node/batch_sys_seq_scan.rs | 224 +++++++++++ .../src/optimizer/plan_node/generic/mod.rs | 2 + .../src/optimizer/plan_node/generic/scan.rs | 5 - .../optimizer/plan_node/generic/sys_scan.rs | 286 +++++++++++++ .../src/optimizer/plan_node/logical_scan.rs | 19 +- .../optimizer/plan_node/logical_sys_scan.rs | 378 ++++++++++++++++++ src/frontend/src/optimizer/plan_node/mod.rs | 8 + .../plan_visitor/cardinality_visitor.rs | 4 +- .../relation_collector_visitor.rs | 12 +- .../plan_visitor/sys_table_visitor.rs | 15 +- src/frontend/src/planner/relation.rs | 7 +- src/frontend/src/scheduler/plan_fragmenter.rs | 33 +- 16 files changed, 985 insertions(+), 143 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/batch_sys_seq_scan.rs create mode 100644 src/frontend/src/optimizer/plan_node/generic/sys_scan.rs create mode 100644 src/frontend/src/optimizer/plan_node/logical_sys_scan.rs diff --git a/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml b/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml index 5ae1827ad95a3..7842e311e47ae 100644 --- a/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml +++ b/src/frontend/planner_test/tests/testdata/output/pg_catalog.yaml @@ -6,8 +6,8 @@ └─LogicalShare { id: 3 } └─LogicalProject { exprs: [rw_types.id, rw_types.name, rw_types.typelem, rw_types.typarray, rw_types.input_oid, false:Boolean, 0:Int32, -1:Int32, 0:Int32, 0:Int32, rw_schemas.id, 'b':Varchar, 0:Int32, null:Varchar, null:Varchar, null:Int32] } └─LogicalJoin { type: Inner, on: (rw_schemas.name = 'pg_catalog':Varchar), output: all } - ├─LogicalScan { table: rw_types, columns: [rw_types.id, rw_types.name, rw_types.input_oid, rw_types.typelem, rw_types.typarray] } - └─LogicalScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } + ├─LogicalSysScan { table: rw_types, columns: [rw_types.id, rw_types.name, rw_types.input_oid, rw_types.typelem, rw_types.typarray] } + └─LogicalSysScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } batch_plan: |- BatchProject { exprs: [rw_types.id, rw_types.name, rw_types.typelem, rw_types.typarray, rw_types.input_oid, false:Boolean, 0:Int32, -1:Int32, 0:Int32, 0:Int32, rw_schemas.id, 'b':Varchar, 0:Int32, null:Varchar, null:Varchar, null:Int32] } └─BatchNestedLoopJoin { type: Inner, predicate: true, output: all } @@ -21,7 +21,7 @@ LogicalProject { exprs: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } - └─LogicalScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } + └─LogicalSysScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } batch_plan: 'BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl], distribution: Single }' - sql: | select * from pg_catalog.pg_cast @@ -211,7 +211,7 @@ └─LogicalFilter { predicate: (1:Int32 = rw_users.id) } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [rw_users.id, rw_users.name, rw_users.create_db, rw_users.is_super, '********':Varchar] } - └─LogicalScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } + └─LogicalSysScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } batch_plan: |- BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index eb76926c35938..5143c7fdee853 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -204,13 +204,13 @@ └─LogicalFilter { predicate: (1:Int32 = rw_users.id) } └─LogicalShare { id: 2 } └─LogicalProject { exprs: [rw_users.id, rw_users.name, rw_users.create_db, rw_users.is_super, '********':Varchar] } - └─LogicalScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } + └─LogicalSysScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } optimized_logical_plan_for_batch: |- LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } └─LogicalJoin { type: LeftOuter, on: true, output: all } ├─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - └─LogicalScan { table: rw_users, output_columns: [rw_users.name], required_columns: [rw_users.name, rw_users.id], predicate: (1:Int32 = rw_users.id) } + └─LogicalSysScan { table: rw_users, output_columns: [rw_users.name], required_columns: [rw_users.name, rw_users.id], predicate: (1:Int32 = rw_users.id) } batch_plan: |- BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } @@ -247,27 +247,27 @@ │ │ │ │ │ ├─LogicalUnion { all: true } │ │ │ │ │ │ ├─LogicalUnion { all: true } │ │ │ │ │ │ │ ├─LogicalProject { exprs: [rw_tables.id, rw_tables.name, 'table':Varchar, rw_tables.schema_id, rw_tables.owner, rw_tables.definition, rw_tables.acl] } - │ │ │ │ │ │ │ │ └─LogicalScan { table: rw_tables, columns: [rw_tables.id, rw_tables.name, rw_tables.schema_id, rw_tables.owner, rw_tables.definition, rw_tables.acl, rw_tables.initialized_at, rw_tables.created_at] } + │ │ │ │ │ │ │ │ └─LogicalSysScan { table: rw_tables, columns: [rw_tables.id, rw_tables.name, rw_tables.schema_id, rw_tables.owner, rw_tables.definition, rw_tables.acl, rw_tables.initialized_at, rw_tables.created_at] } │ │ │ │ │ │ │ └─LogicalProject { exprs: [rw_system_tables.id, rw_system_tables.name, 'system table':Varchar, rw_system_tables.schema_id, rw_system_tables.owner, rw_system_tables.definition, rw_system_tables.acl] } - │ │ │ │ │ │ │ └─LogicalScan { table: rw_system_tables, columns: [rw_system_tables.id, rw_system_tables.name, rw_system_tables.schema_id, rw_system_tables.owner, rw_system_tables.definition, rw_system_tables.acl] } + │ │ │ │ │ │ │ └─LogicalSysScan { table: rw_system_tables, columns: [rw_system_tables.id, rw_system_tables.name, rw_system_tables.schema_id, rw_system_tables.owner, rw_system_tables.definition, rw_system_tables.acl] } │ │ │ │ │ │ └─LogicalProject { exprs: [rw_sources.id, rw_sources.name, 'source':Varchar, rw_sources.schema_id, rw_sources.owner, rw_sources.definition, rw_sources.acl] } - │ │ │ │ │ │ └─LogicalScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id, rw_sources.owner, rw_sources.connector, rw_sources.columns, rw_sources.format, rw_sources.row_encode, rw_sources.append_only, rw_sources.connection_id, rw_sources.definition, rw_sources.acl, rw_sources.initialized_at, rw_sources.created_at] } + │ │ │ │ │ │ └─LogicalSysScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id, rw_sources.owner, rw_sources.connector, rw_sources.columns, rw_sources.format, rw_sources.row_encode, rw_sources.append_only, rw_sources.connection_id, rw_sources.definition, rw_sources.acl, rw_sources.initialized_at, rw_sources.created_at] } │ │ │ │ │ └─LogicalProject { exprs: [rw_indexes.id, rw_indexes.name, 'index':Varchar, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl] } - │ │ │ │ │ └─LogicalScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.primary_table_id, rw_indexes.original_column_ids, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl, rw_indexes.initialized_at, rw_indexes.created_at] } + │ │ │ │ │ └─LogicalSysScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.primary_table_id, rw_indexes.original_column_ids, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl, rw_indexes.initialized_at, rw_indexes.created_at] } │ │ │ │ └─LogicalProject { exprs: [rw_sinks.id, rw_sinks.name, 'sink':Varchar, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.definition, rw_sinks.acl] } - │ │ │ │ └─LogicalScan { table: rw_sinks, columns: [rw_sinks.id, rw_sinks.name, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.connector, rw_sinks.sink_type, rw_sinks.connection_id, rw_sinks.definition, rw_sinks.acl, rw_sinks.initialized_at, rw_sinks.created_at] } + │ │ │ │ └─LogicalSysScan { table: rw_sinks, columns: [rw_sinks.id, rw_sinks.name, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.connector, rw_sinks.sink_type, rw_sinks.connection_id, rw_sinks.definition, rw_sinks.acl, rw_sinks.initialized_at, rw_sinks.created_at] } │ │ │ └─LogicalProject { exprs: [rw_materialized_views.id, rw_materialized_views.name, 'materialized view':Varchar, rw_materialized_views.schema_id, rw_materialized_views.owner, rw_materialized_views.definition, rw_materialized_views.acl] } - │ │ │ └─LogicalScan { table: rw_materialized_views, columns: [rw_materialized_views.id, rw_materialized_views.name, rw_materialized_views.schema_id, rw_materialized_views.owner, rw_materialized_views.definition, rw_materialized_views.acl, rw_materialized_views.initialized_at, rw_materialized_views.created_at] } + │ │ │ └─LogicalSysScan { table: rw_materialized_views, columns: [rw_materialized_views.id, rw_materialized_views.name, rw_materialized_views.schema_id, rw_materialized_views.owner, rw_materialized_views.definition, rw_materialized_views.acl, rw_materialized_views.initialized_at, rw_materialized_views.created_at] } │ │ └─LogicalProject { exprs: [rw_views.id, rw_views.name, 'view':Varchar, rw_views.schema_id, rw_views.owner, rw_views.definition, rw_views.acl] } - │ │ └─LogicalScan { table: rw_views, columns: [rw_views.id, rw_views.name, rw_views.schema_id, rw_views.owner, rw_views.definition, rw_views.acl] } + │ │ └─LogicalSysScan { table: rw_views, columns: [rw_views.id, rw_views.name, rw_views.schema_id, rw_views.owner, rw_views.definition, rw_views.acl] } │ └─LogicalShare { id: 18 } │ └─LogicalProject { exprs: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } - │ └─LogicalScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } + │ └─LogicalSysScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name, rw_schemas.owner, rw_schemas.acl] } └─LogicalProject { exprs: [rw_users.name] } └─LogicalFilter { predicate: (CorrelatedInputRef { index: 3, correlated_id: 1 } = rw_users.id) } └─LogicalShare { id: 22 } └─LogicalProject { exprs: [rw_users.id, rw_users.name, rw_users.create_db, rw_users.is_super, '********':Varchar] } - └─LogicalScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } + └─LogicalSysScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login] } batch_plan: |- BatchExchange { order: [rw_schemas.name ASC, rw_tables.name ASC], dist: Single } └─BatchProject { exprs: [rw_schemas.name, rw_tables.name, Case(($expr1 = 'r':Varchar), 'table':Varchar, ($expr1 = 'v':Varchar), 'view':Varchar, ($expr1 = 'm':Varchar), 'materialized view':Varchar, ($expr1 = 'i':Varchar), 'index':Varchar, ($expr1 = 'S':Varchar), 'sequence':Varchar, ($expr1 = 's':Varchar), 'special':Varchar, ($expr1 = 't':Varchar), 'TOAST table':Varchar, ($expr1 = 'f':Varchar), 'foreign table':Varchar, ($expr1 = 'p':Varchar), 'partitioned table':Varchar, ($expr1 = 'I':Varchar), 'partitioned index':Varchar) as $expr2, rw_users.name] } diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index eb77732339558..70d5ae1769c83 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -747,9 +747,7 @@ fn exist_and_no_exchange_before(plan: &PlanRef, is_candidate: fn(&PlanRef) -> bo /// Returns `true` if we must insert an additional exchange to ensure this. fn require_additional_exchange_on_root_in_distributed_mode(plan: PlanRef) -> bool { fn is_user_table(plan: &PlanRef) -> bool { - plan.as_batch_seq_scan() - .map(|node| !node.core().is_sys_table()) - .unwrap_or(false) + plan.node_type() == PlanNodeType::BatchSeqScan } fn is_source(plan: &PlanRef) -> bool { @@ -780,9 +778,7 @@ fn require_additional_exchange_on_root_in_distributed_mode(plan: PlanRef) -> boo /// them for the different requirement of plan node in different execute mode. fn require_additional_exchange_on_root_in_local_mode(plan: PlanRef) -> bool { fn is_user_table(plan: &PlanRef) -> bool { - plan.as_batch_seq_scan() - .map(|node| !node.core().is_sys_table()) - .unwrap_or(false) + plan.node_type() == PlanNodeType::BatchSeqScan } fn is_source(plan: &PlanRef) -> bool { diff --git a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs index 65487e537420c..1b2582a7e2db2 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -21,8 +21,7 @@ use risingwave_common::types::ScalarImpl; use risingwave_common::util::scan_range::{is_full_range, ScanRange}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::row_seq_scan_node::ChunkSize; -use risingwave_pb::batch_plan::{RowSeqScanNode, SysRowSeqScanNode}; -use risingwave_pb::plan_common::PbColumnDesc; +use risingwave_pb::batch_plan::RowSeqScanNode; use super::batch::prelude::*; use super::utils::{childless_record, Distill}; @@ -78,29 +77,25 @@ impl BatchSeqScan { fn clone_with_dist(&self) -> Self { Self::new_inner( self.core.clone(), - if self.core.is_sys_table() { - Distribution::Single - } else { - match self.core.distribution_key() { - None => Distribution::SomeShard, - Some(distribution_key) => { - if distribution_key.is_empty() { - Distribution::Single - } else { - // For other batch operators, `HashShard` is a simple hashing, i.e., - // `target_shard = hash(dist_key) % shard_num` - // - // But MV is actually sharded by consistent hashing, i.e., - // `target_shard = vnode_mapping.map(hash(dist_key) % vnode_num)` - // - // They are incompatible, so we just specify its distribution as - // `SomeShard` to force an exchange is - // inserted. - Distribution::UpstreamHashShard( - distribution_key, - self.core.table_desc.table_id, - ) - } + match self.core.distribution_key() { + None => Distribution::SomeShard, + Some(distribution_key) => { + if distribution_key.is_empty() { + Distribution::Single + } else { + // For other batch operators, `HashShard` is a simple hashing, i.e., + // `target_shard = hash(dist_key) % shard_num` + // + // But MV is actually sharded by consistent hashing, i.e., + // `target_shard = vnode_mapping.map(hash(dist_key) % vnode_num)` + // + // They are incompatible, so we just specify its distribution as + // `SomeShard` to force an exchange is + // inserted. + Distribution::UpstreamHashShard( + distribution_key, + self.core.table_desc.table_id, + ) } } }, @@ -214,45 +209,29 @@ impl ToDistributedBatch for BatchSeqScan { impl ToBatchPb for BatchSeqScan { fn to_batch_prost_body(&self) -> NodeBody { - let column_descs = self - .core - .column_descs() - .iter() - .map(PbColumnDesc::from) - .collect(); - - if self.core.is_sys_table() { - NodeBody::SysRowSeqScan(SysRowSeqScanNode { - table_id: self.core.table_desc.table_id.table_id, - column_descs, - }) - } else { - NodeBody::RowSeqScan(RowSeqScanNode { - table_desc: Some(self.core.table_desc.to_protobuf()), - column_ids: self - .core - .output_column_ids() - .iter() - .map(ColumnId::get_id) - .collect(), - scan_ranges: self.scan_ranges.iter().map(|r| r.to_protobuf()).collect(), - // To be filled by the scheduler. - vnode_bitmap: None, - ordered: !self.order().is_any(), - chunk_size: self - .core - .chunk_size - .map(|chunk_size| ChunkSize { chunk_size }), - }) - } + NodeBody::RowSeqScan(RowSeqScanNode { + table_desc: Some(self.core.table_desc.to_protobuf()), + column_ids: self + .core + .output_column_ids() + .iter() + .map(ColumnId::get_id) + .collect(), + scan_ranges: self.scan_ranges.iter().map(|r| r.to_protobuf()).collect(), + // To be filled by the scheduler. + vnode_bitmap: None, + ordered: !self.order().is_any(), + chunk_size: self + .core + .chunk_size + .map(|chunk_size| ChunkSize { chunk_size }), + }) } } impl ToLocalBatch for BatchSeqScan { fn to_local(&self) -> Result { - let dist = if self.core.is_sys_table() { - Distribution::Single - } else if let Some(distribution_key) = self.core.distribution_key() + let dist = if let Some(distribution_key) = self.core.distribution_key() && !distribution_key.is_empty() { Distribution::UpstreamHashShard(distribution_key, self.core.table_desc.table_id) diff --git a/src/frontend/src/optimizer/plan_node/batch_sys_seq_scan.rs b/src/frontend/src/optimizer/plan_node/batch_sys_seq_scan.rs new file mode 100644 index 0000000000000..22c24233ada37 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/batch_sys_seq_scan.rs @@ -0,0 +1,224 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::ops::Bound; + +use itertools::Itertools; +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::error::Result; +use risingwave_common::types::ScalarImpl; +use risingwave_common::util::scan_range::{is_full_range, ScanRange}; +use risingwave_pb::batch_plan::plan_node::NodeBody; +use risingwave_pb::batch_plan::SysRowSeqScanNode; +use risingwave_pb::plan_common::PbColumnDesc; + +use super::batch::prelude::*; +use super::utils::{childless_record, Distill}; +use super::{generic, ExprRewritable, PlanBase, PlanRef, ToBatchPb, ToDistributedBatch}; +use crate::expr::ExprRewriter; +use crate::optimizer::plan_node::ToLocalBatch; +use crate::optimizer::property::{Distribution, DistributionDisplay, Order}; + +/// `BatchSysSeqScan` implements [`super::LogicalSysScan`] to scan from a row-oriented table +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct BatchSysSeqScan { + pub base: PlanBase, + core: generic::SysScan, + scan_ranges: Vec, +} + +impl BatchSysSeqScan { + fn new_inner(core: generic::SysScan, dist: Distribution, scan_ranges: Vec) -> Self { + let order = if scan_ranges.len() > 1 { + Order::any() + } else { + core.get_out_column_index_order() + }; + let base = PlanBase::new_batch_with_core(&core, dist, order); + + { + // validate scan_range + scan_ranges.iter().for_each(|scan_range| { + assert!(!scan_range.is_full_table_scan()); + let scan_pk_prefix_len = scan_range.eq_conds.len(); + let order_len = core.table_desc.order_column_indices().len(); + assert!( + scan_pk_prefix_len < order_len + || (scan_pk_prefix_len == order_len && is_full_range(&scan_range.range)), + "invalid scan_range", + ); + }) + } + + Self { + base, + core, + scan_ranges, + } + } + + pub fn new(core: generic::SysScan, scan_ranges: Vec) -> Self { + // Use `Single` by default, will be updated later with `clone_with_dist`. + Self::new_inner(core, Distribution::Single, scan_ranges) + } + + fn clone_with_dist(&self) -> Self { + Self::new_inner( + self.core.clone(), + Distribution::Single, + self.scan_ranges.clone(), + ) + } + + /// Get a reference to the batch seq scan's logical. + #[must_use] + pub fn core(&self) -> &generic::SysScan { + &self.core + } + + pub fn scan_ranges(&self) -> &[ScanRange] { + &self.scan_ranges + } + + fn scan_ranges_as_strs(&self, verbose: bool) -> Vec { + let order_names = match verbose { + true => self.core.order_names_with_table_prefix(), + false => self.core.order_names(), + }; + let mut range_strs = vec![]; + + let explain_max_range = 20; + for scan_range in self.scan_ranges.iter().take(explain_max_range) { + #[expect(clippy::disallowed_methods)] + let mut range_str = scan_range + .eq_conds + .iter() + .zip(order_names.iter()) + .map(|(v, name)| match v { + Some(v) => format!("{} = {:?}", name, v), + None => format!("{} IS NULL", name), + }) + .collect_vec(); + if !is_full_range(&scan_range.range) { + let i = scan_range.eq_conds.len(); + range_str.push(range_to_string(&order_names[i], &scan_range.range)) + } + range_strs.push(range_str.join(" AND ")); + } + if self.scan_ranges.len() > explain_max_range { + range_strs.push("...".to_string()); + } + range_strs + } +} + +impl_plan_tree_node_for_leaf! { BatchSysSeqScan } + +fn lb_to_string(name: &str, lb: &Bound) -> String { + let (op, v) = match lb { + Bound::Included(v) => (">=", v), + Bound::Excluded(v) => (">", v), + Bound::Unbounded => unreachable!(), + }; + format!("{} {} {:?}", name, op, v) +} +fn ub_to_string(name: &str, ub: &Bound) -> String { + let (op, v) = match ub { + Bound::Included(v) => ("<=", v), + Bound::Excluded(v) => ("<", v), + Bound::Unbounded => unreachable!(), + }; + format!("{} {} {:?}", name, op, v) +} +fn range_to_string(name: &str, range: &(Bound, Bound)) -> String { + match (&range.0, &range.1) { + (Bound::Unbounded, Bound::Unbounded) => unreachable!(), + (Bound::Unbounded, ub) => ub_to_string(name, ub), + (lb, Bound::Unbounded) => lb_to_string(name, lb), + (lb, ub) => { + format!("{} AND {}", lb_to_string(name, lb), ub_to_string(name, ub)) + } + } +} + +impl Distill for BatchSysSeqScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let verbose = self.base.ctx().is_explain_verbose(); + let mut vec = Vec::with_capacity(4); + vec.push(("table", Pretty::from(self.core.table_name.clone()))); + vec.push(("columns", self.core.columns_pretty(verbose))); + + if !self.scan_ranges.is_empty() { + let range_strs = self.scan_ranges_as_strs(verbose); + vec.push(( + "scan_ranges", + Pretty::Array(range_strs.into_iter().map(Pretty::from).collect()), + )); + } + + if verbose { + let dist = Pretty::display(&DistributionDisplay { + distribution: self.distribution(), + input_schema: self.base.schema(), + }); + vec.push(("distribution", dist)); + } + + childless_record("BatchScan", vec) + } +} + +impl ToDistributedBatch for BatchSysSeqScan { + fn to_distributed(&self) -> Result { + Ok(self.clone_with_dist().into()) + } +} + +impl ToBatchPb for BatchSysSeqScan { + fn to_batch_prost_body(&self) -> NodeBody { + let column_descs = self + .core + .column_descs() + .iter() + .map(PbColumnDesc::from) + .collect(); + NodeBody::SysRowSeqScan(SysRowSeqScanNode { + table_id: self.core.table_desc.table_id.table_id, + column_descs, + }) + } +} + +impl ToLocalBatch for BatchSysSeqScan { + fn to_local(&self) -> Result { + Ok(Self::new_inner( + self.core.clone(), + Distribution::Single, + self.scan_ranges.clone(), + ) + .into()) + } +} + +impl ExprRewritable for BatchSysSeqScan { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self::new(core, self.scan_ranges.clone()).into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index fe85e0919804e..c0baacffec8a0 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -42,6 +42,8 @@ mod source; pub use source::*; mod scan; pub use scan::*; +mod sys_scan; +pub use sys_scan::*; mod union; pub use union::*; mod top_n; diff --git a/src/frontend/src/optimizer/plan_node/generic/scan.rs b/src/frontend/src/optimizer/plan_node/generic/scan.rs index 7d83ac0ba7e4f..3c615183f9adf 100644 --- a/src/frontend/src/optimizer/plan_node/generic/scan.rs +++ b/src/frontend/src/optimizer/plan_node/generic/scan.rs @@ -34,7 +34,6 @@ use crate::utils::{ColIndexMappingRewriteExt, Condition}; pub enum ScanTableType { #[default] General, - SysTable, CdcTable, } @@ -424,10 +423,6 @@ impl Scan { } } - pub fn is_sys_table(&self) -> bool { - matches!(self.scan_table_type, ScanTableType::SysTable) - } - pub fn is_cdc_table(&self) -> bool { matches!(self.scan_table_type, ScanTableType::CdcTable) } diff --git a/src/frontend/src/optimizer/plan_node/generic/sys_scan.rs b/src/frontend/src/optimizer/plan_node/generic/sys_scan.rs new file mode 100644 index 0000000000000..dadc959fce33c --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/generic/sys_scan.rs @@ -0,0 +1,286 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::rc::Rc; + +use educe::Educe; +use pretty_xmlish::Pretty; +use risingwave_common::catalog::{ColumnDesc, Field, Schema, TableDesc}; +use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_common::util::sort_util::ColumnOrder; + +use super::GenericPlanNode; +use crate::catalog::ColumnId; +use crate::expr::ExprRewriter; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::property::{Cardinality, FunctionalDependencySet, Order}; +use crate::utils::{ColIndexMappingRewriteExt, Condition}; + +/// [`SysScan`] returns contents of a table or other equivalent object +#[derive(Debug, Clone, Educe)] +#[educe(PartialEq, Eq, Hash)] +pub struct SysScan { + pub table_name: String, + /// Include `output_col_idx` and columns required in `predicate` + pub required_col_idx: Vec, + pub output_col_idx: Vec, + /// Descriptor of the table + pub table_desc: Rc, + /// The pushed down predicates. It refers to column indexes of the table. + pub predicate: Condition, + /// Help RowSeqSysScan executor use a better chunk size + pub chunk_size: Option, + /// The cardinality of the table **without** applying the predicate. + pub table_cardinality: Cardinality, + #[educe(PartialEq(ignore))] + #[educe(Hash(ignore))] + pub ctx: OptimizerContextRef, +} + +impl SysScan { + pub(crate) fn rewrite_exprs(&mut self, r: &mut dyn ExprRewriter) { + self.predicate = self.predicate.clone().rewrite_expr(r); + } + + /// Get the ids of the output columns. + pub fn output_column_ids(&self) -> Vec { + self.output_col_idx + .iter() + .map(|i| self.get_table_columns()[*i].column_id) + .collect() + } + + pub fn primary_key(&self) -> &[ColumnOrder] { + &self.table_desc.pk + } + + pub(crate) fn column_names_with_table_prefix(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| format!("{}.{}", self.table_name, self.get_table_columns()[i].name)) + .collect() + } + + pub(crate) fn column_names(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| self.get_table_columns()[i].name.clone()) + .collect() + } + + pub(crate) fn order_names(&self) -> Vec { + self.table_desc + .order_column_indices() + .iter() + .map(|&i| self.get_table_columns()[i].name.clone()) + .collect() + } + + pub(crate) fn order_names_with_table_prefix(&self) -> Vec { + self.table_desc + .order_column_indices() + .iter() + .map(|&i| format!("{}.{}", self.table_name, self.get_table_columns()[i].name)) + .collect() + } + + /// Return indices of fields the output is ordered by and + /// corresponding direction + pub fn get_out_column_index_order(&self) -> Order { + let id_to_tb_idx = self.table_desc.get_id_to_op_idx_mapping(); + let order = Order::new( + self.table_desc + .pk + .iter() + .map(|order| { + let idx = id_to_tb_idx + .get(&self.table_desc.columns[order.column_index].column_id) + .unwrap(); + ColumnOrder::new(*idx, order.order_type) + }) + .collect(), + ); + self.i2o_col_mapping().rewrite_provided_order(&order) + } + + /// get the Mapping of columnIndex from internal column index to output column index + pub fn i2o_col_mapping(&self) -> ColIndexMapping { + ColIndexMapping::with_remaining_columns( + &self.output_col_idx, + self.get_table_columns().len(), + ) + } + + /// Get the ids of the output columns and primary key columns. + pub fn output_and_pk_column_ids(&self) -> Vec { + let mut ids = self.output_column_ids(); + for column_order in self.primary_key() { + let id = self.get_table_columns()[column_order.column_index].column_id; + if !ids.contains(&id) { + ids.push(id); + } + } + ids + } + + /// Create a `LogicalSysScan` node. Used internally by optimizer. + #[allow(clippy::too_many_arguments)] + pub(crate) fn new( + table_name: String, + output_col_idx: Vec, // the column index in the table + table_desc: Rc, + ctx: OptimizerContextRef, + predicate: Condition, // refers to column indexes of the table + table_cardinality: Cardinality, + ) -> Self { + Self::new_inner( + table_name, + output_col_idx, + table_desc, + ctx, + predicate, + table_cardinality, + ) + } + + #[allow(clippy::too_many_arguments)] + pub(crate) fn new_inner( + table_name: String, + output_col_idx: Vec, // the column index in the table + table_desc: Rc, + ctx: OptimizerContextRef, + predicate: Condition, // refers to column indexes of the table + table_cardinality: Cardinality, + ) -> Self { + // here we have 3 concepts + // 1. column_id: ColumnId, stored in catalog and a ID to access data from storage. + // 2. table_idx: usize, column index in the TableDesc or tableCatalog. + // 3. operator_idx: usize, column index in the SysScanOperator's schema. + // In a query we get the same version of catalog, so the mapping from column_id and + // table_idx will not change. And the `required_col_idx` is the `table_idx` of the + // required columns, i.e., the mapping from operator_idx to table_idx. + + let mut required_col_idx = output_col_idx.clone(); + let predicate_col_idx = predicate.collect_input_refs(table_desc.columns.len()); + predicate_col_idx.ones().for_each(|idx| { + if !required_col_idx.contains(&idx) { + required_col_idx.push(idx); + } + }); + + Self { + table_name, + required_col_idx, + output_col_idx, + table_desc, + predicate, + chunk_size: None, + ctx, + table_cardinality, + } + } + + pub(crate) fn columns_pretty<'a>(&self, verbose: bool) -> Pretty<'a> { + Pretty::Array( + match verbose { + true => self.column_names_with_table_prefix(), + false => self.column_names(), + } + .into_iter() + .map(Pretty::from) + .collect(), + ) + } + + pub(crate) fn fields_pretty_schema(&self) -> Schema { + let fields = self + .table_desc + .columns + .iter() + .map(|col| Field::from_with_table_name_prefix(col, &self.table_name)) + .collect(); + Schema { fields } + } +} + +impl GenericPlanNode for SysScan { + fn schema(&self) -> Schema { + let fields = self + .output_col_idx + .iter() + .map(|tb_idx| { + let col = &self.get_table_columns()[*tb_idx]; + Field::from_with_table_name_prefix(col, &self.table_name) + }) + .collect(); + Schema { fields } + } + + fn stream_key(&self) -> Option> { + let id_to_op_idx = Self::get_id_to_op_idx_mapping(&self.output_col_idx, &self.table_desc); + self.table_desc + .stream_key + .iter() + .map(|&c| { + id_to_op_idx + .get(&self.table_desc.columns[c].column_id) + .copied() + }) + .collect::>>() + } + + fn ctx(&self) -> OptimizerContextRef { + self.ctx.clone() + } + + fn functional_dependency(&self) -> FunctionalDependencySet { + let pk_indices = self.stream_key(); + let col_num = self.output_col_idx.len(); + match &pk_indices { + Some(pk_indices) => FunctionalDependencySet::with_key(col_num, pk_indices), + None => FunctionalDependencySet::new(col_num), + } + } +} + +impl SysScan { + pub fn get_table_columns(&self) -> &[ColumnDesc] { + &self.table_desc.columns + } + + /// Get the descs of the output columns. + pub fn column_descs(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| self.get_table_columns()[i].clone()) + .collect() + } + + /// Helper function to create a mapping from `column_id` to `operator_idx` + pub fn get_id_to_op_idx_mapping( + output_col_idx: &[usize], + table_desc: &Rc, + ) -> HashMap { + let mut id_to_op_idx = HashMap::new(); + output_col_idx + .iter() + .enumerate() + .for_each(|(op_idx, tb_idx)| { + let col = &table_desc.columns[*tb_idx]; + id_to_op_idx.insert(col.column_id, op_idx); + }); + id_to_op_idx + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 4a8d9bd872d89..bd1297a4cb92f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -19,7 +19,7 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, TableDesc}; -use risingwave_common::error::{ErrorCode, Result, RwError}; +use risingwave_common::error::Result; use risingwave_common::util::sort_util::ColumnOrder; use super::generic::{GenericPlanNode, GenericPlanRef}; @@ -107,10 +107,6 @@ impl LogicalScan { &self.core.scan_table_type } - pub fn is_sys_table(&self) -> bool { - self.core.is_sys_table() - } - pub fn is_cdc_table(&self) -> bool { matches!(self.core.scan_table_type, ScanTableType::CdcTable) } @@ -557,12 +553,6 @@ impl ToBatch for LogicalScan { impl ToStream for LogicalScan { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { - if self.is_sys_table() { - return Err(RwError::from(ErrorCode::NotImplemented( - "streaming on system table is not allowed".to_string(), - None.into(), - ))); - } if self.predicate().always_true() { if self.is_cdc_table() { Ok(StreamCdcTableScan::new(self.core.clone()).into()) @@ -583,13 +573,6 @@ impl ToStream for LogicalScan { &self, _ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping)> { - if self.is_sys_table() { - return Err(RwError::from(ErrorCode::NotImplemented( - "streaming on system table is not allowed".to_string(), - None.into(), - ))); - } - if self.is_cdc_table() { return Ok(( self.clone().into(), diff --git a/src/frontend/src/optimizer/plan_node/logical_sys_scan.rs b/src/frontend/src/optimizer/plan_node/logical_sys_scan.rs new file mode 100644 index 0000000000000..56985d81a5c27 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_sys_scan.rs @@ -0,0 +1,378 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::rc::Rc; + +use itertools::Itertools; +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::{ColumnDesc, TableDesc}; +use risingwave_common::error::{ErrorCode, Result, RwError}; + +use super::generic::{GenericPlanNode, GenericPlanRef}; +use super::utils::{childless_record, Distill}; +use super::{ + generic, BatchFilter, BatchProject, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, + PredicatePushdown, ToBatch, ToStream, +}; +use crate::expr::{CorrelatedInputRef, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::plan_node::{ + BatchSysSeqScan, ColumnPruningContext, LogicalFilter, LogicalValues, PredicatePushdownContext, + RewriteStreamContext, ToStreamContext, +}; +use crate::optimizer::property::{Cardinality, Order}; +use crate::utils::{ColIndexMapping, Condition, ConditionDisplay}; + +/// `LogicalSysScan` returns contents of a table or other equivalent object +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LogicalSysScan { + pub base: PlanBase, + core: generic::SysScan, +} + +impl From for LogicalSysScan { + fn from(core: generic::SysScan) -> Self { + let base = PlanBase::new_logical_with_core(&core); + Self { base, core } + } +} + +impl From for PlanRef { + fn from(core: generic::SysScan) -> Self { + LogicalSysScan::from(core).into() + } +} + +impl LogicalSysScan { + /// Create a [`LogicalSysScan`] node. Used by planner. + pub fn create( + table_name: String, // explain-only + table_desc: Rc, + ctx: OptimizerContextRef, + table_cardinality: Cardinality, + ) -> Self { + generic::SysScan::new( + table_name, + (0..table_desc.columns.len()).collect(), + table_desc, + ctx, + Condition::true_cond(), + table_cardinality, + ) + .into() + } + + pub fn table_name(&self) -> &str { + &self.core.table_name + } + + /// The cardinality of the table **without** applying the predicate. + pub fn table_cardinality(&self) -> Cardinality { + self.core.table_cardinality + } + + /// Get a reference to the logical scan's table desc. + pub fn table_desc(&self) -> &TableDesc { + self.core.table_desc.as_ref() + } + + /// Get the descs of the output columns. + pub fn column_descs(&self) -> Vec { + self.core.column_descs() + } + + /// Get the logical scan's filter predicate + pub fn predicate(&self) -> &Condition { + &self.core.predicate + } + + /// a vec of `InputRef` corresponding to `output_col_idx`, which can represent a pulled project. + fn output_idx_to_input_ref(&self) -> Vec { + let output_idx = self + .output_col_idx() + .iter() + .enumerate() + .map(|(i, &col_idx)| { + InputRef::new(i, self.table_desc().columns[col_idx].data_type.clone()).into() + }) + .collect_vec(); + output_idx + } + + /// Undo predicate push down when predicate in scan is not supported. + pub fn predicate_pull_up(&self) -> (generic::SysScan, Condition, Option>) { + let mut predicate = self.predicate().clone(); + if predicate.always_true() { + return (self.core.clone(), Condition::true_cond(), None); + } + + let mut inverse_mapping = { + let mapping = ColIndexMapping::new( + self.required_col_idx().iter().map(|i| Some(*i)).collect(), + self.table_desc().columns.len(), + ); + // Since `required_col_idx` mapping is not invertible, we need to inverse manually. + let mut inverse_map = vec![None; mapping.target_size()]; + for (src, dst) in mapping.mapping_pairs() { + inverse_map[dst] = Some(src); + } + ColIndexMapping::new(inverse_map, mapping.source_size()) + }; + + predicate = predicate.rewrite_expr(&mut inverse_mapping); + + let scan_without_predicate = generic::SysScan::new( + self.table_name().to_string(), + self.required_col_idx().to_vec(), + self.core.table_desc.clone(), + self.ctx(), + Condition::true_cond(), + self.table_cardinality(), + ); + let project_expr = if self.required_col_idx() != self.output_col_idx() { + Some(self.output_idx_to_input_ref()) + } else { + None + }; + (scan_without_predicate, predicate, project_expr) + } + + fn clone_with_predicate(&self, predicate: Condition) -> Self { + generic::SysScan::new_inner( + self.table_name().to_string(), + self.output_col_idx().to_vec(), + self.core.table_desc.clone(), + self.base.ctx().clone(), + predicate, + self.table_cardinality(), + ) + .into() + } + + pub fn clone_with_output_indices(&self, output_col_idx: Vec) -> Self { + generic::SysScan::new_inner( + self.table_name().to_string(), + output_col_idx, + self.core.table_desc.clone(), + self.base.ctx().clone(), + self.predicate().clone(), + self.table_cardinality(), + ) + .into() + } + + pub fn output_col_idx(&self) -> &Vec { + &self.core.output_col_idx + } + + pub fn required_col_idx(&self) -> &Vec { + &self.core.required_col_idx + } +} + +impl_plan_tree_node_for_leaf! {LogicalSysScan} + +impl Distill for LogicalSysScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let verbose = self.base.ctx().is_explain_verbose(); + let mut vec = Vec::with_capacity(5); + vec.push(("table", Pretty::from(self.table_name().to_owned()))); + let key_is_columns = + self.predicate().always_true() || self.output_col_idx() == self.required_col_idx(); + let key = if key_is_columns { + "columns" + } else { + "output_columns" + }; + vec.push((key, self.core.columns_pretty(verbose))); + if !key_is_columns { + vec.push(( + "required_columns", + Pretty::Array( + self.required_col_idx() + .iter() + .map(|i| { + let col_name = &self.table_desc().columns[*i].name; + Pretty::from(if verbose { + format!("{}.{}", self.table_name(), col_name) + } else { + col_name.to_string() + }) + }) + .collect(), + ), + )); + } + + if !self.predicate().always_true() { + let input_schema = self.core.fields_pretty_schema(); + vec.push(( + "predicate", + Pretty::display(&ConditionDisplay { + condition: self.predicate(), + input_schema: &input_schema, + }), + )) + } + + if self.table_cardinality() != Cardinality::unknown() { + vec.push(("cardinality", Pretty::display(&self.table_cardinality()))); + } + + childless_record("LogicalSysScan", vec) + } +} + +impl ColPrunable for LogicalSysScan { + fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { + let output_col_idx: Vec = required_cols + .iter() + .map(|i| self.required_col_idx()[*i]) + .collect(); + assert!(output_col_idx + .iter() + .all(|i| self.output_col_idx().contains(i))); + + self.clone_with_output_indices(output_col_idx).into() + } +} + +impl ExprRewritable for LogicalSysScan { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let mut core = self.core.clone(); + core.rewrite_exprs(r); + Self { + base: self.base.clone_with_new_plan_id(), + core, + } + .into() + } +} + +impl PredicatePushdown for LogicalSysScan { + // TODO(kwannoel): Unify this with logical_scan. + fn predicate_pushdown( + &self, + mut predicate: Condition, + _ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + // If the predicate contains `CorrelatedInputRef` or `now()`. We don't push down. + // This case could come from the predicate push down before the subquery unnesting. + struct HasCorrelated {} + impl ExprVisitor for HasCorrelated { + type Result = bool; + + fn merge(a: bool, b: bool) -> bool { + a | b + } + + fn visit_correlated_input_ref(&mut self, _: &CorrelatedInputRef) -> bool { + true + } + } + let non_pushable_predicate: Vec<_> = predicate + .conjunctions + .extract_if(|expr| expr.count_nows() > 0 || HasCorrelated {}.visit_expr(expr)) + .collect(); + let predicate = predicate.rewrite_expr(&mut ColIndexMapping::new( + self.output_col_idx().iter().map(|i| Some(*i)).collect(), + self.table_desc().columns.len(), + )); + if non_pushable_predicate.is_empty() { + self.clone_with_predicate(predicate.and(self.predicate().clone())) + .into() + } else { + return LogicalFilter::create( + self.clone_with_predicate(predicate.and(self.predicate().clone())) + .into(), + Condition { + conjunctions: non_pushable_predicate, + }, + ); + } + } +} + +impl LogicalSysScan { + // TODO(kwannoel): Unify this with logical_scan. + fn to_batch_inner_with_required(&self, required_order: &Order) -> Result { + if self.predicate().always_true() { + required_order + .enforce_if_not_satisfies(BatchSysSeqScan::new(self.core.clone(), vec![]).into()) + } else { + let (scan_ranges, predicate) = self.predicate().clone().split_to_scan_ranges( + self.core.table_desc.clone(), + self.base + .ctx() + .session_ctx() + .config() + .get_max_split_range_gap(), + )?; + let mut scan = self.clone(); + scan.core.predicate = predicate; // We want to keep `required_col_idx` unchanged, so do not call `clone_with_predicate`. + + let plan: PlanRef = if scan.core.predicate.always_false() { + LogicalValues::create(vec![], scan.core.schema(), scan.core.ctx).to_batch()? + } else { + let (scan, predicate, project_expr) = scan.predicate_pull_up(); + + let mut plan: PlanRef = BatchSysSeqScan::new(scan, scan_ranges).into(); + if !predicate.always_true() { + plan = BatchFilter::new(generic::Filter::new(predicate, plan)).into(); + } + if let Some(exprs) = project_expr { + plan = BatchProject::new(generic::Project::new(exprs, plan)).into() + } + plan + }; + + assert_eq!(plan.schema(), self.schema()); + required_order.enforce_if_not_satisfies(plan) + } + } +} + +impl ToBatch for LogicalSysScan { + fn to_batch(&self) -> Result { + self.to_batch_with_order_required(&Order::any()) + } + + fn to_batch_with_order_required(&self, required_order: &Order) -> Result { + let new = self.clone_with_predicate(self.predicate().clone()); + new.to_batch_inner_with_required(required_order) + } +} + +impl ToStream for LogicalSysScan { + fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { + Err(RwError::from(ErrorCode::NotImplemented( + "streaming on system table is not allowed".to_string(), + None.into(), + ))) + } + + fn logical_rewrite_for_stream( + &self, + _ctx: &mut RewriteStreamContext, + ) -> Result<(PlanRef, ColIndexMapping)> { + Err(RwError::from(ErrorCode::NotImplemented( + "streaming on system table is not allowed".to_string(), + None.into(), + ))) + } +} diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index e048f0a4707de..77f8fdb606077 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -768,6 +768,7 @@ mod batch_simple_agg; mod batch_sort; mod batch_sort_agg; mod batch_source; +mod batch_sys_seq_scan; mod batch_table_function; mod batch_topn; mod batch_union; @@ -793,6 +794,7 @@ mod logical_project_set; mod logical_scan; mod logical_share; mod logical_source; +mod logical_sys_scan; mod logical_table_function; mod logical_topn; mod logical_union; @@ -854,6 +856,7 @@ pub use batch_simple_agg::BatchSimpleAgg; pub use batch_sort::BatchSort; pub use batch_sort_agg::BatchSortAgg; pub use batch_source::BatchSource; +pub use batch_sys_seq_scan::BatchSysSeqScan; pub use batch_table_function::BatchTableFunction; pub use batch_topn::BatchTopN; pub use batch_union::BatchUnion; @@ -879,6 +882,7 @@ pub use logical_project_set::LogicalProjectSet; pub use logical_scan::LogicalScan; pub use logical_share::LogicalShare; pub use logical_source::LogicalSource; +pub use logical_sys_scan::LogicalSysScan; pub use logical_table_function::LogicalTableFunction; pub use logical_topn::LogicalTopN; pub use logical_union::LogicalUnion; @@ -944,6 +948,7 @@ macro_rules! for_all_plan_nodes { , { Logical, Filter } , { Logical, Project } , { Logical, Scan } + , { Logical, SysScan } , { Logical, Source } , { Logical, Insert } , { Logical, Delete } @@ -973,6 +978,7 @@ macro_rules! for_all_plan_nodes { , { Batch, Delete } , { Batch, Update } , { Batch, SeqScan } + , { Batch, SysSeqScan } , { Batch, HashJoin } , { Batch, NestedLoopJoin } , { Batch, Values } @@ -1035,6 +1041,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, Filter } , { Logical, Project } , { Logical, Scan } + , { Logical, SysScan } , { Logical, Source } , { Logical, Insert } , { Logical, Delete } @@ -1070,6 +1077,7 @@ macro_rules! for_batch_plan_nodes { , { Batch, Project } , { Batch, Filter } , { Batch, SeqScan } + , { Batch, SysSeqScan } , { Batch, HashJoin } , { Batch, NestedLoopJoin } , { Batch, Values } diff --git a/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs b/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs index a9eb5639fcf85..2072defa62726 100644 --- a/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/cardinality_visitor.rs @@ -56,8 +56,8 @@ impl CardinalityVisitor { return None; } let scan = proj.input(); - let scan = scan.as_logical_scan()?; - if scan.is_sys_table() && scan.table_name() == "rw_schemas" { + let scan = scan.as_logical_sys_scan()?; + if scan.table_name() == "rw_schemas" { if let Some(name) = scan .output_col_idx() .iter() diff --git a/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs b/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs index e899021b8bdef..daabe4d4b4715 100644 --- a/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs @@ -52,22 +52,16 @@ impl PlanVisitor for RelationCollectorVisitor { } fn visit_batch_seq_scan(&mut self, plan: &crate::optimizer::plan_node::BatchSeqScan) { - if !plan.core().is_sys_table() { - self.relations.insert(plan.core().table_desc.table_id); - } + self.relations.insert(plan.core().table_desc.table_id); } fn visit_logical_scan(&mut self, plan: &LogicalScan) { - if !plan.is_sys_table() { - self.relations.insert(plan.table_desc().table_id); - } + self.relations.insert(plan.table_desc().table_id); } fn visit_stream_table_scan(&mut self, plan: &StreamTableScan) { let logical = plan.core(); - if !logical.is_sys_table() { - self.relations.insert(logical.table_desc.table_id); - } + self.relations.insert(logical.table_desc.table_id); } fn visit_batch_source(&mut self, plan: &BatchSource) { diff --git a/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs b/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs index e93a5c4e0bb1c..0f679f7f3030d 100644 --- a/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs @@ -13,7 +13,7 @@ // limitations under the License. use super::{DefaultBehavior, Merge}; -use crate::optimizer::plan_node::{BatchSeqScan, LogicalScan, StreamTableScan}; +use crate::optimizer::plan_node::{BatchSysSeqScan, LogicalSysScan, StreamTableScan}; use crate::optimizer::plan_visitor::PlanVisitor; use crate::PlanRef; @@ -36,15 +36,16 @@ impl PlanVisitor for SysTableVisitor { Merge(|a, b| a | b) } - fn visit_batch_seq_scan(&mut self, batch_seq_scan: &BatchSeqScan) -> bool { - batch_seq_scan.core().is_sys_table() + fn visit_batch_sys_seq_scan(&mut self, _batch_seq_scan: &BatchSysSeqScan) -> bool { + true } - fn visit_logical_scan(&mut self, logical_scan: &LogicalScan) -> bool { - logical_scan.is_sys_table() + fn visit_logical_sys_scan(&mut self, _logical_scan: &LogicalSysScan) -> bool { + true } - fn visit_stream_table_scan(&mut self, stream_table_scan: &StreamTableScan) -> bool { - stream_table_scan.core().is_sys_table() + // Sys scan not allowed for streaming. + fn visit_stream_table_scan(&mut self, _stream_table_scan: &StreamTableScan) -> bool { + false } } diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index d4686bbf15c28..b4987403a7926 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -27,7 +27,7 @@ use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ LogicalApply, LogicalHopWindow, LogicalJoin, LogicalProject, LogicalScan, LogicalShare, - LogicalSource, LogicalTableFunction, LogicalValues, PlanRef, + LogicalSource, LogicalSysScan, LogicalTableFunction, LogicalValues, PlanRef, }; use crate::optimizer::property::Cardinality; use crate::planner::Planner; @@ -57,13 +57,10 @@ impl Planner { } pub(crate) fn plan_sys_table(&mut self, sys_table: BoundSystemTable) -> Result { - Ok(LogicalScan::create( + Ok(LogicalSysScan::create( sys_table.sys_table_catalog.name().to_string(), - ScanTableType::SysTable, Rc::new(sys_table.sys_table_catalog.table_desc()), - vec![], self.ctx(), - false, Cardinality::unknown(), // TODO(card): cardinality of system table ) .into()) diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index b20197f9a491d..fdc546a7d735f 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -916,26 +916,25 @@ impl BatchPlanFragmenter { // Do not visit next stage. return Ok(None); } + if let Some(scan_node) = node.as_batch_sys_seq_scan() { + let name = scan_node.core().table_name.to_owned(); + return Ok(Some(TableScanInfo::system_table(name))); + } if let Some(scan_node) = node.as_batch_seq_scan() { let name = scan_node.core().table_name.to_owned(); - let info = if scan_node.core().is_sys_table() { - TableScanInfo::system_table(name) - } else { - let table_desc = &*scan_node.core().table_desc; - let table_catalog = self - .catalog_reader - .read_guard() - .get_table_by_id(&table_desc.table_id) - .cloned() - .map_err(RwError::from)?; - let vnode_mapping = self - .worker_node_manager - .fragment_mapping(table_catalog.fragment_id)?; - let partitions = - derive_partitions(scan_node.scan_ranges(), table_desc, &vnode_mapping); - TableScanInfo::new(name, partitions) - }; + let table_desc = &*scan_node.core().table_desc; + let table_catalog = self + .catalog_reader + .read_guard() + .get_table_by_id(&table_desc.table_id) + .cloned() + .map_err(RwError::from)?; + let vnode_mapping = self + .worker_node_manager + .fragment_mapping(table_catalog.fragment_id)?; + let partitions = derive_partitions(scan_node.scan_ranges(), table_desc, &vnode_mapping); + let info = TableScanInfo::new(name, partitions); Ok(Some(info)) } else { node.inputs() From 7dd89f26146c206d106583be392472018af6c5c2 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 16 Nov 2023 16:28:15 +0800 Subject: [PATCH 16/36] fix(ci): remove sink json payload and jdbc sink from connector integration test (#13449) Co-authored-by: Noel Kwan <47273164+kwannoel@users.noreply.github.com> --- ci/scripts/connector-node-integration-test.sh | 16 ++---- .../python-client/integration_tests.py | 54 ------------------- 2 files changed, 3 insertions(+), 67 deletions(-) diff --git a/ci/scripts/connector-node-integration-test.sh b/ci/scripts/connector-node-integration-test.sh index 5971cd8986ab5..6b3147bbeb193 100755 --- a/ci/scripts/connector-node-integration-test.sh +++ b/ci/scripts/connector-node-integration-test.sh @@ -99,10 +99,9 @@ else exit 1 fi -sink_input_feature=("" "--input_binary_file=./data/sink_input --data_format_use_json=False") -upsert_sink_input_feature=("--input_file=./data/upsert_sink_input.json" - "--input_binary_file=./data/upsert_sink_input --data_format_use_json=False") -type=("Json format" "StreamChunk format") +sink_input_feature=("--input_binary_file=./data/sink_input --data_format_use_json=False") +upsert_sink_input_feature=("--input_binary_file=./data/upsert_sink_input --data_format_use_json=False") +type=("StreamChunk format") ${MC_PATH} mb minio/bucket for ((i=0; i<${#type[@]}; i++)); do @@ -115,15 +114,6 @@ for ((i=0; i<${#type[@]}; i++)); do exit 1 fi - echo "--- running jdbc ${type[i]} integration tests" - cd ${RISINGWAVE_ROOT}/java/connector-node/python-client - if python3 integration_tests.py --jdbc_sink ${sink_input_feature[i]}; then - echo "Jdbc sink ${type[i]} test passed" - else - echo "Jdbc sink ${type[i]} test failed" - exit 1 - fi - # test upsert mode echo "--- running iceberg upsert mode ${type[i]} integration tests" cd ${RISINGWAVE_ROOT}/java/connector-node/python-client diff --git a/java/connector-node/python-client/integration_tests.py b/java/connector-node/python-client/integration_tests.py index 64fa949f48ce2..99d91b74dd4b4 100644 --- a/java/connector-node/python-client/integration_tests.py +++ b/java/connector-node/python-client/integration_tests.py @@ -211,45 +211,6 @@ def test_sink(prop, format, payload_input, table_schema, is_coordinated=False): exit(1) -def validate_jdbc_sink(input_file): - conn = psycopg2.connect( - "dbname=test user=test password=connector host=localhost port=5432" - ) - cur = conn.cursor() - cur.execute("SELECT * FROM test") - rows = cur.fetchall() - expected = [list(row.values()) for batch in load_input(input_file) for row in batch] - - def convert(b): - return [(item[1]["id"], item[1]["name"]) for item in b] - - expected = convert(expected) - - if len(rows) != len(expected): - print( - "Integration test failed: expected {} rows, but got {}".format( - len(expected), len(rows) - ) - ) - exit(1) - for i in range(len(rows)): - if len(rows[i]) != len(expected[i]): - print( - "Integration test failed: expected {} columns, but got {}".format( - len(expected[i]), len(rows[i]) - ) - ) - exit(1) - for j in range(len(rows[i])): - if rows[i][j] != expected[i][j]: - print( - "Integration test failed: expected {} at row {}, column {}, but got {}".format( - expected[i][j], i, j, rows[i][j] - ) - ) - exit(1) - - def test_file_sink(param): prop = { "connector": "file", @@ -258,18 +219,6 @@ def test_file_sink(param): test_sink(prop, **param) -def test_jdbc_sink(input_file, param): - prop = { - "connector": "jdbc", - "jdbc.url": "jdbc:postgresql://localhost:5432/test?user=test&password=connector", - "table.name": "test", - "type": "upsert", - } - test_sink(prop, **param) - # validate results - validate_jdbc_sink(input_file) - - def test_elasticsearch_sink(param): prop = { "connector": "elasticsearch", @@ -333,7 +282,6 @@ def test_stream_chunk_data_format(param): formatter_class=argparse.ArgumentDefaultsHelpFormatter ) parser.add_argument("--file_sink", action="store_true", help="run file sink test") - parser.add_argument("--jdbc_sink", action="store_true", help="run jdbc sink test") parser.add_argument( "--stream_chunk_format_test", action="store_true", @@ -390,8 +338,6 @@ def test_stream_chunk_data_format(param): if args.file_sink: test_file_sink(param) - if args.jdbc_sink: - test_jdbc_sink(args.input_file, param) if args.iceberg_sink: test_iceberg_sink(param) if args.deltalake_sink: From 98f8e1a4464eee346b09f0d8ed0201a05c9e2e0e Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Thu, 16 Nov 2023 16:41:59 +0800 Subject: [PATCH 17/36] perf(optimizer): omit unnecessary clone (#13466) --- .../tests/testdata/output/cse_expr.yaml | 2 +- .../tests/testdata/output/explain.yaml | 2 +- .../planner_test/tests/testdata/output/expr.yaml | 4 ++-- .../tests/testdata/output/nexmark.yaml | 2 +- .../src/optimizer/heuristic_optimizer.rs | 16 ++++++++++++++-- 5 files changed, 19 insertions(+), 7 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml index 5057fa13473c3..fe41cb83794e2 100644 --- a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml @@ -84,5 +84,5 @@ sql: | with t(v, arr) as (select 1, array[2, 3]) select v < all(arr), v < some(arr) from t; batch_plan: |- - BatchProject { exprs: [All((1:Int32 < $expr10048)) as $expr1, Some((1:Int32 < $expr10048)) as $expr2] } + BatchProject { exprs: [All((1:Int32 < $expr10015)) as $expr1, Some((1:Int32 < $expr10015)) as $expr2] } └─BatchValues { rows: [[1:Int32, ARRAY[2, 3]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index eced1aaf718bc..0faca9c10291f 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -65,7 +65,7 @@ "stages": { "0": { "root": { - "plan_node_id": 10031, + "plan_node_id": 10020, "plan_node_type": "BatchValues", "schema": [ { diff --git a/src/frontend/planner_test/tests/testdata/output/expr.yaml b/src/frontend/planner_test/tests/testdata/output/expr.yaml index 65df2522c81dd..7f117e64be4e9 100644 --- a/src/frontend/planner_test/tests/testdata/output/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/expr.yaml @@ -450,7 +450,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: |- - BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10041, ARRAY[2]:List(Int32)))) as $expr1] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10013, ARRAY[2]:List(Int32)))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List(Int32)]] } @@ -473,7 +473,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: |- - BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10041, ARRAY[2]:List(Int32)))) as $expr1] } + BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10013, ARRAY[2]:List(Int32)))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index 9b754512b1da3..592696190a862 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -1127,7 +1127,7 @@ JOIN side_input FOR SYSTEM_TIME AS OF PROCTIME() S ON mod(B.auction, 10000) = S.key sink_plan: |- - StreamSink { type: append-only, columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr10042(hidden), side_input.key(hidden)] } + StreamSink { type: append-only, columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr10022(hidden), side_input.key(hidden)] } └─StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } ├─StreamExchange { dist: HashShard($expr1) } │ └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] } diff --git a/src/frontend/src/optimizer/heuristic_optimizer.rs b/src/frontend/src/optimizer/heuristic_optimizer.rs index 5e4a6b2c8570d..7db756977bbe7 100644 --- a/src/frontend/src/optimizer/heuristic_optimizer.rs +++ b/src/frontend/src/optimizer/heuristic_optimizer.rs @@ -62,12 +62,17 @@ impl<'a> HeuristicOptimizer<'a> { } fn optimize_inputs(&mut self, plan: PlanRef) -> PlanRef { + let pre_applied = self.stats.total_applied(); let inputs = plan .inputs() .into_iter() .map(|sub_tree| self.optimize(sub_tree)) .collect_vec(); - plan.clone_with_inputs(&inputs) + if pre_applied != self.stats.total_applied() { + plan.clone_with_inputs(&inputs) + } else { + plan + } } pub fn optimize(&mut self, mut plan: PlanRef) -> PlanRef { @@ -102,6 +107,7 @@ impl<'a> HeuristicOptimizer<'a> { } pub struct Stats { + total_applied: usize, rule_counter: HashMap, } @@ -109,10 +115,12 @@ impl Stats { pub fn new() -> Self { Self { rule_counter: HashMap::new(), + total_applied: 0, } } pub fn count_rule(&mut self, rule: &BoxedRule) { + self.total_applied += 1; match self.rule_counter.entry(rule.description().to_string()) { Entry::Occupied(mut entry) => { *entry.get_mut() += 1; @@ -124,7 +132,11 @@ impl Stats { } pub fn has_applied_rule(&self) -> bool { - !self.rule_counter.is_empty() + self.total_applied != 0 + } + + pub fn total_applied(&self) -> usize { + self.total_applied } } From 53d19565c650d3deb6679039c46ad6c0f089ec4a Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 16 Nov 2023 17:04:18 +0800 Subject: [PATCH 18/36] chore(ci): notify for more tests (#13463) --- ci/scripts/notify.py | 3 +++ ci/workflows/main-cron.yml | 3 +++ 2 files changed, 6 insertions(+) diff --git a/ci/scripts/notify.py b/ci/scripts/notify.py index 5ca526115810c..7ea47781a1a8e 100755 --- a/ci/scripts/notify.py +++ b/ci/scripts/notify.py @@ -9,6 +9,9 @@ TEST_MAP = { "test-notify": ["noelkwan", "noelkwan"], "backfill-tests": ["noelkwan"], + "backwards-compat-tests": ["noelkwan"], + "fuzz-test": ["noelkwan"], + "e2e-test-release": ["zhi"], "e2e-iceberg-sink-tests": ["renjie"], "e2e-java-binding-tests": ["yiming"], "e2e-clickhouse-sink-tests": ["bohan"], diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 42fccac95af20..ac7867623dbf4 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -59,6 +59,7 @@ steps: retry: *auto-retry - label: "end-to-end test (release)" + key: "e2e-test-release" command: "ci/scripts/cron-e2e-test.sh -p ci-release -m ci-3streaming-2serving-3fe" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-test") depends_on: @@ -141,6 +142,7 @@ steps: retry: *auto-retry - label: "fuzz test" + key: "fuzz-test" command: "ci/scripts/cron-fuzz-test.sh -p ci-release" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-sqlsmith-fuzzing-tests") depends_on: @@ -481,6 +483,7 @@ steps: # Backwards compatibility tests - label: "Backwards compatibility tests" + key: "backwards-compat-tests" command: "RW_COMMIT=$BUILDKITE_COMMIT ci/scripts/backwards-compat-test.sh -p ci-release" if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-backwards-compat-tests") depends_on: From c6dffdf391dc64fa832061b938cb2eebc3591699 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 16 Nov 2023 17:51:08 +0800 Subject: [PATCH 19/36] feat(datagen): support `timestamptz` (#13451) --- e2e_test/source/basic/datagen.slt | 6 ++++- src/common/src/array/data_chunk.rs | 7 ++++++ src/common/src/field_generator/mod.rs | 23 +++++++++++++++++- src/common/src/field_generator/timestamp.rs | 24 ++++++++++++++++++- .../src/source/datagen/source/reader.rs | 23 ++++++++++++------ 5 files changed, 73 insertions(+), 10 deletions(-) diff --git a/e2e_test/source/basic/datagen.slt b/e2e_test/source/basic/datagen.slt index dcda3718e49d5..91c51f624a1ea 100644 --- a/e2e_test/source/basic/datagen.slt +++ b/e2e_test/source/basic/datagen.slt @@ -102,7 +102,7 @@ statement ok drop table s1; statement ok -create table s1 (v1 struct, t1 timestamp, c1 varchar) with ( +create table s1 (v1 struct, t1 timestamp, z1 timestamptz, c1 varchar) with ( connector = 'datagen', fields.v1.v2.kind = 'random', fields.v1.v2.min = '1', @@ -112,6 +112,10 @@ create table s1 (v1 struct, t1 timestamp, c1 varchar) with ( fields.t1.max_past = '2h 37min', fields.t1.max_past_mode = 'relative', fields.t1.seed = '3', + fields.z1.kind = 'random', + fields.z1.max_past = '2h 37min', + fields.z1.max_past_mode = 'relative', + fields.z1.seed = '3', fields.c1.kind = 'random', fields.c1.length = '100', fields.c1.seed = '3', diff --git a/src/common/src/array/data_chunk.rs b/src/common/src/array/data_chunk.rs index 98a237814176d..7cdeeeee4e2a6 100644 --- a/src/common/src/array/data_chunk.rs +++ b/src/common/src/array/data_chunk.rs @@ -919,6 +919,13 @@ impl DataChunkTestExt for DataChunk { .generate_datum(offset); array_builder.append(datum); } + DataType::Timestamptz => { + let datum = + FieldGeneratorImpl::with_timestamptz(None, None, None, Self::SEED) + .expect("create timestamptz generator should succeed") + .generate_datum(offset); + array_builder.append(datum); + } _ if data_type.is_numeric() => { let mut data_gen = FieldGeneratorImpl::with_number_random( data_type.clone(), diff --git a/src/common/src/field_generator/mod.rs b/src/common/src/field_generator/mod.rs index 02a958aa0ebec..0daa2b640c607 100644 --- a/src/common/src/field_generator/mod.rs +++ b/src/common/src/field_generator/mod.rs @@ -26,7 +26,7 @@ pub use timestamp::*; pub use varchar::*; use crate::array::{ListValue, StructValue}; -use crate::types::{DataType, Datum, ScalarImpl, Timestamp}; +use crate::types::{DataType, Datum, ScalarImpl, Timestamp, Timestamptz}; pub const DEFAULT_MIN: i16 = i16::MIN; pub const DEFAULT_MAX: i16 = i16::MAX; @@ -96,6 +96,7 @@ pub enum FieldGeneratorImpl { VarcharRandomFixedLength(VarcharRandomFixedLengthField), VarcharConstant, Timestamp(ChronoField), + Timestamptz(ChronoField), Struct(Vec<(String, FieldGeneratorImpl)>), List(Box, usize), } @@ -189,6 +190,20 @@ impl FieldGeneratorImpl { )?)) } + pub fn with_timestamptz( + base: Option>, + max_past: Option, + max_past_mode: Option, + seed: u64, + ) -> Result { + Ok(FieldGeneratorImpl::Timestamptz(ChronoField::new( + base, + max_past, + max_past_mode, + seed, + )?)) + } + pub fn with_varchar(varchar_property: &VarcharProperty, seed: u64) -> Self { match varchar_property { VarcharProperty::RandomFixedLength(length_option) => { @@ -235,6 +250,7 @@ impl FieldGeneratorImpl { FieldGeneratorImpl::VarcharRandomVariableLength(f) => f.generate(offset), FieldGeneratorImpl::VarcharConstant => VarcharConstant::generate_json(), FieldGeneratorImpl::Timestamp(f) => f.generate(offset), + FieldGeneratorImpl::Timestamptz(f) => f.generate(offset), FieldGeneratorImpl::Struct(fields) => { let map = fields .iter_mut() @@ -267,6 +283,7 @@ impl FieldGeneratorImpl { FieldGeneratorImpl::VarcharRandomVariableLength(f) => f.generate_datum(offset), FieldGeneratorImpl::VarcharConstant => VarcharConstant::generate_datum(), FieldGeneratorImpl::Timestamp(f) => f.generate_datum(offset), + FieldGeneratorImpl::Timestamptz(f) => f.generate_datum(offset), FieldGeneratorImpl::Struct(fields) => { let data = fields .iter_mut() @@ -328,6 +345,7 @@ mod tests { DataType::Float64, DataType::Varchar, DataType::Timestamp, + DataType::Timestamptz, ] { let mut generator = match data_type { DataType::Varchar => FieldGeneratorImpl::with_varchar( @@ -337,6 +355,9 @@ mod tests { DataType::Timestamp => { FieldGeneratorImpl::with_timestamp(None, None, None, seed).unwrap() } + DataType::Timestamptz => { + FieldGeneratorImpl::with_timestamptz(None, None, None, seed).unwrap() + } _ => FieldGeneratorImpl::with_number_random(data_type, None, None, seed).unwrap(), }; diff --git a/src/common/src/field_generator/timestamp.rs b/src/common/src/field_generator/timestamp.rs index 54c55de273965..faddd6d0a32a8 100644 --- a/src/common/src/field_generator/timestamp.rs +++ b/src/common/src/field_generator/timestamp.rs @@ -22,7 +22,7 @@ use serde_json::Value; use tracing::debug; use super::DEFAULT_MAX_PAST; -use crate::types::{Datum, Scalar, Timestamp}; +use crate::types::{Datum, Scalar, Timestamp, Timestamptz}; pub struct ChronoField { max_past: Duration, @@ -106,3 +106,25 @@ impl ChronoFieldInner for Timestamp { Value::String(self.0.to_string()) } } + +impl ChronoFieldInner for Timestamptz { + fn from_now() -> Self { + Timestamptz::from( + Utc::now() + .duration_round(Duration::microseconds(1)) + .unwrap(), + ) + } + + fn from_base(base: DateTime) -> Self { + Timestamptz::from(base) + } + + fn minus(&self, duration: Duration) -> Self { + Timestamptz::from(self.to_datetime_utc() - duration) + } + + fn to_json(&self) -> Value { + Value::String(self.to_string()) + } +} diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index 11cb9db08c48a..2cc56924876d3 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -213,7 +213,7 @@ fn generator_from_data_type( None => split_index, }; match data_type { - DataType::Timestamp => { + ty @ (DataType::Timestamp | DataType::Timestamptz) => { let max_past_key = format!("fields.{}.max_past", name); let max_past_value = fields_option_map.get(&max_past_key).cloned(); let max_past_mode_key = format!("fields.{}.max_past_mode", name); @@ -230,12 +230,21 @@ fn generator_from_data_type( None => None, }; - FieldGeneratorImpl::with_timestamp( - basetime, - max_past_value, - max_past_mode_value, - random_seed, - ) + if ty == DataType::Timestamptz { + FieldGeneratorImpl::with_timestamptz( + basetime, + max_past_value, + max_past_mode_value, + random_seed, + ) + } else { + FieldGeneratorImpl::with_timestamp( + basetime, + max_past_value, + max_past_mode_value, + random_seed, + ) + } } DataType::Varchar => { let length_key = format!("fields.{}.length", name); From acb33762b65767539deb0e7b2a9b3c56259ceb5f Mon Sep 17 00:00:00 2001 From: Yingjun Wu Date: Thu, 16 Nov 2023 02:31:32 -0800 Subject: [PATCH 20/36] chore: Update README.md (#13468) Co-authored-by: hengm3467 <100685635+hengm3467@users.noreply.github.com> --- README.md | 28 +++++++++++----------------- 1 file changed, 11 insertions(+), 17 deletions(-) diff --git a/README.md b/README.md index f2caa86a23321..ab83b8b80a3e9 100644 --- a/README.md +++ b/README.md @@ -60,7 +60,7 @@ -RisingWave is a distributed SQL streaming database that enables cost-efficient and reliable processing of streaming data. +RisingWave is a distributed SQL streaming database that enables simple, efficient, and reliable processing of streaming data. ![RisingWave](https://github.com/risingwavelabs/risingwave-docs/blob/main/docs/images/new_archi_grey.png) @@ -96,30 +96,24 @@ For **Kubernetes deployments**, please refer to [Kubernetes with Helm](https://d ## Why RisingWave for stream processing? -RisingWave adaptly addresses some of the most challenging problems in stream processing. Compared to existing stream processing systems like [Apache Flink](https://flink.apache.org/), [Apache Spark Streaming](https://spark.apache.org/docs/latest/streaming-programming-guide.html), and [KsqlDB](https://ksqldb.io/), RisingWave stands out in two primary dimensions: **Ease-of-use** and **efficiency**, thanks to its **[PostgreSQL](https://www.postgresql.org/)-style interaction experience** and **[Snowflake](https://snowflake.com/)-like architectural design** (i.e., compute-storage decoupling). +RisingWave specializes in providing **incrementally updated, consistent materialized views** — a persistent data structure that represents the results of stream processing. RisingWave significantly reduces the complexity of building stream processing applications by allowing developers to express intricate stream processing logic through cascaded materialized views. Furthermore, it allows users to persist data directly within the system, eliminating the need to deliver results to external databases for storage and query serving. + +Compared to existing stream processing systems like [Apache Flink](https://flink.apache.org/), [Apache Spark Streaming](https://spark.apache.org/docs/latest/streaming-programming-guide.html), and [KsqlDB](https://ksqldb.io/), RisingWave stands out in two primary dimensions: **Ease-of-use** and **cost efficiency**, thanks to its **[PostgreSQL](https://www.postgresql.org/)-style interaction experience** and **[Snowflake](https://snowflake.com/)-like architectural design** (i.e., decoupled storage and compute). ### Ease-of-use * **Simple to learn** * RisingWave speaks PostgreSQL-style SQL, enabling users to dive into stream processing in much the same way as operating a PostgreSQL database. -* **Simple to verify correctness** - * RisingWave persists results in materialized views and allow users to break down complex stream computation programs into stacked materialized views, simplifying program development and result verification. -* **Simple to maintain and operate** - * RisingWave abstracts away unnecessary low-level details, allowing users to concentrate solely on SQL code-level issues. +* **Simple to develop** + * RisingWave operates as a relational database, allowing users to decompose stream processing logic into smaller, manageable, stacked materialized views, rather than dealing with extensive computational programs. * **Simple to integrate** * With integrations to a diverse range of cloud systems and the PostgreSQL ecosystem, RisingWave boasts a rich and expansive ecosystem, making it straightforward to incorporate into existing infrastructures. -### Efficiency -* **High resource utilization** - * Queries in RisingWave leverage shared computational resources, eliminating the need for users to manually allocate resources for each query. -* **No compromise on large state management** - * The decoupled compute-storage architecture of RisingWave ensures remote persistence of internal states, and users never need to worry about the size of internal states when handling complex queries. -* **Highly efficient in multi-stream joins** - * RisingWave has made significant optimizations for multiple stream join scenarios. Users can easily join 10-20 streams (or more) efficiently in a production environment. +### Cost efficiency +* **Highly efficient in complex queries** + * RisingWave persists internal states in remote storages (e.g., S3), and users can confidently and efficiently perform complex streaming queries (e.g., joining dozens of data streams) in a production environment, without worrying about state size. * **Transparent dynamic scaling** - * RisingWave supports near-instantaneous dynamic scaling without any service interruptions. + * RisingWave's state management mechanism enables near-instantaneous dynamic scaling without any service interruptions. * **Instant failure recovery** - * RisingWave's state management mechanism allows it to recover from failure in seconds, not minutes or hours. -* **Simplified data stack** - * RisingWave's ability to store data and serve queries eliminates the need for separate maintenance of stream processors and databases. Users can effortlessly connect RisingWave to their preferred BI tools or through client libraries. + * RisingWave's state management mechanism also allows it to recover from failure in seconds, not minutes or hours. ## RisingWave's limitations RisingWave isn’t a panacea for all data engineering hurdles. It has its own set of limitations: From 059a84037b157ff4465f862e58a4968003dc2790 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Thu, 16 Nov 2023 20:47:53 +0800 Subject: [PATCH 21/36] fix(test): fix non deterministic group topn test (#13473) --- e2e_test/streaming/group_top_n/main.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/e2e_test/streaming/group_top_n/main.slt b/e2e_test/streaming/group_top_n/main.slt index 16d3187e16d7f..620c93503e286 100644 --- a/e2e_test/streaming/group_top_n/main.slt +++ b/e2e_test/streaming/group_top_n/main.slt @@ -14,7 +14,7 @@ where rank <= 3; statement ok create materialized view mv_with_expr_in_window as select x, y from ( - select *, ROW_NUMBER() OVER (PARTITION BY x/2 ORDER BY 6-y) as rank from t + select *, ROW_NUMBER() OVER (PARTITION BY x/2 ORDER BY 6-y, x) as rank from t ) where rank <= 3; @@ -68,7 +68,7 @@ select * from mv; 3 3 query II rowsort -select * from mv_with_expr_in_window; +select * from mv_with_expr_in_window order by x, y; ---- 1 3 1 4 From 942a526c3a79368072849fa6937b4188871b7882 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Thu, 16 Nov 2023 21:13:10 +0800 Subject: [PATCH 22/36] feat: allow overwrite stream_rate_control in with clause (#13009) Signed-off-by: tabVersion --- e2e_test/ddl/throttle.slt | 23 ++++++++ e2e_test/sink/kafka/create_sink.slt | 29 +++++++++++ e2e_test/source/basic/kafka.slt | 13 +++++ src/frontend/src/handler/create_mv.rs | 10 +++- .../src/optimizer/optimizer_context.rs | 14 ++++- .../optimizer/plan_node/stream_fs_fetch.rs | 7 +-- .../src/optimizer/plan_node/stream_source.rs | 7 +-- .../optimizer/plan_node/stream_table_scan.rs | 8 +-- src/frontend/src/utils/mod.rs | 3 +- src/frontend/src/utils/overwrite_options.rs | 52 +++++++++++++++++++ src/stream/src/from_proto/source/fs_fetch.rs | 2 +- .../src/from_proto/source/trad_source.rs | 2 +- src/stream/src/from_proto/stream_scan.rs | 3 +- 13 files changed, 147 insertions(+), 26 deletions(-) create mode 100644 e2e_test/ddl/throttle.slt create mode 100644 src/frontend/src/utils/overwrite_options.rs diff --git a/e2e_test/ddl/throttle.slt b/e2e_test/ddl/throttle.slt new file mode 100644 index 0000000000000..6395697f0df79 --- /dev/null +++ b/e2e_test/ddl/throttle.slt @@ -0,0 +1,23 @@ +# streaming_rate_limit also applies to create sink and create source, please refer to +# e2e_test/source/basic/kafka.slt and e2e_test/sink/kafka/create_sink.slt for this part + +statement ok +create table t1 (v1 int); + +# tracked in https://github.com/risingwavelabs/risingwave/issues/13474 +# create with duplicate streaming_rate_limit +# statement error +# create materialized view mv1 with (streaming_rate_limit = 1000, streaming_rate_limit = 2000) as select * from t1; + +# create with unknown fields +statement error unexpected options in WITH clause +create materialized view mv1 with (streaming_rate_limit = 1000, unknown_field = 2000) as select * from t1; + +statement ok +create materialized view mv1 with (streaming_rate_limit = 1000) as select * from t1; + +statement ok +drop materialized view mv1; + +statement ok +drop table t1; diff --git a/e2e_test/sink/kafka/create_sink.slt b/e2e_test/sink/kafka/create_sink.slt index 4aabec688840e..6a447bd9fc7ce 100644 --- a/e2e_test/sink/kafka/create_sink.slt +++ b/e2e_test/sink/kafka/create_sink.slt @@ -134,9 +134,38 @@ create sink multiple_pk from t_kafka with ( primary_key = 'id,v_varchar' ); +# throttle option +statement ok +create sink multiple_pk_throttle from t_kafka with ( + connector = 'kafka', + properties.bootstrap.server = 'message_queue:29092', + topic = 'test-rw-sink-debezium', + type = 'debezium', + primary_key = 'id,v_varchar', + streaming_rate_limit = 200 +); + +statement ok +create sink multiple_pk_throttle_1 + as select * from t_kafka + with ( + connector = 'kafka', + properties.bootstrap.server = 'message_queue:29092', + topic = 'test-rw-sink-debezium', + type = 'debezium', + primary_key = 'id,v_varchar', + streaming_rate_limit = 200 +); + statement ok drop sink multiple_pk; +statement ok +drop sink multiple_pk_throttle; + +statement ok +drop sink multiple_pk_throttle_1; + statement error Sink primary key column not found: invalid. create sink invalid_pk_column from t_kafka with ( connector = 'kafka', diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index 941d6ae98c0ae..56097de50809a 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -372,6 +372,16 @@ create table s28 (id bytea, PRIMARY KEY(_rw_key)) with ( scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE BYTES +# throttle option +statement ok +create table s29 (id bytea, PRIMARY KEY(_rw_key)) with ( + connector = 'kafka', + topic = 'kafka_source_format_bytes', + properties.bootstrap.server = 'message_queue:29092', + scan.startup.mode = 'earliest', + streaming_rate_limit = 200 +) FORMAT PLAIN ENCODE BYTES + statement ok CREATE TABLE mongo_customers ( _id BIGINT PRIMARY KEY, @@ -842,6 +852,9 @@ drop table s27 statement ok drop table s28 +statement ok +drop table s29 + statement ok DROP TABLE mongo_customers; diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index aeadf35aa2d8f..4c3beaf922ac5 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -16,7 +16,8 @@ use either::Either; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::acl::AclMode; -use risingwave_common::error::{ErrorCode, Result}; +use risingwave_common::error::ErrorCode::ProtocolError; +use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_pb::catalog::{CreateType, PbTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{EmitMode, Ident, ObjectName, Query}; @@ -165,6 +166,13 @@ pub async fn handle_create_mv( let (mut table, graph) = { let context = OptimizerContext::from_handler_args(handler_args); + if !context.with_options().is_empty() { + // get other useful fields by `remove`, the logic here is to reject unknown options. + return Err(RwError::from(ProtocolError(format!( + "unexpected options in WITH clause: {:?}", + context.with_options().keys() + )))); + } let has_order_by = !query.order_by.is_empty(); if has_order_by { diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index e4b8d3c566813..7f7f867daa3b2 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -24,7 +24,7 @@ use crate::expr::{CorrelatedId, SessionTimezone}; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::PlanNodeId; use crate::session::SessionImpl; -use crate::WithOptions; +use crate::utils::{OverwriteOptions, WithOptions}; const RESERVED_ID_NUM: u16 = 10000; @@ -50,6 +50,9 @@ pub struct OptimizerContext { session_timezone: RefCell, /// Store expr display id. next_expr_display_id: RefCell, + /// Store the configs can be overwritten in with clause + /// if not specified, use the value from session variable. + overwrite_options: OverwriteOptions, } // Still not sure if we need to introduce "on_optimization_finish" or other common callback methods, @@ -71,10 +74,11 @@ impl OptimizerContext { } /// Create a new [`OptimizerContext`] from the given [`HandlerArgs`] and [`ExplainOptions`]. - pub fn new(handler_args: HandlerArgs, explain_options: ExplainOptions) -> Self { + pub fn new(mut handler_args: HandlerArgs, explain_options: ExplainOptions) -> Self { let session_timezone = RefCell::new(SessionTimezone::new( handler_args.session.config().get_timezone().to_owned(), )); + let overwrite_options = OverwriteOptions::new(&mut handler_args); Self { session_ctx: handler_args.session, next_plan_node_id: RefCell::new(RESERVED_ID_NUM.into()), @@ -87,6 +91,7 @@ impl OptimizerContext { with_options: handler_args.with_options, session_timezone, next_expr_display_id: RefCell::new(RESERVED_ID_NUM.into()), + overwrite_options, } } @@ -106,6 +111,7 @@ impl OptimizerContext { with_options: Default::default(), session_timezone: RefCell::new(SessionTimezone::new("UTC".into())), next_expr_display_id: RefCell::new(0), + overwrite_options: OverwriteOptions::default(), } .into() } @@ -189,6 +195,10 @@ impl OptimizerContext { &self.with_options } + pub fn overwrite_options(&self) -> &OverwriteOptions { + &self.overwrite_options + } + pub fn session_ctx(&self) -> &Arc { &self.session_ctx } diff --git a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs index 2fc1aba8127fd..87cbc65b212d1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs +++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs @@ -111,12 +111,7 @@ impl StreamNode for StreamFsFetch { .map(|c| c.to_protobuf()) .collect_vec(), properties: source_catalog.properties.clone().into_iter().collect(), - rate_limit: self - .base - .ctx() - .session_ctx() - .config() - .get_streaming_rate_limit(), + rate_limit: self.base.ctx().overwrite_options().streaming_rate_limit, }); NodeBody::StreamFsFetch(StreamFsFetchNode { node_inner: source_inner, diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index 51a764b53fab2..1f2cd6fc98ad0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -87,12 +87,7 @@ impl StreamNode for StreamSource { .map(|c| c.to_protobuf()) .collect_vec(), properties: source_catalog.properties.clone().into_iter().collect(), - rate_limit: self - .base - .ctx() - .session_ctx() - .config() - .get_streaming_rate_limit(), + rate_limit: self.base.ctx().overwrite_options().streaming_rate_limit, }); PbNodeBody::Source(SourceNode { source_inner }) } diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index e506c69ffb3b2..750548a315e9d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -283,6 +283,7 @@ impl StreamTableScan { }) .collect_vec(); + // TODO: snapshot read of upstream mview let batch_plan_node = BatchPlanNode { table_desc: Some(self.core.table_desc.to_protobuf()), column_ids: upstream_column_ids.clone(), @@ -301,12 +302,7 @@ impl StreamTableScan { // The table desc used by backfill executor table_desc: Some(self.core.table_desc.to_protobuf()), state_table: Some(catalog), - rate_limit: self - .base - .ctx() - .session_ctx() - .config() - .get_streaming_rate_limit(), + rate_limit: self.base.ctx().overwrite_options().streaming_rate_limit, ..Default::default() }); diff --git a/src/frontend/src/utils/mod.rs b/src/frontend/src/utils/mod.rs index 3951da525bb03..6f105e5960b84 100644 --- a/src/frontend/src/utils/mod.rs +++ b/src/frontend/src/utils/mod.rs @@ -31,8 +31,9 @@ mod index_set; pub use index_set::*; pub(crate) mod group_by; pub mod infer_stmt_row_desc; - +pub mod overwrite_options; pub use group_by::*; +pub use overwrite_options::*; use crate::expr::{Expr, ExprImpl, ExprRewriter, InputRef}; diff --git a/src/frontend/src/utils/overwrite_options.rs b/src/frontend/src/utils/overwrite_options.rs new file mode 100644 index 0000000000000..14e838e689d7b --- /dev/null +++ b/src/frontend/src/utils/overwrite_options.rs @@ -0,0 +1,52 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use crate::handler::HandlerArgs; + +#[derive(Debug, Clone, Default)] +pub struct OverwriteOptions { + pub streaming_rate_limit: Option, + // ttl has been deprecated + pub ttl: Option, +} + +impl OverwriteOptions { + const STREAMING_RATE_LIMIT_KEY: &'static str = "streaming_rate_limit"; + const TTL_KEY: &'static str = "ttl"; + + pub fn new(args: &mut HandlerArgs) -> Self { + let streaming_rate_limit = { + if let Some(x) = args + .with_options + .inner_mut() + .remove(Self::STREAMING_RATE_LIMIT_KEY) + { + // FIXME(tabVersion): validate the value + Some(x.parse::().unwrap()) + } else { + args.session.config().get_streaming_rate_limit() + } + }; + let ttl = args + .with_options + .inner_mut() + .remove(Self::TTL_KEY) + // FIXME(tabVersion): validate the value + .map(|x| x.parse::().unwrap()); + Self { + streaming_rate_limit, + ttl, + } + } +} diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index aad6914d4bcef..8c7b79852d6eb 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -97,7 +97,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { ) .boxed(); - let rate_limit = source.get_rate_limit().cloned().ok(); + let rate_limit = source.rate_limit.map(|x| x as _); Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) } } diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 3232c49d8cbb3..ac303e27ee9b8 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -194,7 +194,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { } } }; - let rate_limit = source.get_rate_limit().cloned().ok(); + let rate_limit = source.rate_limit.map(|x| x as _); Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) } else { // If there is no external stream source, then no data should be persisted. We pass a diff --git a/src/stream/src/from_proto/stream_scan.rs b/src/stream/src/from_proto/stream_scan.rs index 5b6136fda9ea1..059a4c983fe20 100644 --- a/src/stream/src/from_proto/stream_scan.rs +++ b/src/stream/src/from_proto/stream_scan.rs @@ -218,7 +218,6 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { } StreamScanType::Unspecified => unreachable!(), }; - let rate_limit = node.get_rate_limit().cloned().ok(); - Ok(FlowControlExecutor::new(executor, rate_limit).boxed()) + Ok(FlowControlExecutor::new(executor, node.rate_limit.map(|x| x as _)).boxed()) } } From 7ebab64057673e92b22b149a1d7de536f655cbe1 Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 16 Nov 2023 23:38:16 +0800 Subject: [PATCH 23/36] feat(optimizer): avoid predicate pushdown for batch queries if unnecessary (#13470) Co-authored-by: stonepage <40830455+st1page@users.noreply.github.com> Co-authored-by: st1page <1245835950@qq.com> --- .../tests/testdata/output/cse_expr.yaml | 2 +- .../tests/testdata/output/explain.yaml | 26 +--------------- .../tests/testdata/output/expr.yaml | 4 +-- .../tests/testdata/output/nexmark_source.yaml | 19 +++++++----- .../testdata/output/nexmark_watermark.yaml | 30 +++++++++---------- .../tests/testdata/output/share.yaml | 6 ++-- .../src/optimizer/logical_optimization.rs | 19 ++++++++++-- .../src/optimizer/optimizer_context.rs | 12 ++++++++ 8 files changed, 63 insertions(+), 55 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml index fe41cb83794e2..4682f091ad1bc 100644 --- a/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/cse_expr.yaml @@ -84,5 +84,5 @@ sql: | with t(v, arr) as (select 1, array[2, 3]) select v < all(arr), v < some(arr) from t; batch_plan: |- - BatchProject { exprs: [All((1:Int32 < $expr10015)) as $expr1, Some((1:Int32 < $expr10015)) as $expr2] } + BatchProject { exprs: [All((1:Int32 < $expr10009)) as $expr1, Some((1:Int32 < $expr10009)) as $expr2] } └─BatchValues { rows: [[1:Int32, ARRAY[2, 3]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index 0faca9c10291f..e44431ffcd3d7 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -16,16 +16,6 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Predicate Push Down: - - LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - - Predicate Push Down: - - LogicalProject { exprs: [1:Int32] } - └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Convert Over Window: apply TrivialProjectToValuesRule 1 time(s) @@ -65,7 +55,7 @@ "stages": { "0": { "root": { - "plan_node_id": 10020, + "plan_node_id": 10016, "plan_node_type": "BatchValues", "schema": [ { @@ -121,13 +111,6 @@ Predicate Push Down: - LogicalProject { exprs: [t1.v1, t2.v2] } - └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1, _row_id] } - └─LogicalScan { table: t2, columns: [v2, _row_id] } - - Predicate Push Down: - LogicalProject { exprs: [t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } ├─LogicalScan { table: t1, columns: [v1, _row_id] } @@ -151,13 +134,6 @@ Prune Columns: - LogicalProject { exprs: [t1.v1, t2.v2] } - └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } - ├─LogicalScan { table: t1, columns: [v1] } - └─LogicalScan { table: t2, columns: [v2] } - - Predicate Push Down: - LogicalProject { exprs: [t1.v1, t2.v2] } └─LogicalJoin { type: Inner, on: (t1.v1 = t2.v2) } ├─LogicalScan { table: t1, columns: [v1] } diff --git a/src/frontend/planner_test/tests/testdata/output/expr.yaml b/src/frontend/planner_test/tests/testdata/output/expr.yaml index 7f117e64be4e9..d0ab974ff81bc 100644 --- a/src/frontend/planner_test/tests/testdata/output/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/expr.yaml @@ -450,7 +450,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: |- - BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10013, ARRAY[2]:List(Int32)))) as $expr1] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr10011, ARRAY[2]:List(Int32)))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List(Int32)]] } @@ -473,7 +473,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: |- - BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10013, ARRAY[2]:List(Int32)))) as $expr1] } + BatchProject { exprs: [All((1:Int32 < ArrayCat($expr10011, ARRAY[2]:List(Int32)))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List(Int32)]] } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml index 6a7d3c0fa04a6..654bc8b0f33e1 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -116,8 +116,8 @@ sql: SELECT auction, price FROM bid WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [auction, price] } - └─BatchFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) } + └─BatchFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) } + └─BatchProject { exprs: [auction, price] } └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } @@ -152,10 +152,12 @@ └─BatchHashJoin { type: Inner, predicate: seller = id, output: [name, city, state, id] } ├─BatchExchange { order: [], dist: HashShard(seller) } │ └─BatchFilter { predicate: (category = 10:Int32) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchProject { exprs: [id, seller, category] } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } - └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id], filter: (None, None) } + └─BatchProject { exprs: [id, name, city, state] } + └─BatchSource { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, seller], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(_row_id, seller, _row_id) } @@ -1015,7 +1017,8 @@ BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra] } └─BatchFilter { predicate: ((0.908:Decimal * price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * price::Decimal) < 50000000:Decimal) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchProject { exprs: [auction, bidder, price, date_time, extra] } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra, _row_id] } @@ -1634,7 +1637,8 @@ │ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (category = 10:Int32) } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category] } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(auction, _row_id, _row_id) } @@ -1705,7 +1709,8 @@ BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1] } └─BatchFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(channel), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) } - └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } + └─BatchProject { exprs: [auction, bidder, price, channel, url] } + └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml index 7f41cfffe04f9..53df7f872633e 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -118,16 +118,16 @@ A.category = 10 and (P.state = 'or' OR P.state = 'id' OR P.state = 'ca'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: Inner, predicate: $expr3 = $expr5, output: [$expr6, $expr7, $expr8, $expr2] } - ├─BatchExchange { order: [], dist: HashShard($expr3) } - │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 7:Int32) as $expr3] } + └─BatchHashJoin { type: Inner, predicate: $expr2 = $expr3, output: [$expr4, $expr5, $expr6, $expr1] } + ├─BatchExchange { order: [], dist: HashShard($expr2) } + │ └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr1, Field(auction, 7:Int32) as $expr2] } │ └─BatchFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─BatchProject { exprs: [event_type, auction] } │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } - └─BatchExchange { order: [], dist: HashShard($expr5) } - └─BatchProject { exprs: [Field(person, 0:Int32) as $expr5, Field(person, 1:Int32) as $expr6, Field(person, 4:Int32) as $expr7, Field(person, 5:Int32) as $expr8] } + └─BatchExchange { order: [], dist: HashShard($expr3) } + └─BatchProject { exprs: [Field(person, 0:Int32) as $expr3, Field(person, 1:Int32) as $expr4, Field(person, 4:Int32) as $expr5, Field(person, 5:Int32) as $expr6] } └─BatchFilter { predicate: (((Field(person, 5:Int32) = 'or':Varchar) OR (Field(person, 5:Int32) = 'id':Varchar)) OR (Field(person, 5:Int32) = 'ca':Varchar)) AND (event_type = 0:Int32) } - └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr4, _row_id] } + └─BatchProject { exprs: [event_type, person] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), $expr3(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, $expr3], pk_columns: [_row_id, _row_id#1, $expr3], pk_conflict: NoCheck } @@ -1719,12 +1719,12 @@ ├─BatchExchange { order: [], dist: HashShard($expr2) } │ └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6, $expr1] } │ └─BatchFilter { predicate: (event_type = 2:Int32) } - │ └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─BatchProject { exprs: [event_type, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1] } │ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard($expr8) } └─BatchProject { exprs: [Field(auction, 0:Int32) as $expr8, Field(auction, 1:Int32) as $expr9, Field(auction, 2:Int32) as $expr10, Field(auction, 3:Int32) as $expr11, Field(auction, 4:Int32) as $expr12, $expr7, Field(auction, 6:Int32) as $expr13, Field(auction, 7:Int32) as $expr14, Field(auction, 8:Int32) as $expr15] } └─BatchFilter { predicate: (Field(auction, 8:Int32) = 10:Int32) AND (event_type = 1:Int32) } - └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr7, _row_id] } + └─BatchProject { exprs: [event_type, auction, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr7] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } @@ -1819,10 +1819,10 @@ lower(channel) in ('apple', 'google', 'facebook', 'baidu'); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [$expr2, $expr3, $expr4, $expr5, Case((Lower($expr5) = 'apple':Varchar), '0':Varchar, (Lower($expr5) = 'google':Varchar), '1':Varchar, (Lower($expr5) = 'facebook':Varchar), '2':Varchar, (Lower($expr5) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch($expr6, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr7] } - └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6] } + └─BatchProject { exprs: [$expr1, $expr2, $expr3, $expr4, Case((Lower($expr4) = 'apple':Varchar), '0':Varchar, (Lower($expr4) = 'google':Varchar), '1':Varchar, (Lower($expr4) = 'facebook':Varchar), '2':Varchar, (Lower($expr4) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch($expr5, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr6] } + └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5] } └─BatchFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(Field(bid, 4:Int32), '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(Field(bid, 3:Int32)), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) AND (event_type = 2:Int32) } - └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─BatchProject { exprs: [event_type, bid] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } @@ -1861,10 +1861,10 @@ SPLIT_PART(url, '/', 6) as dir3 FROM bid; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [$expr2, $expr3, $expr4, $expr5, SplitPart($expr6, '/':Varchar, 4:Int32) as $expr7, SplitPart($expr6, '/':Varchar, 5:Int32) as $expr8, SplitPart($expr6, '/':Varchar, 6:Int32) as $expr9] } - └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 1:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, Field(bid, 3:Int32) as $expr5, Field(bid, 4:Int32) as $expr6] } + └─BatchProject { exprs: [$expr1, $expr2, $expr3, $expr4, SplitPart($expr5, '/':Varchar, 4:Int32) as $expr6, SplitPart($expr5, '/':Varchar, 5:Int32) as $expr7, SplitPart($expr5, '/':Varchar, 6:Int32) as $expr8] } + └─BatchProject { exprs: [Field(bid, 0:Int32) as $expr1, Field(bid, 1:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, Field(bid, 3:Int32) as $expr4, Field(bid, 4:Int32) as $expr5] } └─BatchFilter { predicate: (event_type = 2:Int32) } - └─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─BatchProject { exprs: [event_type, bid] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index d446b23fab883..799eadf1cdf8a 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -337,10 +337,12 @@ └─BatchHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time] } ├─BatchExchange { order: [], dist: HashShard(id) } │ └─BatchFilter { predicate: (initial_bid = 1:Int32) AND (AtTimeZone(date_time, 'UTC':Varchar) > '2021-03-31 23:59:59+00:00':Timestamptz) } - │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + │ └─BatchProject { exprs: [id, initial_bid, date_time] } + │ └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard(id) } └─BatchFilter { predicate: (initial_bid = 2:Int32) } - └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } + └─BatchProject { exprs: [id, initial_bid, date_time] } + └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [a_id, b_id, a_ts, b_ts, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a_id], pk_columns: [_row_id, _row_id#1, a_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id, _row_id) } diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index 887f393e81c6a..cfae8a04504a2 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -47,6 +47,7 @@ impl PlanRef { ctx.trace(format!("{}", stats)); ctx.trace(plan.explain_to_string()); } + ctx.add_rule_applied(stats.total_applied()); plan } @@ -71,6 +72,7 @@ impl PlanRef { ctx.trace(format!("{}", stats)); ctx.trace(output_plan.explain_to_string()); } + ctx.add_rule_applied(stats.total_applied()); if !stats.has_applied_rule() { return output_plan; @@ -638,6 +640,7 @@ impl LogicalOptimizer { plan = Self::subquery_unnesting(plan, false, explain_trace, &ctx)?; // Predicate Push-down + let mut last_total_rule_applied_before_predicate_pushdown = ctx.total_rule_applied(); plan = Self::predicate_pushdown(plan, explain_trace, &ctx); if plan.ctx().session_ctx().config().get_enable_join_ordering() { @@ -652,7 +655,10 @@ impl LogicalOptimizer { // Predicate Push-down: apply filter pushdown rules again since we pullup all join // conditions into a filter above the multijoin. - plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + if last_total_rule_applied_before_predicate_pushdown != ctx.total_rule_applied() { + last_total_rule_applied_before_predicate_pushdown = ctx.total_rule_applied(); + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + } // Push down the calculation of inputs of join's condition. plan = plan.optimize_by_rules(&PUSH_CALC_OF_JOIN); @@ -660,7 +666,10 @@ impl LogicalOptimizer { plan = plan.optimize_by_rules(&SPLIT_OVER_WINDOW); // Must push down predicates again after split over window so that OverWindow can be // optimized to TopN. - plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + if last_total_rule_applied_before_predicate_pushdown != ctx.total_rule_applied() { + last_total_rule_applied_before_predicate_pushdown = ctx.total_rule_applied(); + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + } plan = plan.optimize_by_rules(&CONVERT_OVER_WINDOW); plan = plan.optimize_by_rules(&MERGE_OVER_WINDOW); @@ -673,7 +682,11 @@ impl LogicalOptimizer { // Do a final column pruning and predicate pushing down to clean up the plan. plan = Self::column_pruning(plan, explain_trace, &ctx); - plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + if last_total_rule_applied_before_predicate_pushdown != ctx.total_rule_applied() { + #[allow(unused_assignments)] + last_total_rule_applied_before_predicate_pushdown = ctx.total_rule_applied(); + plan = Self::predicate_pushdown(plan, explain_trace, &ctx); + } plan = plan.optimize_by_rules(&PROJECT_REMOVE); diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index 7f7f867daa3b2..abf86b5fa5b71 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -50,6 +50,8 @@ pub struct OptimizerContext { session_timezone: RefCell, /// Store expr display id. next_expr_display_id: RefCell, + /// Total number of optimization rules have been applied. + total_rule_applied: RefCell, /// Store the configs can be overwritten in with clause /// if not specified, use the value from session variable. overwrite_options: OverwriteOptions, @@ -91,6 +93,7 @@ impl OptimizerContext { with_options: handler_args.with_options, session_timezone, next_expr_display_id: RefCell::new(RESERVED_ID_NUM.into()), + total_rule_applied: RefCell::new(0), overwrite_options, } } @@ -111,6 +114,7 @@ impl OptimizerContext { with_options: Default::default(), session_timezone: RefCell::new(SessionTimezone::new("UTC".into())), next_expr_display_id: RefCell::new(0), + total_rule_applied: RefCell::new(0), overwrite_options: OverwriteOptions::default(), } .into() @@ -147,6 +151,14 @@ impl OptimizerContext { *self.next_correlated_id.borrow() } + pub fn add_rule_applied(&self, num: usize) { + *self.total_rule_applied.borrow_mut() += num; + } + + pub fn total_rule_applied(&self) -> usize { + *self.total_rule_applied.borrow() + } + pub fn is_explain_verbose(&self) -> bool { self.explain_options.verbose } From 483347725eacbe39a9a60b0817ac145aaf205f80 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 16 Nov 2023 19:21:47 +0000 Subject: [PATCH 24/36] chore(deps): Bump itertools from 0.11.0 to 0.12.0 (#13444) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] Co-authored-by: TennyZhuang --- Cargo.lock | 72 +++++++++++++++------------- src/batch/Cargo.toml | 2 +- src/bench/Cargo.toml | 2 +- src/common/Cargo.toml | 2 +- src/compute/Cargo.toml | 2 +- src/connector/Cargo.toml | 2 +- src/ctl/Cargo.toml | 2 +- src/expr/core/Cargo.toml | 2 +- src/expr/impl/Cargo.toml | 2 +- src/expr/macro/Cargo.toml | 2 +- src/frontend/Cargo.toml | 2 +- src/frontend/planner_test/Cargo.toml | 2 +- src/jni_core/Cargo.toml | 2 +- src/meta/Cargo.toml | 2 +- src/meta/node/Cargo.toml | 2 +- src/meta/service/Cargo.toml | 2 +- src/object_store/Cargo.toml | 2 +- src/risedevtool/Cargo.toml | 2 +- src/risedevtool/config/Cargo.toml | 2 +- src/rpc_client/Cargo.toml | 2 +- src/source/Cargo.toml | 2 +- src/sqlparser/Cargo.toml | 2 +- src/storage/Cargo.toml | 2 +- src/storage/backup/Cargo.toml | 2 +- src/storage/hummock_sdk/Cargo.toml | 2 +- src/storage/hummock_test/Cargo.toml | 2 +- src/stream/Cargo.toml | 2 +- src/tests/simulation/Cargo.toml | 2 +- src/tests/sqlsmith/Cargo.toml | 2 +- src/utils/pgwire/Cargo.toml | 2 +- src/workspace-hack/Cargo.toml | 6 +-- 31 files changed, 71 insertions(+), 65 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6be570b2eabd9..b44d512affc53 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4250,6 +4250,15 @@ dependencies = [ "either", ] +[[package]] +name = "itertools" +version = "0.12.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "25db6b064527c5d482d0423354fcd07a89a2dfe07b67892e62411946db7f07b0" +dependencies = [ + "either", +] + [[package]] name = "itoa" version = "1.0.9" @@ -6024,7 +6033,7 @@ dependencies = [ "byteorder", "bytes", "futures", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "openssl", "panic-message", @@ -6560,7 +6569,7 @@ checksum = "8bdf592881d821b83d471f8af290226c8d51402259e9bb5be7f9f8bdebbb11ac" dependencies = [ "bytes", "heck 0.4.1", - "itertools 0.11.0", + "itertools 0.10.5", "log", "multimap 0.8.3", "once_cell", @@ -6594,7 +6603,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "265baba7fabd416cf5078179f7d2cbeca4ce7a9041111900675ea7c4cb8a4c32" dependencies = [ "anyhow", - "itertools 0.11.0", + "itertools 0.10.5", "proc-macro2", "quote", "syn 2.0.37", @@ -7163,7 +7172,7 @@ dependencies = [ "glob", "google-cloud-pubsub", "indicatif", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-rdkafka", "madsim-tokio", "redis", @@ -7190,7 +7199,7 @@ dependencies = [ "dialoguer", "enum-iterator", "fs-err", - "itertools 0.11.0", + "itertools 0.12.0", ] [[package]] @@ -7201,7 +7210,7 @@ dependencies = [ "async-trait", "bincode 1.3.3", "bytes", - "itertools 0.11.0", + "itertools 0.12.0", "parking_lot 0.12.1", "prost 0.12.1", "risingwave_common", @@ -7230,7 +7239,7 @@ dependencies = [ "futures-util", "hashbrown 0.14.0", "hytra", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "madsim-tonic", "parking_lot 0.12.1", @@ -7273,7 +7282,7 @@ dependencies = [ "clap", "futures", "hdrhistogram", - "itertools 0.11.0", + "itertools 0.12.0", "libc", "madsim-tokio", "nix 0.27.1", @@ -7384,7 +7393,7 @@ dependencies = [ "humantime", "hyper", "hytra", - "itertools 0.11.0", + "itertools 0.12.0", "itoa", "jsonbb", "libc", @@ -7548,7 +7557,7 @@ dependencies = [ "futures", "futures-async-stream", "hyper", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "madsim-tonic", "maplit", @@ -7618,7 +7627,7 @@ dependencies = [ "hyper-tls", "icelake", "indexmap 1.9.3", - "itertools 0.11.0", + "itertools 0.12.0", "jni", "jsonschema-transpiler", "madsim-rdkafka", @@ -7689,7 +7698,7 @@ dependencies = [ "comfy-table", "futures", "inquire", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-etcd-client", "madsim-tokio", "regex", @@ -7762,7 +7771,7 @@ dependencies = [ "expect-test", "futures-async-stream", "futures-util", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "num-traits", "parse-display", @@ -7793,7 +7802,7 @@ dependencies = [ "futures-async-stream", "futures-util", "hex", - "itertools 0.11.0", + "itertools 0.12.0", "jsonbb", "madsim-tokio", "md5", @@ -7817,7 +7826,7 @@ dependencies = [ name = "risingwave_expr_macro" version = "0.1.0" dependencies = [ - "itertools 0.11.0", + "itertools 0.12.0", "proc-macro2", "quote", "syn 2.0.37", @@ -7848,7 +7857,7 @@ dependencies = [ "futures", "futures-async-stream", "iana-time-zone", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "madsim-tonic", "maplit", @@ -7897,7 +7906,7 @@ version = "1.3.0-alpha" dependencies = [ "bytes", "hex", - "itertools 0.11.0", + "itertools 0.12.0", "parse-display", "risingwave_common", "risingwave_pb", @@ -7917,7 +7926,7 @@ dependencies = [ "fail", "futures", "futures-async-stream", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "parking_lot 0.12.1", "rand", @@ -7982,7 +7991,7 @@ dependencies = [ "cfg-or-panic", "expect-test", "futures", - "itertools 0.11.0", + "itertools 0.12.0", "jni", "madsim-tokio", "paste", @@ -8022,7 +8031,7 @@ dependencies = [ "futures", "hex", "hyper", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-etcd-client", "madsim-tokio", "madsim-tonic", @@ -8092,7 +8101,7 @@ dependencies = [ "clap", "either", "futures", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-etcd-client", "madsim-tokio", "madsim-tonic", @@ -8120,7 +8129,7 @@ dependencies = [ "async-trait", "either", "futures", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "madsim-tonic", "rand", @@ -8156,7 +8165,7 @@ dependencies = [ "hyper", "hyper-rustls", "hyper-tls", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-aws-sdk-s3", "madsim-tokio", "opendal", @@ -8194,7 +8203,7 @@ version = "1.3.0-alpha" dependencies = [ "anyhow", "expect-test", - "itertools 0.11.0", + "itertools 0.12.0", "libtest-mimic", "madsim-tokio", "paste", @@ -8234,7 +8243,7 @@ dependencies = [ "either", "futures", "hyper", - "itertools 0.11.0", + "itertools 0.12.0", "lru 0.10.1", "madsim-tokio", "madsim-tonic", @@ -8291,7 +8300,7 @@ dependencies = [ "console", "futures", "glob", - "itertools 0.11.0", + "itertools 0.12.0", "lru 0.7.6", "madsim", "madsim-aws-sdk-s3", @@ -8338,7 +8347,7 @@ dependencies = [ "criterion", "futures", "futures-async-stream", - "itertools 0.11.0", + "itertools 0.12.0", "madsim-tokio", "parking_lot 0.12.1", "paste", @@ -8355,7 +8364,7 @@ dependencies = [ name = "risingwave_sqlparser" version = "1.3.0-alpha" dependencies = [ - "itertools 0.11.0", + "itertools 0.12.0", "matches", "serde", "tracing", @@ -8386,7 +8395,7 @@ dependencies = [ "chrono", "clap", "expect-test", - "itertools 0.11.0", + "itertools 0.12.0", "libtest-mimic", "madsim-tokio", "rand", @@ -8448,7 +8457,7 @@ dependencies = [ "futures", "futures-async-stream", "hex", - "itertools 0.11.0", + "itertools 0.12.0", "libc", "lz4", "mach2", @@ -8511,7 +8520,7 @@ dependencies = [ "futures-async-stream", "governor", "hytra", - "itertools 0.11.0", + "itertools 0.12.0", "local_stats_alloc", "lru 0.7.6", "madsim-tokio", @@ -11420,7 +11429,6 @@ dependencies = [ "hyper", "indexmap 1.9.3", "itertools 0.10.5", - "itertools 0.11.0", "jni", "lazy_static", "lexical-core", diff --git a/src/batch/Cargo.toml b/src/batch/Cargo.toml index 60141e1638c28..c91023f5b1b2d 100644 --- a/src/batch/Cargo.toml +++ b/src/batch/Cargo.toml @@ -24,7 +24,7 @@ futures-async-stream = { workspace = true } futures-util = "0.3" hashbrown = { workspace = true } hytra = "0.1.2" -itertools = "0.11" +itertools = "0.12" parking_lot = { version = "0.12", features = ["arc_lock"] } paste = "1" prometheus = { version = "0.13", features = ["process"] } diff --git a/src/bench/Cargo.toml b/src/bench/Cargo.toml index 5227c39272c38..e5f6d8e7ede1a 100644 --- a/src/bench/Cargo.toml +++ b/src/bench/Cargo.toml @@ -19,7 +19,7 @@ bytesize = { version = "1", features = ["serde"] } clap = { version = "4", features = ["derive"] } futures = { version = "0.3", default-features = false, features = ["alloc"] } hdrhistogram = "7" -itertools = "0.11" +itertools = "0.12" libc = "0.2" opentelemetry = { version = "0.20", default-features = false, features = ["rt-tokio"], optional = true } parking_lot = "0.12" diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index 27b3a4249e3bc..921c02ee6ae4e 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -48,7 +48,7 @@ http = "0.2" humantime = "2.1" hyper = "0.14" hytra = { workspace = true } -itertools = "0.11" +itertools = "0.12" itoa = "1.0" jsonbb = "0.1.2" lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "cb2d7c7" } diff --git a/src/compute/Cargo.toml b/src/compute/Cargo.toml index 5dc1b90f14ba1..54c8b6896f0a3 100644 --- a/src/compute/Cargo.toml +++ b/src/compute/Cargo.toml @@ -23,7 +23,7 @@ either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } hyper = "0.14" -itertools = "0.11" +itertools = "0.12" maplit = "1.0.2" pprof = { version = "0.13", features = ["flamegraph"] } prometheus = { version = "0.13" } diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index d336b482d4f97..db1cae16bb67d 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -65,7 +65,7 @@ hyper = { version = "0.14", features = [ hyper-tls = "0.5" icelake = { workspace = true } indexmap = { version = "1.9.3", features = ["serde"] } -itertools = "0.11" +itertools = "0.12" jni = { version = "0.21.1", features = ["invocation"] } jst = { package = 'jsonschema-transpiler', git = "https://github.com/mozilla/jsonschema-transpiler", rev = "c1a89d720d118843d8bcca51084deb0ed223e4b4" } maplit = "1.0.2" diff --git a/src/ctl/Cargo.toml b/src/ctl/Cargo.toml index f8bd5d9d4d27d..0097f5624f70b 100644 --- a/src/ctl/Cargo.toml +++ b/src/ctl/Cargo.toml @@ -22,7 +22,7 @@ comfy-table = "7" etcd-client = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } inquire = "0.6.2" -itertools = "0.11" +itertools = "0.12" regex = "1.10.0" risingwave_common = { workspace = true } risingwave_connector = { workspace = true } diff --git a/src/expr/core/Cargo.toml b/src/expr/core/Cargo.toml index ab8dd697e220d..db3795b8a20ed 100644 --- a/src/expr/core/Cargo.toml +++ b/src/expr/core/Cargo.toml @@ -34,7 +34,7 @@ either = "1" enum-as-inner = "0.6" futures-async-stream = { workspace = true } futures-util = "0.3" -itertools = "0.11" +itertools = "0.12" num-traits = "0.2" parse-display = "0.8" paste = "1" diff --git a/src/expr/impl/Cargo.toml b/src/expr/impl/Cargo.toml index dbc3e60c880ee..ee6d52977cf6d 100644 --- a/src/expr/impl/Cargo.toml +++ b/src/expr/impl/Cargo.toml @@ -28,7 +28,7 @@ fancy-regex = "0.12" futures-async-stream = { workspace = true } futures-util = "0.3" hex = "0.4" -itertools = "0.11" +itertools = "0.12" jsonbb = "0.1.2" md5 = "0.7" num-traits = "0.2" diff --git a/src/expr/macro/Cargo.toml b/src/expr/macro/Cargo.toml index bf761b142061f..9b388920d2ebb 100644 --- a/src/expr/macro/Cargo.toml +++ b/src/expr/macro/Cargo.toml @@ -8,7 +8,7 @@ edition = "2021" proc-macro = true [dependencies] -itertools = "0.11" +itertools = "0.12" proc-macro2 = "1" quote = "1" syn = { version = "2", features = ["full", "extra-traits"] } diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 73fcb205fafcb..ada566ad81797 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -35,7 +35,7 @@ fixedbitset = "0.4.2" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } iana-time-zone = "0.1" -itertools = "0.11" +itertools = "0.12" maplit = "1" md5 = "0.7.0" num-integer = "0.1" diff --git a/src/frontend/planner_test/Cargo.toml b/src/frontend/planner_test/Cargo.toml index 47d6ba3a68289..4de959bf16ce7 100644 --- a/src/frontend/planner_test/Cargo.toml +++ b/src/frontend/planner_test/Cargo.toml @@ -16,7 +16,7 @@ normal = ["workspace-hack"] [dependencies] anyhow = "1" expect-test = "1" -itertools = "0.11" +itertools = "0.12" paste = "1" risingwave_expr_impl = { workspace = true } risingwave_frontend = { workspace = true } diff --git a/src/jni_core/Cargo.toml b/src/jni_core/Cargo.toml index c1bdde44b43d9..8953d15a99a98 100644 --- a/src/jni_core/Cargo.toml +++ b/src/jni_core/Cargo.toml @@ -14,7 +14,7 @@ anyhow = "1" bytes = "1" cfg-or-panic = "0.2" futures = { version = "0.3", default-features = false, features = ["alloc"] } -itertools = "0.11" +itertools = "0.12" jni = "0.21.1" paste = "1" prost = { workspace = true } diff --git a/src/meta/Cargo.toml b/src/meta/Cargo.toml index e0b58defa9b57..9acbf84ff121a 100644 --- a/src/meta/Cargo.toml +++ b/src/meta/Cargo.toml @@ -33,7 +33,7 @@ function_name = "0.3.0" futures = { version = "0.3", default-features = false, features = ["alloc"] } hex = "0.4" hyper = "0.14" -itertools = "0.11" +itertools = "0.12" memcomparable = { version = "0.2" } mime_guess = "2" num-integer = "0.1" diff --git a/src/meta/node/Cargo.toml b/src/meta/node/Cargo.toml index e7597c0d041f3..04fbea0705a87 100644 --- a/src/meta/node/Cargo.toml +++ b/src/meta/node/Cargo.toml @@ -19,7 +19,7 @@ clap = { version = "4", features = ["derive", "env"] } either = "1" etcd-client = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } -itertools = "0.11" +itertools = "0.12" prometheus-http-query = "0.7" redact = "0.1.5" regex = "1" diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index d1c08a642c8ca..b734b62106495 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -18,7 +18,7 @@ anyhow = "1" async-trait = "0.1" either = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } -itertools = "0.11" +itertools = "0.12" rand = "0.8" regex = "1" risingwave_common = { workspace = true } diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index 87fd07be7810a..a3f77e101bd4f 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -25,7 +25,7 @@ futures = { version = "0.3", default-features = false, features = ["alloc"] } hyper = "0.14" hyper-rustls = { version = "0.24.2", features = ["webpki-roots"] } hyper-tls = "0.5.0" -itertools = "0.11" +itertools = "0.12" opendal = "0.41" prometheus = { version = "0.13", features = ["process"] } risingwave_common = { workspace = true } diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index e3caf3c7d34da..ff2acfe3cbec0 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -22,7 +22,7 @@ fs-err = "2.10.0" glob = "0.3" google-cloud-pubsub = "0.20" indicatif = "0.17" -itertools = "0.11" +itertools = "0.12" rdkafka = { workspace = true } redis = "0.23" regex = "1" diff --git a/src/risedevtool/config/Cargo.toml b/src/risedevtool/config/Cargo.toml index c0eecd8ce0c46..bafa8584922ec 100644 --- a/src/risedevtool/config/Cargo.toml +++ b/src/risedevtool/config/Cargo.toml @@ -14,7 +14,7 @@ console = "0.15" dialoguer = "0.11" enum-iterator = "1" fs-err = "2.10.0" -itertools = "0.11" +itertools = "0.12" [lints] workspace = true diff --git a/src/rpc_client/Cargo.toml b/src/rpc_client/Cargo.toml index f49a9b26ad5ec..6fb11d7f5a7ff 100644 --- a/src/rpc_client/Cargo.toml +++ b/src/rpc_client/Cargo.toml @@ -20,7 +20,7 @@ easy-ext = "1" either = "1.9.0" futures = { version = "0.3", default-features = false, features = ["alloc"] } hyper = "0.14" -itertools = "0.11.0" +itertools = "0.12.0" lru = "0.10.1" moka = { version = "0.12", features = ["future"] } rand = "0.8" diff --git a/src/source/Cargo.toml b/src/source/Cargo.toml index aedb0b9158908..735ca5f10d9b6 100644 --- a/src/source/Cargo.toml +++ b/src/source/Cargo.toml @@ -17,7 +17,7 @@ normal = ["workspace-hack"] anyhow = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } -itertools = "0.11" +itertools = "0.12" parking_lot = "0.12" rand = "0.8" risingwave_common = { workspace = true } diff --git a/src/sqlparser/Cargo.toml b/src/sqlparser/Cargo.toml index 56566b4090a53..2bb58461302b3 100644 --- a/src/sqlparser/Cargo.toml +++ b/src/sqlparser/Cargo.toml @@ -25,7 +25,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] -itertools = "0.11" +itertools = "0.12" serde = { version = "1.0", features = ["derive"], optional = true } tracing = "0.1" diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index 6150a6e862fdc..a53aca9ff1bc5 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -30,7 +30,7 @@ foyer = { git = "https://github.com/MrCroxx/foyer", rev = "ce2e222" } futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } hex = "0.4" -itertools = "0.11" +itertools = "0.12" libc = "0.2" lz4 = "1.24.0" memcomparable = "0.2" diff --git a/src/storage/backup/Cargo.toml b/src/storage/backup/Cargo.toml index 9e98e6c9076a8..2b4f1c60ec5c1 100644 --- a/src/storage/backup/Cargo.toml +++ b/src/storage/backup/Cargo.toml @@ -18,7 +18,7 @@ anyhow = "1" async-trait = "0.1" bincode = "1.3" bytes = { version = "1", features = ["serde"] } -itertools = "0.11" +itertools = "0.12" parking_lot = { version = "0.12", features = ["arc_lock"] } prost = { workspace = true } risingwave_common = { workspace = true } diff --git a/src/storage/hummock_sdk/Cargo.toml b/src/storage/hummock_sdk/Cargo.toml index 4e8e47a019c2b..2bc06424280e9 100644 --- a/src/storage/hummock_sdk/Cargo.toml +++ b/src/storage/hummock_sdk/Cargo.toml @@ -16,7 +16,7 @@ normal = ["workspace-hack"] [dependencies] bytes = "1" hex = "0.4" -itertools = "0.11" +itertools = "0.12" parse-display = "0.8" risingwave_common = { workspace = true } risingwave_pb = { workspace = true } diff --git a/src/storage/hummock_test/Cargo.toml b/src/storage/hummock_test/Cargo.toml index c3482f142d46b..39af3db64241a 100644 --- a/src/storage/hummock_test/Cargo.toml +++ b/src/storage/hummock_test/Cargo.toml @@ -21,7 +21,7 @@ clap = { version = "4", features = ["derive"] } fail = "0.5" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = "0.2.9" -itertools = "0.11" +itertools = "0.12" parking_lot = "0.12" rand = "0.8" risingwave_common = { workspace = true } diff --git a/src/stream/Cargo.toml b/src/stream/Cargo.toml index d85914a87b0df..4cdb6ad02b364 100644 --- a/src/stream/Cargo.toml +++ b/src/stream/Cargo.toml @@ -33,7 +33,7 @@ governor = { version = "0.6", default-features = false, features = [ "jitter", ] } hytra = "0.1.2" -itertools = "0.11" +itertools = "0.12" local_stats_alloc = { path = "../utils/local_stats_alloc" } lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "cb2d7c7" } maplit = "1.0.2" diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index c2e37b78e14a1..eceab4c434eb4 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -20,7 +20,7 @@ console = "0.15" etcd-client = { workspace = true } futures = { version = "0.3", default-features = false, features = ["alloc"] } glob = "0.3" -itertools = "0.11" +itertools = "0.12" lru = { git = "https://github.com/risingwavelabs/lru-rs.git", rev = "cb2d7c7" } madsim = "0.2.22" paste = "1" diff --git a/src/tests/sqlsmith/Cargo.toml b/src/tests/sqlsmith/Cargo.toml index 726b8d15ea1ec..2ddaa196eb15e 100644 --- a/src/tests/sqlsmith/Cargo.toml +++ b/src/tests/sqlsmith/Cargo.toml @@ -17,7 +17,7 @@ normal = ["workspace-hack"] anyhow = "1" chrono = "0.4" clap = { version = "4", features = ["derive"] } -itertools = "0.11" +itertools = "0.12" rand = { version = "0.8", features = ["small_rng"] } rand_chacha = { version = "0.3.1" } regex = "1" diff --git a/src/utils/pgwire/Cargo.toml b/src/utils/pgwire/Cargo.toml index c62def532c362..73ed5a4200b1c 100644 --- a/src/utils/pgwire/Cargo.toml +++ b/src/utils/pgwire/Cargo.toml @@ -20,7 +20,7 @@ auto_enums = { version = "0.8", features = ["tokio1"] } byteorder = "1.5" bytes = "1" futures = { version = "0.3", default-features = false, features = ["alloc"] } -itertools = "0.11" +itertools = "0.12" openssl = "0.10.57" panic-message = "0.3" risingwave_common = { workspace = true } diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index b263a9c5bcefb..6a38db45a2762 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -59,8 +59,7 @@ hashbrown-5ef9efb8ec2df382 = { package = "hashbrown", version = "0.12", features hmac = { version = "0.12", default-features = false, features = ["reset"] } hyper = { version = "0.14", features = ["full"] } indexmap = { version = "1", default-features = false, features = ["serde", "std"] } -itertools-93f6ce9d446188ac = { package = "itertools", version = "0.10" } -itertools-a6292c17cd707f01 = { package = "itertools", version = "0.11" } +itertools = { version = "0.10" } jni = { version = "0.21", features = ["invocation"] } lazy_static = { version = "1", default-features = false, features = ["spin_no_std"] } lexical-core = { version = "0.8", features = ["format"] } @@ -158,8 +157,7 @@ either = { version = "1", features = ["serde"] } fixedbitset = { version = "0.4" } frunk_core = { version = "0.4", default-features = false, features = ["std"] } hashbrown-582f2526e08bb6a0 = { package = "hashbrown", version = "0.14", features = ["nightly", "raw"] } -itertools-93f6ce9d446188ac = { package = "itertools", version = "0.10" } -itertools-a6292c17cd707f01 = { package = "itertools", version = "0.11" } +itertools = { version = "0.10" } lazy_static = { version = "1", default-features = false, features = ["spin_no_std"] } libc = { version = "0.2", features = ["extra_traits"] } log = { version = "0.4", default-features = false, features = ["kv_unstable", "std"] } From a34f46a0ac3d79363e93f11f710dbdf32233e0c6 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Fri, 17 Nov 2023 18:38:04 +0800 Subject: [PATCH 25/36] refactor: separate `CdcScan` from `Scan` in `logical` and `core` (#13494) Co-authored-by: Eric Fu --- .../tests/testdata/input/create_source.yaml | 20 ++ .../tests/testdata/output/create_source.yaml | 20 ++ src/frontend/src/handler/create_index.rs | 2 - src/frontend/src/handler/create_table.rs | 4 +- .../optimizer/plan_node/generic/cdc_scan.rs | 173 +++++++++++++++ .../src/optimizer/plan_node/generic/mod.rs | 4 + .../src/optimizer/plan_node/generic/scan.rs | 99 ++------- .../optimizer/plan_node/logical_cdc_scan.rs | 206 ++++++++++++++++++ .../src/optimizer/plan_node/logical_scan.rs | 56 +---- src/frontend/src/optimizer/plan_node/mod.rs | 4 + .../plan_node/stream_cdc_table_scan.rs | 8 +- .../optimizer/rule/index_selection_rule.rs | 7 +- src/frontend/src/planner/relation.rs | 2 - .../src/scheduler/distributed/query.rs | 2 - 14 files changed, 452 insertions(+), 155 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/generic/cdc_scan.rs create mode 100644 src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs diff --git a/src/frontend/planner_test/tests/testdata/input/create_source.yaml b/src/frontend/planner_test/tests/testdata/input/create_source.yaml index 0496ad1c13b13..32fd9c806ae30 100644 --- a/src/frontend/planner_test/tests/testdata/input/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/input/create_source.yaml @@ -33,6 +33,26 @@ ) FORMAT PLAIN ENCODE CSV (delimiter = E'\t', without_header = true); expected_outputs: - explain_output +- id: create_source_with_cdc_backfill + sql: | + create source mysql_mydb with ( + connector = 'mysql-cdc', + hostname = '127.0.0.1', + port = '8306', + username = 'root', + password = '123456', + database.name = 'mydb', + server.id = 5888 + ); + explain (logical) create table t1_rw ( + v1 int, + v2 int, + primary key(v1) + ) from mysql_mydb table 'mydb.t1'; + expected_outputs: + - explain_output + with_config_map: + CDC_BACKFILL: 'true' - id: create_source_with_cdc_backfill sql: | create source mysql_mydb with ( diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index c8082fb3b7138..2aeea6f914d2b 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -45,6 +45,26 @@ └─StreamExchange { dist: HashShard(_row_id) } └─StreamDml { columns: [v1, v2, _row_id] } └─StreamSource +- id: create_source_with_cdc_backfill + sql: | + create source mysql_mydb with ( + connector = 'mysql-cdc', + hostname = '127.0.0.1', + port = '8306', + username = 'root', + password = '123456', + database.name = 'mydb', + server.id = 5888 + ); + explain (logical) create table t1_rw ( + v1 int, + v2 int, + primary key(v1) + ) from mysql_mydb table 'mydb.t1'; + explain_output: | + LogicalCdcScan { table: mydb.t1, columns: [v1, v2] } + with_config_map: + CDC_BACKFILL: 'true' - id: create_source_with_cdc_backfill sql: | create source mysql_mydb with ( diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index 3eb93f2900c95..2cc7bb3b49df8 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -35,7 +35,6 @@ use crate::catalog::root_catalog::SchemaPath; use crate::expr::{Expr, ExprImpl, InputRef}; use crate::handler::privilege::ObjectCheckItem; use crate::handler::HandlerArgs; -use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{Explain, LogicalProject, LogicalScan, StreamMaterialize}; use crate::optimizer::property::{Cardinality, Distribution, Order, RequiredDist}; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot}; @@ -325,7 +324,6 @@ fn assemble_materialize( let logical_scan = LogicalScan::create( table_name, - ScanTableType::default(), table_desc.clone(), // Index table has no indexes. vec![], diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index c9b96065f2971..e79571e5e0c94 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -55,7 +55,7 @@ use crate::handler::create_source::{ check_source_schema, validate_compatibility, UPSTREAM_SOURCE_KEY, }; use crate::handler::HandlerArgs; -use crate::optimizer::plan_node::{LogicalScan, LogicalSource}; +use crate::optimizer::plan_node::{LogicalCdcScan, LogicalSource}; use crate::optimizer::property::{Order, RequiredDist}; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot}; use crate::session::SessionImpl; @@ -863,7 +863,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( tracing::debug!(?cdc_table_desc, "create cdc table"); - let logical_scan = LogicalScan::create_for_cdc( + let logical_scan = LogicalCdcScan::create( external_table_name, Rc::new(cdc_table_desc), context.clone(), diff --git a/src/frontend/src/optimizer/plan_node/generic/cdc_scan.rs b/src/frontend/src/optimizer/plan_node/generic/cdc_scan.rs new file mode 100644 index 0000000000000..c33cdc3330f27 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/generic/cdc_scan.rs @@ -0,0 +1,173 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::rc::Rc; + +use educe::Educe; +use fixedbitset::FixedBitSet; +use pretty_xmlish::Pretty; +use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, Field, Schema}; +use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_common::util::sort_util::ColumnOrder; + +use super::GenericPlanNode; +use crate::catalog::ColumnId; +use crate::expr::ExprRewriter; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::property::FunctionalDependencySet; + +/// [`CdcScan`] reads rows of a table from an external upstream database +#[derive(Debug, Clone, Educe)] +#[educe(PartialEq, Eq, Hash)] +pub struct CdcScan { + pub table_name: String, + /// Include `output_col_idx` and columns required in `predicate` + pub output_col_idx: Vec, + /// Descriptor of the external table for CDC + pub cdc_table_desc: Rc, + #[educe(PartialEq(ignore))] + #[educe(Hash(ignore))] + pub ctx: OptimizerContextRef, +} + +impl CdcScan { + pub fn rewrite_exprs(&self, _rewriter: &mut dyn ExprRewriter) {} + + /// Get the ids of the output columns. + pub fn output_column_ids(&self) -> Vec { + self.output_col_idx + .iter() + .map(|i| self.get_table_columns()[*i].column_id) + .collect() + } + + pub fn primary_key(&self) -> &[ColumnOrder] { + &self.cdc_table_desc.pk + } + + pub fn watermark_columns(&self) -> FixedBitSet { + FixedBitSet::with_capacity(self.get_table_columns().len()) + } + + pub(crate) fn column_names_with_table_prefix(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| format!("{}.{}", self.table_name, self.get_table_columns()[i].name)) + .collect() + } + + pub(crate) fn column_names(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| self.get_table_columns()[i].name.clone()) + .collect() + } + + /// get the Mapping of columnIndex from internal column index to output column index + pub fn i2o_col_mapping(&self) -> ColIndexMapping { + ColIndexMapping::with_remaining_columns( + &self.output_col_idx, + self.get_table_columns().len(), + ) + } + + /// Get the ids of the output columns and primary key columns. + pub fn output_and_pk_column_ids(&self) -> Vec { + let mut ids = self.output_column_ids(); + for column_order in self.primary_key() { + let id = self.get_table_columns()[column_order.column_index].column_id; + if !ids.contains(&id) { + ids.push(id); + } + } + ids + } + + /// Create a logical scan node for CDC backfill + pub(crate) fn new( + table_name: String, + output_col_idx: Vec, // the column index in the table + cdc_table_desc: Rc, + ctx: OptimizerContextRef, + ) -> Self { + Self { + table_name, + output_col_idx, + cdc_table_desc, + ctx, + } + } + + pub(crate) fn columns_pretty<'a>(&self, verbose: bool) -> Pretty<'a> { + Pretty::Array( + match verbose { + true => self.column_names_with_table_prefix(), + false => self.column_names(), + } + .into_iter() + .map(Pretty::from) + .collect(), + ) + } +} + +// TODO: extend for cdc table +impl GenericPlanNode for CdcScan { + fn schema(&self) -> Schema { + let fields = self + .output_col_idx + .iter() + .map(|tb_idx| { + let col = &self.get_table_columns()[*tb_idx]; + Field::from_with_table_name_prefix(col, &self.table_name) + }) + .collect(); + Schema { fields } + } + + fn stream_key(&self) -> Option> { + Some(self.cdc_table_desc.stream_key.clone()) + } + + fn ctx(&self) -> OptimizerContextRef { + self.ctx.clone() + } + + fn functional_dependency(&self) -> FunctionalDependencySet { + let pk_indices = self.stream_key(); + let col_num = self.output_col_idx.len(); + match &pk_indices { + Some(pk_indices) => FunctionalDependencySet::with_key(col_num, pk_indices), + None => FunctionalDependencySet::new(col_num), + } + } +} + +impl CdcScan { + pub fn get_table_columns(&self) -> &[ColumnDesc] { + &self.cdc_table_desc.columns + } + + pub fn append_only(&self) -> bool { + false + } + + /// Get the descs of the output columns. + pub fn column_descs(&self) -> Vec { + self.output_col_idx + .iter() + .map(|&i| self.get_table_columns()[i].clone()) + .collect() + } +} diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index c0baacffec8a0..9da00cb056065 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -44,6 +44,10 @@ mod scan; pub use scan::*; mod sys_scan; pub use sys_scan::*; + +mod cdc_scan; +pub use cdc_scan::*; + mod union; pub use union::*; mod top_n; diff --git a/src/frontend/src/optimizer/plan_node/generic/scan.rs b/src/frontend/src/optimizer/plan_node/generic/scan.rs index 3c615183f9adf..d0890eb72ff3d 100644 --- a/src/frontend/src/optimizer/plan_node/generic/scan.rs +++ b/src/frontend/src/optimizer/plan_node/generic/scan.rs @@ -18,7 +18,7 @@ use std::rc::Rc; use educe::Educe; use fixedbitset::FixedBitSet; use pretty_xmlish::Pretty; -use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, Field, Schema, TableDesc}; +use risingwave_common::catalog::{ColumnDesc, Field, Schema, TableDesc}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::sort_util::ColumnOrder; @@ -29,27 +29,16 @@ use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::property::{Cardinality, FunctionalDependencySet, Order}; use crate::utils::{ColIndexMappingRewriteExt, Condition}; -#[derive(Debug, Default, Clone, Educe)] -#[educe(PartialEq, Eq, Hash)] -pub enum ScanTableType { - #[default] - General, - CdcTable, -} - /// [`Scan`] returns contents of a table or other equivalent object #[derive(Debug, Clone, Educe)] #[educe(PartialEq, Eq, Hash)] pub struct Scan { pub table_name: String, - pub scan_table_type: ScanTableType, /// Include `output_col_idx` and columns required in `predicate` pub required_col_idx: Vec, pub output_col_idx: Vec, /// Descriptor of the table pub table_desc: Rc, - /// Descriptor of the external table for CDC - pub cdc_table_desc: Rc, /// Descriptors of all indexes on this table pub indexes: Vec>, /// The pushed down predicates. It refers to column indexes of the table. @@ -77,10 +66,6 @@ impl Scan { /// /// Return `None` if the table's distribution key are not all in the `output_col_idx`. pub fn distribution_key(&self) -> Option> { - if self.is_cdc_table() { - return None; - } - let tb_idx_to_op_idx = self .output_col_idx .iter() @@ -103,20 +88,12 @@ impl Scan { } pub fn primary_key(&self) -> &[ColumnOrder] { - if self.is_cdc_table() { - &self.cdc_table_desc.pk - } else { - &self.table_desc.pk - } + &self.table_desc.pk } pub fn watermark_columns(&self) -> FixedBitSet { - if self.is_cdc_table() { - FixedBitSet::with_capacity(self.get_table_columns().len()) - } else { - let watermark_columns = &self.table_desc.watermark_columns; - self.i2o_col_mapping().rewrite_bitset(watermark_columns) - } + let watermark_columns = &self.table_desc.watermark_columns; + self.i2o_col_mapping().rewrite_bitset(watermark_columns) } pub(crate) fn column_names_with_table_prefix(&self) -> Vec { @@ -241,7 +218,6 @@ impl Scan { Self::new( index_name.to_string(), - ScanTableType::default(), new_output_col_idx, index_table_desc, vec![], @@ -256,7 +232,6 @@ impl Scan { #[allow(clippy::too_many_arguments)] pub(crate) fn new( table_name: String, - scan_table_type: ScanTableType, output_col_idx: Vec, // the column index in the table table_desc: Rc, indexes: Vec>, @@ -267,10 +242,8 @@ impl Scan { ) -> Self { Self::new_inner( table_name, - scan_table_type, output_col_idx, table_desc, - Rc::new(CdcTableDesc::default()), indexes, ctx, predicate, @@ -279,34 +252,11 @@ impl Scan { ) } - /// Create a logical scan node for CDC backfill - pub(crate) fn new_for_cdc( - table_name: String, - output_col_idx: Vec, // the column index in the table - cdc_table_desc: Rc, - ctx: OptimizerContextRef, - ) -> Self { - Self::new_inner( - table_name, - ScanTableType::CdcTable, - output_col_idx, - Rc::new(TableDesc::default()), - cdc_table_desc, - vec![], - ctx, - Condition::true_cond(), - false, - Cardinality::unknown(), - ) - } - #[allow(clippy::too_many_arguments)] pub(crate) fn new_inner( table_name: String, - scan_table_type: ScanTableType, output_col_idx: Vec, // the column index in the table table_desc: Rc, - cdc_table_desc: Rc, indexes: Vec>, ctx: OptimizerContextRef, predicate: Condition, // refers to column indexes of the table @@ -331,11 +281,9 @@ impl Scan { Self { table_name, - scan_table_type, required_col_idx, output_col_idx, table_desc, - cdc_table_desc, indexes, predicate, chunk_size: None, @@ -368,7 +316,6 @@ impl Scan { } } -// TODO: extend for cdc table impl GenericPlanNode for Scan { fn schema(&self) -> Schema { let fields = self @@ -383,21 +330,16 @@ impl GenericPlanNode for Scan { } fn stream_key(&self) -> Option> { - if self.is_cdc_table() { - Some(self.cdc_table_desc.stream_key.clone()) - } else { - let id_to_op_idx = - Self::get_id_to_op_idx_mapping(&self.output_col_idx, &self.table_desc); - self.table_desc - .stream_key - .iter() - .map(|&c| { - id_to_op_idx - .get(&self.table_desc.columns[c].column_id) - .copied() - }) - .collect::>>() - } + let id_to_op_idx = Self::get_id_to_op_idx_mapping(&self.output_col_idx, &self.table_desc); + self.table_desc + .stream_key + .iter() + .map(|&c| { + id_to_op_idx + .get(&self.table_desc.columns[c].column_id) + .copied() + }) + .collect::>>() } fn ctx(&self) -> OptimizerContextRef { @@ -416,21 +358,10 @@ impl GenericPlanNode for Scan { impl Scan { pub fn get_table_columns(&self) -> &[ColumnDesc] { - if self.is_cdc_table() { - &self.cdc_table_desc.columns - } else { - &self.table_desc.columns - } - } - - pub fn is_cdc_table(&self) -> bool { - matches!(self.scan_table_type, ScanTableType::CdcTable) + &self.table_desc.columns } pub fn append_only(&self) -> bool { - if self.is_cdc_table() { - return false; - } self.table_desc.append_only } diff --git a/src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs b/src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs new file mode 100644 index 0000000000000..d155581f01966 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/logical_cdc_scan.rs @@ -0,0 +1,206 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::rc::Rc; + +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::{CdcTableDesc, ColumnDesc}; +use risingwave_common::error::Result; + +use super::generic::GenericPlanRef; +use super::utils::{childless_record, Distill}; +use super::{ + generic, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, PredicatePushdown, ToBatch, + ToStream, +}; +use crate::catalog::ColumnId; +use crate::expr::ExprRewriter; +use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::plan_node::{ + ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamCdcTableScan, + ToStreamContext, +}; +use crate::optimizer::property::Order; +use crate::utils::{ColIndexMapping, Condition}; + +/// `LogicalCdcScan` reads rows of a table from an external upstream database +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct LogicalCdcScan { + pub base: PlanBase, + core: generic::CdcScan, +} + +impl From for LogicalCdcScan { + fn from(core: generic::CdcScan) -> Self { + let base = PlanBase::new_logical_with_core(&core); + Self { base, core } + } +} + +impl From for PlanRef { + fn from(core: generic::CdcScan) -> Self { + LogicalCdcScan::from(core).into() + } +} + +impl LogicalCdcScan { + pub fn create( + table_name: String, // explain-only + cdc_table_desc: Rc, + ctx: OptimizerContextRef, + ) -> Self { + generic::CdcScan::new( + table_name, + (0..cdc_table_desc.columns.len()).collect(), + cdc_table_desc, + ctx, + ) + .into() + } + + pub fn table_name(&self) -> &str { + &self.core.table_name + } + + pub fn cdc_table_desc(&self) -> &CdcTableDesc { + self.core.cdc_table_desc.as_ref() + } + + /// Get the descs of the output columns. + pub fn column_descs(&self) -> Vec { + self.core.column_descs() + } + + /// Get the ids of the output columns. + pub fn output_column_ids(&self) -> Vec { + self.core.output_column_ids() + } + + pub fn clone_with_output_indices(&self, output_col_idx: Vec) -> Self { + generic::CdcScan::new( + self.table_name().to_string(), + output_col_idx, + self.core.cdc_table_desc.clone(), + self.base.ctx().clone(), + ) + .into() + } + + pub fn output_col_idx(&self) -> &Vec { + &self.core.output_col_idx + } +} + +impl_plan_tree_node_for_leaf! {LogicalCdcScan} + +impl Distill for LogicalCdcScan { + fn distill<'a>(&self) -> XmlNode<'a> { + let verbose = self.base.ctx().is_explain_verbose(); + let mut vec = Vec::with_capacity(5); + vec.push(("table", Pretty::from(self.table_name().to_owned()))); + let key_is_columns = true; + let key = if key_is_columns { + "columns" + } else { + "output_columns" + }; + vec.push((key, self.core.columns_pretty(verbose))); + if !key_is_columns { + vec.push(( + "required_columns", + Pretty::Array( + self.output_col_idx() + .iter() + .map(|i| { + let col_name = &self.cdc_table_desc().columns[*i].name; + Pretty::from(if verbose { + format!("{}.{}", self.table_name(), col_name) + } else { + col_name.to_string() + }) + }) + .collect(), + ), + )); + } + + childless_record("LogicalCdcScan", vec) + } +} + +impl ColPrunable for LogicalCdcScan { + fn prune_col(&self, required_cols: &[usize], _ctx: &mut ColumnPruningContext) -> PlanRef { + let output_col_idx: Vec = required_cols + .iter() + .map(|i| self.output_col_idx()[*i]) + .collect(); + assert!(output_col_idx + .iter() + .all(|i| self.output_col_idx().contains(i))); + + self.clone_with_output_indices(output_col_idx).into() + } +} + +impl ExprRewritable for LogicalCdcScan { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + let core = self.core.clone(); + core.rewrite_exprs(r); + Self { + base: self.base.clone_with_new_plan_id(), + core, + } + .into() + } +} + +impl PredicatePushdown for LogicalCdcScan { + fn predicate_pushdown( + &self, + _predicate: Condition, + _ctx: &mut PredicatePushdownContext, + ) -> PlanRef { + self.clone().into() + } +} + +impl ToBatch for LogicalCdcScan { + fn to_batch(&self) -> Result { + unreachable!() + } + + fn to_batch_with_order_required(&self, _required_order: &Order) -> Result { + unreachable!() + } +} + +impl ToStream for LogicalCdcScan { + fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { + Ok(StreamCdcTableScan::new(self.core.clone()).into()) + } + + fn logical_rewrite_for_stream( + &self, + _ctx: &mut RewriteStreamContext, + ) -> Result<(PlanRef, ColIndexMapping)> { + Ok(( + self.clone().into(), + ColIndexMapping::identity(self.schema().len()), + )) + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index bd1297a4cb92f..8c4aedf524920 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -18,7 +18,7 @@ use std::rc::Rc; use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, TableDesc}; +use risingwave_common::catalog::{ColumnDesc, TableDesc}; use risingwave_common::error::Result; use risingwave_common::util::sort_util::ColumnOrder; @@ -31,10 +31,9 @@ use super::{ use crate::catalog::{ColumnId, IndexCatalog}; use crate::expr::{CorrelatedInputRef, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::optimizer_context::OptimizerContextRef; -use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ BatchSeqScan, ColumnPruningContext, LogicalFilter, LogicalProject, LogicalValues, - PredicatePushdownContext, RewriteStreamContext, StreamCdcTableScan, ToStreamContext, + PredicatePushdownContext, RewriteStreamContext, ToStreamContext, }; use crate::optimizer::property::{Cardinality, Order}; use crate::optimizer::rule::IndexSelectionRule; @@ -64,7 +63,6 @@ impl LogicalScan { /// Create a [`LogicalScan`] node. Used by planner. pub fn create( table_name: String, // explain-only - scan_table_type: ScanTableType, table_desc: Rc, indexes: Vec>, ctx: OptimizerContextRef, @@ -73,7 +71,6 @@ impl LogicalScan { ) -> Self { generic::Scan::new( table_name, - scan_table_type, (0..table_desc.columns.len()).collect(), table_desc, indexes, @@ -85,32 +82,10 @@ impl LogicalScan { .into() } - pub fn create_for_cdc( - table_name: String, // explain-only - cdc_table_desc: Rc, - ctx: OptimizerContextRef, - ) -> Self { - generic::Scan::new_for_cdc( - table_name, - (0..cdc_table_desc.columns.len()).collect(), - cdc_table_desc, - ctx, - ) - .into() - } - pub fn table_name(&self) -> &str { &self.core.table_name } - pub fn scan_table_type(&self) -> &ScanTableType { - &self.core.scan_table_type - } - - pub fn is_cdc_table(&self) -> bool { - matches!(self.core.scan_table_type, ScanTableType::CdcTable) - } - pub fn for_system_time_as_of_proctime(&self) -> bool { self.core.for_system_time_as_of_proctime } @@ -125,10 +100,6 @@ impl LogicalScan { self.core.table_desc.as_ref() } - pub fn cdc_table_desc(&self) -> &CdcTableDesc { - self.core.cdc_table_desc.as_ref() - } - /// Get the descs of the output columns. pub fn column_descs(&self) -> Vec { self.core.column_descs() @@ -272,7 +243,6 @@ impl LogicalScan { let scan_without_predicate = generic::Scan::new( self.table_name().to_string(), - self.scan_table_type().clone(), self.required_col_idx().to_vec(), self.core.table_desc.clone(), self.indexes().to_vec(), @@ -292,10 +262,8 @@ impl LogicalScan { fn clone_with_predicate(&self, predicate: Condition) -> Self { generic::Scan::new_inner( self.table_name().to_string(), - self.scan_table_type().clone(), self.output_col_idx().to_vec(), self.core.table_desc.clone(), - self.core.cdc_table_desc.clone(), self.indexes().to_vec(), self.base.ctx().clone(), predicate, @@ -308,10 +276,8 @@ impl LogicalScan { pub fn clone_with_output_indices(&self, output_col_idx: Vec) -> Self { generic::Scan::new_inner( self.table_name().to_string(), - self.scan_table_type().clone(), output_col_idx, self.core.table_desc.clone(), - self.core.cdc_table_desc.clone(), self.indexes().to_vec(), self.base.ctx().clone(), self.predicate().clone(), @@ -419,11 +385,6 @@ impl PredicatePushdown for LogicalScan { mut predicate: Condition, _ctx: &mut PredicatePushdownContext, ) -> PlanRef { - // skip pushdown if the table is cdc table - if self.is_cdc_table() { - return self.clone().into(); - } - // If the predicate contains `CorrelatedInputRef` or `now()`. We don't push down. // This case could come from the predicate push down before the subquery unnesting. struct HasCorrelated {} @@ -554,11 +515,7 @@ impl ToBatch for LogicalScan { impl ToStream for LogicalScan { fn to_stream(&self, ctx: &mut ToStreamContext) -> Result { if self.predicate().always_true() { - if self.is_cdc_table() { - Ok(StreamCdcTableScan::new(self.core.clone()).into()) - } else { - Ok(StreamTableScan::new(self.core.clone()).into()) - } + Ok(StreamTableScan::new(self.core.clone()).into()) } else { let (scan, predicate, project_expr) = self.predicate_pull_up(); let mut plan = LogicalFilter::create(scan.into(), predicate); @@ -573,13 +530,6 @@ impl ToStream for LogicalScan { &self, _ctx: &mut RewriteStreamContext, ) -> Result<(PlanRef, ColIndexMapping)> { - if self.is_cdc_table() { - return Ok(( - self.clone().into(), - ColIndexMapping::identity(self.schema().len()), - )); - } - match self.base.stream_key().is_none() { true => { let mut col_ids = HashSet::new(); diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 77f8fdb606077..a0be5132c92ac 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -776,6 +776,7 @@ mod batch_update; mod batch_values; mod logical_agg; mod logical_apply; +mod logical_cdc_scan; mod logical_dedup; mod logical_delete; mod logical_except; @@ -864,6 +865,7 @@ pub use batch_update::BatchUpdate; pub use batch_values::BatchValues; pub use logical_agg::LogicalAgg; pub use logical_apply::LogicalApply; +pub use logical_cdc_scan::LogicalCdcScan; pub use logical_dedup::LogicalDedup; pub use logical_delete::LogicalDelete; pub use logical_except::LogicalExcept; @@ -948,6 +950,7 @@ macro_rules! for_all_plan_nodes { , { Logical, Filter } , { Logical, Project } , { Logical, Scan } + , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } , { Logical, Insert } @@ -1041,6 +1044,7 @@ macro_rules! for_logical_plan_nodes { , { Logical, Filter } , { Logical, Project } , { Logical, Scan } + , { Logical, CdcScan } , { Logical, SysScan } , { Logical, Source } , { Logical, Insert } diff --git a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs index 4731227bc54c7..4ef394f3748fd 100644 --- a/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_cdc_table_scan.rs @@ -36,13 +36,13 @@ use crate::{Explain, TableCatalog}; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamCdcTableScan { pub base: PlanBase, - core: generic::Scan, + core: generic::CdcScan, batch_plan_id: PlanNodeId, stream_scan_type: StreamScanType, } impl StreamCdcTableScan { - pub fn new(core: generic::Scan) -> Self { + pub fn new(core: generic::CdcScan) -> Self { let batch_plan_id = core.ctx.next_plan_node_id(); let distribution = Distribution::SomeShard; let base = PlanBase::new_stream_with_core( @@ -64,7 +64,7 @@ impl StreamCdcTableScan { &self.core.table_name } - pub fn core(&self) -> &generic::Scan { + pub fn core(&self) -> &generic::CdcScan { &self.core } @@ -263,7 +263,7 @@ impl ExprRewritable for StreamCdcTableScan { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - let mut core = self.core.clone(); + let core = self.core.clone(); core.rewrite_exprs(r); Self::new(core).into() } diff --git a/src/frontend/src/optimizer/rule/index_selection_rule.rs b/src/frontend/src/optimizer/rule/index_selection_rule.rs index 7aeefe4f9154e..3920924d8146e 100644 --- a/src/frontend/src/optimizer/rule/index_selection_rule.rs +++ b/src/frontend/src/optimizer/rule/index_selection_rule.rs @@ -66,7 +66,7 @@ use crate::expr::{ FunctionCall, InputRef, }; use crate::optimizer::optimizer_context::OptimizerContextRef; -use crate::optimizer::plan_node::generic::{GenericPlanRef, ScanTableType}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, LogicalJoin, LogicalScan, LogicalUnion, PlanTreeNode, PlanTreeNodeBinary, PredicatePushdown, PredicatePushdownContext, @@ -222,7 +222,6 @@ impl IndexSelectionRule { let index_scan = LogicalScan::create( index.index_table.name.clone(), - ScanTableType::default(), index.index_table.table_desc().into(), vec![], logical_scan.ctx(), @@ -232,7 +231,6 @@ impl IndexSelectionRule { let primary_table_scan = LogicalScan::create( index.primary_table.name.clone(), - ScanTableType::default(), index.primary_table.table_desc().into(), vec![], logical_scan.ctx(), @@ -332,7 +330,6 @@ impl IndexSelectionRule { let primary_table_scan = LogicalScan::create( logical_scan.table_name().to_string(), - ScanTableType::default(), primary_table_desc.clone().into(), vec![], logical_scan.ctx(), @@ -560,7 +557,6 @@ impl IndexSelectionRule { let primary_access = generic::Scan::new( logical_scan.table_name().to_string(), - ScanTableType::default(), primary_table_desc .pk .iter() @@ -603,7 +599,6 @@ impl IndexSelectionRule { Some( generic::Scan::new( index.index_table.name.to_string(), - ScanTableType::default(), index .primary_table_pk_ref_to_index_table() .iter() diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index b4987403a7926..7ff5806eceb08 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -24,7 +24,6 @@ use crate::binder::{ BoundWindowTableFunction, Relation, WindowTableFunctionKind, }; use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; -use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ LogicalApply, LogicalHopWindow, LogicalJoin, LogicalProject, LogicalScan, LogicalShare, LogicalSource, LogicalSysScan, LogicalTableFunction, LogicalValues, PlanRef, @@ -69,7 +68,6 @@ impl Planner { pub(super) fn plan_base_table(&mut self, base_table: &BoundBaseTable) -> Result { Ok(LogicalScan::create( base_table.table_catalog.name().to_string(), - ScanTableType::default(), Rc::new(base_table.table_catalog.table_desc()), base_table .table_indexes diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index f35b4771ba13b..db5581aea856c 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -449,7 +449,6 @@ pub(crate) mod tests { use crate::catalog::catalog_service::CatalogReader; use crate::catalog::root_catalog::Catalog; use crate::expr::InputRef; - use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ generic, BatchExchange, BatchFilter, BatchHashJoin, EqJoinPredicate, LogicalScan, ToBatch, }; @@ -513,7 +512,6 @@ pub(crate) mod tests { let table_id = 0.into(); let batch_plan_node: PlanRef = LogicalScan::create( "".to_string(), - ScanTableType::default(), Rc::new(TableDesc { table_id, stream_key: vec![], From b289d384c74afa9fac3c415960b5e013b619b666 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Sat, 18 Nov 2023 00:46:36 +0800 Subject: [PATCH 26/36] fix: reject duplicate entries in with clause (#13488) --- e2e_test/ddl/throttle.slt | 4 +- e2e_test/schema_registry/pb.slt | 6 +-- e2e_test/source/basic/kafka.slt | 16 +++++--- .../basic/old_row_format_syntax/kafka.slt | 6 +-- src/frontend/src/utils/with_options.rs | 37 +++++++++++-------- 5 files changed, 38 insertions(+), 31 deletions(-) diff --git a/e2e_test/ddl/throttle.slt b/e2e_test/ddl/throttle.slt index 6395697f0df79..9b6c2f053bf63 100644 --- a/e2e_test/ddl/throttle.slt +++ b/e2e_test/ddl/throttle.slt @@ -6,8 +6,8 @@ create table t1 (v1 int); # tracked in https://github.com/risingwavelabs/risingwave/issues/13474 # create with duplicate streaming_rate_limit -# statement error -# create materialized view mv1 with (streaming_rate_limit = 1000, streaming_rate_limit = 2000) as select * from t1; +statement error Duplicated option +create materialized view mv1 with (streaming_rate_limit = 1000, streaming_rate_limit = 2000) as select * from t1; # create with unknown fields statement error unexpected options in WITH clause diff --git a/e2e_test/schema_registry/pb.slt b/e2e_test/schema_registry/pb.slt index 618ecd71e960c..fb40759d34ada 100644 --- a/e2e_test/schema_registry/pb.slt +++ b/e2e_test/schema_registry/pb.slt @@ -7,8 +7,7 @@ create table sr_pb_test with ( connector = 'kafka', topic = 'sr_pb_test', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - message = 'test.User') + scan.startup.mode = 'earliest') FORMAT plain ENCODE protobuf( schema.registry = 'http://message_queue:8081', message = 'test.User' @@ -20,8 +19,7 @@ create table sr_pb_test_bk with ( connector = 'kafka', topic = 'sr_pb_test', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - message = 'test.User') + scan.startup.mode = 'earliest') FORMAT plain ENCODE protobuf( schema.registry = 'http://message_queue:8081,http://message_queue:8081', message = 'test.User' diff --git a/e2e_test/source/basic/kafka.slt b/e2e_test/source/basic/kafka.slt index 56097de50809a..b1ad66bda7f99 100644 --- a/e2e_test/source/basic/kafka.slt +++ b/e2e_test/source/basic/kafka.slt @@ -184,9 +184,11 @@ create table s11 with ( connector = 'kafka', topic = 'proto_c_bin', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - proto.message = 'test.User' -) FORMAT PLAIN ENCODE PROTOBUF (message = 'test.User', schema.location = 'file:///risingwave/proto-complex-schema'); + scan.startup.mode = 'earliest') +FORMAT PLAIN ENCODE PROTOBUF ( + message = 'test.User', + schema.location = 'file:///risingwave/proto-complex-schema' +); statement ok CREATE TABLE s12( @@ -268,9 +270,11 @@ create source s17 with ( connector = 'kafka', topic = 'proto_c_bin', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - proto.message = 'test.User' -) FORMAT PLAIN ENCODE PROTOBUF (message = 'test.User', schema.location = 'file:///risingwave/proto-complex-schema'); + scan.startup.mode = 'earliest') +FORMAT PLAIN ENCODE PROTOBUF ( + message = 'test.User', + schema.location = 'file:///risingwave/proto-complex-schema' +); statement ok create source s18 with ( diff --git a/e2e_test/source/basic/old_row_format_syntax/kafka.slt b/e2e_test/source/basic/old_row_format_syntax/kafka.slt index 3d2e4719d744d..05e0d55c28c48 100644 --- a/e2e_test/source/basic/old_row_format_syntax/kafka.slt +++ b/e2e_test/source/basic/old_row_format_syntax/kafka.slt @@ -176,8 +176,7 @@ create table s11 with ( connector = 'kafka', topic = 'proto_c_bin', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - proto.message = 'test.User' + scan.startup.mode = 'earliest' ) row format protobuf message 'test.User' row schema location 'file:///risingwave/proto-complex-schema' statement ok @@ -260,8 +259,7 @@ create source s17 with ( connector = 'kafka', topic = 'proto_c_bin', properties.bootstrap.server = 'message_queue:29092', - scan.startup.mode = 'earliest', - proto.message = 'test.User' + scan.startup.mode = 'earliest' ) row format protobuf message 'test.User' row schema location 'file:///risingwave/proto-complex-schema' statement ok diff --git a/src/frontend/src/utils/with_options.rs b/src/frontend/src/utils/with_options.rs index 4b0a70ef856dc..77d4185804215 100644 --- a/src/frontend/src/utils/with_options.rs +++ b/src/frontend/src/utils/with_options.rs @@ -16,7 +16,6 @@ use std::collections::{BTreeMap, HashMap}; use std::convert::TryFrom; use std::num::NonZeroU32; -use itertools::Itertools; use risingwave_common::error::{ErrorCode, Result as RwResult, RwError}; use risingwave_connector::source::kafka::{ insert_privatelink_broker_rewrite_map, PRIVATELINK_ENDPOINT_KEY, @@ -172,20 +171,28 @@ impl TryFrom<&[SqlOption]> for WithOptions { type Error = RwError; fn try_from(options: &[SqlOption]) -> Result { - let inner = options - .iter() - .cloned() - .map(|x| match x.value { - Value::CstyleEscapedString(s) => Ok((x.name.real_value(), s.value)), - Value::SingleQuotedString(s) => Ok((x.name.real_value(), s)), - Value::Number(n) => Ok((x.name.real_value(), n)), - Value::Boolean(b) => Ok((x.name.real_value(), b.to_string())), - _ => Err(ErrorCode::InvalidParameterValue( - "`with options` or `with properties` only support single quoted string value and C style escaped string" - .to_owned(), - )), - }) - .try_collect()?; + let mut inner: BTreeMap = BTreeMap::new(); + for option in options { + let key = option.name.real_value(); + let value: String = match option.value.clone() { + Value::CstyleEscapedString(s) => s.value, + Value::SingleQuotedString(s) => s, + Value::Number(n) => n, + Value::Boolean(b) => b.to_string(), + _ => { + return Err(RwError::from(ErrorCode::InvalidParameterValue( + "`with options` or `with properties` only support single quoted string value and C style escaped string" + .to_owned(), + ))) + } + }; + if inner.insert(key.clone(), value).is_some() { + return Err(RwError::from(ErrorCode::InvalidParameterValue(format!( + "Duplicated option: {}", + key + )))); + } + } Ok(Self { inner }) } From 47c53a15ffc753c0feb3ae55a29776e3d1d3974f Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Sun, 19 Nov 2023 11:43:19 -0500 Subject: [PATCH 27/36] feat(expr): add local make_timestamptz() (#13456) --- Cargo.lock | 1 + .../basic/local/make_timestamptz.slt.part | 111 ++++++++++++++++++ e2e_test/batch/types/timestamptz_utc.slt.part | 3 + proto/expr.proto | 1 + src/frontend/Cargo.toml | 1 + src/frontend/src/binder/expr/function.rs | 1 + .../src/expr/function_impl/context.rs | 1 + .../expr/function_impl/make_timestamptz.rs | 101 ++++++++++++++++ src/frontend/src/expr/function_impl/mod.rs | 1 + src/frontend/src/expr/pure.rs | 3 +- src/frontend/src/lib.rs | 1 + src/frontend/src/scheduler/local.rs | 4 +- 12 files changed, 227 insertions(+), 2 deletions(-) create mode 100644 e2e_test/batch/basic/local/make_timestamptz.slt.part create mode 100644 src/frontend/src/expr/function_impl/make_timestamptz.rs diff --git a/Cargo.lock b/Cargo.lock index b44d512affc53..93a2918d5d8b6 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7846,6 +7846,7 @@ dependencies = [ "auto_impl", "bk-tree", "bytes", + "chrono", "clap", "downcast-rs", "dyn-clone", diff --git a/e2e_test/batch/basic/local/make_timestamptz.slt.part b/e2e_test/batch/basic/local/make_timestamptz.slt.part new file mode 100644 index 0000000000000..4b6cdfc6efb66 --- /dev/null +++ b/e2e_test/batch/basic/local/make_timestamptz.slt.part @@ -0,0 +1,111 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +query T +SELECT make_timestamptz(1973, 07, 15, 08, 15, 55.33); +---- +1973-07-15 08:15:55.330+00:00 + +query T +SELECT make_timestamptz(-1973, 07, 15, 08, 15, 55.33); +---- +-1973-07-15 08:15:55.330+00:00 + +statement error +SELECT make_timestamptz(1973, 07, 15, 08, 15, -55.33); +---- +db error: ERROR: QueryError + +Caused by these errors (recent errors listed first): + 1: Expr error + 2: Invalid parameter sec: invalid sec + + +statement error +SELECT make_timestamptz(1973, 07, 15, 08, -15, 55.33); +---- +db error: ERROR: QueryError + +Caused by these errors (recent errors listed first): + 1: Expr error + 2: Invalid parameter hour, min, sec: invalid time + + +statement error +SELECT make_timestamptz(1973, -07, 15, 08, 15, 55.33); +---- +db error: ERROR: QueryError + +Caused by these errors (recent errors listed first): + 1: Expr error + 2: Invalid parameter year, month, day: invalid date + + +statement error +SELECT make_timestamptz(1973, 06, 31, 08, 15, 55.33); +---- +db error: ERROR: QueryError + +Caused by these errors (recent errors listed first): + 1: Expr error + 2: Invalid parameter year, month, day: invalid date + + +statement ok +set TimeZone to 'America/New_York'; + +query T +SELECT make_timestamptz(1973, 07, 15, 08, 15, 55.33); +---- +1973-07-15 08:15:55.330-04:00 + +statement error +SELECT make_timestamptz(1910, 12, 24, 0, 0, 0, 'Nehwon/Lankhmar'); +---- +db error: ERROR: QueryError + +Caused by these errors (recent errors listed first): + 1: Expr error + 2: Invalid parameter time_zone: 'Nehwon/Lankhmar' is not a valid timezone + + +query TT +WITH tzs (tz) AS (VALUES ('Europe/Prague'), ('Europe/Paris'), ('America/New_York'), ('EST'), ('EST5EDT'), ('PST8PDT')) SELECT make_timestamptz(2010, 2, 27, 3, 45, 00, tz), tz FROM tzs; +---- +2010-02-26 21:45:00-05:00 Europe/Prague +2010-02-26 21:45:00-05:00 Europe/Paris +2010-02-27 03:45:00-05:00 America/New_York +2010-02-27 03:45:00-05:00 EST +2010-02-27 03:45:00-05:00 EST5EDT +2010-02-27 06:45:00-05:00 PST8PDT + +query TT +WITH tzs (tz) AS (VALUES ('Europe/Prague'), ('Europe/Paris'), ('America/New_York'), ('EST'), ('EST5EDT'), ('PST8PDT')) SELECT make_timestamptz(2010, 2, 27, 3, 45, 00, tz) AT TIME ZONE 'EST5EDT', tz FROM tzs; +---- +2010-02-26 21:45:00 Europe/Prague +2010-02-26 21:45:00 Europe/Paris +2010-02-27 03:45:00 America/New_York +2010-02-27 03:45:00 EST +2010-02-27 03:45:00 EST5EDT +2010-02-27 06:45:00 PST8PDT + +query T +SELECT make_timestamptz(1973, 07, 15, 08, 15, 55.33, 'Asia/Manila') = '1973-07-14 20:15:55.33'::timestamptz; +---- +t + +statement ok +set TimeZone to 'Europe/London'; + +query T +SELECT make_timestamptz(2013, 7, 15, 8, 15, 23.5); +---- +2013-07-15 08:15:23.500+01:00 + +query T +SELECT make_timestamptz(2013, 7, 15, 8, 15, 23.5, 'America/New_York'); +---- +2013-07-15 13:15:23.500+01:00 + +statement ok +set timezone to 'UTC'; \ No newline at end of file diff --git a/e2e_test/batch/types/timestamptz_utc.slt.part b/e2e_test/batch/types/timestamptz_utc.slt.part index 5a0fe6ebd13cc..48d6ab179d194 100644 --- a/e2e_test/batch/types/timestamptz_utc.slt.part +++ b/e2e_test/batch/types/timestamptz_utc.slt.part @@ -2,6 +2,9 @@ # Input in whatever timezone # Output always in UTC (rather than session TimeZone yet) +statement ok +SET RW_IMPLICIT_FLUSH TO true; + query T select '2022-10-01 12:00:00-08:00'::timestamp with time zone; ---- diff --git a/proto/expr.proto b/proto/expr.proto index f7923525f68e0..faf9422a1cb68 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -66,6 +66,7 @@ message ExprNode { CAST_WITH_TIME_ZONE = 108; ADD_WITH_TIME_ZONE = 109; SUBTRACT_WITH_TIME_ZONE = 110; + MAKE_TIMESTAMPTZ = 112; // other functions CAST = 201; SUBSTR = 202; diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index ada566ad81797..c90804f169b49 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -24,6 +24,7 @@ auto_enums = { version = "0.8", features = ["futures03"] } auto_impl = "1" bk-tree = "0.5.0" bytes = "1" +chrono = { version = "0.4" } clap = { version = "4", features = ["derive"] } downcast-rs = "1.2" dyn-clone = "1.0.14" diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 635f475f5396a..96435d4e9f656 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -758,6 +758,7 @@ impl Binder { ("date_trunc", raw_call(ExprType::DateTrunc)), ("date_part", raw_call(ExprType::DatePart)), ("to_date", raw_call(ExprType::CharToDate)), + ("make_timestamptz", raw_call(ExprType::MakeTimestamptz)), // string ("substr", raw_call(ExprType::Substr)), ("length", raw_call(ExprType::Length)), diff --git a/src/frontend/src/expr/function_impl/context.rs b/src/frontend/src/expr/function_impl/context.rs index 13a7175fabb54..ae81e31cc270b 100644 --- a/src/frontend/src/expr/function_impl/context.rs +++ b/src/frontend/src/expr/function_impl/context.rs @@ -24,4 +24,5 @@ define_context! { pub(super) AUTH_CONTEXT: Arc, pub(super) DB_NAME: String, pub(super) SEARCH_PATH: SearchPath, + pub(super) TIME_ZONE: String, } diff --git a/src/frontend/src/expr/function_impl/make_timestamptz.rs b/src/frontend/src/expr/function_impl/make_timestamptz.rs new file mode 100644 index 0000000000000..8d7ea62a696b3 --- /dev/null +++ b/src/frontend/src/expr/function_impl/make_timestamptz.rs @@ -0,0 +1,101 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use chrono::{NaiveDate, NaiveDateTime, NaiveTime}; +use risingwave_common::types::{FloatExt, Timestamptz, F64}; +use risingwave_expr::{capture_context, function, ExprError, Result}; + +use super::context::TIME_ZONE; + +/// Just a wrapper to reuse the `map_err` logic. +#[inline(always)] +pub fn time_zone_err(inner_err: String) -> ExprError { + ExprError::InvalidParam { + name: "time_zone", + reason: inner_err.into(), + } +} + +// year int, month int, day int, hour int, min int, sec double precision +#[function("make_timestamptz(int4, int4, int4, int4, int4, float8) -> timestamptz")] +pub fn make_timestamptz( + year: i32, + month: i32, + day: i32, + hour: i32, + min: i32, + sec: F64, +) -> Result { + make_timestamptz_impl_captured(year, month, day, hour, min, sec) +} + +// year int, month int, day int, hour int, min int, sec double precision, timezone text +#[function("make_timestamptz(int4, int4, int4, int4, int4, float8, varchar) -> timestamptz")] +pub fn make_timestamptz_with_time_zone( + year: i32, + month: i32, + day: i32, + hour: i32, + min: i32, + sec: F64, + time_zone: &str, +) -> Result { + make_timestamptz_impl(time_zone, year, month, day, hour, min, sec) +} + +#[capture_context(TIME_ZONE)] +fn make_timestamptz_impl( + time_zone: &str, + year: i32, + month: i32, + day: i32, + hour: i32, + min: i32, + sec: F64, +) -> Result { + let time_zone = Timestamptz::lookup_time_zone(time_zone).map_err(time_zone_err)?; + if !sec.is_finite() || sec.0.is_sign_negative() { + return Err(ExprError::InvalidParam { + name: "sec", + reason: "invalid sec".into(), + }); + } + let sec_u32 = sec.0.trunc() as u32; + let microsecond_u32 = ((sec.0 - sec.0.trunc()) * 1_000_000.0).round_ties_even() as u32; + let naive_date_time = NaiveDateTime::new( + NaiveDate::from_ymd_opt(year, month as u32, day as u32).ok_or_else(|| { + ExprError::InvalidParam { + name: "year, month, day", + reason: "invalid date".into(), + } + })?, + NaiveTime::from_hms_micro_opt(hour as u32, min as u32, sec_u32, microsecond_u32) + .ok_or_else(|| ExprError::InvalidParam { + name: "hour, min, sec", + reason: "invalid time".into(), + })?, + ); + let date_time = naive_date_time + .and_local_timezone(time_zone) + .latest() + .ok_or_else(|| ExprError::InvalidParam { + name: "time_zone", + reason: format!( + "fail to interpret local timestamp \"{:?}\" in time zone \"{}\"", + naive_date_time, time_zone + ) + .into(), + })?; + Ok(Timestamptz::from(date_time)) +} diff --git a/src/frontend/src/expr/function_impl/mod.rs b/src/frontend/src/expr/function_impl/mod.rs index 1f31b7f307dac..ff13a93b6da7f 100644 --- a/src/frontend/src/expr/function_impl/mod.rs +++ b/src/frontend/src/expr/function_impl/mod.rs @@ -15,3 +15,4 @@ mod cast_regclass; mod col_description; pub mod context; +mod make_timestamptz; diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index e78e4305ed0e5..5c1cc37b0b99e 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -239,7 +239,8 @@ impl ExprVisitor for ImpureAnalyzer { | expr_node::Type::PgSleepFor | expr_node::Type::PgSleepUntil | expr_node::Type::ColDescription - | expr_node::Type::CastRegclass => true, + | expr_node::Type::CastRegclass + | expr_node::Type::MakeTimestamptz => true, } } } diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 309e240211389..516adba5cf95c 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -34,6 +34,7 @@ #![feature(impl_trait_in_assoc_type)] #![feature(result_flattening)] #![feature(error_generic_member_access)] +#![feature(round_ties_even)] #![recursion_limit = "256"] #[cfg(test)] diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index d3d558ef4eff2..fd9f1474c8815 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -147,6 +147,7 @@ impl LocalQueryExecution { let auth_context = self.session.auth_context().clone(); let db_name = self.session.database().to_string(); let search_path = self.session.config().get_search_path().clone(); + let time_zone = self.session.config().get_timezone().to_owned(); let exec = async move { let mut data_stream = self.run().map(|r| r.map_err(|e| Box::new(e) as BoxedError)); @@ -163,13 +164,14 @@ impl LocalQueryExecution { }; use crate::expr::function_impl::context::{ - AUTH_CONTEXT, CATALOG_READER, DB_NAME, SEARCH_PATH, + AUTH_CONTEXT, CATALOG_READER, DB_NAME, SEARCH_PATH, TIME_ZONE, }; let exec = async move { CATALOG_READER::scope(catalog_reader, exec).await }; let exec = async move { DB_NAME::scope(db_name, exec).await }; let exec = async move { SEARCH_PATH::scope(search_path, exec).await }; let exec = async move { AUTH_CONTEXT::scope(auth_context, exec).await }; + let exec = async move { TIME_ZONE::scope(time_zone, exec).await }; compute_runtime.spawn(exec); From d552b5a2278e0a1bc3cbf2675500113af07fcfb8 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Mon, 20 Nov 2023 11:31:42 +0800 Subject: [PATCH 28/36] fix(meta): use only `StreamTableScan` actors on recovery to track progress (#13498) --- src/meta/src/barrier/mod.rs | 4 +- src/meta/src/barrier/progress.rs | 9 ++- src/meta/src/barrier/recovery.rs | 2 +- src/meta/src/manager/catalog/fragment.rs | 23 +++++-- .../recovery/background_ddl.rs | 63 ++++++++++++++++++- 5 files changed, 91 insertions(+), 10 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index c0752bd484055..2d19b33216d48 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -100,7 +100,7 @@ impl From> for HashMap { } } -pub(crate) type TableActorMap = TableMap>; +pub(crate) type TableActorMap = TableMap>; pub(crate) type TableUpstreamMvCountMap = TableMap>; pub(crate) type TableDefinitionMap = TableMap; pub(crate) type TableNotifierMap = TableMap; @@ -1085,7 +1085,9 @@ impl GlobalBarrierManager { commands.push(command); } for progress in resps.iter().flat_map(|r| &r.create_mview_progress) { + tracing::trace!(?progress, "update progress"); if let Some(command) = tracker.update(progress, &version_stats) { + tracing::trace!(?progress, "update progress"); commands.push(command); } } diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index ce0dc788424bf..2fc012c031a9b 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -91,6 +91,7 @@ impl Progress { /// Update the progress of `actor`. fn update(&mut self, actor: ActorId, new_state: BackfillState, upstream_total_key_count: u64) { self.upstream_total_key_count = upstream_total_key_count; + let total_actors = self.states.len(); match self.states.remove(&actor).unwrap() { BackfillState::Init => {} BackfillState::ConsumingUpstream(_, old_consumed_rows) => { @@ -104,8 +105,14 @@ impl Progress { self.consumed_rows += new_consumed_rows; } BackfillState::Done(new_consumed_rows) => { + tracing::debug!("actor {} done", actor); self.consumed_rows += new_consumed_rows; self.done_count += 1; + tracing::debug!( + "{} actors out of {} complete", + self.done_count, + total_actors, + ); } }; self.states.insert(actor, new_state); @@ -263,7 +270,7 @@ impl CreateMviewProgressTracker { ) -> Self { let mut actor_map = HashMap::new(); let mut progress_map = HashMap::new(); - let table_map: HashMap<_, Vec> = table_map.into(); + let table_map: HashMap<_, HashSet> = table_map.into(); for (creating_table_id, actors) in table_map { // 1. Recover `BackfillState` in the tracker. let mut states = HashMap::new(); diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index fc18ba4fbb612..1527b279bc53f 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -147,7 +147,7 @@ impl GlobalBarrierManager { let table_map = self .fragment_manager - .get_table_id_actor_mapping(&creating_table_ids) + .get_table_id_stream_scan_actor_mapping(&creating_table_ids) .await; let table_fragment_map = self .fragment_manager diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 2df089e57667d..ab6d93d0d1dca 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -169,18 +169,33 @@ impl FragmentManager { /// 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_actor_mapping( + pub async fn get_table_id_stream_scan_actor_mapping( &self, table_ids: &[TableId], - ) -> HashMap> { + ) -> 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_stream_scan(stream_node: &StreamNode) -> bool { + let is_node_scan = if let Some(node) = &stream_node.node_body { + node.is_stream_scan() + } else { + false + }; + is_node_scan || stream_node.get_input().iter().any(has_stream_scan) + } for table_id in table_ids { if let Some(table_fragment) = map.get(table_id) { - let mut actors = vec![]; + let mut actors = HashSet::new(); for fragment in table_fragment.fragments.values() { for actor in &fragment.actors { - actors.push(actor.actor_id) + if let Some(node) = &actor.nodes + && has_stream_scan(node) + { + actors.insert(actor.actor_id); + } else { + tracing::trace!("ignoring actor: {:?}", actor); + } } } table_map.insert(*table_id, actors); diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs index 0f8fceb46fd58..014c7fd1c0ff7 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -38,7 +38,24 @@ async fn kill_cn_and_wait_recover(cluster: &Cluster) { sleep(Duration::from_secs(10)).await; } -async fn kill_and_wait_recover(cluster: &Cluster) { +async fn kill_cn_and_meta_and_wait_recover(cluster: &Cluster) { + cluster + .kill_nodes( + [ + "compute-1", + "compute-2", + "compute-3", + "meta-1", + "meta-2", + "meta-3", + ], + 0, + ) + .await; + sleep(Duration::from_secs(10)).await; +} + +async fn kill_random_and_wait_recover(cluster: &Cluster) { // Kill it again for _ in 0..3 { sleep(Duration::from_secs(2)).await; @@ -102,7 +119,7 @@ async fn test_background_mv_barrier_recovery() -> Result<()> { .await?; session.flush().await?; - kill_and_wait_recover(&cluster).await; + kill_random_and_wait_recover(&cluster).await; // Now just wait for it to complete. session.run(WAIT).await?; @@ -129,6 +146,46 @@ async fn test_background_mv_barrier_recovery() -> Result<()> { Ok(()) } +#[tokio::test] +async fn test_background_join_mv_recovery() -> Result<()> { + init_logger(); + let mut cluster = Cluster::start(Configuration::for_background_ddl()).await?; + let mut session = cluster.start_session(); + + session.run("CREATE TABLE t1 (v1 int)").await?; + session.run("CREATE TABLE t2 (v1 int)").await?; + session + .run("INSERT INTO t1 SELECT generate_series FROM generate_series(1, 200);") + .await?; + session + .run("INSERT INTO t2 SELECT generate_series FROM generate_series(1, 200);") + .await?; + session.flush().await?; + session.run(SET_RATE_LIMIT_2).await?; + session.run(SET_BACKGROUND_DDL).await?; + session + .run("CREATE MATERIALIZED VIEW mv1 as select t1.v1 from t1 join t2 on t1.v1 = t2.v1;") + .await?; + sleep(Duration::from_secs(2)).await; + + kill_cn_and_meta_and_wait_recover(&cluster).await; + + // Now just wait for it to complete. + session.run(WAIT).await?; + + let t_count = session.run("SELECT COUNT(v1) FROM t1").await?; + let mv1_count = session.run("SELECT COUNT(v1) FROM mv1").await?; + assert_eq!(t_count, mv1_count); + + // Make sure that if MV killed and restarted + // it will not be dropped. + session.run("DROP MATERIALIZED VIEW mv1;").await?; + session.run("DROP TABLE t1;").await?; + session.run("DROP TABLE t2;").await?; + + Ok(()) +} + #[tokio::test] async fn test_background_ddl_cancel() -> Result<()> { init_logger(); @@ -160,7 +217,7 @@ async fn test_background_ddl_cancel() -> Result<()> { create_mv(&mut session).await?; // Test cancel after kill meta - kill_and_wait_recover(&cluster).await; + kill_random_and_wait_recover(&cluster).await; let ids = cancel_stream_jobs(&mut session).await?; assert_eq!(ids.len(), 1); From 4fa7645c2a502cde49fc24c9f5305bb0089bbbae Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Mon, 20 Nov 2023 11:42:55 +0800 Subject: [PATCH 29/36] fix: incorrect auto dump threshold (#13495) --- src/common/heap_profiling/src/profiler.rs | 4 ++++ src/common/src/config.rs | 3 +-- src/compute/src/server.rs | 27 ++++++++++++++--------- src/storage/compactor/src/server.rs | 16 +++++++------- 4 files changed, 29 insertions(+), 21 deletions(-) diff --git a/src/common/heap_profiling/src/profiler.rs b/src/common/heap_profiling/src/profiler.rs index 49e81e8526241..de7a885d83331 100644 --- a/src/common/heap_profiling/src/profiler.rs +++ b/src/common/heap_profiling/src/profiler.rs @@ -36,6 +36,10 @@ pub struct HeapProfiler { } impl HeapProfiler { + /// # Arguments + /// + /// `total_memory` must be the total available memory for the process. + /// It will be compared with jemalloc's allocated memory. pub fn new(total_memory: usize, config: HeapProfilingConfig) -> Self { let threshold_auto_dump_heap_profile = (total_memory as f64 * config.threshold_auto as f64) as usize; diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 0090612d20ea3..f80afbebf0ee0 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -524,8 +524,7 @@ pub struct StorageConfig { pub compactor_max_task_multiplier: f32, /// The percentage of memory available when compactor is deployed separately. - /// total_memory_available_bytes = system_memory_available_bytes * - /// compactor_memory_available_proportion + /// non_reserved_memory_bytes = system_memory_available_bytes * compactor_memory_available_proportion #[serde(default = "default::storage::compactor_memory_available_proportion")] pub compactor_memory_available_proportion: f64, diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 672e0f43bea06..5ce4139ec3327 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -148,15 +148,6 @@ pub async fn compute_node_serve( reserved_memory_bytes, ); - // NOTE: Due to some limits, we use `compute_memory_bytes + storage_memory_bytes` as - // `total_compute_memory_bytes` for memory control. This is just a workaround for some - // memory control issues and should be modified as soon as we figure out a better solution. - // - // Related issues: - // - https://github.com/risingwavelabs/risingwave/issues/8696 - // - https://github.com/risingwavelabs/risingwave/issues/8822 - let total_memory_bytes = compute_memory_bytes + storage_memory_bytes; - let storage_opts = Arc::new(StorageOpts::from(( &config, &system_params, @@ -285,7 +276,18 @@ pub async fn compute_node_serve( let batch_mgr_clone = batch_mgr.clone(); let stream_mgr_clone = stream_mgr.clone(); - let memory_mgr = GlobalMemoryManager::new(streaming_metrics.clone(), total_memory_bytes); + // NOTE: Due to some limits, we use `compute_memory_bytes + storage_memory_bytes` as + // `total_compute_memory_bytes` for memory control. This is just a workaround for some + // memory control issues and should be modified as soon as we figure out a better solution. + // + // Related issues: + // - https://github.com/risingwavelabs/risingwave/issues/8696 + // - https://github.com/risingwavelabs/risingwave/issues/8822 + let memory_mgr = GlobalMemoryManager::new( + streaming_metrics.clone(), + compute_memory_bytes + storage_memory_bytes, + ); + // Run a background memory manager tokio::spawn(memory_mgr.clone().run( batch_mgr_clone, @@ -294,7 +296,10 @@ pub async fn compute_node_serve( system_params_manager.watch_params(), )); - let heap_profiler = HeapProfiler::new(total_memory_bytes, config.server.heap_profiling.clone()); + let heap_profiler = HeapProfiler::new( + opts.total_memory_bytes, + config.server.heap_profiling.clone(), + ); // Run a background heap profiler heap_profiler.start(); diff --git a/src/storage/compactor/src/server.rs b/src/storage/compactor/src/server.rs index 3ad23bf68cc3b..7480b51b1cc55 100644 --- a/src/storage/compactor/src/server.rs +++ b/src/storage/compactor/src/server.rs @@ -27,7 +27,7 @@ use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::telemetry::manager::TelemetryManager; use risingwave_common::telemetry::telemetry_env_enabled; use risingwave_common::util::addr::HostAddr; -use risingwave_common::util::resource_util; +use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_heap_profiling::HeapProfiler; use risingwave_common_service::metrics_manager::MetricsManager; @@ -87,18 +87,18 @@ pub async fn prepare_start_parameters( &system_params_reader, &storage_memory_config, ))); - let total_memory_available_bytes = - (resource_util::memory::system_memory_available_bytes() as f64 - * config.storage.compactor_memory_available_proportion) as usize; + let non_reserved_memory_bytes = (system_memory_available_bytes() as f64 + * config.storage.compactor_memory_available_proportion) + as usize; let meta_cache_capacity_bytes = storage_opts.meta_cache_capacity_mb * (1 << 20); let compactor_memory_limit_bytes = match config.storage.compactor_memory_limit_mb { Some(compactor_memory_limit_mb) => compactor_memory_limit_mb as u64 * (1 << 20), - None => (total_memory_available_bytes - meta_cache_capacity_bytes) as u64, + None => (non_reserved_memory_bytes - meta_cache_capacity_bytes) as u64, }; tracing::info!( - "Compactor total_memory_available_bytes {} meta_cache_capacity_bytes {} compactor_memory_limit_bytes {} sstable_size_bytes {} block_size_bytes {}", - total_memory_available_bytes, meta_cache_capacity_bytes, compactor_memory_limit_bytes, + "Compactor non_reserved_memory_bytes {} meta_cache_capacity_bytes {} compactor_memory_limit_bytes {} sstable_size_bytes {} block_size_bytes {}", + non_reserved_memory_bytes, meta_cache_capacity_bytes, compactor_memory_limit_bytes, storage_opts.sstable_size_mb * (1 << 20), storage_opts.block_size_kb * (1 << 10), ); @@ -145,7 +145,7 @@ pub async fn prepare_start_parameters( )); let heap_profiler = HeapProfiler::new( - total_memory_available_bytes, + system_memory_available_bytes(), config.server.heap_profiling.clone(), ); From ece7f2d7c74a750b7d60ed0ddc21582c3e2acf58 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 20 Nov 2023 03:53:09 +0000 Subject: [PATCH 30/36] chore(deps): Bump arrow-array from 48.0.0 to 49.0.0 (#13445) Signed-off-by: dependabot[bot] Signed-off-by: Runji Wang Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Runji Wang Co-authored-by: wangrunji0408 --- Cargo.lock | 54 ++++++++++++++++++----------------- Cargo.toml | 16 ++++++----- src/workspace-hack/Cargo.toml | 6 ++-- 3 files changed, 41 insertions(+), 35 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 93a2918d5d8b6..065950598061d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -243,9 +243,9 @@ checksum = "96d30a06541fbafbc7f82ed10c06164cfbd2c401138f6addd8404629c4b16711" [[package]] name = "arrow-arith" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c5c3d17fc5b006e7beeaebfb1d2edfc92398b981f82d9744130437909b72a468" +checksum = "34ccd45e217ffa6e53bbb0080990e77113bdd4e91ddb84e97b77649810bcf1a7" dependencies = [ "arrow-array", "arrow-buffer", @@ -258,9 +258,9 @@ dependencies = [ [[package]] name = "arrow-array" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55705ada5cdde4cb0f202ffa6aa756637e33fea30e13d8d0d0fd6a24ffcee1e3" +checksum = "6bda9acea48b25123c08340f3a8ac361aa0f74469bb36f5ee9acf923fce23e9d" dependencies = [ "ahash 0.8.3", "arrow-buffer", @@ -274,9 +274,9 @@ dependencies = [ [[package]] name = "arrow-buffer" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a722f90a09b94f295ab7102542e97199d3500128843446ef63e410ad546c5333" +checksum = "01a0fc21915b00fc6c2667b069c1b64bdd920982f426079bc4a7cab86822886c" dependencies = [ "bytes", "half 2.3.1", @@ -285,15 +285,16 @@ dependencies = [ [[package]] name = "arrow-cast" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af01fc1a06f6f2baf31a04776156d47f9f31ca5939fe6d00cd7a059f95a46ff1" +checksum = "5dc0368ed618d509636c1e3cc20db1281148190a78f43519487b2daf07b63b4a" dependencies = [ "arrow-array", "arrow-buffer", "arrow-data", "arrow-schema", "arrow-select", + "base64 0.21.4", "chrono", "half 2.3.1", "lexical-core", @@ -302,9 +303,9 @@ dependencies = [ [[package]] name = "arrow-data" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d0a547195e607e625e7fafa1a7269b8df1a4a612c919efd9b26bd86e74538f3a" +checksum = "907fafe280a3874474678c1858b9ca4cb7fd83fb8034ff5b6d6376205a08c634" dependencies = [ "arrow-buffer", "arrow-schema", @@ -314,9 +315,9 @@ dependencies = [ [[package]] name = "arrow-flight" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c58645809ced5acd6243e89a63ae8535a2ab50d780affcd7efe8c7473a0da661" +checksum = "624e0dcb6b5a7a06222bfd2be3f7e905ce849a6b714ec989f18cdba330c77d38" dependencies = [ "arrow-array", "arrow-buffer", @@ -334,9 +335,9 @@ dependencies = [ [[package]] name = "arrow-ipc" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e36bf091502ab7e37775ff448413ef1ffff28ff93789acb669fffdd51b394d51" +checksum = "79a43d6808411886b8c7d4f6f7dd477029c1e77ffffffb7923555cc6579639cd" dependencies = [ "arrow-array", "arrow-buffer", @@ -348,9 +349,9 @@ dependencies = [ [[package]] name = "arrow-ord" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4502123d2397319f3a13688432bc678c61cb1582f2daa01253186da650bf5841" +checksum = "9b23b0e53c0db57c6749997fd343d4c0354c994be7eca67152dd2bdb9a3e1bb4" dependencies = [ "arrow-array", "arrow-buffer", @@ -363,9 +364,9 @@ dependencies = [ [[package]] name = "arrow-row" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "249fc5a07906ab3f3536a6e9f118ec2883fbcde398a97a5ba70053f0276abda4" +checksum = "361249898d2d6d4a6eeb7484be6ac74977e48da12a4dd81a708d620cc558117a" dependencies = [ "ahash 0.8.3", "arrow-array", @@ -378,15 +379,15 @@ dependencies = [ [[package]] name = "arrow-schema" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9d7a8c3f97f5ef6abd862155a6f39aaba36b029322462d72bbcfa69782a50614" +checksum = "09e28a5e781bf1b0f981333684ad13f5901f4cd2f20589eab7cf1797da8fc167" [[package]] name = "arrow-select" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f868f4a5001429e20f7c1994b5cd1aa68b82e3db8cf96c559cdb56dc8be21410" +checksum = "4f6208466590960efc1d2a7172bc4ff18a67d6e25c529381d7f96ddaf0dc4036" dependencies = [ "ahash 0.8.3", "arrow-array", @@ -5851,9 +5852,9 @@ dependencies = [ [[package]] name = "parquet" -version = "48.0.0" +version = "49.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "239229e6a668ab50c61de3dce61cf0fa1069345f7aa0f4c934491f92205a4945" +checksum = "af88740a842787da39b3d69ce5fbf6fce97d20211d3b299fee0a0da6430c74d4" dependencies = [ "ahash 0.8.3", "arrow-array", @@ -6569,7 +6570,7 @@ checksum = "8bdf592881d821b83d471f8af290226c8d51402259e9bb5be7f9f8bdebbb11ac" dependencies = [ "bytes", "heck 0.4.1", - "itertools 0.10.5", + "itertools 0.11.0", "log", "multimap 0.8.3", "once_cell", @@ -6603,7 +6604,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "265baba7fabd416cf5078179f7d2cbeca4ce7a9041111900675ea7c4cb8a4c32" dependencies = [ "anyhow", - "itertools 0.10.5", + "itertools 0.11.0", "proc-macro2", "quote", "syn 2.0.37", @@ -11430,6 +11431,7 @@ dependencies = [ "hyper", "indexmap 1.9.3", "itertools 0.10.5", + "itertools 0.11.0", "jni", "lazy_static", "lexical-core", diff --git a/Cargo.toml b/Cargo.toml index dd6fcf7269853..5a50c9367ed56 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -115,13 +115,15 @@ prost = { version = "0.12" } icelake = { git = "https://github.com/icelake-io/icelake", rev = "5cdcdffd24f4624a0a43f92c5f368988169a799b", features = [ "prometheus", ] } -arrow-array = "48" -arrow-cast = "48" -arrow-schema = "48" -arrow-buffer = "48" -arrow-flight = "48" -arrow-select = "48" -arrow-ord = "48" +arrow-array = "49" +arrow-arith = "49" +arrow-cast = "49" +arrow-schema = "49" +arrow-buffer = "49" +arrow-flight = "49" +arrow-select = "49" +arrow-ord = "49" +arrow-row = "49" thiserror-ext = "0.0.6" tikv-jemalloc-ctl = { git = "https://github.com/risingwavelabs/jemallocator.git", rev = "64a2d9" } tikv-jemallocator = { git = "https://github.com/risingwavelabs/jemallocator.git", features = [ diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index 6a38db45a2762..b263a9c5bcefb 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -59,7 +59,8 @@ hashbrown-5ef9efb8ec2df382 = { package = "hashbrown", version = "0.12", features hmac = { version = "0.12", default-features = false, features = ["reset"] } hyper = { version = "0.14", features = ["full"] } indexmap = { version = "1", default-features = false, features = ["serde", "std"] } -itertools = { version = "0.10" } +itertools-93f6ce9d446188ac = { package = "itertools", version = "0.10" } +itertools-a6292c17cd707f01 = { package = "itertools", version = "0.11" } jni = { version = "0.21", features = ["invocation"] } lazy_static = { version = "1", default-features = false, features = ["spin_no_std"] } lexical-core = { version = "0.8", features = ["format"] } @@ -157,7 +158,8 @@ either = { version = "1", features = ["serde"] } fixedbitset = { version = "0.4" } frunk_core = { version = "0.4", default-features = false, features = ["std"] } hashbrown-582f2526e08bb6a0 = { package = "hashbrown", version = "0.14", features = ["nightly", "raw"] } -itertools = { version = "0.10" } +itertools-93f6ce9d446188ac = { package = "itertools", version = "0.10" } +itertools-a6292c17cd707f01 = { package = "itertools", version = "0.11" } lazy_static = { version = "1", default-features = false, features = ["spin_no_std"] } libc = { version = "0.2", features = ["extra_traits"] } log = { version = "0.4", default-features = false, features = ["kv_unstable", "std"] } From 88ddc29a08b33b79808dc1f597aeb6fa4afe269a Mon Sep 17 00:00:00 2001 From: hengm3467 <100685635+hengm3467@users.noreply.github.com> Date: Mon, 20 Nov 2023 13:18:35 +0800 Subject: [PATCH 31/36] chore: version update and typo fixes in readme (#13485) --- README.md | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index ab83b8b80a3e9..fc631f9236a14 100644 --- a/README.md +++ b/README.md @@ -77,8 +77,8 @@ risingwave playground ``` **Ubuntu** ``` -wget https://github.com/risingwavelabs/risingwave/releases/download/v1.3.0/risingwave-v1.3.0-x86_64-unknown-linux.tar.gz -tar xvf risingwave-v1.3.0-x86_64-unknown-linux.tar.gz +wget https://github.com/risingwavelabs/risingwave/releases/download/v1.4.0/risingwave-v1.4.0-x86_64-unknown-linux-all-in-one.tar.gz +tar xvf risingwave-v1.4.0-x86_64-unknown-linux-all-in-one.tar.gz ./risingwave playground ``` Now connect to RisingWave using `psql`: @@ -88,6 +88,7 @@ psql -h localhost -p 4566 -d dev -U root Learn more at [Quick Start](https://docs.risingwave.com/docs/current/get-started/). ## Production deployments + For **single-node Docker deployments**, please refer to [Docker Compose](https://docs.risingwave.com/docs/current/risingwave-trial/?method=docker-compose). For **Kubernetes deployments**, please refer to [Kubernetes with Helm](https://docs.risingwave.com/docs/current/risingwave-k8s-helm/) or [Kubernetes with Operator](https://docs.risingwave.com/docs/current/risingwave-kubernetes/). @@ -96,10 +97,13 @@ For **Kubernetes deployments**, please refer to [Kubernetes with Helm](https://d ## Why RisingWave for stream processing? + RisingWave specializes in providing **incrementally updated, consistent materialized views** — a persistent data structure that represents the results of stream processing. RisingWave significantly reduces the complexity of building stream processing applications by allowing developers to express intricate stream processing logic through cascaded materialized views. Furthermore, it allows users to persist data directly within the system, eliminating the need to deliver results to external databases for storage and query serving. -Compared to existing stream processing systems like [Apache Flink](https://flink.apache.org/), [Apache Spark Streaming](https://spark.apache.org/docs/latest/streaming-programming-guide.html), and [KsqlDB](https://ksqldb.io/), RisingWave stands out in two primary dimensions: **Ease-of-use** and **cost efficiency**, thanks to its **[PostgreSQL](https://www.postgresql.org/)-style interaction experience** and **[Snowflake](https://snowflake.com/)-like architectural design** (i.e., decoupled storage and compute). +Compared to existing stream processing systems like [Apache Flink](https://flink.apache.org/), [Apache Spark Streaming](https://spark.apache.org/docs/latest/streaming-programming-guide.html), and [ksqlDB](https://ksqldb.io/), RisingWave stands out in two primary dimensions: **Ease-of-use** and **cost efficiency**, thanks to its **[PostgreSQL](https://www.postgresql.org/)-style interaction experience** and **[Snowflake](https://snowflake.com/)-like architectural design** (i.e., decoupled storage and compute). + ### Ease-of-use + * **Simple to learn** * RisingWave speaks PostgreSQL-style SQL, enabling users to dive into stream processing in much the same way as operating a PostgreSQL database. * **Simple to develop** @@ -108,8 +112,9 @@ Compared to existing stream processing systems like [Apache Flink](https://flink * With integrations to a diverse range of cloud systems and the PostgreSQL ecosystem, RisingWave boasts a rich and expansive ecosystem, making it straightforward to incorporate into existing infrastructures. ### Cost efficiency + * **Highly efficient in complex queries** - * RisingWave persists internal states in remote storages (e.g., S3), and users can confidently and efficiently perform complex streaming queries (e.g., joining dozens of data streams) in a production environment, without worrying about state size. + * RisingWave persists internal states in remote storage systems such as S3, and users can confidently and efficiently perform complex streaming queries (for example, joining dozens of data streams) in a production environment, without worrying about state size. * **Transparent dynamic scaling** * RisingWave's state management mechanism enables near-instantaneous dynamic scaling without any service interruptions. * **Instant failure recovery** From 4972ccd19ca890de3e8c31d25264d416870de895 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Mon, 20 Nov 2023 13:30:46 +0800 Subject: [PATCH 32/36] fix(telemetry): fix memory stats (#13482) --- src/common/src/telemetry/mod.rs | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/common/src/telemetry/mod.rs b/src/common/src/telemetry/mod.rs index 65e314d232aeb..b0974b1a988d1 100644 --- a/src/common/src/telemetry/mod.rs +++ b/src/common/src/telemetry/mod.rs @@ -76,7 +76,6 @@ pub struct SystemData { #[derive(Debug, Serialize, Deserialize)] struct Memory { used: usize, - available: usize, total: usize, } @@ -98,13 +97,9 @@ impl SystemData { let mut sys = System::new(); let memory = { - let available = system_memory_available_bytes(); + let total = system_memory_available_bytes(); let used = total_memory_used_bytes(); - Memory { - available, - used, - total: available + used, - } + Memory { used, total } }; let os = { @@ -172,7 +167,6 @@ mod tests { fn test_system_data_new() { let system_data = SystemData::new(); - assert!(system_data.memory.available > 0); assert!(system_data.memory.used > 0); assert!(system_data.memory.total > 0); assert!(!system_data.os.name.is_empty()); From aed05eba38bda72ec6463f7e69f8607e3a09a6d4 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Mon, 20 Nov 2023 13:58:07 +0800 Subject: [PATCH 33/36] feat(storage): implement hummock iterator for btreemap (#13304) --- src/storage/hummock_sdk/src/key.rs | 7 + src/storage/src/hummock/compactor/iterator.rs | 77 +++--- .../src/hummock/iterator/concat_inner.rs | 97 +++---- .../src/hummock/iterator/forward_merge.rs | 18 +- .../src/hummock/iterator/merge_inner.rs | 173 ++++++------ src/storage/src/hummock/iterator/mod.rs | 252 +++++++++++++----- .../shared_buffer/shared_buffer_batch.rs | 170 ++++++------ .../sstable/backward_sstable_iterator.rs | 75 +++--- .../sstable/forward_sstable_iterator.rs | 73 +++-- .../src/hummock/store/hummock_storage.rs | 6 +- .../hummock/store/local_hummock_storage.rs | 79 ++++-- src/storage/src/hummock/store/version.rs | 49 +++- src/storage/src/mem_table.rs | 204 +++++++++++++- src/storage/src/store.rs | 5 +- 14 files changed, 800 insertions(+), 485 deletions(-) diff --git a/src/storage/hummock_sdk/src/key.rs b/src/storage/hummock_sdk/src/key.rs index bbaa972cadf6d..8c38a2c98e4b4 100644 --- a/src/storage/hummock_sdk/src/key.rs +++ b/src/storage/hummock_sdk/src/key.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::borrow::Borrow; use std::cmp::Ordering; use std::fmt::Debug; use std::ops::Bound::*; @@ -412,6 +413,12 @@ impl> TableKey { } } +impl> Borrow<[u8]> for TableKey { + fn borrow(&self) -> &[u8] { + self.0.as_ref() + } +} + impl EstimateSize for TableKey { fn estimated_heap_size(&self) -> usize { self.0.estimated_heap_size() diff --git a/src/storage/src/hummock/compactor/iterator.rs b/src/storage/src/hummock/compactor/iterator.rs index f925d871a610e..fc624993c8f98 100644 --- a/src/storage/src/hummock/compactor/iterator.rs +++ b/src/storage/src/hummock/compactor/iterator.rs @@ -14,7 +14,6 @@ use std::cmp::Ordering; use std::collections::HashSet; -use std::future::Future; use std::sync::atomic::AtomicU64; use std::sync::{atomic, Arc}; use std::time::Instant; @@ -432,23 +431,17 @@ impl ConcatSstableIterator { impl HummockIterator for ConcatSstableIterator { type Direction = Forward; - type NextFuture<'a> = impl Future> + 'a; - type RewindFuture<'a> = impl Future> + 'a; - type SeekFuture<'a> = impl Future> + 'a; - - fn next(&mut self) -> Self::NextFuture<'_> { - async { - let sstable_iter = self.sstable_iter.as_mut().expect("no table iter"); - - // Does just calling `next()` suffice? - sstable_iter.next().await?; - if sstable_iter.is_valid() { - Ok(()) - } else { - // No, seek to next table. - self.seek_idx(self.cur_idx + 1, None).await?; - Ok(()) - } + async fn next(&mut self) -> HummockResult<()> { + let sstable_iter = self.sstable_iter.as_mut().expect("no table iter"); + + // Does just calling `next()` suffice? + sstable_iter.next().await?; + if sstable_iter.is_valid() { + Ok(()) + } else { + // No, seek to next table. + self.seek_idx(self.cur_idx + 1, None).await?; + Ok(()) } } @@ -464,34 +457,32 @@ impl HummockIterator for ConcatSstableIterator { self.sstable_iter.as_ref().map_or(false, |i| i.is_valid()) } - fn rewind(&mut self) -> Self::RewindFuture<'_> { - async { self.seek_idx(0, None).await } + async fn rewind(&mut self) -> HummockResult<()> { + self.seek_idx(0, None).await } /// Resets the iterator and seeks to the first position where the stored key >= `key`. - fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { - async move { - let seek_key = if self.key_range.left.is_empty() { - key - } else { - match key.cmp(&FullKey::decode(&self.key_range.left)) { - Ordering::Less | Ordering::Equal => FullKey::decode(&self.key_range.left), - Ordering::Greater => key, - } - }; - let table_idx = self.sstables.partition_point(|table| { - // We use the maximum key of an SST for the search. That way, we guarantee that the - // resulting SST contains either that key or the next-larger KV-pair. Subsequently, - // we avoid calling `seek_idx()` twice if the determined SST does not contain `key`. - - // Note that we need to use `<` instead of `<=` to ensure that all keys in an SST - // (including its max. key) produce the same search result. - let max_sst_key = &table.key_range.as_ref().unwrap().right; - FullKey::decode(max_sst_key).cmp(&seek_key) == Ordering::Less - }); - - self.seek_idx(table_idx, Some(key)).await - } + async fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> HummockResult<()> { + let seek_key = if self.key_range.left.is_empty() { + key + } else { + match key.cmp(&FullKey::decode(&self.key_range.left)) { + Ordering::Less | Ordering::Equal => FullKey::decode(&self.key_range.left), + Ordering::Greater => key, + } + }; + let table_idx = self.sstables.partition_point(|table| { + // We use the maximum key of an SST for the search. That way, we guarantee that the + // resulting SST contains either that key or the next-larger KV-pair. Subsequently, + // we avoid calling `seek_idx()` twice if the determined SST does not contain `key`. + + // Note that we need to use `<` instead of `<=` to ensure that all keys in an SST + // (including its max. key) produce the same search result. + let max_sst_key = &table.key_range.as_ref().unwrap().right; + FullKey::decode(max_sst_key).cmp(&seek_key) == Ordering::Less + }); + + self.seek_idx(table_idx, Some(key)).await } fn collect_local_statistic(&self, stats: &mut StoreLocalStatistic) { diff --git a/src/storage/src/hummock/iterator/concat_inner.rs b/src/storage/src/hummock/iterator/concat_inner.rs index 419375eeaa60d..5402f5bfb987f 100644 --- a/src/storage/src/hummock/iterator/concat_inner.rs +++ b/src/storage/src/hummock/iterator/concat_inner.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::cmp::Ordering::{Equal, Greater, Less}; -use std::future::Future; use std::sync::Arc; use risingwave_hummock_sdk::key::FullKey; @@ -108,26 +107,20 @@ impl ConcatIteratorInner { impl HummockIterator for ConcatIteratorInner { type Direction = TI::Direction; - type NextFuture<'a> = impl Future> + 'a; - type RewindFuture<'a> = impl Future> + 'a; - type SeekFuture<'a> = impl Future> + 'a; + async fn next(&mut self) -> HummockResult<()> { + let sstable_iter = self.sstable_iter.as_mut().expect("no table iter"); + sstable_iter.next().await?; - fn next(&mut self) -> Self::NextFuture<'_> { - async move { - let sstable_iter = self.sstable_iter.as_mut().expect("no table iter"); - sstable_iter.next().await?; - - if sstable_iter.is_valid() { - Ok(()) - } else { - // seek to next table - let mut table_idx = self.cur_idx + 1; - while !self.is_valid() && table_idx < self.tables.len() { - self.seek_idx(table_idx, None).await?; - table_idx += 1; - } - Ok(()) + if sstable_iter.is_valid() { + Ok(()) + } else { + // seek to next table + let mut table_idx = self.cur_idx + 1; + while !self.is_valid() && table_idx < self.tables.len() { + self.seek_idx(table_idx, None).await?; + table_idx += 1; } + Ok(()) } } @@ -143,46 +136,42 @@ impl HummockIterator for ConcatIteratorInner { self.sstable_iter.as_ref().map_or(false, |i| i.is_valid()) } - fn rewind(&mut self) -> Self::RewindFuture<'_> { - async move { - self.seek_idx(0, None).await?; - let mut table_idx = 1; - while !self.is_valid() && table_idx < self.tables.len() { - // Seek to next table - self.seek_idx(table_idx, None).await?; - table_idx += 1; - } - Ok(()) + async fn rewind(&mut self) -> HummockResult<()> { + self.seek_idx(0, None).await?; + let mut table_idx = 1; + while !self.is_valid() && table_idx < self.tables.len() { + // Seek to next table + self.seek_idx(table_idx, None).await?; + table_idx += 1; } + Ok(()) } - fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { - async move { - let mut table_idx = self - .tables - .partition_point(|table| match Self::Direction::direction() { - DirectionEnum::Forward => { - let ord = FullKey::decode(smallest_key(table)).cmp(&key); - - ord == Less || ord == Equal - } - DirectionEnum::Backward => { - let ord = FullKey::decode(largest_key(table)).cmp(&key); - ord == Greater - || (ord == Equal && !table.key_range.as_ref().unwrap().right_exclusive) - } - }) - .saturating_sub(1); // considering the boundary of 0 - - self.seek_idx(table_idx, Some(key)).await?; + async fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> HummockResult<()> { + let mut table_idx = self + .tables + .partition_point(|table| match Self::Direction::direction() { + DirectionEnum::Forward => { + let ord = FullKey::decode(smallest_key(table)).cmp(&key); + + ord == Less || ord == Equal + } + DirectionEnum::Backward => { + let ord = FullKey::decode(largest_key(table)).cmp(&key); + ord == Greater + || (ord == Equal && !table.key_range.as_ref().unwrap().right_exclusive) + } + }) + .saturating_sub(1); // considering the boundary of 0 + + self.seek_idx(table_idx, Some(key)).await?; + table_idx += 1; + while !self.is_valid() && table_idx < self.tables.len() { + // Seek to next table + self.seek_idx(table_idx, None).await?; table_idx += 1; - while !self.is_valid() && table_idx < self.tables.len() { - // Seek to next table - self.seek_idx(table_idx, None).await?; - table_idx += 1; - } - Ok(()) } + Ok(()) } fn collect_local_statistic(&self, stats: &mut StoreLocalStatistic) { diff --git a/src/storage/src/hummock/iterator/forward_merge.rs b/src/storage/src/hummock/iterator/forward_merge.rs index 56aa73a078f58..70b5c72f9db28 100644 --- a/src/storage/src/hummock/iterator/forward_merge.rs +++ b/src/storage/src/hummock/iterator/forward_merge.rs @@ -14,7 +14,7 @@ #[cfg(test)] mod test { - use std::future::{pending, poll_fn, Future}; + use std::future::{pending, poll_fn}; use std::iter::once; use std::sync::Arc; use std::task::Poll; @@ -304,12 +304,8 @@ mod test { impl HummockIterator for CancellationTestIterator { type Direction = Forward; - type NextFuture<'a> = impl Future> + 'a; - type RewindFuture<'a> = impl Future> + 'a; - type SeekFuture<'a> = impl Future> + 'a; - - fn next(&mut self) -> Self::NextFuture<'_> { - async { pending::>().await } + async fn next(&mut self) -> HummockResult<()> { + pending::>().await } fn key(&self) -> FullKey<&[u8]> { @@ -330,12 +326,12 @@ mod test { true } - fn rewind(&mut self) -> Self::RewindFuture<'_> { - async { Ok(()) } + async fn rewind(&mut self) -> HummockResult<()> { + Ok(()) } - fn seek<'a>(&'a mut self, _key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { - async { Ok(()) } + async fn seek<'a>(&'a mut self, _key: FullKey<&'a [u8]>) -> HummockResult<()> { + Ok(()) } fn collect_local_statistic(&self, _stats: &mut StoreLocalStatistic) {} diff --git a/src/storage/src/hummock/iterator/merge_inner.rs b/src/storage/src/hummock/iterator/merge_inner.rs index e3c2df4e51067..9865287f0b588 100644 --- a/src/storage/src/hummock/iterator/merge_inner.rs +++ b/src/storage/src/hummock/iterator/merge_inner.rs @@ -211,10 +211,7 @@ where /// The behaviour of `next` of order aware merge iterator is different from the normal one, so we /// extract this trait. trait MergeIteratorNext { - type HummockResultFuture<'a>: Future> + Send + 'a - where - Self: 'a; - fn next_inner(&mut self) -> Self::HummockResultFuture<'_>; + fn next_inner(&mut self) -> impl Future> + Send + '_; } /// This is a wrapper for the `PeekMut` of heap. @@ -286,105 +283,93 @@ impl<'a, T: Ord> Drop for PeekMutGuard<'a, T> { } impl MergeIteratorNext for OrderedMergeIteratorInner { - type HummockResultFuture<'a> = impl Future> + 'a; - - fn next_inner(&mut self) -> Self::HummockResultFuture<'_> { - async { - let top_key = { - let top_key = self.heap.peek().expect("no inner iter").iter.key(); - self.last_table_key.clear(); - self.last_table_key - .extend_from_slice(top_key.user_key.table_key.0); - FullKey { - user_key: UserKey { - table_id: top_key.user_key.table_id, - table_key: TableKey(self.last_table_key.as_slice()), - }, - epoch_with_gap: top_key.epoch_with_gap, - } + async fn next_inner(&mut self) -> HummockResult<()> { + let top_key = { + let top_key = self.heap.peek().expect("no inner iter").iter.key(); + self.last_table_key.clear(); + self.last_table_key + .extend_from_slice(top_key.user_key.table_key.0); + FullKey { + user_key: UserKey { + table_id: top_key.user_key.table_id, + table_key: TableKey(self.last_table_key.as_slice()), + }, + epoch_with_gap: top_key.epoch_with_gap, + } + }; + loop { + let Some(mut node) = PeekMutGuard::peek_mut(&mut self.heap, &mut self.unused_iters) + else { + break; }; - loop { - let Some(mut node) = PeekMutGuard::peek_mut(&mut self.heap, &mut self.unused_iters) - else { - break; + // WARNING: within scope of BinaryHeap::PeekMut, we must carefully handle all places + // of return. Once the iterator enters an invalid state, we should + // remove it from heap before returning. + + if node.iter.key() == top_key { + if let Err(e) = node.iter.next().await { + node.pop(); + self.heap.clear(); + return Err(e); }; - // WARNING: within scope of BinaryHeap::PeekMut, we must carefully handle all places - // of return. Once the iterator enters an invalid state, we should - // remove it from heap before returning. - - if node.iter.key() == top_key { - if let Err(e) = node.iter.next().await { - node.pop(); - self.heap.clear(); - return Err(e); - }; - if !node.iter.is_valid() { - let node = node.pop(); - self.unused_iters.push_back(node); - } else { - node.used(); - } + if !node.iter.is_valid() { + let node = node.pop(); + self.unused_iters.push_back(node); } else { node.used(); - break; } + } else { + node.used(); + break; } - - Ok(()) } + + Ok(()) } } impl MergeIteratorNext for UnorderedMergeIteratorInner { - type HummockResultFuture<'a> = impl Future> + 'a; - - fn next_inner(&mut self) -> Self::HummockResultFuture<'_> { - async { - let mut node = PeekMutGuard::peek_mut(&mut self.heap, &mut self.unused_iters) - .expect("no inner iter"); - - // WARNING: within scope of BinaryHeap::PeekMut, we must carefully handle all places of - // return. Once the iterator enters an invalid state, we should remove it from heap - // before returning. - - match node.iter.next().await { - Ok(_) => {} - Err(e) => { - // If the iterator returns error, we should clear the heap, so that this - // iterator becomes invalid. - node.pop(); - self.heap.clear(); - return Err(e); - } - } - - if !node.iter.is_valid() { - // Put back to `unused_iters` - let node = node.pop(); - self.unused_iters.push_back(node); - } else { - // This will update the heap top. - node.used(); + async fn next_inner(&mut self) -> HummockResult<()> { + let mut node = + PeekMutGuard::peek_mut(&mut self.heap, &mut self.unused_iters).expect("no inner iter"); + + // WARNING: within scope of BinaryHeap::PeekMut, we must carefully handle all places of + // return. Once the iterator enters an invalid state, we should remove it from heap + // before returning. + + match node.iter.next().await { + Ok(_) => {} + Err(e) => { + // If the iterator returns error, we should clear the heap, so that this + // iterator becomes invalid. + node.pop(); + self.heap.clear(); + return Err(e); } + } - Ok(()) + if !node.iter.is_valid() { + // Put back to `unused_iters` + let node = node.pop(); + self.unused_iters.push_back(node); + } else { + // This will update the heap top. + node.used(); } + + Ok(()) } } impl HummockIterator for MergeIteratorInner where - Self: MergeIteratorNext + 'static, + Self: MergeIteratorNext, Node: Ord, { type Direction = I::Direction; - type NextFuture<'a> = impl Future> + 'a; - type RewindFuture<'a> = impl Future> + 'a; - type SeekFuture<'a> = impl Future> + 'a; - - fn next(&mut self) -> Self::NextFuture<'_> { - self.next_inner() + async fn next(&mut self) -> HummockResult<()> { + self.next_inner().await } fn key(&self) -> FullKey<&[u8]> { @@ -399,24 +384,20 @@ where self.heap.peek().map_or(false, |n| n.iter.is_valid()) } - fn rewind(&mut self) -> Self::RewindFuture<'_> { - async move { - self.reset_heap(); - futures::future::try_join_all(self.unused_iters.iter_mut().map(|x| x.iter.rewind())) - .await?; - self.build_heap(); - Ok(()) - } + async fn rewind(&mut self) -> HummockResult<()> { + self.reset_heap(); + futures::future::try_join_all(self.unused_iters.iter_mut().map(|x| x.iter.rewind())) + .await?; + self.build_heap(); + Ok(()) } - fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { - async move { - self.reset_heap(); - futures::future::try_join_all(self.unused_iters.iter_mut().map(|x| x.iter.seek(key))) - .await?; - self.build_heap(); - Ok(()) - } + async fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> HummockResult<()> { + self.reset_heap(); + futures::future::try_join_all(self.unused_iters.iter_mut().map(|x| x.iter.seek(key))) + .await?; + self.build_heap(); + Ok(()) } fn collect_local_statistic(&self, stats: &mut StoreLocalStatistic) { diff --git a/src/storage/src/hummock/iterator/mod.rs b/src/storage/src/hummock/iterator/mod.rs index 2a20039eecba8..b584363b64a43 100644 --- a/src/storage/src/hummock/iterator/mod.rs +++ b/src/storage/src/hummock/iterator/mod.rs @@ -16,6 +16,8 @@ use std::future::Future; use std::marker::PhantomData; use std::ops::{Deref, DerefMut}; +use more_asserts::assert_gt; + use super::{HummockResult, HummockValue}; mod forward_concat; @@ -34,7 +36,7 @@ pub mod forward_user; mod merge_inner; pub use forward_user::*; pub use merge_inner::{OrderedMergeIteratorInner, UnorderedMergeIteratorInner}; -use risingwave_hummock_sdk::key::FullKey; +use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKey}; use crate::hummock::iterator::HummockIteratorUnion::{First, Fourth, Second, Third}; @@ -46,6 +48,8 @@ pub mod test_utils; pub use delete_range_iterator::{ DeleteRangeIterator, ForwardMergeRangeIterator, RangeIteratorTyped, }; +use risingwave_common::catalog::TableId; +use risingwave_hummock_sdk::EpochWithGap; use crate::monitor::StoreLocalStatistic; @@ -55,17 +59,8 @@ use crate::monitor::StoreLocalStatistic; /// After creating the iterator instance, /// - if you want to iterate from the beginning, you need to then call its `rewind` method. /// - if you want to iterate from some specific position, you need to then call its `seek` method. -pub trait HummockIterator: Send + 'static { +pub trait HummockIterator: Send + Sync { type Direction: HummockIteratorDirection; - type NextFuture<'a>: Future> + Send + 'a - where - Self: 'a; - type RewindFuture<'a>: Future> + Send + 'a - where - Self: 'a; - type SeekFuture<'a>: Future> + Send + 'a - where - Self: 'a; /// Moves a valid iterator to the next key. /// /// Note: @@ -77,7 +72,7 @@ pub trait HummockIterator: Send + 'static { /// /// # Panics /// This function will panic if the iterator is invalid. - fn next(&mut self) -> Self::NextFuture<'_>; + fn next(&mut self) -> impl Future> + Send + '_; /// Retrieves the current key. /// @@ -113,7 +108,7 @@ pub trait HummockIterator: Send + 'static { /// - Do not decide whether the position is valid or not by checking the returned error of this /// function. This function WON'T return an `Err` if invalid. You should check `is_valid` /// before starting iteration. - fn rewind(&mut self) -> Self::RewindFuture<'_>; + fn rewind(&mut self) -> impl Future> + Send + '_; /// Resets iterator and seeks to the first position where the key >= provided key, or key <= /// provided key if this is a backward iterator. @@ -122,7 +117,10 @@ pub trait HummockIterator: Send + 'static { /// - Do not decide whether the position is valid or not by checking the returned error of this /// function. This function WON'T return an `Err` if invalid. You should check `is_valid` /// before starting iteration. - fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a>; + fn seek<'a>( + &'a mut self, + key: FullKey<&'a [u8]>, + ) -> impl Future> + Send + '_; /// take local statistic info from iterator to report metrics. fn collect_local_statistic(&self, _stats: &mut StoreLocalStatistic); @@ -136,12 +134,8 @@ pub struct PhantomHummockIterator { impl HummockIterator for PhantomHummockIterator { type Direction = D; - type NextFuture<'a> = impl Future> + 'a; - type RewindFuture<'a> = impl Future> + 'a; - type SeekFuture<'a> = impl Future> + 'a; - - fn next(&mut self) -> Self::NextFuture<'_> { - async { unreachable!() } + async fn next(&mut self) -> HummockResult<()> { + unreachable!() } fn key(&self) -> FullKey<&[u8]> { @@ -156,12 +150,12 @@ impl HummockIterator for PhantomHummockIterator unreachable!() } - fn rewind(&mut self) -> Self::RewindFuture<'_> { - async { unreachable!() } + async fn rewind(&mut self) -> HummockResult<()> { + unreachable!() } - fn seek<'a>(&'a mut self, _key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { - async { unreachable!() } + async fn seek<'a>(&'a mut self, _key: FullKey<&'a [u8]>) -> HummockResult<()> { + unreachable!() } fn collect_local_statistic(&self, _stats: &mut StoreLocalStatistic) {} @@ -202,18 +196,12 @@ impl< { type Direction = D; - type NextFuture<'a> = impl Future> + 'a; - type RewindFuture<'a> = impl Future> + 'a; - type SeekFuture<'a> = impl Future> + 'a; - - fn next(&mut self) -> Self::NextFuture<'_> { - async move { - match self { - First(iter) => iter.next().await, - Second(iter) => iter.next().await, - Third(iter) => iter.next().await, - Fourth(iter) => iter.next().await, - } + async fn next(&mut self) -> HummockResult<()> { + match self { + First(iter) => iter.next().await, + Second(iter) => iter.next().await, + Third(iter) => iter.next().await, + Fourth(iter) => iter.next().await, } } @@ -244,25 +232,21 @@ impl< } } - fn rewind(&mut self) -> Self::RewindFuture<'_> { - async move { - match self { - First(iter) => iter.rewind().await, - Second(iter) => iter.rewind().await, - Third(iter) => iter.rewind().await, - Fourth(iter) => iter.rewind().await, - } + async fn rewind(&mut self) -> HummockResult<()> { + match self { + First(iter) => iter.rewind().await, + Second(iter) => iter.rewind().await, + Third(iter) => iter.rewind().await, + Fourth(iter) => iter.rewind().await, } } - fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { - async move { - match self { - First(iter) => iter.seek(key).await, - Second(iter) => iter.seek(key).await, - Third(iter) => iter.seek(key).await, - Fourth(iter) => iter.seek(key).await, - } + async fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> HummockResult<()> { + match self { + First(iter) => iter.seek(key).await, + Second(iter) => iter.seek(key).await, + Third(iter) => iter.seek(key).await, + Fourth(iter) => iter.seek(key).await, } } @@ -279,12 +263,8 @@ impl< impl HummockIterator for Box { type Direction = I::Direction; - type NextFuture<'a> = impl Future> + 'a; - type RewindFuture<'a> = impl Future> + 'a; - type SeekFuture<'a> = impl Future> + 'a; - - fn next(&mut self) -> Self::NextFuture<'_> { - (*self).deref_mut().next() + async fn next(&mut self) -> HummockResult<()> { + (*self).deref_mut().next().await } fn key(&self) -> FullKey<&[u8]> { @@ -299,12 +279,12 @@ impl HummockIterator for Box { (*self).deref().is_valid() } - fn rewind(&mut self) -> Self::RewindFuture<'_> { - (*self).deref_mut().rewind() + async fn rewind(&mut self) -> HummockResult<()> { + (*self).deref_mut().rewind().await } - fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { - (*self).deref_mut().seek(key) + async fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> HummockResult<()> { + (*self).deref_mut().seek(key).await } fn collect_local_statistic(&self, stats: &mut StoreLocalStatistic) { @@ -312,6 +292,154 @@ impl HummockIterator for Box { } } +pub enum RustIteratorOfBuilder<'a, B: RustIteratorBuilder> { + Seek(B::SeekIter<'a>), + Rewind(B::RewindIter<'a>), +} + +impl<'a, B: RustIteratorBuilder> Iterator for RustIteratorOfBuilder<'a, B> { + type Item = (TableKey<&'a [u8]>, HummockValue<&'a [u8]>); + + fn next(&mut self) -> Option { + match self { + RustIteratorOfBuilder::Seek(i) => i.next(), + RustIteratorOfBuilder::Rewind(i) => i.next(), + } + } +} + +pub trait RustIteratorBuilder: Send + Sync + 'static { + type Iterable: Send + Sync; + type RewindIter<'a>: Iterator, HummockValue<&'a [u8]>)> + + Send + + Sync + + 'a; + type SeekIter<'a>: Iterator, HummockValue<&'a [u8]>)> + + Send + + Sync + + 'a; + + fn seek<'a>(iterable: &'a Self::Iterable, seek_key: TableKey<&[u8]>) -> Self::SeekIter<'a>; + fn rewind(iterable: &Self::Iterable) -> Self::RewindIter<'_>; +} + +pub struct FromRustIterator<'a, B: RustIteratorBuilder> { + inner: &'a B::Iterable, + #[expect(clippy::type_complexity)] + iter: Option<( + RustIteratorOfBuilder<'a, B>, + TableKey<&'a [u8]>, + HummockValue<&'a [u8]>, + )>, + epoch: EpochWithGap, + table_id: TableId, +} + +impl<'a, B: RustIteratorBuilder> FromRustIterator<'a, B> { + pub fn new(inner: &'a B::Iterable, epoch: EpochWithGap, table_id: TableId) -> Self { + Self { + inner, + iter: None, + epoch, + table_id, + } + } +} + +impl<'a, B: RustIteratorBuilder> HummockIterator for FromRustIterator<'a, B> { + type Direction = Forward; + + async fn next(&mut self) -> HummockResult<()> { + let (iter, key, value) = self.iter.as_mut().expect("should be valid"); + if let Some((new_key, new_value)) = iter.next() { + *key = new_key; + *value = new_value; + } else { + self.iter = None; + } + Ok(()) + } + + fn key(&self) -> FullKey<&[u8]> { + let (_, key, _) = self.iter.as_ref().expect("should be valid"); + FullKey { + epoch_with_gap: self.epoch, + user_key: UserKey { + table_id: self.table_id, + table_key: *key, + }, + } + } + + fn value(&self) -> HummockValue<&[u8]> { + let (_, _, value) = self.iter.as_ref().expect("should be valid"); + *value + } + + fn is_valid(&self) -> bool { + self.iter.is_some() + } + + async fn rewind(&mut self) -> HummockResult<()> { + let mut iter = B::rewind(self.inner); + if let Some((key, value)) = iter.next() { + self.iter = Some((RustIteratorOfBuilder::Rewind(iter), key, value)); + } else { + self.iter = None; + } + Ok(()) + } + + async fn seek<'b>(&'b mut self, key: FullKey<&'b [u8]>) -> HummockResult<()> { + if self.table_id < key.user_key.table_id { + // returns None when the range of self.table_id must not include the given key + self.iter = None; + return Ok(()); + } + if self.table_id > key.user_key.table_id { + return self.rewind().await; + } + let mut iter = B::seek(self.inner, key.user_key.table_key); + match iter.next() { + Some((first_key, first_value)) => { + let first_full_key = FullKey { + epoch_with_gap: self.epoch, + user_key: UserKey { + table_id: self.table_id, + table_key: first_key, + }, + }; + if first_full_key < key { + // The semantic of `seek_fn` will ensure that `first_key` >= table_key of `key`. + // At the beginning we have checked that `self.table_id` >= table_id of `key`. + // Therefore, when `first_full_key` < `key`, the only possibility is that + // `first_key` == table_key of `key`, and `self.table_id` == table_id of `key`, + // the `self.epoch` < epoch of `key`. + assert_eq!(first_key, key.user_key.table_key); + match iter.next() { + Some((next_key, next_value)) => { + assert_gt!(next_key, first_key); + self.iter = + Some((RustIteratorOfBuilder::Seek(iter), next_key, next_value)); + } + None => { + self.iter = None; + } + } + } else { + self.iter = Some((RustIteratorOfBuilder::Seek(iter), first_key, first_value)); + } + } + None => { + self.iter = None; + } + } + Ok(()) + } + + fn collect_local_statistic(&self, _stats: &mut StoreLocalStatistic) {} +} + #[derive(PartialEq, Eq, Debug)] pub enum DirectionEnum { Forward, diff --git a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs index 5c517a1155150..a1e3a77f00c6e 100644 --- a/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs +++ b/src/storage/src/hummock/shared_buffer/shared_buffer_batch.rs @@ -703,34 +703,28 @@ impl SharedBufferBatchIterator { impl HummockIterator for SharedBufferBatchIterator { type Direction = D; - type NextFuture<'a> = impl Future> + 'a; - type RewindFuture<'a> = impl Future> + 'a; - type SeekFuture<'a> = impl Future> + 'a; - - fn next(&mut self) -> Self::NextFuture<'_> { - async move { - assert!(self.is_valid()); - match D::direction() { - DirectionEnum::Forward => { - // If the current key has more versions, we need to advance the value index - if self.current_version_idx + 1 < self.current_versions_len() { - self.current_version_idx += 1; - } else { - self.current_idx += 1; - self.current_version_idx = 0; - } + async fn next(&mut self) -> HummockResult<()> { + assert!(self.is_valid()); + match D::direction() { + DirectionEnum::Forward => { + // If the current key has more versions, we need to advance the value index + if self.current_version_idx + 1 < self.current_versions_len() { + self.current_version_idx += 1; + } else { + self.current_idx += 1; + self.current_version_idx = 0; } - DirectionEnum::Backward => { - if self.current_version_idx > 0 { - self.current_version_idx -= 1; - } else { - self.current_idx += 1; - self.current_version_idx = self.current_versions_len() - 1; - } + } + DirectionEnum::Backward => { + if self.current_version_idx > 0 { + self.current_version_idx -= 1; + } else { + self.current_idx += 1; + self.current_version_idx = self.current_versions_len() - 1; } } - Ok(()) } + Ok(()) } fn key(&self) -> FullKey<&[u8]> { @@ -751,91 +745,87 @@ impl HummockIterator for SharedBufferBatchIterator< && self.current_version_idx < self.current_versions().len() as i32 } - fn rewind(&mut self) -> Self::RewindFuture<'_> { - async move { - self.current_idx = 0; + async fn rewind(&mut self) -> HummockResult<()> { + self.current_idx = 0; - match D::direction() { - DirectionEnum::Forward => { - self.current_version_idx = 0; - } - DirectionEnum::Backward => { - self.current_version_idx = self.current_versions_len() - 1; - } + match D::direction() { + DirectionEnum::Forward => { + self.current_version_idx = 0; + } + DirectionEnum::Backward => { + self.current_version_idx = self.current_versions_len() - 1; } - Ok(()) } - } + Ok(()) + } + + async fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> HummockResult<()> { + debug_assert_eq!(key.user_key.table_id, self.table_id); + // Perform binary search on table key because the items in SharedBufferBatch is ordered + // by table key. + let partition_point = self + .inner + .binary_search_by(|probe| probe.0[..].cmp(*key.user_key.table_key)); + let seek_key_epoch = key.epoch_with_gap; + match D::direction() { + DirectionEnum::Forward => match partition_point { + Ok(i) => { + self.current_idx = i; + // seek to the first version that is <= the seek key epoch + let mut idx: i32 = 0; + for (epoch_with_gap, _) in self.current_versions() { + if epoch_with_gap <= &seek_key_epoch { + break; + } + idx += 1; + } - fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { - async move { - debug_assert_eq!(key.user_key.table_id, self.table_id); - // Perform binary search on table key because the items in SharedBufferBatch is ordered - // by table key. - let partition_point = self - .inner - .binary_search_by(|probe| probe.0[..].cmp(*key.user_key.table_key)); - let seek_key_epoch = key.epoch_with_gap; - match D::direction() { - DirectionEnum::Forward => match partition_point { + // Move onto the next key for forward iteration if seek key epoch is smaller + // than all versions + if idx >= self.current_versions().len() as i32 { + self.current_idx += 1; + self.current_version_idx = 0; + } else { + self.current_version_idx = idx; + } + } + Err(i) => { + self.current_idx = i; + self.current_version_idx = 0; + } + }, + DirectionEnum::Backward => { + match partition_point { Ok(i) => { - self.current_idx = i; - // seek to the first version that is <= the seek key epoch - let mut idx: i32 = 0; - for (epoch_with_gap, _) in self.current_versions() { - if epoch_with_gap <= &seek_key_epoch { + self.current_idx = self.inner.len() - i - 1; + // seek from back to the first version that is >= seek_key_epoch + let values = self.current_versions(); + let mut idx: i32 = (values.len() - 1) as i32; + for (epoch_with_gap, _) in values.iter().rev() { + if epoch_with_gap >= &seek_key_epoch { break; } - idx += 1; + idx -= 1; } - // Move onto the next key for forward iteration if seek key epoch is smaller - // than all versions - if idx >= self.current_versions().len() as i32 { + if idx < 0 { self.current_idx += 1; - self.current_version_idx = 0; + self.current_version_idx = self.current_versions_len() - 1; } else { self.current_version_idx = idx; } } + // Seek to one item before the seek partition_point: + // If i == 0, the iterator will be invalidated with self.current_idx == + // self.inner.len(). Err(i) => { - self.current_idx = i; - self.current_version_idx = 0; - } - }, - DirectionEnum::Backward => { - match partition_point { - Ok(i) => { - self.current_idx = self.inner.len() - i - 1; - // seek from back to the first version that is >= seek_key_epoch - let values = self.current_versions(); - let mut idx: i32 = (values.len() - 1) as i32; - for (epoch_with_gap, _) in values.iter().rev() { - if epoch_with_gap >= &seek_key_epoch { - break; - } - idx -= 1; - } - - if idx < 0 { - self.current_idx += 1; - self.current_version_idx = self.current_versions_len() - 1; - } else { - self.current_version_idx = idx; - } - } - // Seek to one item before the seek partition_point: - // If i == 0, the iterator will be invalidated with self.current_idx == - // self.inner.len(). - Err(i) => { - self.current_idx = self.inner.len() - i; - self.current_version_idx = self.current_versions_len() - 1; - } + self.current_idx = self.inner.len() - i; + self.current_version_idx = self.current_versions_len() - 1; } } } - Ok(()) } + Ok(()) } fn collect_local_statistic(&self, _stats: &mut crate::monitor::StoreLocalStatistic) {} diff --git a/src/storage/src/hummock/sstable/backward_sstable_iterator.rs b/src/storage/src/hummock/sstable/backward_sstable_iterator.rs index 36c4e166eae90..cd2de8d234b94 100644 --- a/src/storage/src/hummock/sstable/backward_sstable_iterator.rs +++ b/src/storage/src/hummock/sstable/backward_sstable_iterator.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::cmp::Ordering::{Equal, Less}; -use std::future::Future; use std::sync::Arc; use risingwave_common::cache::CachePriority; @@ -90,20 +89,14 @@ impl BackwardSstableIterator { impl HummockIterator for BackwardSstableIterator { type Direction = Backward; - type NextFuture<'a> = impl Future> + 'a; - type RewindFuture<'a> = impl Future> + 'a; - type SeekFuture<'a> = impl Future> + 'a; - - fn next(&mut self) -> Self::NextFuture<'_> { + async fn next(&mut self) -> HummockResult<()> { self.stats.total_key_count += 1; - async move { - let block_iter = self.block_iter.as_mut().expect("no block iter"); - if block_iter.try_prev() { - Ok(()) - } else { - // seek to the previous block - self.seek_idx(self.cur_idx as isize - 1, None).await - } + let block_iter = self.block_iter.as_mut().expect("no block iter"); + if block_iter.try_prev() { + Ok(()) + } else { + // seek to the previous block + self.seek_idx(self.cur_idx as isize - 1, None).await } } @@ -123,38 +116,34 @@ impl HummockIterator for BackwardSstableIterator { /// Instead of setting idx to 0th block, a `BackwardSstableIterator` rewinds to the last block /// in the sstable. - fn rewind(&mut self) -> Self::RewindFuture<'_> { - async move { - self.seek_idx(self.sst.value().block_count() as isize - 1, None) - .await - } + async fn rewind(&mut self) -> HummockResult<()> { + self.seek_idx(self.sst.value().block_count() as isize - 1, None) + .await } - fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { - async move { - let block_idx = self - .sst - .value() - .meta - .block_metas - .partition_point(|block_meta| { - // Compare by version comparator - // Note: we are comparing against the `smallest_key` of the `block`, thus the - // partition point should be `prev(<=)` instead of `<`. - let ord = FullKey::decode(&block_meta.smallest_key).cmp(&key); - ord == Less || ord == Equal - }) - .saturating_sub(1); // considering the boundary of 0 - let block_idx = block_idx as isize; - - self.seek_idx(block_idx, Some(key)).await?; - if !self.is_valid() { - // Seek to prev block - self.seek_idx(block_idx - 1, None).await?; - } - - Ok(()) + async fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> HummockResult<()> { + let block_idx = self + .sst + .value() + .meta + .block_metas + .partition_point(|block_meta| { + // Compare by version comparator + // Note: we are comparing against the `smallest_key` of the `block`, thus the + // partition point should be `prev(<=)` instead of `<`. + let ord = FullKey::decode(&block_meta.smallest_key).cmp(&key); + ord == Less || ord == Equal + }) + .saturating_sub(1); // considering the boundary of 0 + let block_idx = block_idx as isize; + + self.seek_idx(block_idx, Some(key)).await?; + if !self.is_valid() { + // Seek to prev block + self.seek_idx(block_idx - 1, None).await?; } + + Ok(()) } fn collect_local_statistic(&self, stats: &mut StoreLocalStatistic) { diff --git a/src/storage/src/hummock/sstable/forward_sstable_iterator.rs b/src/storage/src/hummock/sstable/forward_sstable_iterator.rs index f7639cddbda57..08d6ad0ee2881 100644 --- a/src/storage/src/hummock/sstable/forward_sstable_iterator.rs +++ b/src/storage/src/hummock/sstable/forward_sstable_iterator.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::cmp::Ordering::{Equal, Less}; -use std::future::Future; use std::ops::Bound::*; use std::sync::Arc; @@ -216,20 +215,14 @@ impl SstableIterator { impl HummockIterator for SstableIterator { type Direction = Forward; - type NextFuture<'a> = impl Future> + 'a; - type RewindFuture<'a> = impl Future> + 'a; - type SeekFuture<'a> = impl Future> + 'a; - - fn next(&mut self) -> Self::NextFuture<'_> { + async fn next(&mut self) -> HummockResult<()> { self.stats.total_key_count += 1; - async move { - let block_iter = self.block_iter.as_mut().expect("no block iter"); - if !block_iter.try_next() { - // seek to next block - self.seek_idx(self.cur_idx + 1, None).await?; - } - Ok(()) + let block_iter = self.block_iter.as_mut().expect("no block iter"); + if !block_iter.try_next() { + // seek to next block + self.seek_idx(self.cur_idx + 1, None).await?; } + Ok(()) } fn key(&self) -> FullKey<&[u8]> { @@ -246,38 +239,34 @@ impl HummockIterator for SstableIterator { self.block_iter.as_ref().map_or(false, |i| i.is_valid()) } - fn rewind(&mut self) -> Self::RewindFuture<'_> { - async move { - self.init_block_prefetch_range(0); - self.seek_idx(0, None).await?; - Ok(()) - } + async fn rewind(&mut self) -> HummockResult<()> { + self.init_block_prefetch_range(0); + self.seek_idx(0, None).await?; + Ok(()) } - fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> Self::SeekFuture<'a> { - async move { - let block_idx = self - .sst - .value() - .meta - .block_metas - .partition_point(|block_meta| { - // compare by version comparator - // Note: we are comparing against the `smallest_key` of the `block`, thus the - // partition point should be `prev(<=)` instead of `<`. - let ord = FullKey::decode(&block_meta.smallest_key).cmp(&key); - ord == Less || ord == Equal - }) - .saturating_sub(1); // considering the boundary of 0 - self.init_block_prefetch_range(block_idx); - - self.seek_idx(block_idx, Some(key)).await?; - if !self.is_valid() { - // seek to next block - self.seek_idx(block_idx + 1, None).await?; - } - Ok(()) + async fn seek<'a>(&'a mut self, key: FullKey<&'a [u8]>) -> HummockResult<()> { + let block_idx = self + .sst + .value() + .meta + .block_metas + .partition_point(|block_meta| { + // compare by version comparator + // Note: we are comparing against the `smallest_key` of the `block`, thus the + // partition point should be `prev(<=)` instead of `<`. + let ord = FullKey::decode(&block_meta.smallest_key).cmp(&key); + ord == Less || ord == Equal + }) + .saturating_sub(1); // considering the boundary of 0 + self.init_block_prefetch_range(block_idx); + + self.seek_idx(block_idx, Some(key)).await?; + if !self.is_valid() { + // seek to next block + self.seek_idx(block_idx + 1, None).await?; } + Ok(()) } fn collect_local_statistic(&self, stats: &mut StoreLocalStatistic) { diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index f58a37c1f7488..c018ce167bff9 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -35,7 +35,7 @@ use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; use tokio::sync::oneshot; use tracing::log::error; -use super::local_hummock_storage::{HummockStorageIterator, LocalHummockStorage}; +use super::local_hummock_storage::LocalHummockStorage; use super::version::{CommittedVersion, HummockVersionReader}; use crate::error::StorageResult; use crate::filter_key_extractor::{FilterKeyExtractorManager, RpcFilterKeyExtractorManager}; @@ -50,8 +50,8 @@ use crate::hummock::store::version::read_filter_for_batch; use crate::hummock::utils::{validate_safe_epoch, wait_for_epoch}; use crate::hummock::write_limiter::{WriteLimiter, WriteLimiterRef}; use crate::hummock::{ - HummockEpoch, HummockError, HummockResult, MemoryLimiter, SstableObjectIdManager, - SstableObjectIdManagerRef, SstableStoreRef, + HummockEpoch, HummockError, HummockResult, HummockStorageIterator, MemoryLimiter, + SstableObjectIdManager, SstableObjectIdManagerRef, SstableStoreRef, }; use crate::mem_table::ImmutableMemtable; use crate::monitor::{CompactorMetrics, HummockStateStoreMetrics, StoreLocalStatistic}; diff --git a/src/storage/src/hummock/store/local_hummock_storage.rs b/src/storage/src/hummock/store/local_hummock_storage.rs index db4fb7b9d8930..26730dcaeac5b 100644 --- a/src/storage/src/hummock/store/local_hummock_storage.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -23,7 +23,7 @@ use prometheus::IntGauge; use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::util::epoch::{MAX_EPOCH, MAX_SPILL_TIMES}; use risingwave_hummock_sdk::key::{TableKey, TableKeyRange}; -use risingwave_hummock_sdk::HummockEpoch; +use risingwave_hummock_sdk::{EpochWithGap, HummockEpoch}; use tokio::sync::mpsc; use tracing::{warn, Instrument}; @@ -44,11 +44,10 @@ use crate::hummock::utils::{ }; use crate::hummock::write_limiter::WriteLimiterRef; use crate::hummock::{MemoryLimiter, SstableIterator}; -use crate::mem_table::{merge_stream, KeyOp, MemTable}; +use crate::mem_table::{KeyOp, MemTable, MemTableHummockIterator}; use crate::monitor::{HummockStateStoreMetrics, IterLocalMetricsGuard, StoreLocalStatistic}; use crate::storage_value::StorageValue; use crate::store::*; -use crate::StateStoreIter; /// `LocalHummockStorage` is a handle for a state table shard to access data from and write data to /// the hummock state backend. It is created via `HummockStorage::new_local`. @@ -133,7 +132,7 @@ impl LocalHummockStorage { wait_for_epoch(&self.version_update_notifier_tx, wait_epoch).await } - pub async fn iter_inner( + pub async fn iter_flushed( &self, table_key_range: TableKeyRange, epoch: u64, @@ -151,6 +150,38 @@ impl LocalHummockStorage { .await } + fn mem_table_iter(&self) -> MemTableHummockIterator<'_> { + MemTableHummockIterator::new( + &self.mem_table.buffer, + EpochWithGap::new(self.epoch(), self.spill_offset), + self.table_id, + ) + } + + pub async fn iter_all( + &self, + table_key_range: TableKeyRange, + epoch: u64, + read_options: ReadOptions, + ) -> StorageResult>> { + let read_snapshot = read_filter_for_local( + epoch, + read_options.table_id, + &table_key_range, + self.read_version.clone(), + )?; + + self.hummock_version_reader + .iter_with_memtable( + table_key_range, + epoch, + read_options, + read_snapshot, + self.mem_table_iter(), + ) + .await + } + pub async fn may_exist_inner( &self, key_range: TableKeyRange, @@ -193,13 +224,13 @@ impl StateStoreRead for LocalHummockStorage { read_options: ReadOptions, ) -> impl Future> + '_ { assert!(epoch <= self.epoch()); - self.iter_inner(key_range, epoch, read_options) + self.iter_flushed(key_range, epoch, read_options) .instrument(tracing::trace_span!("hummock_iter")) } } impl LocalStateStore for LocalHummockStorage { - type IterStream<'a> = impl StateStoreIterItemStream + 'a; + type IterStream<'a> = StreamTypeOfIter>; fn may_exist( &self, @@ -223,23 +254,13 @@ impl LocalStateStore for LocalHummockStorage { } } - #[allow(clippy::manual_async_fn)] - fn iter( + async fn iter( &self, key_range: TableKeyRange, read_options: ReadOptions, - ) -> impl Future>> + Send + '_ { - async move { - let stream = self - .iter_inner(key_range.clone(), self.epoch(), read_options) - .await?; - Ok(merge_stream( - self.mem_table.iter(key_range), - stream, - self.table_id, - self.epoch(), - )) - } + ) -> StorageResult> { + self.iter_all(key_range.clone(), self.epoch(), read_options) + .await } fn insert( @@ -552,21 +573,25 @@ impl LocalHummockStorage { pub type StagingDataIterator = OrderedMergeIteratorInner< HummockIteratorUnion, SstableIterator>, >; -type HummockStorageIteratorPayload = UnorderedMergeIteratorInner< +pub type HummockStorageIteratorPayloadInner<'a> = UnorderedMergeIteratorInner< HummockIteratorUnion< Forward, StagingDataIterator, SstableIterator, ConcatIteratorInner, + MemTableHummockIterator<'a>, >, >; -pub struct HummockStorageIterator { - inner: UserIterator, +pub type HummockStorageIterator = HummockStorageIteratorInner<'static>; +pub type LocalHummockStorageIterator<'a> = HummockStorageIteratorInner<'a>; + +pub struct HummockStorageIteratorInner<'a> { + inner: UserIterator>, stats_guard: IterLocalMetricsGuard, } -impl StateStoreIter for HummockStorageIterator { +impl<'a> StateStoreIter for HummockStorageIteratorInner<'a> { type Item = StateStoreIterItem; async fn next(&mut self) -> StorageResult> { @@ -582,9 +607,9 @@ impl StateStoreIter for HummockStorageIterator { } } -impl HummockStorageIterator { +impl<'a> HummockStorageIteratorInner<'a> { pub fn new( - inner: UserIterator, + inner: UserIterator>, metrics: Arc, table_id: TableId, local_stats: StoreLocalStatistic, @@ -596,7 +621,7 @@ impl HummockStorageIterator { } } -impl Drop for HummockStorageIterator { +impl<'a> Drop for HummockStorageIteratorInner<'a> { fn drop(&mut self) { self.inner .collect_local_statistic(&mut self.stats_guard.local_stats); diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 0d6a2f7a17a13..3a3ca91e9f6ab 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -41,16 +41,16 @@ use crate::hummock::iterator::{ use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::sstable::SstableIteratorReadOptions; use crate::hummock::sstable_store::SstableStoreRef; -use crate::hummock::store::HummockStorageIterator; use crate::hummock::utils::{ check_subset_preserve_order, filter_single_sst, prune_nonoverlapping_ssts, prune_overlapping_ssts, range_overlap, search_sst_idx, }; use crate::hummock::{ - get_from_batch, get_from_sstable_info, hit_sstable_bloom_filter, Sstable, - SstableDeleteRangeIterator, SstableIterator, + get_from_batch, get_from_sstable_info, hit_sstable_bloom_filter, HummockStorageIterator, + HummockStorageIteratorInner, LocalHummockStorageIterator, Sstable, SstableDeleteRangeIterator, + SstableIterator, }; -use crate::mem_table::{ImmId, ImmutableMemtable}; +use crate::mem_table::{ImmId, ImmutableMemtable, MemTableHummockIterator}; use crate::monitor::{ GetLocalMetricsGuard, HummockStateStoreMetrics, MayExistLocalMetricsGuard, StoreLocalStatistic, }; @@ -691,6 +691,42 @@ impl HummockVersionReader { read_options: ReadOptions, read_version_tuple: (Vec, Vec, CommittedVersion), ) -> StorageResult> { + self.iter_inner( + table_key_range, + epoch, + read_options, + read_version_tuple, + None, + ) + .await + } + + pub async fn iter_with_memtable<'a>( + &'a self, + table_key_range: TableKeyRange, + epoch: u64, + read_options: ReadOptions, + read_version_tuple: (Vec, Vec, CommittedVersion), + memtable_iter: MemTableHummockIterator<'a>, + ) -> StorageResult>> { + self.iter_inner( + table_key_range, + epoch, + read_options, + read_version_tuple, + Some(memtable_iter), + ) + .await + } + + pub async fn iter_inner<'a, 'b>( + &'a self, + table_key_range: TableKeyRange, + epoch: u64, + read_options: ReadOptions, + read_version_tuple: (Vec, Vec, CommittedVersion), + mem_table: Option>, + ) -> StorageResult>> { let table_id_string = read_options.table_id.to_string(); let table_id_label = table_id_string.as_str(); let (imms, uncommitted_ssts, committed) = read_version_tuple; @@ -908,7 +944,8 @@ impl HummockVersionReader { non_overlapping_iters .into_iter() .map(HummockIteratorUnion::Third), - ), + ) + .chain(mem_table.into_iter().map(HummockIteratorUnion::Fourth)), ); let user_key_range = ( @@ -936,7 +973,7 @@ impl HummockVersionReader { + local_stats.overlapping_iter_count + local_stats.non_overlapping_iter_count; - Ok(HummockStorageIterator::new( + Ok(HummockStorageIteratorInner::new( user_iter, self.state_store_metrics.clone(), read_options.table_id, diff --git a/src/storage/src/mem_table.rs b/src/storage/src/mem_table.rs index 52c8f0b0c200d..79a7fb50f381b 100644 --- a/src/storage/src/mem_table.rs +++ b/src/storage/src/mem_table.rs @@ -16,6 +16,7 @@ use std::cmp::Ordering; use std::collections::btree_map::Entry; use std::collections::BTreeMap; use std::future::Future; +use std::ops::Bound::{Included, Unbounded}; use std::ops::{Bound, RangeBounds}; use bytes::Bytes; @@ -27,11 +28,13 @@ use risingwave_hummock_sdk::key::{FullKey, TableKey, TableKeyRange}; use thiserror::Error; use crate::error::{StorageError, StorageResult}; +use crate::hummock::iterator::{FromRustIterator, RustIteratorBuilder}; use crate::hummock::shared_buffer::shared_buffer_batch::{SharedBufferBatch, SharedBufferBatchId}; use crate::hummock::utils::{ cmp_delete_range_left_bounds, do_delete_sanity_check, do_insert_sanity_check, do_update_sanity_check, filter_with_delete_range, ENABLE_SANITY_CHECK, }; +use crate::hummock::value::HummockValue; use crate::row_serde::value_serde::ValueRowSerde; use crate::storage_value::StorageValue; use crate::store::*; @@ -50,7 +53,7 @@ pub enum KeyOp { /// `MemTable` is a buffer for modify operations without encoding #[derive(Clone)] pub struct MemTable { - pub(crate) buffer: BTreeMap, KeyOp>, + pub(crate) buffer: MemTableStore, pub(crate) is_consistent_op: bool, pub(crate) kv_size: KvSize, } @@ -67,6 +70,42 @@ pub enum MemTableError { type Result = std::result::Result>; +pub type MemTableStore = BTreeMap, KeyOp>; +pub struct MemTableIteratorBuilder; + +fn map_to_hummock_value<'a>( + (key, op): (&'a TableKey, &'a KeyOp), +) -> (TableKey<&'a [u8]>, HummockValue<&'a [u8]>) { + ( + TableKey(key.0.as_ref()), + match op { + KeyOp::Insert(value) | KeyOp::Update((_, value)) => HummockValue::Put(value), + KeyOp::Delete(_) => HummockValue::Delete, + }, + ) +} + +impl RustIteratorBuilder for MemTableIteratorBuilder { + type Iterable = MemTableStore; + + type RewindIter<'a> = + impl Iterator, HummockValue<&'a [u8]>)> + Send + 'a; + type SeekIter<'a> = + impl Iterator, HummockValue<&'a [u8]>)> + Send + 'a; + + fn seek<'a>(iterable: &'a Self::Iterable, seek_key: TableKey<&[u8]>) -> Self::SeekIter<'a> { + iterable + .range::<[u8], _>((Included(seek_key.0), Unbounded)) + .map(map_to_hummock_value) + } + + fn rewind(iterable: &Self::Iterable) -> Self::RewindIter<'_> { + iterable.iter().map(map_to_hummock_value) + } +} + +pub type MemTableHummockIterator<'a> = FromRustIterator<'a, MemTableIteratorBuilder>; + impl MemTable { pub fn new(is_consistent_op: bool) -> Self { Self { @@ -582,10 +621,18 @@ impl LocalStateStore for MemtableLocalState #[cfg(test)] mod tests { - use bytes::Bytes; - use risingwave_hummock_sdk::key::TableKey; - - use crate::mem_table::{KeyOp, MemTable}; + use bytes::{BufMut, Bytes, BytesMut}; + use itertools::Itertools; + use rand::seq::SliceRandom; + use rand::{thread_rng, Rng}; + use risingwave_common::catalog::TableId; + use risingwave_common::hash::VirtualNode; + use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKey}; + use risingwave_hummock_sdk::EpochWithGap; + + use crate::hummock::iterator::HummockIterator; + use crate::hummock::value::HummockValue; + use crate::mem_table::{KeyOp, MemTable, MemTableHummockIterator}; #[tokio::test] async fn test_mem_table_memory_size() { @@ -755,4 +802,151 @@ mod tests { + Bytes::from("value4444").len() ); } + + #[tokio::test] + async fn test_mem_table_hummock_iterator() { + let mut rng = thread_rng(); + + fn get_key(i: usize) -> TableKey { + let mut bytes = BytesMut::new(); + bytes.put(&VirtualNode::ZERO.to_be_bytes()[..]); + bytes.put(format!("key_{:20}", i).as_bytes()); + TableKey(bytes.freeze()) + } + + let ordered_test_data = (0..10000) + .map(|i| { + let key_op = match rng.gen::() % 3 { + 0 => KeyOp::Insert(Bytes::from("insert")), + 1 => KeyOp::Delete(Bytes::from("delete")), + 2 => KeyOp::Update((Bytes::from("old_value"), Bytes::from("new_value"))), + _ => unreachable!(), + }; + (get_key(i), key_op) + }) + .collect_vec(); + + let mut test_data = ordered_test_data.clone(); + + test_data.shuffle(&mut rng); + let mut mem_table = MemTable::new(true); + for (key, op) in test_data { + match op { + KeyOp::Insert(value) => { + mem_table.insert(key, value).unwrap(); + } + KeyOp::Delete(value) => mem_table.delete(key, value).unwrap(), + KeyOp::Update((old_value, new_value)) => { + mem_table.update(key, old_value, new_value).unwrap(); + } + } + } + + const TEST_TABLE_ID: TableId = TableId::new(233); + const TEST_EPOCH: u64 = 10; + + async fn check_data( + iter: &mut MemTableHummockIterator<'_>, + test_data: &[(TableKey, KeyOp)], + ) { + let mut idx = 0; + while iter.is_valid() { + let key = iter.key(); + let value = iter.value(); + + let (expected_key, expected_value) = test_data[idx].clone(); + assert_eq!(key.epoch_with_gap, EpochWithGap::new_from_epoch(TEST_EPOCH)); + assert_eq!(key.user_key.table_id, TEST_TABLE_ID); + assert_eq!(key.user_key.table_key.0, expected_key.0.as_ref()); + match expected_value { + KeyOp::Insert(expected_value) | KeyOp::Update((_, expected_value)) => { + assert_eq!(value, HummockValue::Put(expected_value.as_ref())); + } + KeyOp::Delete(_) => { + assert_eq!(value, HummockValue::Delete); + } + } + + idx += 1; + iter.next().await.unwrap(); + } + assert_eq!(idx, test_data.len()); + } + + let mut iter = MemTableHummockIterator::new( + &mem_table.buffer, + EpochWithGap::new_from_epoch(TEST_EPOCH), + TEST_TABLE_ID, + ); + + // Test rewind + iter.rewind().await.unwrap(); + check_data(&mut iter, &ordered_test_data).await; + + // Test seek with a later epoch, the first key is not skipped + let later_epoch = EpochWithGap::new_from_epoch(TEST_EPOCH + 1); + let seek_idx = 500; + iter.seek(FullKey { + user_key: UserKey { + table_id: TEST_TABLE_ID, + table_key: TableKey(&get_key(seek_idx)), + }, + epoch_with_gap: later_epoch, + }) + .await + .unwrap(); + check_data(&mut iter, &ordered_test_data[seek_idx..]).await; + + // Test seek with a earlier epoch, the first key is skipped + let early_epoch = EpochWithGap::new_from_epoch(TEST_EPOCH - 1); + let seek_idx = 500; + iter.seek(FullKey { + user_key: UserKey { + table_id: TEST_TABLE_ID, + table_key: TableKey(&get_key(seek_idx)), + }, + epoch_with_gap: early_epoch, + }) + .await + .unwrap(); + check_data(&mut iter, &ordered_test_data[seek_idx + 1..]).await; + + // Test seek to over the end + iter.seek(FullKey { + user_key: UserKey { + table_id: TEST_TABLE_ID, + table_key: TableKey(&get_key(ordered_test_data.len() + 10)), + }, + epoch_with_gap: EpochWithGap::new_from_epoch(TEST_EPOCH), + }) + .await + .unwrap(); + check_data(&mut iter, &[]).await; + + // Test seek with a smaller table id + let smaller_table_id = TableId::new(TEST_TABLE_ID.table_id() - 1); + iter.seek(FullKey { + user_key: UserKey { + table_id: smaller_table_id, + table_key: TableKey(&get_key(ordered_test_data.len() + 10)), + }, + epoch_with_gap: EpochWithGap::new_from_epoch(TEST_EPOCH), + }) + .await + .unwrap(); + check_data(&mut iter, &ordered_test_data).await; + + // Test seek with a greater table id + let greater_table_id = TableId::new(TEST_TABLE_ID.table_id() + 1); + iter.seek(FullKey { + user_key: UserKey { + table_id: greater_table_id, + table_key: TableKey(&get_key(0)), + }, + epoch_with_gap: EpochWithGap::new_from_epoch(TEST_EPOCH), + }) + .await + .unwrap(); + check_data(&mut iter, &[]).await; + } } diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index b2cc8062f0579..004ff7937ea92 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -36,15 +36,14 @@ use crate::storage_value::StorageValue; pub trait StaticSendSync = Send + Sync + 'static; -pub trait StateStoreIter: StaticSendSync { +pub trait StateStoreIter: Send + Sync { type Item: Send; fn next(&mut self) -> impl Future>> + Send + '_; } -pub trait StateStoreIterStreamTrait = Stream> + Send + 'static; pub trait StateStoreIterExt: StateStoreIter { - type ItemStream: StateStoreIterStreamTrait<::Item>; + type ItemStream: Stream::Item>> + Send; fn into_stream(self) -> Self::ItemStream; } From 2e13b1e681340ec3148c84c0290dd1c3ba50cbbe Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Mon, 20 Nov 2023 14:17:43 +0800 Subject: [PATCH 34/36] refactor(storage): use easy_ext for HummockVersionExt (#13507) --- Cargo.lock | 1 + src/storage/hummock_sdk/Cargo.toml | 1 + .../compaction_group/hummock_version_ext.rs | 107 +++++------------- 3 files changed, 29 insertions(+), 80 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 065950598061d..a9e3e13f69b43 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7907,6 +7907,7 @@ name = "risingwave_hummock_sdk" version = "1.3.0-alpha" dependencies = [ "bytes", + "easy-ext", "hex", "itertools 0.12.0", "parse-display", diff --git a/src/storage/hummock_sdk/Cargo.toml b/src/storage/hummock_sdk/Cargo.toml index 2bc06424280e9..10752894f4cc7 100644 --- a/src/storage/hummock_sdk/Cargo.toml +++ b/src/storage/hummock_sdk/Cargo.toml @@ -15,6 +15,7 @@ normal = ["workspace-hack"] [dependencies] bytes = "1" +easy-ext = "1" hex = "0.4" itertools = "0.12" parse-display = "0.8" diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index 3e4286eb856bc..681fef7a896c7 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -118,62 +118,17 @@ pub struct SstDeltaInfo { pub delete_sst_object_ids: Vec, } -pub trait HummockVersionExt { - type CombinedLevelsIter<'a>: Iterator + 'a - where - Self: 'a; - - /// Gets `compaction_group_id`'s levels - fn get_compaction_group_levels(&self, compaction_group_id: CompactionGroupId) -> &Levels; - /// Gets `compaction_group_id`'s levels - fn get_compaction_group_levels_mut( - &mut self, - compaction_group_id: CompactionGroupId, - ) -> &mut Levels; - /// Gets all levels. - /// - /// Levels belonging to the same compaction group retain their relative order. - fn get_combined_levels(&self) -> Self::CombinedLevelsIter<'_>; - fn num_levels(&self, compaction_group_id: CompactionGroupId) -> usize; - fn level_iter bool>(&self, compaction_group_id: CompactionGroupId, f: F); - - fn get_object_ids(&self) -> Vec; -} - pub type BranchedSstInfo = HashMap; -pub trait HummockVersionUpdateExt { - fn count_new_ssts_in_group_split( - &self, - parent_group_id: CompactionGroupId, - member_table_ids: HashSet, - ) -> u64; - fn init_with_parent_group( - &mut self, - parent_group_id: CompactionGroupId, - group_id: CompactionGroupId, - member_table_ids: HashSet, - new_sst_start_id: u64, - allow_trivial_split: bool, - ) -> Vec; - - fn apply_version_delta(&mut self, version_delta: &HummockVersionDelta) -> Vec; - - fn build_compaction_group_info(&self) -> HashMap; - fn build_branched_sst_info(&self) -> BTreeMap; - fn build_sst_delta_infos(&self, version_delta: &HummockVersionDelta) -> Vec; -} - -impl HummockVersionExt for HummockVersion { - type CombinedLevelsIter<'a> = impl Iterator + 'a; - - fn get_compaction_group_levels(&self, compaction_group_id: CompactionGroupId) -> &Levels { +#[easy_ext::ext(HummockVersionExt)] +impl HummockVersion { + pub fn get_compaction_group_levels(&self, compaction_group_id: CompactionGroupId) -> &Levels { self.levels .get(&compaction_group_id) .unwrap_or_else(|| panic!("compaction group {} does not exist", compaction_group_id)) } - fn get_compaction_group_levels_mut( + pub fn get_compaction_group_levels_mut( &mut self, compaction_group_id: CompactionGroupId, ) -> &mut Levels { @@ -182,7 +137,7 @@ impl HummockVersionExt for HummockVersion { .unwrap_or_else(|| panic!("compaction group {} does not exist", compaction_group_id)) } - fn get_combined_levels(&self) -> Self::CombinedLevelsIter<'_> { + pub fn get_combined_levels(&self) -> impl Iterator + '_ { self.levels.values().flat_map(|level| { level .l0 @@ -196,7 +151,7 @@ impl HummockVersionExt for HummockVersion { } /// This function does NOT dedup. - fn get_object_ids(&self) -> Vec { + pub fn get_object_ids(&self) -> Vec { self.get_combined_levels() .flat_map(|level| { level @@ -207,7 +162,7 @@ impl HummockVersionExt for HummockVersion { .collect_vec() } - fn level_iter bool>( + pub fn level_iter bool>( &self, compaction_group_id: CompactionGroupId, mut f: F, @@ -226,7 +181,7 @@ impl HummockVersionExt for HummockVersion { } } - fn num_levels(&self, compaction_group_id: CompactionGroupId) -> usize { + pub fn num_levels(&self, compaction_group_id: CompactionGroupId) -> usize { // l0 is currently separated from all levels self.levels .get(&compaction_group_id) @@ -246,8 +201,9 @@ pub type SstSplitInfo = ( HummockSstableId, ); -impl HummockVersionUpdateExt for HummockVersion { - fn count_new_ssts_in_group_split( +#[easy_ext::ext(HummockVersionUpdateExt)] +impl HummockVersion { + pub fn count_new_ssts_in_group_split( &self, parent_group_id: CompactionGroupId, member_table_ids: HashSet, @@ -274,7 +230,7 @@ impl HummockVersionUpdateExt for HummockVersion { }) } - fn init_with_parent_group( + pub fn init_with_parent_group( &mut self, parent_group_id: CompactionGroupId, group_id: CompactionGroupId, @@ -386,7 +342,7 @@ impl HummockVersionUpdateExt for HummockVersion { split_id_vers } - fn build_sst_delta_infos(&self, version_delta: &HummockVersionDelta) -> Vec { + pub fn build_sst_delta_infos(&self, version_delta: &HummockVersionDelta) -> Vec { let mut infos = vec![]; for (group_id, group_deltas) in &version_delta.group_deltas { @@ -467,7 +423,10 @@ impl HummockVersionUpdateExt for HummockVersion { infos } - fn apply_version_delta(&mut self, version_delta: &HummockVersionDelta) -> Vec { + pub fn apply_version_delta( + &mut self, + version_delta: &HummockVersionDelta, + ) -> Vec { let mut sst_split_info = vec![]; for (compaction_group_id, group_deltas) in &version_delta.group_deltas { let summary = summarize_group_deltas(group_deltas); @@ -575,7 +534,7 @@ impl HummockVersionUpdateExt for HummockVersion { sst_split_info } - fn build_compaction_group_info(&self) -> HashMap { + pub fn build_compaction_group_info(&self) -> HashMap { let mut ret = HashMap::new(); for (compaction_group_id, levels) in &self.levels { for table_id in &levels.member_table_ids { @@ -585,7 +544,7 @@ impl HummockVersionUpdateExt for HummockVersion { ret } - fn build_branched_sst_info(&self) -> BTreeMap { + pub fn build_branched_sst_info(&self) -> BTreeMap { let mut ret: BTreeMap<_, _> = BTreeMap::new(); for (compaction_group_id, group) in &self.levels { let mut levels = vec![]; @@ -609,33 +568,21 @@ impl HummockVersionUpdateExt for HummockVersion { } } -pub trait HummockLevelsExt { - fn get_level0(&self) -> &OverlappingLevel; - fn get_level(&self, idx: usize) -> &Level; - fn get_level_mut(&mut self, idx: usize) -> &mut Level; - fn count_ssts(&self) -> usize; - fn apply_compact_ssts(&mut self, summary: GroupDeltasSummary); - fn check_deleted_sst_exist( - &self, - delete_sst_levels: &[u32], - delete_sst_ids_set: HashSet, - ) -> bool; -} - -impl HummockLevelsExt for Levels { - fn get_level0(&self) -> &OverlappingLevel { +#[easy_ext::ext(HummockLevelsExt)] +impl Levels { + pub fn get_level0(&self) -> &OverlappingLevel { self.l0.as_ref().unwrap() } - fn get_level(&self, level_idx: usize) -> &Level { + pub fn get_level(&self, level_idx: usize) -> &Level { &self.levels[level_idx - 1] } - fn get_level_mut(&mut self, level_idx: usize) -> &mut Level { + pub fn get_level_mut(&mut self, level_idx: usize) -> &mut Level { &mut self.levels[level_idx - 1] } - fn count_ssts(&self) -> usize { + pub fn count_ssts(&self) -> usize { self.get_level0() .get_sub_levels() .iter() @@ -644,7 +591,7 @@ impl HummockLevelsExt for Levels { .sum() } - fn apply_compact_ssts(&mut self, summary: GroupDeltasSummary) { + pub fn apply_compact_ssts(&mut self, summary: GroupDeltasSummary) { let GroupDeltasSummary { delete_sst_levels, delete_sst_ids_set, @@ -726,7 +673,7 @@ impl HummockLevelsExt for Levels { } } - fn check_deleted_sst_exist( + pub fn check_deleted_sst_exist( &self, delete_sst_levels: &[u32], mut delete_sst_ids_set: HashSet, From 6d88344c286f250ea8a7e7ef6b9d74dea838269e Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Mon, 20 Nov 2023 14:25:36 +0800 Subject: [PATCH 35/36] ci: opt-in steps with build.env rather than pr labels (#13464) --- ci/workflows/main-cron.yml | 193 +++++++++++++++++++++++++++------- ci/workflows/pull-request.yml | 118 +++++++++++++++------ 2 files changed, 238 insertions(+), 73 deletions(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index ac7867623dbf4..3933ddf6b9ac1 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -7,7 +7,10 @@ auto-retry: &auto-retry steps: - label: "build" command: "ci/scripts/build.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-build") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-build" + || build.env("CI_STEPS") =~ /(^|,)build(,|$$)/ key: "build" plugins: - docker-compose#v4.9.0: @@ -19,7 +22,10 @@ steps: - label: "build other components" command: "ci/scripts/build-other.sh" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-build-other") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-build-other" + || build.env("CI_STEPS") =~ /(^|,)build-other(,|$$)/ key: "build-other" plugins: - seek-oss/aws-sm#v2.3.1: @@ -36,7 +42,10 @@ steps: - label: "build (deterministic simulation)" command: "ci/scripts/build-simulation.sh" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-build-simulation") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-build-simulation" + || build.env("CI_STEPS") =~ /(^|,)build-simulation(,|$$)/ key: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -48,7 +57,10 @@ steps: - label: "docslt" command: "ci/scripts/docslt.sh" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-docslt") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-docslt" + || build.env("CI_STEPS") =~ /(^|,)docslt(,|$$)/ key: "docslt" plugins: - docker-compose#v4.9.0: @@ -61,7 +73,10 @@ steps: - label: "end-to-end test (release)" key: "e2e-test-release" command: "ci/scripts/cron-e2e-test.sh -p ci-release -m ci-3streaming-2serving-3fe" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-test") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-test" + || build.env("CI_STEPS") =~ /(^|,)e2e-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -77,7 +92,10 @@ steps: - label: "end-to-end test (parallel) (release)" command: "ci/scripts/e2e-test-parallel.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-parallel-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-parallel-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-parallel-tests?(,|$$)/ depends_on: - "build" - "docslt" @@ -98,7 +116,10 @@ steps: - label: "end-to-end test (parallel, in-memory) (release)" command: "ci/scripts/e2e-test-parallel-in-memory.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-parallel-in-memory-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-parallel-in-memory-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-parallel-in-memory-tests?(,|$$)/ depends_on: - "build" - "docslt" @@ -113,7 +134,10 @@ steps: - label: "end-to-end source test (release)" command: "ci/scripts/e2e-source-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-source-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-source-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-source-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -128,7 +152,10 @@ steps: - label: "end-to-end sink test (release)" command: "ci/scripts/e2e-sink-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-sink-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-sink-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-sink-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -144,7 +171,10 @@ steps: - label: "fuzz test" key: "fuzz-test" command: "ci/scripts/cron-fuzz-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-sqlsmith-fuzzing-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-sqlsmith-fuzzing-tests" + || build.env("CI_STEPS") =~ /(^|,)sqlsmith-fuzzing-tests?(,|$$)/ depends_on: - "build" - "build-simulation" @@ -162,7 +192,10 @@ steps: # This ensures our `main-cron` workflow will be stable. - label: "unit test" command: "ci/scripts/unit-test.sh" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-unit-test") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-unit-test" + || build.env("CI_STEPS") =~ /(^|,)unit-tests?(,|$$)/ plugins: - ./ci/plugins/swapfile - seek-oss/aws-sm#v2.3.1: @@ -178,7 +211,10 @@ steps: - label: "unit test (deterministic simulation)" command: "MADSIM_TEST_NUM=100 timeout 15m ci/scripts/deterministic-unit-test.sh" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-unit-test-deterministic-simulation") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-unit-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)unit-tests?-deterministic-simulation(,|$$)/ plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -189,7 +225,10 @@ steps: - label: "integration test (deterministic simulation) - scale" command: "TEST_NUM=60 ci/scripts/deterministic-it-test.sh scale::" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)integration-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -202,7 +241,10 @@ steps: - label: "integration test (deterministic simulation) - recovery" command: "TEST_NUM=60 ci/scripts/deterministic-it-test.sh recovery::" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)integration-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -215,7 +257,10 @@ steps: - label: "integration test (deterministic simulation) - others" command: "TEST_NUM=30 ci/scripts/deterministic-it-test.sh backfill_tests:: storage:: sink::" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)integration-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -228,7 +273,10 @@ steps: - label: "end-to-end test (deterministic simulation)" command: "TEST_NUM=64 timeout 55m ci/scripts/deterministic-e2e-test.sh" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)e2e-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" plugins: - seek-oss/aws-sm#v2.3.1: @@ -246,7 +294,10 @@ steps: - label: "recovery test (deterministic simulation)" command: "TEST_NUM=12 KILL_RATE=1.0 timeout 55m ci/scripts/deterministic-recovery-test.sh" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)recovery-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -259,7 +310,10 @@ steps: - label: "misc check" command: "ci/scripts/misc-check.sh" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-misc-check") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-misc-check" + || build.env("CI_STEPS") =~ /(^|,)misc-check(,|$$)/ plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -272,7 +326,10 @@ steps: - label: "end-to-end iceberg sink test (release)" key: "e2e-iceberg-sink-tests" command: "ci/scripts/e2e-iceberg-sink-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-iceberg-sink-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -288,7 +345,10 @@ steps: - label: "end-to-end iceberg sink v2 test (release)" command: "ci/scripts/e2e-iceberg-sink-v2-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-iceberg-sink-v2-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-iceberg-sink-v2-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-iceberg-sink-v2-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -304,7 +364,10 @@ steps: - label: "e2e java-binding test (release)" key: "e2e-java-binding-tests" command: "ci/scripts/java-binding-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-java-binding-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-java-binding-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-java-binding-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -321,7 +384,10 @@ steps: - label: "S3 source check on AWS (json parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s run.py" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") + 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: - seek-oss/aws-sm#v2.3.1: @@ -339,7 +405,10 @@ steps: - label: "S3 source check on AWS (json parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s json_file.py" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") + 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: - seek-oss/aws-sm#v2.3.1: @@ -357,7 +426,10 @@ steps: - label: "S3 source check on AWS (csv parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s run_csv.py" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") + 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: - seek-oss/aws-sm#v2.3.1: @@ -375,7 +447,10 @@ steps: - label: "S3_v2 source check on AWS (json parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s 'fs_source_v2.py json'" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") + 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: - seek-oss/aws-sm#v2.3.1: @@ -393,7 +468,10 @@ steps: - label: "S3_v2 source check on AWS (csv parser)" command: "ci/scripts/s3-source-test.sh -p ci-release -s 'fs_source_v2.py csv_without_header'" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") + 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: - seek-oss/aws-sm#v2.3.1: @@ -412,7 +490,10 @@ steps: - 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" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-s3-source-tests") + 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: - seek-oss/aws-sm#v2.3.1: @@ -431,7 +512,10 @@ steps: - label: "pulsar source check" key: "pulsar-source-tests" command: "ci/scripts/pulsar-source-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-pulsar-source-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-pulsar-source-tests" + || build.env("CI_STEPS") =~ /(^|,)pulsar-source-tests?(,|$$)/ depends_on: - build - build-other @@ -452,7 +536,10 @@ steps: - label: "micro benchmark" command: "ci/scripts/run-micro-benchmarks.sh" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-micro-benchmarks") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-micro-benchmarks" + || build.env("CI_STEPS") =~ /(^|,)micro-benchmarks?(,|$$)/ key: "run-micro-benchmarks" plugins: - docker-compose#v4.9.0: @@ -463,7 +550,11 @@ steps: retry: *auto-retry - label: "upload micro-benchmark" - if: build.branch == "main" || (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-micro-benchmarks") + if: | + build.branch == "main" + || !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-micro-benchmarks" + || build.env("CI_STEPS") =~ /(^|,)micro-benchmarks?(,|$$)/ command: - "BUILDKITE_BUILD_NUMBER=$BUILDKITE_BUILD_NUMBER ci/scripts/upload-micro-bench-results.sh" depends_on: "run-micro-benchmarks" @@ -485,7 +576,10 @@ steps: - label: "Backwards compatibility tests" key: "backwards-compat-tests" command: "RW_COMMIT=$BUILDKITE_COMMIT ci/scripts/backwards-compat-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-backwards-compat-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-backwards-compat-tests" + || build.env("CI_STEPS") =~ /(^|,)backwards?-compat-tests?(,|$$)/ depends_on: - "build" plugins: @@ -500,7 +594,10 @@ steps: # Sqlsmith differential testing - label: "Sqlsmith Differential Testing" command: "ci/scripts/sqlsmith-differential-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-sqlsmith-differential-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-sqlsmith-differential-tests" + || build.env("CI_STEPS") =~ /(^|,)sqlsmith-differential-tests?(,|$$)/ depends_on: - "build" plugins: @@ -514,7 +611,10 @@ steps: - label: "Backfill tests" key: "backfill-tests" command: "ci/scripts/backfill-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-backfill-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-backfill-tests" + || build.env("CI_STEPS") =~ /(^|,)backfill-tests?(,|$$)/ depends_on: - "build" plugins: @@ -528,7 +628,10 @@ steps: - label: "e2e standalone binary test" command: "ci/scripts/e2e-test.sh -p ci-release -m standalone" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-standalone-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-standalone-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-standalone-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -544,7 +647,10 @@ steps: - label: "end-to-end test for opendal (parallel)" command: "ci/scripts/e2e-test-parallel-for-opendal.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-parallel-tests-for-opendal") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-parallel-tests-for-opendal" + || build.env("CI_STEPS") =~ /(^|,)e2e-parallel-tests?-for-opendal(,|$$)/ depends_on: - "build" - "docslt" @@ -560,7 +666,10 @@ steps: - label: "end-to-end clickhouse sink test" key: "e2e-clickhouse-sink-tests" command: "ci/scripts/e2e-clickhouse-sink-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-clickhouse-sink-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-clickhouse-sink-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-clickhouse-sink-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -576,7 +685,10 @@ steps: - label: "end-to-end pulsar sink test" key: "e2e-pulsar-sink-tests" command: "ci/scripts/e2e-pulsar-sink-test.sh -p ci-release" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-e2e-pulsar-sink-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-pulsar-sink-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-pulsar-sink-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -592,7 +704,10 @@ steps: - label: "connector node integration test Java {{matrix.java_version}}" key: "connector-node-integration-test" command: "ci/scripts/connector-node-integration-test.sh -p ci-release -v {{matrix.java_version}}" - if: (!build.pull_request.labels includes "ci/main-cron/skip-ci" || build.pull_request.labels includes "ci/run-connector-node-integration-tests") + if: | + !(build.pull_request.labels includes "ci/main-cron/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-connector-node-integration-tests" + || build.env("CI_STEPS") =~ /(^|,)connector-node-integration-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -688,4 +803,4 @@ steps: # This should be the LAST part of the main-cron file. - label: "trigger failed test notification" if: build.pull_request.labels includes "ci/main-cron/test-notify" || build.branch == "main" - command: "ci/scripts/notify.py" \ No newline at end of file + command: "ci/scripts/notify.py" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index bd71c7f7044ef..df4a12ba8ac8c 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -19,7 +19,10 @@ steps: - label: "build" command: "ci/scripts/build.sh -p ci-dev" key: "build" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-build") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-build" + || build.env("CI_STEPS") =~ /(^|,)build(,|$$)/ plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -31,7 +34,10 @@ steps: - label: "build other components" command: "ci/scripts/build-other.sh" key: "build-other" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-build-other") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-build-other" + || build.env("CI_STEPS") =~ /(^|,)build-other(,|$$)/ plugins: - seek-oss/aws-sm#v2.3.1: env: @@ -48,7 +54,10 @@ steps: - label: "build (deterministic simulation)" command: "ci/scripts/build-simulation.sh" key: "build-simulation" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-build-simulation") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-build-simulation" + || build.env("CI_STEPS") =~ /(^|,)build-simulation(,|$$)/ plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -60,7 +69,10 @@ steps: - label: "docslt" command: "ci/scripts/docslt.sh" key: "docslt" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-docslt") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-docslt" + || build.env("CI_STEPS") =~ /(^|,)docslt(,|$$)/ plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -71,7 +83,10 @@ steps: - label: "end-to-end test" command: "ci/scripts/e2e-test.sh -p ci-dev -m ci-3streaming-2serving-3fe" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-e2e-test") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-test" + || build.env("CI_STEPS") =~ /(^|,)e2e-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -87,7 +102,10 @@ steps: - label: "end-to-end test (parallel)" command: "ci/scripts/e2e-test-parallel.sh -p ci-dev" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-e2e-parallel-tests") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-parallel-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-parallel-tests?(,|$$)/ depends_on: - "build" - "docslt" @@ -101,7 +119,7 @@ steps: retry: *auto-retry - label: "end-to-end test for opendal (parallel)" - if: build.pull_request.labels includes "ci/run-opendal-tests" + if: build.pull_request.labels includes "ci/run-opendal-tests" || build.env("CI_STEPS") =~ /(^|,)opendal-tests?(,|$$)/ command: "ci/scripts/e2e-test-parallel-for-opendal.sh -p ci-dev" depends_on: - "build" @@ -116,7 +134,7 @@ steps: retry: *auto-retry - label: "end-to-end test (parallel, in-memory)" - if: build.pull_request.labels includes "ci/run-e2e-parallel-in-memory-tests" + if: build.pull_request.labels includes "ci/run-e2e-parallel-in-memory-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-parallel-in-memory-tests?(,|$$)/ command: "ci/scripts/e2e-test-parallel-in-memory.sh -p ci-dev" depends_on: "build" plugins: @@ -130,7 +148,10 @@ steps: - label: "end-to-end source test" command: "ci/scripts/e2e-source-test.sh -p ci-dev" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-e2e-source-tests") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-source-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-source-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -145,7 +166,10 @@ steps: - label: "end-to-end sink test" command: "ci/scripts/e2e-sink-test.sh -p ci-dev" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-e2e-sink-tests") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-sink-tests" + || build.env("CI_STEPS") =~ /(^|,)e2e-sink-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -160,7 +184,7 @@ steps: retry: *auto-retry - label: "connector node integration test Java {{matrix.java_version}}" - if: build.pull_request.labels includes "ci/run-java-connector-node-integration-tests" + if: build.pull_request.labels includes "ci/run-java-connector-node-integration-tests" || build.env("CI_STEPS") =~ /(^|,)java-connector-node-integration-tests?(,|$$)/ command: "ci/scripts/connector-node-integration-test.sh -p ci-dev -v {{matrix.java_version}}" depends_on: - "build" @@ -180,7 +204,7 @@ steps: retry: *auto-retry - label: "end-to-end iceberg sink test" - if: build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests" + if: build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-iceberg-sink-tests?(,|$$)/ command: "ci/scripts/e2e-iceberg-sink-test.sh -p ci-dev" depends_on: - "build" @@ -195,7 +219,7 @@ steps: retry: *auto-retry - label: "end-to-end iceberg sink v2 test" - if: build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests" + if: build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-iceberg-sink-tests?(,|$$)/ command: "ci/scripts/e2e-iceberg-sink-v2-test.sh -p ci-dev" depends_on: - "build" @@ -210,7 +234,7 @@ steps: retry: *auto-retry - label: "end-to-end iceberg cdc test" - if: build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests" + if: build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-iceberg-sink-tests?(,|$$)/ command: "ci/scripts/e2e-iceberg-cdc.sh -p ci-dev" depends_on: - "build" @@ -225,7 +249,7 @@ steps: retry: *auto-retry - label: "end-to-end pulsar sink test" - if: build.pull_request.labels includes "ci/run-e2e-pulsar-sink-tests" + if: build.pull_request.labels includes "ci/run-e2e-pulsar-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-pulsar-sink-tests?(,|$$)/ command: "ci/scripts/e2e-pulsar-sink-test.sh -p ci-dev" depends_on: - "build" @@ -240,7 +264,7 @@ steps: retry: *auto-retry - label: "end-to-end clickhouse sink test" - if: build.pull_request.labels includes "ci/run-e2e-clickhouse-sink-tests" + if: build.pull_request.labels includes "ci/run-e2e-clickhouse-sink-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-clickhouse-sink-tests?(,|$$)/ command: "ci/scripts/e2e-clickhouse-sink-test.sh -p ci-dev" depends_on: - "build" @@ -255,7 +279,7 @@ steps: retry: *auto-retry - label: "e2e java-binding test" - if: build.pull_request.labels includes "ci/run-java-binding-tests" + if: build.pull_request.labels includes "ci/run-java-binding-tests" || build.env("CI_STEPS") =~ /(^|,)java-binding-tests?(,|$$)/ command: "ci/scripts/java-binding-test.sh -p ci-dev" depends_on: - "build" @@ -271,7 +295,10 @@ steps: - label: "regress test" command: "ci/scripts/regress-test.sh -p ci-dev" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-regress-test") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-regress-test" + || build.env("CI_STEPS") =~ /(^|,)regress-tests?(,|$$)/ depends_on: "build" plugins: - docker-compose#v4.9.0: @@ -287,7 +314,10 @@ steps: # This ensures our `main-cron` workflow will be stable. - label: "unit test" command: "ci/scripts/pr-unit-test.sh" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-unit-test") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-unit-test" + || build.env("CI_STEPS") =~ /(^|,)unit-tests?(,|$$)/ plugins: - ./ci/plugins/swapfile - seek-oss/aws-sm#v2.3.1: @@ -303,7 +333,10 @@ steps: - label: "check" command: "ci/scripts/check.sh" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-check") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-check" + || build.env("CI_STEPS") =~ /(^|,)check(,|$$)/ plugins: - gencer/cache#v2.4.10: id: cache @@ -325,7 +358,10 @@ steps: - label: "unit test (deterministic simulation)" command: "ci/scripts/deterministic-unit-test.sh" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-unit-test-deterministic-simulation") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-unit-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)unit-tests?-deterministic-simulation(,|$$)/ plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -337,7 +373,10 @@ steps: - label: "integration test (deterministic simulation)" command: "TEST_NUM=5 ci/scripts/deterministic-it-test.sh" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-integration-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)integration-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" plugins: - docker-compose#v4.9.0: @@ -350,7 +389,10 @@ steps: - label: "end-to-end test (deterministic simulation)" command: "TEST_NUM=16 ci/scripts/deterministic-e2e-test.sh" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)e2e-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" plugins: - seek-oss/aws-sm#v2.3.1: @@ -369,7 +411,10 @@ steps: - label: "recovery test (deterministic simulation)" command: "TEST_NUM=8 KILL_RATE=0.5 ci/scripts/deterministic-recovery-test.sh" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation" + || build.env("CI_STEPS") =~ /(^|,)recovery-tests?-deterministic-simulation(,|$$)/ depends_on: "build-simulation" plugins: # - seek-oss/aws-sm#v2.3.1: @@ -389,7 +434,10 @@ steps: - label: "misc check" command: "ci/scripts/misc-check.sh" - if: (!build.pull_request.labels includes "ci/skip-ci" || build.pull_request.labels includes "ci/run-misc-check") + if: | + !(build.pull_request.labels includes "ci/skip-ci") && build.env("CI_STEPS") == null + || build.pull_request.labels includes "ci/run-misc-check" + || build.env("CI_STEPS") =~ /(^|,)misc-check(,|$$)/ plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -410,6 +458,7 @@ steps: || build.pull_request.labels includes "ci/run-cpu-flamegraph" || build.pull_request.labels includes "heap_flamegraph" || build.pull_request.labels includes "ci/run-heap-flamegraph" + || build.env("CI_STEPS") =~ /(^|,)(cpu-flamegraph|heap-flamegraph)(,|$$)/ plugins: - seek-oss/aws-sm#v2.3.1: env: @@ -426,7 +475,7 @@ steps: - label: "Generate CPU flamegraph" command: "PULL_REQUEST=$BUILDKITE_PULL_REQUEST ci/scripts/gen-flamegraph.sh cpu" depends_on: "flamegraph-env-build" - if: build.pull_request.labels includes "cpu_flamegraph" || build.pull_request.labels includes "ci/run-cpu-flamegraph" + if: build.pull_request.labels includes "cpu_flamegraph" || build.pull_request.labels includes "ci/run-cpu-flamegraph" || build.env("CI_STEPS") =~ /(^|,)cpu-flamegraph(,|$$)/ plugins: - seek-oss/aws-sm#v2.3.1: env: @@ -448,7 +497,7 @@ steps: command: "PULL_REQUEST=$BUILDKITE_PULL_REQUEST ci/scripts/gen-flamegraph.sh heap" depends_on: "flamegraph-env-build" - if: build.pull_request.labels includes "heap_flamegraph" || build.pull_request.labels includes "ci/run-heap-flamegraph" + if: build.pull_request.labels includes "heap_flamegraph" || build.pull_request.labels includes "ci/run-heap-flamegraph" || build.env("CI_STEPS") =~ /(^|,)heap-flamegraph(,|$$)/ plugins: - seek-oss/aws-sm#v2.3.1: @@ -471,7 +520,8 @@ steps: command: "RW_COMMIT=$BUILDKITE_COMMIT ci/scripts/backwards-compat-test.sh -p ci-dev" if: | build.pull_request.labels includes "breaking-change" || - build.pull_request.labels includes "ci/run-backwards-compat-tests" + build.pull_request.labels includes "ci/run-backwards-compat-tests" || + build.env("CI_STEPS") =~ /(^|,)backwards?-compat-tests?(,|$$)/ depends_on: - "build" plugins: @@ -485,7 +535,7 @@ steps: # Sqlsmith differential testing - label: "Sqlsmith Differential Testing" command: "ci/scripts/sqlsmith-differential-test.sh -p ci-dev" - if: build.pull_request.labels includes "ci/run-sqlsmith-differential-tests" + if: build.pull_request.labels includes "ci/run-sqlsmith-differential-tests" || build.env("CI_STEPS") =~ /(^|,)sqlsmith-differential-tests?(,|$$)/ depends_on: - "build" plugins: @@ -497,7 +547,7 @@ steps: - label: "Backfill tests" command: "ci/scripts/backfill-test.sh -p ci-dev" - if: build.pull_request.labels includes "ci/run-backfill-tests" + if: build.pull_request.labels includes "ci/run-backfill-tests" || build.env("CI_STEPS") =~ /(^|,)backfill-tests?(,|$$)/ depends_on: - "build" plugins: @@ -510,7 +560,7 @@ steps: - label: "e2e standalone binary test" command: "ci/scripts/e2e-test.sh -p ci-dev -m standalone" - if: build.pull_request.labels includes "ci/run-e2e-standalone-tests" + if: build.pull_request.labels includes "ci/run-e2e-standalone-tests" || build.env("CI_STEPS") =~ /(^|,)e2e-standalone-tests?(,|$$)/ depends_on: - "build" - "build-other" @@ -527,7 +577,7 @@ steps: # FIXME(kwannoel): Let the github PR labeller label it, if sqlsmith source files has changes. - label: "fuzz test" command: "ci/scripts/pr-fuzz-test.sh -p ci-dev" - if: build.pull_request.labels includes "ci/run-sqlsmith-fuzzing-tests" + if: build.pull_request.labels includes "ci/run-sqlsmith-fuzzing-tests" || build.env("CI_STEPS") =~ /(^|,)sqlsmith-fuzzing-tests?(,|$$)/ depends_on: - "build" - "build-simulation" @@ -550,7 +600,7 @@ steps: - label: "micro benchmark" command: "ci/scripts/run-micro-benchmarks.sh" key: "run-micro-benchmarks" - if: build.pull_request.labels includes "ci/run-micro-benchmarks" + if: build.pull_request.labels includes "ci/run-micro-benchmarks" || build.env("CI_STEPS") =~ /(^|,)micro-benchmarks?(,|$$)/ plugins: - docker-compose#v4.9.0: run: rw-build-env @@ -560,7 +610,7 @@ steps: retry: *auto-retry - label: "upload micro-benchmark" - if: build.pull_request.labels includes "ci/run-upload-micro-benchmark" + if: build.pull_request.labels includes "ci/run-upload-micro-benchmark" || build.env("CI_STEPS") =~ /(^|,)upload-micro-benchmarks?(,|$$)/ command: - "BUILDKITE_BUILD_NUMBER=$BUILDKITE_BUILD_NUMBER ci/scripts/upload-micro-bench-results.sh" depends_on: "run-micro-benchmarks" From 0c35dc7b229b12f55740687d7950e503845720c9 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Mon, 20 Nov 2023 15:06:09 +0800 Subject: [PATCH 36/36] refactor(rw_catalog): make rw_worker_nodes list all nodes and resource (#13487) --- proto/common.proto | 13 ++ proto/meta.proto | 3 +- src/ctl/src/cmd_impl/await_tree.rs | 6 +- src/ctl/src/cmd_impl/meta/serving.rs | 2 +- .../src/catalog/system_catalog/mod.rs | 2 +- .../rw_catalog/rw_worker_nodes.rs | 33 +++-- src/frontend/src/meta_client.rs | 7 ++ .../src/scheduler/distributed/query.rs | 3 + .../src/scheduler/worker_node_manager.rs | 2 + src/frontend/src/test_utils.rs | 5 + src/meta/node/src/lib.rs | 3 +- src/meta/service/src/cluster_service.rs | 5 +- src/meta/service/src/notification_service.rs | 2 +- src/meta/service/src/scale_service.rs | 2 +- src/meta/src/controller/cluster.rs | 35 +++--- src/meta/src/dashboard/mod.rs | 12 +- src/meta/src/hummock/manager/gc.rs | 8 +- src/meta/src/hummock/manager/tests.rs | 2 + .../src/hummock/mock_hummock_meta_client.rs | 1 + src/meta/src/hummock/test_utils.rs | 1 + src/meta/src/manager/cluster.rs | 114 +++++++++++++----- src/meta/src/manager/env.rs | 2 + src/meta/src/model/cluster.rs | 33 ++--- src/meta/src/rpc/metrics.rs | 2 +- src/meta/src/stream/stream_manager.rs | 1 + src/rpc_client/src/meta_client.rs | 16 ++- .../hummock_test/src/compactor_tests.rs | 1 + 27 files changed, 216 insertions(+), 100 deletions(-) diff --git a/proto/common.proto b/proto/common.proto index fe976fb5c8e87..4f0d56b4823a9 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -52,6 +52,11 @@ message WorkerNode { bool is_serving = 2; bool is_unschedulable = 3; } + message Resource { + string rw_version = 1; + uint64 total_memory_bytes = 2; + uint64 total_cpu_cores = 3; + } uint32 id = 1; WorkerType type = 2; HostAddress host = 3; @@ -62,6 +67,14 @@ message WorkerNode { // Ranges from 0 to 1023, used to generate the machine ID field in the global unique ID. optional uint32 transactional_id = 7; + // Resource spec. + // It's populated by meta node with the value reported by worker node, when the worker node is added by meta node. + // It's not persistent in meta store. + optional Resource resource = 8; + // Timestamp the worker node is added by meta node, in seconds. + // It's populated by meta node, when the worker node is added by meta node. + // It's not persistent in meta store. + optional uint64 started_at = 9; } message Buffer { diff --git a/proto/meta.proto b/proto/meta.proto index f2375eed7653a..8085d5801ca51 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -262,6 +262,7 @@ message AddWorkerNodeRequest { common.HostAddress host = 2; reserved 3; Property property = 4; + common.WorkerNode.Resource resource = 5; } message AddWorkerNodeResponse { @@ -304,7 +305,7 @@ message UpdateWorkerNodeSchedulabilityResponse { } message ListAllNodesRequest { - common.WorkerType worker_type = 1; + optional common.WorkerType worker_type = 1; // Whether to include nodes still starting bool include_starting_nodes = 2; } diff --git a/src/ctl/src/cmd_impl/await_tree.rs b/src/ctl/src/cmd_impl/await_tree.rs index f01b07b6cd105..901b7f8d35650 100644 --- a/src/ctl/src/cmd_impl/await_tree.rs +++ b/src/ctl/src/cmd_impl/await_tree.rs @@ -31,7 +31,7 @@ pub async fn dump(context: &CtlContext) -> anyhow::Result<()> { let meta_client = context.meta_client().await?; let compute_nodes = meta_client - .list_worker_nodes(WorkerType::ComputeNode) + .list_worker_nodes(Some(WorkerType::ComputeNode)) .await?; let clients = ComputeClientPool::default(); @@ -43,7 +43,9 @@ pub async fn dump(context: &CtlContext) -> anyhow::Result<()> { merge(&mut all, response); } - let compactor_nodes = meta_client.list_worker_nodes(WorkerType::Compactor).await?; + let compactor_nodes = meta_client + .list_worker_nodes(Some(WorkerType::Compactor)) + .await?; for compactor in compactor_nodes { let addr: HostAddr = compactor.get_host().unwrap().into(); diff --git a/src/ctl/src/cmd_impl/meta/serving.rs b/src/ctl/src/cmd_impl/meta/serving.rs index cff3c6f911282..148adf8726b84 100644 --- a/src/ctl/src/cmd_impl/meta/serving.rs +++ b/src/ctl/src/cmd_impl/meta/serving.rs @@ -25,7 +25,7 @@ pub async fn list_serving_fragment_mappings(context: &CtlContext) -> anyhow::Res let meta_client = context.meta_client().await?; let mappings = meta_client.list_serving_vnode_mappings().await?; let workers = meta_client - .list_worker_nodes(WorkerType::ComputeNode) + .list_worker_nodes(Some(WorkerType::ComputeNode)) .await?; let mut pu_to_worker: HashMap = HashMap::new(); for w in &workers { diff --git a/src/frontend/src/catalog/system_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/mod.rs index c1510e38270f4..214761bff0768 100644 --- a/src/frontend/src/catalog/system_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/mod.rs @@ -382,7 +382,7 @@ prepare_sys_catalog! { { BuiltinCatalog::Table(&RW_CONNECTIONS), read_rw_connections_info }, { BuiltinCatalog::Table(&RW_FUNCTIONS), read_rw_functions_info }, { BuiltinCatalog::Table(&RW_VIEWS), read_rw_views_info }, - { BuiltinCatalog::Table(&RW_WORKER_NODES), read_rw_worker_nodes_info }, + { BuiltinCatalog::Table(&RW_WORKER_NODES), read_rw_worker_nodes_info await }, { BuiltinCatalog::Table(&RW_PARALLEL_UNITS), read_rw_parallel_units_info }, { BuiltinCatalog::Table(&RW_TABLE_FRAGMENTS), read_rw_table_fragments_info await }, { BuiltinCatalog::Table(&RW_FRAGMENTS), read_rw_fragment_distributions_info await }, diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs index 0c37783ea7dd3..4f50d18ba7835 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs @@ -16,12 +16,11 @@ use itertools::Itertools; use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_common::types::{DataType, ScalarImpl, Timestamptz}; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; /// `rw_worker_nodes` contains all information about the compute nodes in the cluster. -/// TODO: Add other type of nodes if necessary in the future. pub const RW_WORKER_NODES: BuiltinTable = BuiltinTable { name: "rw_worker_nodes", schema: RW_CATALOG_SCHEMA_NAME, @@ -35,23 +34,29 @@ pub const RW_WORKER_NODES: BuiltinTable = BuiltinTable { (DataType::Boolean, "is_streaming"), (DataType::Boolean, "is_serving"), (DataType::Boolean, "is_unschedulable"), + (DataType::Varchar, "rw_version"), + (DataType::Int64, "total_memory_bytes"), + (DataType::Int64, "total_cpu_cores"), + (DataType::Timestamptz, "started_at"), ], pk: &[0], }; impl SysCatalogReaderImpl { - pub fn read_rw_worker_nodes_info(&self) -> Result> { - let workers = self.worker_node_manager.list_worker_nodes(); + pub async fn read_rw_worker_nodes_info(&self) -> Result> { + let workers = self.meta_client.list_all_nodes().await?; Ok(workers .into_iter() + .sorted_by_key(|w| w.id) .map(|worker| { - let host = worker.host.as_ref().unwrap(); - let property = worker.property.as_ref().unwrap(); + let host = worker.host.as_ref(); + let property = worker.property.as_ref(); + let resource = worker.resource.as_ref(); OwnedRow::new(vec![ Some(ScalarImpl::Int32(worker.id as i32)), - Some(ScalarImpl::Utf8(host.host.clone().into())), - Some(ScalarImpl::Utf8(host.port.to_string().into())), + host.map(|h| ScalarImpl::Utf8(h.host.clone().into())), + host.map(|h| ScalarImpl::Utf8(h.port.to_string().into())), Some(ScalarImpl::Utf8( worker.get_type().unwrap().as_str_name().into(), )), @@ -59,9 +64,15 @@ impl SysCatalogReaderImpl { worker.get_state().unwrap().as_str_name().into(), )), Some(ScalarImpl::Int32(worker.parallel_units.len() as i32)), - Some(ScalarImpl::Bool(property.is_streaming)), - Some(ScalarImpl::Bool(property.is_serving)), - Some(ScalarImpl::Bool(property.is_unschedulable)), + property.map(|p| ScalarImpl::Bool(p.is_streaming)), + property.map(|p| ScalarImpl::Bool(p.is_serving)), + property.map(|p| ScalarImpl::Bool(p.is_unschedulable)), + resource.map(|r| ScalarImpl::Utf8(r.rw_version.to_owned().into())), + resource.map(|r| ScalarImpl::Int64(r.total_memory_bytes as _)), + resource.map(|r| ScalarImpl::Int64(r.total_cpu_cores as _)), + worker.started_at.map(|ts| { + ScalarImpl::Timestamptz(Timestamptz::from_secs(ts as i64).unwrap()) + }), ]) }) .collect_vec()) diff --git a/src/frontend/src/meta_client.rs b/src/frontend/src/meta_client.rs index 00c1e2c04f257..0d2752f71c191 100644 --- a/src/frontend/src/meta_client.rs +++ b/src/frontend/src/meta_client.rs @@ -18,6 +18,7 @@ use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::util::epoch::MAX_EPOCH; use risingwave_pb::backup_service::MetaSnapshotMetadata; use risingwave_pb::catalog::Table; +use risingwave_pb::common::WorkerNode; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ @@ -96,6 +97,8 @@ pub trait FrontendMetaClient: Send + Sync { async fn list_hummock_active_write_limits(&self) -> Result>; async fn list_hummock_meta_configs(&self) -> Result>; + + async fn list_all_nodes(&self) -> Result>; } pub struct FrontendMetaClientImpl(pub MetaClient); @@ -240,4 +243,8 @@ impl FrontendMetaClient for FrontendMetaClientImpl { async fn list_hummock_meta_configs(&self) -> Result> { self.0.list_hummock_meta_config().await } + + async fn list_all_nodes(&self) -> Result> { + self.0.list_worker_nodes(None).await + } } diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index db5581aea856c..6be22bc180b08 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -611,6 +611,7 @@ pub(crate) mod tests { is_streaming: true, }), transactional_id: Some(0), + ..Default::default() }; let worker2 = WorkerNode { id: 1, @@ -627,6 +628,7 @@ pub(crate) mod tests { is_streaming: true, }), transactional_id: Some(1), + ..Default::default() }; let worker3 = WorkerNode { id: 2, @@ -643,6 +645,7 @@ pub(crate) mod tests { is_streaming: true, }), transactional_id: Some(2), + ..Default::default() }; let workers = vec![worker1, worker2, worker3]; let worker_node_manager = Arc::new(WorkerNodeManager::mock(workers)); diff --git a/src/frontend/src/scheduler/worker_node_manager.rs b/src/frontend/src/scheduler/worker_node_manager.rs index ed1ce5d2aaa6b..469048cd3d62e 100644 --- a/src/frontend/src/scheduler/worker_node_manager.rs +++ b/src/frontend/src/scheduler/worker_node_manager.rs @@ -413,6 +413,7 @@ mod tests { is_streaming: true, }), transactional_id: Some(1), + ..Default::default() }, WorkerNode { id: 2, @@ -426,6 +427,7 @@ mod tests { is_streaming: false, }), transactional_id: Some(2), + ..Default::default() }, ]; worker_nodes diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 4e336b29c002d..3bfd4a5667c20 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -36,6 +36,7 @@ use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ PbComment, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, Table, }; +use risingwave_pb::common::WorkerNode; use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::ddl_service::{create_connection_request, DdlProgress, PbTableJobType}; use risingwave_pb::hummock::write_limits::WriteLimit; @@ -895,6 +896,10 @@ impl FrontendMetaClient for MockFrontendMetaClient { async fn list_hummock_meta_configs(&self) -> RpcResult> { unimplemented!() } + + async fn list_all_nodes(&self) -> RpcResult> { + unimplemented!() + } } #[cfg(test)] diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index dfd3cdf77d4b1..44cc885851b04 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -234,7 +234,7 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { }); let add_info = AddressInfo { - advertise_addr: opts.advertise_addr, + advertise_addr: opts.advertise_addr.to_owned(), listen_addr, prometheus_addr, dashboard_addr, @@ -300,6 +300,7 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { .meta .compaction_task_max_heartbeat_interval_secs, compaction_config: Some(config.meta.compaction_config), + advertise_addr: opts.advertise_addr, }, config.system.into_init_system_params(), ) diff --git a/src/meta/service/src/cluster_service.rs b/src/meta/service/src/cluster_service.rs index 1d6f87b5f9d01..e67431cfe7103 100644 --- a/src/meta/service/src/cluster_service.rs +++ b/src/meta/service/src/cluster_service.rs @@ -48,9 +48,10 @@ impl ClusterService for ClusterServiceImpl { let property = req .property .ok_or_else(|| MetaError::invalid_parameter("worker node property is not provided"))?; + let resource = req.resource.unwrap_or_default(); let result = self .cluster_manager - .add_worker_node(worker_type, host, property) + .add_worker_node(worker_type, host, property, resource) .await; match result { Ok(worker_node) => Ok(Response::new(AddWorkerNodeResponse { @@ -116,7 +117,7 @@ impl ClusterService for ClusterServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let worker_type = req.get_worker_type()?; + let worker_type = req.worker_type.map(|wt| wt.try_into().unwrap()); let worker_states = if req.include_starting_nodes { None } else { diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index 0fcc470a70e39..b7b63dcc6c164 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -113,7 +113,7 @@ impl NotificationServiceImpl { async fn get_worker_node_snapshot(&self) -> (Vec, NotificationVersion) { let cluster_guard = self.cluster_manager.get_cluster_core_guard().await; - let nodes = cluster_guard.list_worker_node(WorkerType::ComputeNode, Some(Running)); + let nodes = cluster_guard.list_worker_node(Some(WorkerType::ComputeNode), Some(Running)); let notification_version = self.env.notification_manager().current_version().await; (nodes, notification_version) } diff --git a/src/meta/service/src/scale_service.rs b/src/meta/service/src/scale_service.rs index c054b9b18bd6d..39fc0bd90acca 100644 --- a/src/meta/service/src/scale_service.rs +++ b/src/meta/service/src/scale_service.rs @@ -88,7 +88,7 @@ impl ScaleService for ScaleServiceImpl { let worker_nodes = self .cluster_manager - .list_worker_node(WorkerType::ComputeNode, None) + .list_worker_node(Some(WorkerType::ComputeNode), None) .await; let actor_splits = self diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index 6f4e99ff161f6..8d3168bf8fe42 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -86,6 +86,8 @@ impl From for PbWorkerNode { is_unschedulable: p.is_unschedulable, }), transactional_id: info.0.transaction_id.map(|id| id as _), + resource: None, + started_at: None, } } } @@ -293,7 +295,7 @@ impl ClusterController { /// * `worker_state` Filter by this state if it is not None. pub async fn list_workers( &self, - worker_type: WorkerType, + worker_type: Option, worker_status: Option, ) -> MetaResult> { self.inner @@ -661,27 +663,20 @@ impl ClusterControllerInner { pub async fn list_workers( &self, - worker_type: WorkerType, + worker_type: Option, worker_status: Option, ) -> MetaResult> { - let workers = if let Some(status) = worker_status { - Worker::find() - .filter( - worker::Column::WorkerType - .eq(worker_type) - .and(worker::Column::Status.eq(status)), - ) - .find_also_related(WorkerProperty) - .all(&self.db) - .await? - } else { - Worker::find() - .filter(worker::Column::WorkerType.eq(worker_type)) - .find_also_related(WorkerProperty) - .all(&self.db) - .await? - }; - + let mut find = Worker::find(); + if let Some(worker_type) = worker_type { + find = find.filter(worker::Column::WorkerType.eq(worker_type)); + } + if let Some(worker_status) = worker_status { + find = find.filter(worker::Column::Status.eq(worker_status)); + } + let workers = find.find_also_related(WorkerProperty).all(&self.db).await?; + // TODO include meta node + // TODO include resource spec + // TODO set started_at Ok(workers .into_iter() .map(|(worker, property)| WorkerInfo(worker, property).into()) diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 6a10dd9c02bb3..63439d4f09d37 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -101,14 +101,12 @@ pub(super) mod handlers { Path(ty): Path, Extension(srv): Extension, ) -> Result>> { + let worker_type = WorkerType::try_from(ty) + .map_err(|_| anyhow!("invalid worker type")) + .map_err(err)?; let mut result = srv .cluster_manager - .list_worker_node( - WorkerType::try_from(ty) - .map_err(|_| anyhow!("invalid worker type")) - .map_err(err)?, - None, - ) + .list_worker_node(Some(worker_type), None) .await; result.sort_unstable_by_key(|n| n.id); Ok(result.into()) @@ -224,7 +222,7 @@ pub(super) mod handlers { ) -> Result> { let worker_nodes = srv .cluster_manager - .list_worker_node(WorkerType::ComputeNode, None) + .list_worker_node(Some(WorkerType::ComputeNode), None) .await; dump_await_tree_inner(&worker_nodes, &srv.compute_clients).await diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index 5533d9be68e85..837564d6e6477 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -209,7 +209,7 @@ pub async fn collect_global_gc_watermark( let workers = [ cluster_manager.list_active_streaming_compute_nodes().await, cluster_manager - .list_worker_node(WorkerType::Compactor, Some(Running)) + .list_worker_node(Some(WorkerType::Compactor), Some(Running)) .await, ] .concat(); @@ -237,11 +237,11 @@ pub async fn collect_global_gc_watermark( }; match init_version_id.as_ref() { None => { - init_version_id = Some(worker_info.info_version_id()); + init_version_id = Some(worker_info.info_version_id); } Some(init_version_id) => { - if worker_info.info_version_id() >= *init_version_id + 2 { - return worker_info.hummock_gc_watermark(); + if worker_info.info_version_id >= *init_version_id + 2 { + return worker_info.hummock_gc_watermark; } } } diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 4d5de0cc19011..359e9e76294dc 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -382,6 +382,7 @@ async fn test_release_context_resource() { is_serving: true, is_unschedulable: false, }, + Default::default(), ) .await .unwrap(); @@ -469,6 +470,7 @@ async fn test_hummock_manager_basic() { is_serving: true, is_unschedulable: false, }, + Default::default(), ) .await .unwrap(); diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index 678c701ca2891..d0d68ce357032 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -217,6 +217,7 @@ impl HummockMetaClient for MockHummockMetaClient { port: 0, }, Default::default(), + Default::default(), ) .await .unwrap(); diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 3d42442ae7c67..c396cd63e0ffc 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -346,6 +346,7 @@ pub async fn setup_compute_env_with_metric( is_serving: true, is_unschedulable: false, }, + Default::default(), ) .await .unwrap(); diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 0546231cb79fa..2ca44beb64cab 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -15,11 +15,16 @@ use std::cmp::Ordering; use std::collections::{HashMap, HashSet, VecDeque}; use std::hash::{Hash, Hasher}; +use std::iter; use std::sync::Arc; use std::time::{Duration, SystemTime}; use itertools::Itertools; use risingwave_common::hash::ParallelUnitId; +use risingwave_common::util::addr::HostAddr; +use risingwave_common::util::resource_util::cpu::total_cpu_available; +use risingwave_common::util::resource_util::memory::system_memory_available_bytes; +use risingwave_common::RW_VERSION; use risingwave_pb::common::worker_node::{Property, State}; use risingwave_pb::common::{HostAddress, ParallelUnit, WorkerNode, WorkerType}; use risingwave_pb::meta::add_worker_node_request::Property as AddNodeProperty; @@ -32,7 +37,7 @@ use tokio::task::JoinHandle; use crate::manager::{IdCategory, LocalNotification, MetaSrvEnv}; use crate::model::{MetadataModel, ValTransaction, VarTransaction, Worker, INVALID_EXPIRE_AT}; -use crate::storage::{MetaStore, MetaStoreRef, Transaction}; +use crate::storage::{MetaStore, Transaction}; use crate::{MetaError, MetaResult}; pub type WorkerId = u32; @@ -71,7 +76,7 @@ pub struct ClusterManager { impl ClusterManager { pub async fn new(env: MetaSrvEnv, max_heartbeat_interval: Duration) -> MetaResult { - let core = ClusterManagerCore::new(env.meta_store_ref()).await?; + let core = ClusterManagerCore::new(env.clone()).await?; Ok(Self { env, @@ -99,12 +104,15 @@ impl ClusterManager { r#type: WorkerType, host_address: HostAddress, property: AddNodeProperty, + resource: risingwave_pb::common::worker_node::Resource, ) -> MetaResult { let worker_node_parallelism = property.worker_node_parallelism as usize; let mut property = self.parse_property(r#type, property); let mut core = self.core.write().await; if let Some(worker) = core.get_worker_by_host_mut(host_address.clone()) { + worker.update_resource(Some(resource)); + worker.update_started_at(timestamp_now_sec()); if let Some(property) = &mut property { property.is_unschedulable = worker .worker_node @@ -118,7 +126,7 @@ impl ClusterManager { if current_parallelism == worker_node_parallelism && worker.worker_node.property == property { - worker.update_ttl(self.max_heartbeat_interval); + worker.update_expire_at(self.max_heartbeat_interval); return Ok(worker.to_protobuf()); } @@ -162,7 +170,7 @@ impl ClusterManager { new_worker.worker_node.property = property; } - new_worker.update_ttl(self.max_heartbeat_interval); + new_worker.update_expire_at(self.max_heartbeat_interval); new_worker.insert(self.env.meta_store()).await?; *worker = new_worker; return Ok(worker.to_protobuf()); @@ -202,9 +210,14 @@ impl ClusterManager { parallel_units, property, transactional_id, + // resource doesn't need persist + resource: None, + started_at: None, }; - let worker = Worker::from_protobuf(worker_node.clone()); + let mut worker = Worker::from_protobuf(worker_node.clone()); + worker.update_started_at(timestamp_now_sec()); + worker.update_resource(Some(resource)); // Persist worker node. worker.insert(self.env.meta_store()).await?; // Update core. @@ -318,7 +331,7 @@ impl ClusterManager { let mut core = self.core.write().await; for worker in core.workers.values_mut() { if worker.worker_id() == worker_id { - worker.update_ttl(self.max_heartbeat_interval); + worker.update_expire_at(self.max_heartbeat_interval); worker.update_info(info); return Ok(()); } @@ -353,19 +366,16 @@ impl ClusterManager { // 1. Initialize new workers' TTL. for worker in workers .values_mut() - .filter(|worker| worker.expire_at() == INVALID_EXPIRE_AT) + .filter(|worker| worker.expire_at == INVALID_EXPIRE_AT) { - worker.update_ttl(cluster_manager.max_heartbeat_interval); + worker.update_expire_at(cluster_manager.max_heartbeat_interval); } // 2. Collect expired workers. - let now = SystemTime::now() - .duration_since(SystemTime::UNIX_EPOCH) - .expect("Clock may have gone backwards") - .as_secs(); + let now = timestamp_now_sec(); ( workers .values() - .filter(|worker| worker.expire_at() < now) + .filter(|worker| worker.expire_at < now) .map(|worker| (worker.worker_id(), worker.key().unwrap())) .collect_vec(), now, @@ -413,11 +423,11 @@ impl ClusterManager { /// Get live nodes with the specified type and state. /// # Arguments - /// * `worker_type` `WorkerType` of the nodes + /// * `worker_type` `WorkerType` of the nodes if it is not None. /// * `worker_state` Filter by this state if it is not None. pub async fn list_worker_node( &self, - worker_type: WorkerType, + worker_type: Option, worker_state: Option, ) -> Vec { let core = self.core.read().await; @@ -504,19 +514,22 @@ pub struct StreamingClusterInfo { } pub struct ClusterManagerCore { + env: MetaSrvEnv, /// Record for workers in the cluster. workers: HashMap, - /// Record for tracking available machine ids, one is available. available_transactional_ids: VecDeque, + /// Used as timestamp when meta node starts in sec. + started_at: u64, } impl ClusterManagerCore { pub const MAX_WORKER_REUSABLE_ID_BITS: usize = 10; pub const MAX_WORKER_REUSABLE_ID_COUNT: usize = 1 << Self::MAX_WORKER_REUSABLE_ID_BITS; - async fn new(meta_store: MetaStoreRef) -> MetaResult { - let mut workers = Worker::list(&meta_store).await?; + async fn new(env: MetaSrvEnv) -> MetaResult { + let meta_store = env.meta_store(); + let mut workers = Worker::list(meta_store).await?; let used_transactional_ids: HashSet<_> = workers .iter() @@ -569,11 +582,13 @@ impl ClusterManagerCore { } Ok(Self { + env, workers: workers .into_iter() .map(|w| (WorkerKey(w.key().unwrap()), w)) .collect(), available_transactional_ids, + started_at: timestamp_now_sec(), }) } @@ -623,24 +638,34 @@ impl ClusterManagerCore { pub fn list_worker_node( &self, - worker_type: WorkerType, + worker_type: Option, worker_state: Option, ) -> Vec { let worker_state = worker_state.map(|worker_state| worker_state as i32); - self.workers .values() - .map(|worker| worker.to_protobuf()) - .filter(|w| w.r#type == worker_type as i32) + .map(|worker| WorkerNode { + resource: worker.resource.to_owned(), + started_at: worker.started_at, + ..worker.to_protobuf() + }) + .chain(iter::once(meta_node_info( + &self.env.opts.advertise_addr, + Some(self.started_at), + ))) + .filter(|w| match worker_type { + None => true, + Some(worker_type) => w.r#type == worker_type as i32, + }) .filter(|w| match worker_state { None => true, Some(state) => state == w.state, }) - .collect_vec() + .collect() } pub fn list_streaming_worker_node(&self, worker_state: Option) -> Vec { - self.list_worker_node(WorkerType::ComputeNode, worker_state) + self.list_worker_node(Some(WorkerType::ComputeNode), worker_state) .into_iter() .filter(|w| w.property.as_ref().map_or(false, |p| p.is_streaming)) .collect() @@ -648,7 +673,7 @@ impl ClusterManagerCore { // List all parallel units on running nodes pub fn list_serving_worker_node(&self, worker_state: Option) -> Vec { - self.list_worker_node(WorkerType::ComputeNode, worker_state) + self.list_worker_node(Some(WorkerType::ComputeNode), worker_state) .into_iter() .filter(|w| w.property.as_ref().map_or(false, |p| p.is_serving)) .collect() @@ -721,6 +746,34 @@ impl ClusterManagerCore { } } +fn timestamp_now_sec() -> u64 { + SystemTime::now() + .duration_since(SystemTime::UNIX_EPOCH) + .expect("Clock may have gone backwards") + .as_secs() +} + +fn meta_node_info(host: &str, started_at: Option) -> WorkerNode { + WorkerNode { + id: META_NODE_ID, + r#type: WorkerType::Meta as _, + host: HostAddr::try_from(host) + .as_ref() + .map(HostAddr::to_protobuf) + .ok(), + state: State::Running as _, + parallel_units: vec![], + property: None, + transactional_id: None, + resource: Some(risingwave_pb::common::worker_node::Resource { + rw_version: RW_VERSION.to_string(), + total_memory_bytes: system_memory_available_bytes() as _, + total_cpu_cores: total_cpu_available() as _, + }), + started_at, + } +} + #[cfg(test)] mod tests { use super::*; @@ -750,6 +803,7 @@ mod tests { is_serving: true, is_unschedulable: false, }, + Default::default(), ) .await .unwrap(); @@ -790,6 +844,7 @@ mod tests { is_serving: true, is_unschedulable: false, }, + Default::default(), ) .await .unwrap(); @@ -811,6 +866,7 @@ mod tests { is_serving: true, is_unschedulable: false, }, + Default::default(), ) .await .unwrap(); @@ -852,6 +908,7 @@ mod tests { is_serving: true, is_unschedulable: false, }, + Default::default(), ) .await .unwrap(); @@ -904,13 +961,14 @@ mod tests { is_serving: true, is_unschedulable: false, }, + Default::default(), ) .await .unwrap(); // Two live nodes assert_eq!( cluster_manager - .list_worker_node(WorkerType::ComputeNode, None) + .list_worker_node(Some(WorkerType::ComputeNode), None) .await .len(), 2 @@ -937,7 +995,7 @@ mod tests { // started. assert_eq!( cluster_manager - .list_worker_node(WorkerType::ComputeNode, None) + .list_worker_node(Some(WorkerType::ComputeNode), None) .await .len(), 2 @@ -950,7 +1008,7 @@ mod tests { // One live node left. assert_eq!( cluster_manager - .list_worker_node(WorkerType::ComputeNode, None) + .list_worker_node(Some(WorkerType::ComputeNode), None) .await .len(), 1 diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index c9c22ff438cf8..841e2b931a894 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -169,6 +169,7 @@ pub struct MetaOpts { pub compaction_task_max_heartbeat_interval_secs: u64, pub compaction_config: Option, + pub advertise_addr: String, } impl MetaOpts { @@ -209,6 +210,7 @@ impl MetaOpts { partition_vnode_count: 32, compaction_task_max_heartbeat_interval_secs: 0, compaction_config: None, + advertise_addr: "".to_string(), } } } diff --git a/src/meta/src/model/cluster.rs b/src/meta/src/model/cluster.rs index 3d654a1d6b8c9..dd3228dc98da4 100644 --- a/src/meta/src/model/cluster.rs +++ b/src/meta/src/model/cluster.rs @@ -17,6 +17,7 @@ use std::ops::{Add, Deref}; use std::time::{Duration, SystemTime}; use risingwave_hummock_sdk::HummockSstableObjectId; +use risingwave_pb::common::worker_node::Resource; use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; use risingwave_pb::meta::heartbeat_request::extra_info::Info; use uuid::Uuid; @@ -37,14 +38,16 @@ pub struct Worker { // Volatile values updated by meta node as follows. // // Unix timestamp that the worker will expire at. - expire_at: u64, + pub expire_at: u64, + pub started_at: Option, // Volatile values updated by worker as follows: // // Monotonic increasing id since meta node bootstrap. - info_version_id: u64, + pub info_version_id: u64, // GC watermark. - hummock_gc_watermark: Option, + pub hummock_gc_watermark: Option, + pub resource: Option, } impl MetadataModel for Worker { @@ -63,8 +66,10 @@ impl MetadataModel for Worker { Self { worker_node: prost, expire_at: INVALID_EXPIRE_AT, + started_at: None, info_version_id: 0, - hummock_gc_watermark: Default::default(), + hummock_gc_watermark: None, + resource: None, } } @@ -86,13 +91,9 @@ impl Worker { self.worker_node.r#type() } - pub fn expire_at(&self) -> u64 { - self.expire_at - } - - pub fn update_ttl(&mut self, ttl: Duration) { + pub fn update_expire_at(&mut self, ttl: Duration) { let expire_at = cmp::max( - self.expire_at(), + self.expire_at, SystemTime::now() .add(ttl) .duration_since(SystemTime::UNIX_EPOCH) @@ -102,6 +103,10 @@ impl Worker { self.expire_at = expire_at; } + pub fn update_started_at(&mut self, started_at: u64) { + self.started_at = Some(started_at); + } + pub fn update_info(&mut self, info: Vec) { self.info_version_id += 1; for i in info { @@ -113,12 +118,8 @@ impl Worker { } } - pub fn hummock_gc_watermark(&self) -> Option { - self.hummock_gc_watermark - } - - pub fn info_version_id(&self) -> u64 { - self.info_version_id + pub fn update_resource(&mut self, resource: Option) { + self.resource = resource; } } diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index 3183007753cbd..149c426fc59ef 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -768,7 +768,7 @@ pub fn start_fragment_info_monitor( meta_metrics.actor_info.reset(); meta_metrics.table_info.reset(); let workers: HashMap = cluster_manager - .list_worker_node(WorkerType::ComputeNode, None) + .list_worker_node(Some(WorkerType::ComputeNode), None) .await .into_iter() .map(|worker_node| match worker_node.host { diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index acd874f4bdc00..ea13e2326dffb 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -899,6 +899,7 @@ mod tests { is_serving: true, is_unschedulable: false, }, + Default::default(), ) .await?; cluster_manager.activate_worker_node(host).await?; diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 0b9de6439d19b..5f74ed1608235 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -32,6 +32,9 @@ use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::telemetry::report::TelemetryInfoFetcher; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_common::util::resource_util::cpu::total_cpu_available; +use risingwave_common::util::resource_util::memory::system_memory_available_bytes; +use risingwave_common::RW_VERSION; use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::{ CompactionGroupId, HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, @@ -245,6 +248,11 @@ impl MetaClient { worker_type: worker_type as i32, host: Some(addr.to_protobuf()), property: Some(property.clone()), + resource: Some(risingwave_pb::common::worker_node::Resource { + rw_version: RW_VERSION.to_string(), + total_memory_bytes: system_memory_available_bytes() as _, + total_cpu_cores: total_cpu_available() as _, + }), }) .await?; if let Some(status) = &add_worker_resp.status @@ -663,9 +671,12 @@ impl MetaClient { Ok(resp) } - pub async fn list_worker_nodes(&self, worker_type: WorkerType) -> Result> { + pub async fn list_worker_nodes( + &self, + worker_type: Option, + ) -> Result> { let request = ListAllNodesRequest { - worker_type: worker_type as _, + worker_type: worker_type.map(Into::into), include_starting_nodes: true, }; let resp = self.inner.list_all_nodes(request).await?; @@ -1724,7 +1735,6 @@ macro_rules! for_all_meta_rpc { ,{ cluster_client, activate_worker_node, ActivateWorkerNodeRequest, ActivateWorkerNodeResponse } ,{ cluster_client, delete_worker_node, DeleteWorkerNodeRequest, DeleteWorkerNodeResponse } ,{ cluster_client, update_worker_node_schedulability, UpdateWorkerNodeSchedulabilityRequest, UpdateWorkerNodeSchedulabilityResponse } - //(not used) ,{ cluster_client, list_all_nodes, ListAllNodesRequest, ListAllNodesResponse } ,{ cluster_client, list_all_nodes, ListAllNodesRequest, ListAllNodesResponse } ,{ heartbeat_client, heartbeat, HeartbeatRequest, HeartbeatResponse } ,{ stream_client, flush, FlushRequest, FlushResponse } diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index de3ff915d010e..294acd39df31f 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -249,6 +249,7 @@ pub(crate) mod tests { WorkerType::ComputeNode, HostAddress::default(), Property::default(), + Default::default(), ) .await .unwrap();